You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wa...@apache.org on 2015/09/30 17:42:01 UTC

[01/50] [abbrv] hadoop git commit: HDFS-8220. Erasure Coding: StripedDataStreamer fails to handle the blocklocations which doesn't satisfy BlockGroupSize.

Repository: hadoop
Updated Branches:
  refs/heads/trunk 854d25b0c -> 6c17d3152


HDFS-8220. Erasure Coding: StripedDataStreamer fails to handle the blocklocations which doesn't satisfy BlockGroupSize.


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

Branch: refs/heads/trunk
Commit: b57c9a35f76612a02e16497ea92fbc3f4dc524ec
Parents: 1d37a88
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Fri Aug 14 15:16:22 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Fri Aug 14 15:16:58 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +
 .../apache/hadoop/hdfs/StripedDataStreamer.java | 42 +++++++---
 .../TestDFSStripedOutputStreamWithFailure.java  | 84 ++++++++++++++++++++
 3 files changed, 120 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b57c9a35/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index b752d5b..dad997a 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -397,3 +397,6 @@
 
     HDFS-8854. Erasure coding: add ECPolicy to replace schema+cellSize in
     hadoop-hdfs. (Walter Su via zhz)
+
+    HDFS-8220. Erasure Coding: StripedDataStreamer fails to handle the
+    blocklocations which doesn't satisfy BlockGroupSize. (Rakesh R via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b57c9a35/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
index 2d51dc4..f533bf9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator;
 import org.apache.hadoop.hdfs.DFSStripedOutputStream.MultipleBlockingQueue;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@@ -167,18 +168,33 @@ public class StripedDataStreamer extends DataStreamer {
 
         final LocatedBlock lb = StripedDataStreamer.super.locateFollowingBlock(
             excludedNodes);
+        if (lb.getLocations().length < HdfsConstants.NUM_DATA_BLOCKS) {
+          throw new IOException(
+              "Failed to get datablocks number of nodes from namenode: blockGroupSize= "
+                  + (HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS)
+                  + ", blocks.length= " + lb.getLocations().length);
+        }
         final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
             (LocatedStripedBlock)lb,
             BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
 
         for (int i = 0; i < blocks.length; i++) {
-          if (!coordinator.getStripedDataStreamer(i).isFailed()) {
-            if (blocks[i] == null) {
-              getLastException().set(
-                  new IOException("Failed to get following block, i=" + i));
-            } else {
-              followingBlocks.offer(i, blocks[i]);
-            }
+          StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
+          if (si.isFailed()) {
+            continue; // skipping failed data streamer
+          }
+          if (blocks[i] == null) {
+            // Set exception and close streamer as there is no block locations
+            // found for the parity block.
+            LOG.warn("Failed to get block location for parity block, index="
+                + i);
+            si.getLastException().set(
+                new IOException("Failed to get following block, i=" + i));
+            si.setFailed(true);
+            si.endBlock();
+            si.close(true);
+          } else {
+            followingBlocks.offer(i, blocks[i]);
           }
         }
       }
@@ -199,7 +215,11 @@ public class StripedDataStreamer extends DataStreamer {
             .parseStripedBlockGroup((LocatedStripedBlock) updated,
                 BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
         for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) {
-          final ExtendedBlock bi = coordinator.getStripedDataStreamer(i).getBlock();
+          StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
+          if (si.isFailed()) {
+            continue; // skipping failed data streamer
+          }
+          final ExtendedBlock bi = si.getBlock();
           if (bi != null) {
             final LocatedBlock lb = new LocatedBlock(newBlock(bi, newGS),
                 null, null, null, -1, updated.isCorrupt(), null);
@@ -225,7 +245,11 @@ public class StripedDataStreamer extends DataStreamer {
         final ExtendedBlock newBG = newBlock(bg, newGS);
         final ExtendedBlock updated = callUpdatePipeline(bg, newBG);
         for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) {
-          final ExtendedBlock bi = coordinator.getStripedDataStreamer(i).getBlock();
+          StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
+          if (si.isFailed()) {
+            continue; // skipping failed data streamer
+          }
+          final ExtendedBlock bi = si.getBlock();
           updateBlocks.offer(i, newBlock(bi, updated.getGenerationStamp()));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b57c9a35/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index fed9f16..f65d0c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.junit.Assert.assertEquals;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -33,6 +35,7 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@@ -40,6 +43,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.SecurityTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.security.token.Token;
@@ -145,6 +149,86 @@ public class TestDFSStripedOutputStreamWithFailure {
     }
   }
 
+  @Test(timeout = 90000)
+  public void testAddBlockWhenNoSufficientDataBlockNumOfNodes()
+      throws IOException {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    try {
+      setup(conf);
+      ArrayList<DataNode> dataNodes = cluster.getDataNodes();
+      // shutdown few datanodes to avoid getting sufficient data blocks number
+      // of datanodes
+      int killDns = dataNodes.size() / 2;
+      int numDatanodes = dataNodes.size() - killDns;
+      for (int i = 0; i < killDns; i++) {
+        cluster.stopDataNode(i);
+      }
+      cluster.restartNameNodes();
+      cluster.triggerHeartbeats();
+      DatanodeInfo[] info = dfs.getClient().datanodeReport(
+          DatanodeReportType.LIVE);
+      assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
+      final Path dirFile = new Path(dir, "ecfile");
+      FSDataOutputStream out = null;
+      try {
+        out = dfs.create(dirFile, true);
+        out.write("something".getBytes());
+        out.flush();
+        out.close();
+        Assert.fail("Failed to validate available dns against blkGroupSize");
+      } catch (IOException ioe) {
+        // expected
+        GenericTestUtils.assertExceptionContains("Failed: the number of "
+            + "remaining blocks = 5 < the number of data blocks = 6", ioe);
+        DFSStripedOutputStream dfsout = (DFSStripedOutputStream) out
+            .getWrappedStream();
+
+        // get leading streamer and verify the last exception
+        StripedDataStreamer datastreamer = dfsout.getStripedDataStreamer(0);
+        try {
+          datastreamer.getLastException().check(true);
+          Assert.fail("Failed to validate available dns against blkGroupSize");
+        } catch (IOException le) {
+          GenericTestUtils.assertExceptionContains(
+              "Failed to get datablocks number of nodes from"
+                  + " namenode: blockGroupSize= 9, blocks.length= "
+                  + numDatanodes, le);
+        }
+      }
+    } finally {
+      tearDown();
+    }
+  }
+
+  @Test(timeout = 90000)
+  public void testAddBlockWhenNoSufficientParityNumOfNodes() throws IOException {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    try {
+      setup(conf);
+      ArrayList<DataNode> dataNodes = cluster.getDataNodes();
+      // shutdown few data nodes to avoid writing parity blocks
+      int killDns = (NUM_PARITY_BLOCKS - 1);
+      int numDatanodes = dataNodes.size() - killDns;
+      for (int i = 0; i < killDns; i++) {
+        cluster.stopDataNode(i);
+      }
+      cluster.restartNameNodes();
+      cluster.triggerHeartbeats();
+      DatanodeInfo[] info = dfs.getClient().datanodeReport(
+          DatanodeReportType.LIVE);
+      assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
+      Path srcPath = new Path(dir, "testAddBlockWhenNoSufficientParityNodes");
+      int fileLength = HdfsConstants.BLOCK_STRIPED_CELL_SIZE - 1000;
+      final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
+      DFSTestUtil.writeFile(dfs, srcPath, new String(expected));
+      StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength);
+    } finally {
+      tearDown();
+    }
+  }
+
   private void runTest(final Path p, final int length, final int killPos,
       final int dnIndex, final boolean tokenExpire) throws Exception {
     LOG.info("p=" + p + ", length=" + length + ", killPos=" + killPos


[16/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
Merge remote-tracking branch 'apache/trunk' into HDFS-7285


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

Branch: refs/heads/trunk
Commit: 53358fe680a11c1b66a7f60733d11c1f4efe0232
Parents: ab56fcd 2e251a7
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Tue Sep 1 00:29:55 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue Sep 1 14:48:37 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   9 +
 hadoop-common-project/hadoop-common/pom.xml     |  19 +-
 .../fs/CommonConfigurationKeysPublic.java       |   7 +
 .../src/main/resources/core-default.xml         |  14 +-
 .../src/site/markdown/FileSystemShell.md        |  13 +-
 .../java/org/apache/hadoop/fs/test-untar.tar    | Bin 20480 -> 0 bytes
 .../java/org/apache/hadoop/fs/test-untar.tgz    | Bin 2024 -> 0 bytes
 .../fs/viewfs/ViewFileSystemBaseTest.java       |   2 +-
 .../apache/hadoop/fs/viewfs/ViewFsBaseTest.java |   2 +-
 .../src/test/resources/test-untar.tar           | Bin 0 -> 20480 bytes
 .../src/test/resources/test-untar.tgz           | Bin 0 -> 2024 bytes
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  |   5 +
 .../org/apache/hadoop/hdfs/BlockReader.java     | 110 +++
 .../apache/hadoop/hdfs/BlockReaderLocal.java    | 748 +++++++++++++++++++
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     | 743 ++++++++++++++++++
 .../org/apache/hadoop/hdfs/BlockReaderUtil.java |  57 ++
 .../org/apache/hadoop/hdfs/ClientContext.java   | 196 +++++
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  68 ++
 .../apache/hadoop/hdfs/ExternalBlockReader.java | 126 ++++
 .../apache/hadoop/hdfs/KeyProviderCache.java    | 112 +++
 .../java/org/apache/hadoop/hdfs/PeerCache.java  | 291 ++++++++
 .../apache/hadoop/hdfs/RemoteBlockReader.java   | 517 +++++++++++++
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  | 485 ++++++++++++
 .../hadoop/hdfs/client/BlockReportOptions.java  |  59 ++
 .../hdfs/client/HdfsClientConfigKeys.java       |  13 +
 .../hdfs/protocol/BlockLocalPathInfo.java       |  70 ++
 .../hdfs/protocol/ClientDatanodeProtocol.java   | 152 ++++
 .../InvalidEncryptionKeyException.java          |  40 +
 .../protocol/datatransfer/PacketHeader.java     | 214 ++++++
 .../protocol/datatransfer/PacketReceiver.java   | 310 ++++++++
 .../protocolPB/ClientDatanodeProtocolPB.java    |  37 +
 .../ClientDatanodeProtocolTranslatorPB.java     | 326 ++++++++
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  13 +
 .../token/block/BlockTokenSelector.java         |  48 ++
 .../hdfs/util/ByteBufferOutputStream.java       |  49 ++
 .../hadoop/hdfs/web/URLConnectionFactory.java   |  30 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  15 +-
 .../hdfs/web/oauth2/AccessTokenProvider.java    |  66 ++
 .../hdfs/web/oauth2/AccessTokenTimer.java       | 103 +++
 .../ConfCredentialBasedAccessTokenProvider.java |  62 ++
 ...onfRefreshTokenBasedAccessTokenProvider.java | 146 ++++
 .../CredentialBasedAccessTokenProvider.java     | 135 ++++
 .../oauth2/OAuth2ConnectionConfigurator.java    |  79 ++
 .../hadoop/hdfs/web/oauth2/OAuth2Constants.java |  46 ++
 .../apache/hadoop/hdfs/web/oauth2/Utils.java    |  63 ++
 .../hadoop/hdfs/web/oauth2/package-info.java    |  26 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  22 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   6 +
 .../bkjournal/BookKeeperEditLogInputStream.java |   2 +-
 .../org/apache/hadoop/hdfs/BlockReader.java     | 110 ---
 .../apache/hadoop/hdfs/BlockReaderLocal.java    | 746 ------------------
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     | 740 ------------------
 .../org/apache/hadoop/hdfs/BlockReaderUtil.java |  57 --
 .../org/apache/hadoop/hdfs/ClientContext.java   | 195 -----
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   1 -
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  14 +-
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |   2 +-
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  68 +-
 .../apache/hadoop/hdfs/ExternalBlockReader.java | 126 ----
 .../apache/hadoop/hdfs/KeyProviderCache.java    | 111 ---
 .../java/org/apache/hadoop/hdfs/PeerCache.java  | 290 -------
 .../apache/hadoop/hdfs/RemoteBlockReader.java   | 513 -------------
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  | 482 ------------
 .../hadoop/hdfs/client/BlockReportOptions.java  |  59 --
 .../hdfs/protocol/BlockLocalPathInfo.java       |  70 --
 .../hdfs/protocol/ClientDatanodeProtocol.java   | 152 ----
 .../hadoop/hdfs/protocol/LayoutVersion.java     |   2 +-
 .../InvalidEncryptionKeyException.java          |  40 -
 .../protocol/datatransfer/PacketHeader.java     | 214 ------
 .../protocol/datatransfer/PacketReceiver.java   | 310 --------
 .../hdfs/protocol/datatransfer/Receiver.java    |  15 +-
 .../protocolPB/ClientDatanodeProtocolPB.java    |  37 -
 ...tDatanodeProtocolServerSideTranslatorPB.java |   6 +-
 .../ClientDatanodeProtocolTranslatorPB.java     | 326 --------
 ...tNamenodeProtocolServerSideTranslatorPB.java |  14 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   2 +-
 ...rDatanodeProtocolServerSideTranslatorPB.java |   2 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  33 +-
 .../token/block/BlockTokenSelector.java         |  48 --
 .../hdfs/server/blockmanagement/BlockInfo.java  |  19 +-
 .../blockmanagement/BlockInfoContiguous.java    |  15 -
 .../blockmanagement/BlockInfoStriped.java       |  19 -
 .../server/blockmanagement/BlockManager.java    | 552 +++++---------
 .../BlockPlacementPolicyDefault.java            | 147 +---
 .../blockmanagement/BlockRecoveryWork.java      | 111 +++
 .../blockmanagement/BlockToMarkCorrupt.java     |  82 ++
 .../hdfs/server/blockmanagement/BlocksMap.java  |  16 -
 .../blockmanagement/DatanodeDescriptor.java     |  35 +-
 .../server/blockmanagement/DatanodeManager.java |   9 +-
 .../blockmanagement/ErasureCodingWork.java      |  60 ++
 .../server/blockmanagement/HostFileManager.java |  19 +
 .../server/blockmanagement/ReplicationWork.java |  53 ++
 .../hadoop/hdfs/server/datanode/DNConf.java     |   4 +-
 .../namenode/EditLogBackupInputStream.java      |   2 +-
 .../server/namenode/EditLogFileInputStream.java |   2 +-
 .../hdfs/server/namenode/FSDirDeleteOp.java     |  40 +
 .../hdfs/server/namenode/FSDirectory.java       |  63 ++
 .../hdfs/server/namenode/FSEditLogLoader.java   |   4 +-
 .../hdfs/server/namenode/FSEditLogOp.java       | 354 ++++++---
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |   3 +-
 .../hdfs/util/ByteBufferOutputStream.java       |  49 --
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    |  25 +
 .../hadoop/hdfs/TestBlockReaderLocal.java       |  30 +-
 .../hadoop/hdfs/TestBlockReaderLocalLegacy.java |   2 +-
 .../hdfs/TestClientBlockVerification.java       |   4 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |   2 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |  15 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |   4 +-
 .../security/token/block/TestBlockToken.java    |  10 +-
 .../blockmanagement/TestBlockInfoStriped.java   |  30 -
 .../blockmanagement/TestDatanodeManager.java    | 103 ++-
 .../blockmanagement/TestHostFileManager.java    |   7 +-
 .../blockmanagement/TestReplicationPolicy.java  |  26 +-
 .../hdfs/server/namenode/TestEditLog.java       |   2 +-
 .../namenode/TestEditLogFileInputStream.java    |  80 ++
 .../namenode/TestProtectedDirectories.java      | 373 +++++++++
 .../shortcircuit/TestShortCircuitLocalRead.java |   4 +-
 .../hadoop/hdfs/web/TestWebHDFSOAuth2.java      | 216 ++++++
 .../hdfs/web/oauth2/TestAccessTokenTimer.java   |  63 ++
 ...ClientCredentialTimeBasedTokenRefresher.java | 138 ++++
 ...TestRefreshTokenTimeBasedTokenRefresher.java | 138 ++++
 hadoop-project/src/site/site.xml                |   1 +
 .../org/apache/hadoop/fs/s3a/Constants.java     |   4 +-
 .../src/site/markdown/tools/hadoop-aws/index.md |   4 +-
 hadoop-yarn-project/CHANGES.txt                 |  10 +-
 .../hadoop/yarn/client/TestRMFailover.java      |  27 +
 .../hadoop/yarn/webapp/YarnWebParams.java       |   1 +
 .../scheduler/capacity/AbstractCSQueue.java     |  27 +
 .../scheduler/capacity/CSQueue.java             |  26 +
 .../scheduler/capacity/CapacityScheduler.java   |  40 +-
 .../scheduler/capacity/LeafQueue.java           |  16 +
 .../scheduler/common/fica/FiCaSchedulerApp.java |   9 +
 .../resourcemanager/webapp/RMWebAppFilter.java  |  90 ++-
 .../TestCapacitySchedulerNodeLabelUpdate.java   | 249 +++++-
 .../src/site/markdown/NodeLabel.md              | 140 ++++
 135 files changed, 8407 insertions(+), 5608 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
index 0000000,aa3e8ba..8f988af
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
@@@ -1,0 -1,102 +1,110 @@@
+ /**
+  * 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.hdfs;
+ 
++import java.io.Closeable;
+ import java.io.IOException;
+ import java.util.EnumSet;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.fs.ByteBufferReadable;
+ import org.apache.hadoop.fs.ReadOption;
+ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
++import org.apache.hadoop.util.DataChecksum;
+ 
+ /**
+  * A BlockReader is responsible for reading a single block
+  * from a single datanode.
+  */
+ @InterfaceAudience.Private
 -public interface BlockReader extends ByteBufferReadable {
++public interface BlockReader extends ByteBufferReadable, Closeable {
+   
+ 
+   /* same interface as inputStream java.io.InputStream#read()
+    * used by DFSInputStream#read()
+    * This violates one rule when there is a checksum error:
+    * "Read should not modify user buffer before successful read"
+    * because it first reads the data to user buffer and then checks
+    * the checksum.
+    * Note: this must return -1 on EOF, even in the case of a 0-byte read.
+    * See HDFS-5762 for details.
+    */
+   int read(byte[] buf, int off, int len) throws IOException;
+ 
+   /**
+    * Skip the given number of bytes
+    */
+   long skip(long n) throws IOException;
+ 
+   /**
+    * Returns an estimate of the number of bytes that can be read
+    * (or skipped over) from this input stream without performing
+    * network I/O.
+    * This may return more than what is actually present in the block.
+    */
+   int available() throws IOException;
+ 
+   /**
+    * Close the block reader.
+    *
+    * @throws IOException
+    */
++  @Override // java.io.Closeable
+   void close() throws IOException;
+ 
+   /**
+    * Read exactly the given amount of data, throwing an exception
+    * if EOF is reached before that amount
+    */
+   void readFully(byte[] buf, int readOffset, int amtToRead) throws IOException;
+ 
+   /**
+    * Similar to {@link #readFully(byte[], int, int)} except that it will
+    * not throw an exception on EOF. However, it differs from the simple
+    * {@link #read(byte[], int, int)} call in that it is guaranteed to
+    * read the data if it is available. In other words, if this call
+    * does not throw an exception, then either the buffer has been
+    * filled or the next call will return EOF.
+    */
+   int readAll(byte[] buf, int offset, int len) throws IOException;
+ 
+   /**
+    * @return              true only if this is a local read.
+    */
+   boolean isLocal();
+   
+   /**
+    * @return              true only if this is a short-circuit read.
+    *                      All short-circuit reads are also local.
+    */
+   boolean isShortCircuit();
+ 
+   /**
+    * Get a ClientMmap object for this BlockReader.
+    *
+    * @param opts          The read options to use.
+    * @return              The ClientMmap object, or null if mmap is not
+    *                      supported.
+    */
+   ClientMmap getClientMmap(EnumSet<ReadOption> opts);
++
++  /**
++   * @return              The DataChecksum used by the read block
++   */
++  DataChecksum getDataChecksum();
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
index 0000000,2a0e21b..8d7f294
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
@@@ -1,0 -1,743 +1,748 @@@
+ /**
+  * 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.hdfs;
+ 
+ import java.io.IOException;
+ import java.nio.ByteBuffer;
+ import java.nio.channels.FileChannel;
+ import java.util.EnumSet;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.fs.ReadOption;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+ import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
+ import org.apache.hadoop.util.DataChecksum;
+ import org.apache.hadoop.util.DirectBufferPool;
+ import org.apache.htrace.Sampler;
+ import org.apache.htrace.Trace;
+ import org.apache.htrace.TraceScope;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.base.Preconditions;
+ 
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ /**
+  * BlockReaderLocal enables local short circuited reads. If the DFS client is on
+  * the same machine as the datanode, then the client can read files directly
+  * from the local file system rather than going through the datanode for better
+  * performance. <br>
+  * {@link BlockReaderLocal} works as follows:
+  * <ul>
+  * <li>The client performing short circuit reads must be configured at the
+  * datanode.</li>
+  * <li>The client gets the file descriptors for the metadata file and the data 
+  * file for the block using
+  * {@link org.apache.hadoop.hdfs.server.datanode.DataXceiver#requestShortCircuitFds}.
+  * </li>
+  * <li>The client reads the file descriptors.</li>
+  * </ul>
+  */
+ @InterfaceAudience.Private
+ class BlockReaderLocal implements BlockReader {
+   static final Logger LOG = LoggerFactory.getLogger(BlockReaderLocal.class);
+ 
+   private static final DirectBufferPool bufferPool = new DirectBufferPool();
+ 
+   public static class Builder {
+     private final int bufferSize;
+     private boolean verifyChecksum;
+     private int maxReadahead;
+     private String filename;
+     private ShortCircuitReplica replica;
+     private long dataPos;
+     private ExtendedBlock block;
+     private StorageType storageType;
+ 
+     public Builder(ShortCircuitConf conf) {
+       this.maxReadahead = Integer.MAX_VALUE;
+       this.verifyChecksum = !conf.isSkipShortCircuitChecksums();
+       this.bufferSize = conf.getShortCircuitBufferSize();
+     }
+ 
+     public Builder setVerifyChecksum(boolean verifyChecksum) {
+       this.verifyChecksum = verifyChecksum;
+       return this;
+     }
+ 
+     public Builder setCachingStrategy(CachingStrategy cachingStrategy) {
+       long readahead = cachingStrategy.getReadahead() != null ?
+           cachingStrategy.getReadahead() :
+               HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT;
+       this.maxReadahead = (int)Math.min(Integer.MAX_VALUE, readahead);
+       return this;
+     }
+ 
+     public Builder setFilename(String filename) {
+       this.filename = filename;
+       return this;
+     }
+ 
+     public Builder setShortCircuitReplica(ShortCircuitReplica replica) {
+       this.replica = replica;
+       return this;
+     }
+ 
+     public Builder setStartOffset(long startOffset) {
+       this.dataPos = Math.max(0, startOffset);
+       return this;
+     }
+ 
+     public Builder setBlock(ExtendedBlock block) {
+       this.block = block;
+       return this;
+     }
+ 
+     public Builder setStorageType(StorageType storageType) {
+       this.storageType = storageType;
+       return this;
+     }
+ 
+     public BlockReaderLocal build() {
+       Preconditions.checkNotNull(replica);
+       return new BlockReaderLocal(this);
+     }
+   }
+ 
+   private boolean closed = false;
+ 
+   /**
+    * Pair of streams for this block.
+    */
+   private final ShortCircuitReplica replica;
+ 
+   /**
+    * The data FileChannel.
+    */
+   private final FileChannel dataIn;
+ 
+   /**
+    * The next place we'll read from in the block data FileChannel.
+    *
+    * If data is buffered in dataBuf, this offset will be larger than the
+    * offset of the next byte which a read() operation will give us.
+    */
+   private long dataPos;
+ 
+   /**
+    * The Checksum FileChannel.
+    */
+   private final FileChannel checksumIn;
+   
+   /**
+    * Checksum type and size.
+    */
+   private final DataChecksum checksum;
+ 
+   /**
+    * If false, we will always skip the checksum.
+    */
+   private final boolean verifyChecksum;
+ 
+   /**
+    * Name of the block, for logging purposes.
+    */
+   private final String filename;
+   
+   /**
+    * Block ID and Block Pool ID.
+    */
+   private final ExtendedBlock block;
+   
+   /**
+    * Cache of Checksum#bytesPerChecksum.
+    */
+   private final int bytesPerChecksum;
+ 
+   /**
+    * Cache of Checksum#checksumSize.
+    */
+   private final int checksumSize;
+ 
+   /**
+    * Maximum number of chunks to allocate.
+    *
+    * This is used to allocate dataBuf and checksumBuf, in the event that
+    * we need them.
+    */
+   private final int maxAllocatedChunks;
+ 
+   /**
+    * True if zero readahead was requested.
+    */
+   private final boolean zeroReadaheadRequested;
+ 
+   /**
+    * Maximum amount of readahead we'll do.  This will always be at least the,
+    * size of a single chunk, even if {@link #zeroReadaheadRequested} is true.
+    * The reason is because we need to do a certain amount of buffering in order
+    * to do checksumming.
+    * 
+    * This determines how many bytes we'll use out of dataBuf and checksumBuf.
+    * Why do we allocate buffers, and then (potentially) only use part of them?
+    * The rationale is that allocating a lot of buffers of different sizes would
+    * make it very difficult for the DirectBufferPool to re-use buffers. 
+    */
+   private final int maxReadaheadLength;
+ 
+   /**
+    * Buffers data starting at the current dataPos and extending on
+    * for dataBuf.limit().
+    *
+    * This may be null if we don't need it.
+    */
+   private ByteBuffer dataBuf;
+ 
+   /**
+    * Buffers checksums starting at the current checksumPos and extending on
+    * for checksumBuf.limit().
+    *
+    * This may be null if we don't need it.
+    */
+   private ByteBuffer checksumBuf;
+ 
+   /**
+    * StorageType of replica on DataNode.
+    */
+   private StorageType storageType;
+ 
+   private BlockReaderLocal(Builder builder) {
+     this.replica = builder.replica;
+     this.dataIn = replica.getDataStream().getChannel();
+     this.dataPos = builder.dataPos;
+     this.checksumIn = replica.getMetaStream().getChannel();
+     BlockMetadataHeader header = builder.replica.getMetaHeader();
+     this.checksum = header.getChecksum();
+     this.verifyChecksum = builder.verifyChecksum &&
+         (this.checksum.getChecksumType().id != DataChecksum.CHECKSUM_NULL);
+     this.filename = builder.filename;
+     this.block = builder.block;
+     this.bytesPerChecksum = checksum.getBytesPerChecksum();
+     this.checksumSize = checksum.getChecksumSize();
+ 
+     this.maxAllocatedChunks = (bytesPerChecksum == 0) ? 0 :
+         ((builder.bufferSize + bytesPerChecksum - 1) / bytesPerChecksum);
+     // Calculate the effective maximum readahead.
+     // We can't do more readahead than there is space in the buffer.
+     int maxReadaheadChunks = (bytesPerChecksum == 0) ? 0 :
+         ((Math.min(builder.bufferSize, builder.maxReadahead) +
+             bytesPerChecksum - 1) / bytesPerChecksum);
+     if (maxReadaheadChunks == 0) {
+       this.zeroReadaheadRequested = true;
+       maxReadaheadChunks = 1;
+     } else {
+       this.zeroReadaheadRequested = false;
+     }
+     this.maxReadaheadLength = maxReadaheadChunks * bytesPerChecksum;
+     this.storageType = builder.storageType;
+   }
+ 
+   private synchronized void createDataBufIfNeeded() {
+     if (dataBuf == null) {
+       dataBuf = bufferPool.getBuffer(maxAllocatedChunks * bytesPerChecksum);
+       dataBuf.position(0);
+       dataBuf.limit(0);
+     }
+   }
+ 
+   private synchronized void freeDataBufIfExists() {
+     if (dataBuf != null) {
+       // When disposing of a dataBuf, we have to move our stored file index
+       // backwards.
+       dataPos -= dataBuf.remaining();
+       dataBuf.clear();
+       bufferPool.returnBuffer(dataBuf);
+       dataBuf = null;
+     }
+   }
+ 
+   private synchronized void createChecksumBufIfNeeded() {
+     if (checksumBuf == null) {
+       checksumBuf = bufferPool.getBuffer(maxAllocatedChunks * checksumSize);
+       checksumBuf.position(0);
+       checksumBuf.limit(0);
+     }
+   }
+ 
+   private synchronized void freeChecksumBufIfExists() {
+     if (checksumBuf != null) {
+       checksumBuf.clear();
+       bufferPool.returnBuffer(checksumBuf);
+       checksumBuf = null;
+     }
+   }
+ 
+   private synchronized int drainDataBuf(ByteBuffer buf) {
+     if (dataBuf == null) return -1;
+     int oldLimit = dataBuf.limit();
+     int nRead = Math.min(dataBuf.remaining(), buf.remaining());
+     if (nRead == 0) {
+       return (dataBuf.remaining() == 0) ? -1 : 0;
+     }
+     try {
+       dataBuf.limit(dataBuf.position() + nRead);
+       buf.put(dataBuf);
+     } finally {
+       dataBuf.limit(oldLimit);
+     }
+     return nRead;
+   }
+ 
+   /**
+    * Read from the block file into a buffer.
+    *
+    * This function overwrites checksumBuf.  It will increment dataPos.
+    *
+    * @param buf   The buffer to read into.  May be dataBuf.
+    *              The position and limit of this buffer should be set to
+    *              multiples of the checksum size.
+    * @param canSkipChecksum  True if we can skip checksumming.
+    *
+    * @return      Total bytes read.  0 on EOF.
+    */
+   private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
+       throws IOException {
+     TraceScope scope = Trace.startSpan("BlockReaderLocal#fillBuffer(" +
+         block.getBlockId() + ")", Sampler.NEVER);
+     try {
+       int total = 0;
+       long startDataPos = dataPos;
+       int startBufPos = buf.position();
+       while (buf.hasRemaining()) {
+         int nRead = dataIn.read(buf, dataPos);
+         if (nRead < 0) {
+           break;
+         }
+         dataPos += nRead;
+         total += nRead;
+       }
+       if (canSkipChecksum) {
+         freeChecksumBufIfExists();
+         return total;
+       }
+       if (total > 0) {
+         try {
+           buf.limit(buf.position());
+           buf.position(startBufPos);
+           createChecksumBufIfNeeded();
+           int checksumsNeeded = (total + bytesPerChecksum - 1) / bytesPerChecksum;
+           checksumBuf.clear();
+           checksumBuf.limit(checksumsNeeded * checksumSize);
+           long checksumPos = BlockMetadataHeader.getHeaderSize()
+               + ((startDataPos / bytesPerChecksum) * checksumSize);
+           while (checksumBuf.hasRemaining()) {
+             int nRead = checksumIn.read(checksumBuf, checksumPos);
+             if (nRead < 0) {
+               throw new IOException("Got unexpected checksum file EOF at " +
+                   checksumPos + ", block file position " + startDataPos + " for " +
+                   "block " + block + " of file " + filename);
+             }
+             checksumPos += nRead;
+           }
+           checksumBuf.flip();
+ 
+           checksum.verifyChunkedSums(buf, checksumBuf, filename, startDataPos);
+         } finally {
+           buf.position(buf.limit());
+         }
+       }
+       return total;
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   private boolean createNoChecksumContext() {
+     if (verifyChecksum) {
+       if (storageType != null && storageType.isTransient()) {
+         // Checksums are not stored for replicas on transient storage.  We do not
+         // anchor, because we do not intend for client activity to block eviction
+         // from transient storage on the DataNode side.
+         return true;
+       } else {
+         return replica.addNoChecksumAnchor();
+       }
+     } else {
+       return true;
+     }
+   }
+ 
+   private void releaseNoChecksumContext() {
+     if (verifyChecksum) {
+       if (storageType == null || !storageType.isTransient()) {
+         replica.removeNoChecksumAnchor();
+       }
+     }
+   }
+ 
+   @Override
+   public synchronized int read(ByteBuffer buf) throws IOException {
+     boolean canSkipChecksum = createNoChecksumContext();
+     try {
+       String traceString = null;
+       if (LOG.isTraceEnabled()) {
+         traceString = new StringBuilder().
+             append("read(").
+             append("buf.remaining=").append(buf.remaining()).
+             append(", block=").append(block).
+             append(", filename=").append(filename).
+             append(", canSkipChecksum=").append(canSkipChecksum).
+             append(")").toString();
+         LOG.info(traceString + ": starting");
+       }
+       int nRead;
+       try {
+         if (canSkipChecksum && zeroReadaheadRequested) {
+           nRead = readWithoutBounceBuffer(buf);
+         } else {
+           nRead = readWithBounceBuffer(buf, canSkipChecksum);
+         }
+       } catch (IOException e) {
+         if (LOG.isTraceEnabled()) {
+           LOG.info(traceString + ": I/O error", e);
+         }
+         throw e;
+       }
+       if (LOG.isTraceEnabled()) {
+         LOG.info(traceString + ": returning " + nRead);
+       }
+       return nRead;
+     } finally {
+       if (canSkipChecksum) releaseNoChecksumContext();
+     }
+   }
+ 
+   private synchronized int readWithoutBounceBuffer(ByteBuffer buf)
+       throws IOException {
+     freeDataBufIfExists();
+     freeChecksumBufIfExists();
+     int total = 0;
+     while (buf.hasRemaining()) {
+       int nRead = dataIn.read(buf, dataPos);
+       if (nRead <= 0) break;
+       dataPos += nRead;
+       total += nRead;
+     }
+     return (total == 0 && (dataPos == dataIn.size())) ? -1 : total;
+   }
+ 
+   /**
+    * Fill the data buffer.  If necessary, validate the data against the
+    * checksums.
+    * 
+    * We always want the offsets of the data contained in dataBuf to be
+    * aligned to the chunk boundary.  If we are validating checksums, we
+    * accomplish this by seeking backwards in the file until we're on a
+    * chunk boundary.  (This is necessary because we can't checksum a
+    * partial chunk.)  If we are not validating checksums, we simply only
+    * fill the latter part of dataBuf.
+    * 
+    * @param canSkipChecksum  true if we can skip checksumming.
+    * @return                 true if we hit EOF.
+    * @throws IOException
+    */
+   private synchronized boolean fillDataBuf(boolean canSkipChecksum)
+       throws IOException {
+     createDataBufIfNeeded();
+     final int slop = (int)(dataPos % bytesPerChecksum);
+     final long oldDataPos = dataPos;
+     dataBuf.limit(maxReadaheadLength);
+     if (canSkipChecksum) {
+       dataBuf.position(slop);
+       fillBuffer(dataBuf, canSkipChecksum);
+     } else {
+       dataPos -= slop;
+       dataBuf.position(0);
+       fillBuffer(dataBuf, canSkipChecksum);
+     }
+     dataBuf.limit(dataBuf.position());
+     dataBuf.position(Math.min(dataBuf.position(), slop));
+     if (LOG.isTraceEnabled()) {
+       LOG.trace("loaded " + dataBuf.remaining() + " bytes into bounce " +
+           "buffer from offset " + oldDataPos + " of " + block);
+     }
+     return dataBuf.limit() != maxReadaheadLength;
+   }
+ 
+   /**
+    * Read using the bounce buffer.
+    *
+    * A 'direct' read actually has three phases. The first drains any
+    * remaining bytes from the slow read buffer. After this the read is
+    * guaranteed to be on a checksum chunk boundary. If there are still bytes
+    * to read, the fast direct path is used for as many remaining bytes as
+    * possible, up to a multiple of the checksum chunk size. Finally, any
+    * 'odd' bytes remaining at the end of the read cause another slow read to
+    * be issued, which involves an extra copy.
+    *
+    * Every 'slow' read tries to fill the slow read buffer in one go for
+    * efficiency's sake. As described above, all non-checksum-chunk-aligned
+    * reads will be served from the slower read path.
+    *
+    * @param buf              The buffer to read into. 
+    * @param canSkipChecksum  True if we can skip checksums.
+    */
+   private synchronized int readWithBounceBuffer(ByteBuffer buf,
+         boolean canSkipChecksum) throws IOException {
+     int total = 0;
+     int bb = drainDataBuf(buf); // drain bounce buffer if possible
+     if (bb >= 0) {
+       total += bb;
+       if (buf.remaining() == 0) return total;
+     }
+     boolean eof = true, done = false;
+     do {
+       if (buf.isDirect() && (buf.remaining() >= maxReadaheadLength)
+             && ((dataPos % bytesPerChecksum) == 0)) {
+         // Fast lane: try to read directly into user-supplied buffer, bypassing
+         // bounce buffer.
+         int oldLimit = buf.limit();
+         int nRead;
+         try {
+           buf.limit(buf.position() + maxReadaheadLength);
+           nRead = fillBuffer(buf, canSkipChecksum);
+         } finally {
+           buf.limit(oldLimit);
+         }
+         if (nRead < maxReadaheadLength) {
+           done = true;
+         }
+         if (nRead > 0) {
+           eof = false;
+         }
+         total += nRead;
+       } else {
+         // Slow lane: refill bounce buffer.
+         if (fillDataBuf(canSkipChecksum)) {
+           done = true;
+         }
+         bb = drainDataBuf(buf); // drain bounce buffer if possible
+         if (bb >= 0) {
+           eof = false;
+           total += bb;
+         }
+       }
+     } while ((!done) && (buf.remaining() > 0));
+     return (eof && total == 0) ? -1 : total;
+   }
+ 
+   @Override
+   public synchronized int read(byte[] arr, int off, int len)
+         throws IOException {
+     boolean canSkipChecksum = createNoChecksumContext();
+     int nRead;
+     try {
+       String traceString = null;
+       if (LOG.isTraceEnabled()) {
+         traceString = new StringBuilder().
+             append("read(arr.length=").append(arr.length).
+             append(", off=").append(off).
+             append(", len=").append(len).
+             append(", filename=").append(filename).
+             append(", block=").append(block).
+             append(", canSkipChecksum=").append(canSkipChecksum).
+             append(")").toString();
+         LOG.trace(traceString + ": starting");
+       }
+       try {
+         if (canSkipChecksum && zeroReadaheadRequested) {
+           nRead = readWithoutBounceBuffer(arr, off, len);
+         } else {
+           nRead = readWithBounceBuffer(arr, off, len, canSkipChecksum);
+         }
+       } catch (IOException e) {
+         if (LOG.isTraceEnabled()) {
+           LOG.trace(traceString + ": I/O error", e);
+         }
+         throw e;
+       }
+       if (LOG.isTraceEnabled()) {
+         LOG.trace(traceString + ": returning " + nRead);
+       }
+     } finally {
+       if (canSkipChecksum) releaseNoChecksumContext();
+     }
+     return nRead;
+   }
+ 
+   private synchronized int readWithoutBounceBuffer(byte arr[], int off,
+         int len) throws IOException {
+     freeDataBufIfExists();
+     freeChecksumBufIfExists();
+     int nRead = dataIn.read(ByteBuffer.wrap(arr, off, len), dataPos);
+     if (nRead > 0) {
+       dataPos += nRead;
+     } else if ((nRead == 0) && (dataPos == dataIn.size())) {
+       return -1;
+     }
+     return nRead;
+   }
+ 
+   private synchronized int readWithBounceBuffer(byte arr[], int off, int len,
+         boolean canSkipChecksum) throws IOException {
+     createDataBufIfNeeded();
+     if (!dataBuf.hasRemaining()) {
+       dataBuf.position(0);
+       dataBuf.limit(maxReadaheadLength);
+       fillDataBuf(canSkipChecksum);
+     }
+     if (dataBuf.remaining() == 0) return -1;
+     int toRead = Math.min(dataBuf.remaining(), len);
+     dataBuf.get(arr, off, toRead);
+     return toRead;
+   }
+ 
+   @Override
+   public synchronized long skip(long n) throws IOException {
+     int discardedFromBuf = 0;
+     long remaining = n;
+     if ((dataBuf != null) && dataBuf.hasRemaining()) {
+       discardedFromBuf = (int)Math.min(dataBuf.remaining(), n);
+       dataBuf.position(dataBuf.position() + discardedFromBuf);
+       remaining -= discardedFromBuf;
+     }
+     if (LOG.isTraceEnabled()) {
+       LOG.trace("skip(n=" + n + ", block=" + block + ", filename=" + 
+         filename + "): discarded " + discardedFromBuf + " bytes from " +
+         "dataBuf and advanced dataPos by " + remaining);
+     }
+     dataPos += remaining;
+     return n;
+   }
+ 
+   @Override
+   public int available() throws IOException {
+     // We never do network I/O in BlockReaderLocal.
+     return Integer.MAX_VALUE;
+   }
+ 
+   @Override
+   public synchronized void close() throws IOException {
+     if (closed) return;
+     closed = true;
+     if (LOG.isTraceEnabled()) {
+       LOG.trace("close(filename=" + filename + ", block=" + block + ")");
+     }
+     replica.unref();
+     freeDataBufIfExists();
+     freeChecksumBufIfExists();
+   }
+ 
+   @Override
+   public synchronized void readFully(byte[] arr, int off, int len)
+       throws IOException {
+     BlockReaderUtil.readFully(this, arr, off, len);
+   }
+ 
+   @Override
+   public synchronized int readAll(byte[] buf, int off, int len)
+       throws IOException {
+     return BlockReaderUtil.readAll(this, buf, off, len);
+   }
+ 
+   @Override
+   public boolean isLocal() {
+     return true;
+   }
+ 
+   @Override
+   public boolean isShortCircuit() {
+     return true;
+   }
+ 
+   /**
+    * Get or create a memory map for this replica.
+    * 
+    * There are two kinds of ClientMmap objects we could fetch here: one that 
+    * will always read pre-checksummed data, and one that may read data that
+    * hasn't been checksummed.
+    *
+    * If we fetch the former, "safe" kind of ClientMmap, we have to increment
+    * the anchor count on the shared memory slot.  This will tell the DataNode
+    * not to munlock the block until this ClientMmap is closed.
+    * If we fetch the latter, we don't bother with anchoring.
+    *
+    * @param opts     The options to use, such as SKIP_CHECKSUMS.
+    * 
+    * @return         null on failure; the ClientMmap otherwise.
+    */
+   @Override
+   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
+     boolean anchor = verifyChecksum &&
+         (opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
+     if (anchor) {
+       if (!createNoChecksumContext()) {
+         if (LOG.isTraceEnabled()) {
+           LOG.trace("can't get an mmap for " + block + " of " + filename + 
+               " since SKIP_CHECKSUMS was not given, " +
+               "we aren't skipping checksums, and the block is not mlocked.");
+         }
+         return null;
+       }
+     }
+     ClientMmap clientMmap = null;
+     try {
+       clientMmap = replica.getOrCreateClientMmap(anchor);
+     } finally {
+       if ((clientMmap == null) && anchor) {
+         releaseNoChecksumContext();
+       }
+     }
+     return clientMmap;
+   }
+   
+   @VisibleForTesting
+   boolean getVerifyChecksum() {
+     return this.verifyChecksum;
+   }
+ 
+   @VisibleForTesting
+   int getMaxReadaheadLength() {
+     return this.maxReadaheadLength;
+   }
+   
+   /**
+    * Make the replica anchorable.  Normally this can only be done by the
+    * DataNode.  This method is only for testing.
+    */
+   @VisibleForTesting
+   void forceAnchorable() {
+     replica.getSlot().makeAnchorable();
+   }
+ 
+   /**
+    * Make the replica unanchorable.  Normally this can only be done by the
+    * DataNode.  This method is only for testing.
+    */
+   @VisibleForTesting
+   void forceUnanchorable() {
+     replica.getSlot().makeUnanchorable();
+   }
++
++  @Override
++  public DataChecksum getDataChecksum() {
++    return checksum;
++  }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
index 0000000,eea3f06..9920438
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
@@@ -1,0 -1,738 +1,743 @@@
+ /**
+  * 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.hdfs;
+ 
+ import java.io.DataInputStream;
+ import java.io.File;
+ import java.io.FileInputStream;
+ import java.io.IOException;
+ import java.nio.ByteBuffer;
+ import java.security.PrivilegedExceptionAction;
+ import java.util.Collections;
+ import java.util.EnumSet;
+ import java.util.HashMap;
+ import java.util.LinkedHashMap;
+ import java.util.Map;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.fs.ReadOption;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+ import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+ import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
+ import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
+ import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+ import org.apache.hadoop.hdfs.util.IOUtilsClient;
+ import org.apache.hadoop.io.IOUtils;
+ import org.apache.hadoop.ipc.RPC;
+ import org.apache.hadoop.security.UserGroupInformation;
+ import org.apache.hadoop.security.token.Token;
+ import org.apache.hadoop.util.DataChecksum;
+ import org.apache.hadoop.util.DirectBufferPool;
+ import org.apache.htrace.Sampler;
+ import org.apache.htrace.Trace;
+ import org.apache.htrace.TraceScope;
+ 
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ /**
+  * BlockReaderLocalLegacy enables local short circuited reads. If the DFS client is on
+  * the same machine as the datanode, then the client can read files directly
+  * from the local file system rather than going through the datanode for better
+  * performance. <br>
+  *
+  * This is the legacy implementation based on HDFS-2246, which requires
+  * permissions on the datanode to be set so that clients can directly access the
+  * blocks. The new implementation based on HDFS-347 should be preferred on UNIX
+  * systems where the required native code has been implemented.<br>
+  *
+  * {@link BlockReaderLocalLegacy} works as follows:
+  * <ul>
+  * <li>The client performing short circuit reads must be configured at the
+  * datanode.</li>
+  * <li>The client gets the path to the file where block is stored using
+  * {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
+  * RPC call</li>
+  * <li>Client uses kerberos authentication to connect to the datanode over RPC,
+  * if security is enabled.</li>
+  * </ul>
+  */
+ @InterfaceAudience.Private
+ class BlockReaderLocalLegacy implements BlockReader {
+   private static final Logger LOG = LoggerFactory.getLogger(
+       BlockReaderLocalLegacy.class);
+ 
+   //Stores the cache and proxy for a local datanode.
+   private static class LocalDatanodeInfo {
+     private ClientDatanodeProtocol proxy = null;
+     private final Map<ExtendedBlock, BlockLocalPathInfo> cache;
+ 
+     LocalDatanodeInfo() {
+       final int cacheSize = 10000;
+       final float hashTableLoadFactor = 0.75f;
+       int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
+       cache = Collections
+           .synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
+               hashTableCapacity, hashTableLoadFactor, true) {
+             private static final long serialVersionUID = 1;
+ 
+             @Override
+             protected boolean removeEldestEntry(
+                 Map.Entry<ExtendedBlock, BlockLocalPathInfo> eldest) {
+               return size() > cacheSize;
+             }
+           });
+     }
+ 
+     private synchronized ClientDatanodeProtocol getDatanodeProxy(
+         UserGroupInformation ugi, final DatanodeInfo node,
+         final Configuration conf, final int socketTimeout,
+         final boolean connectToDnViaHostname) throws IOException {
+       if (proxy == null) {
+         try {
+           proxy = ugi.doAs(new PrivilegedExceptionAction<ClientDatanodeProtocol>() {
+             @Override
+             public ClientDatanodeProtocol run() throws Exception {
+               return DFSUtilClient.createClientDatanodeProtocolProxy(node, conf,
+                   socketTimeout, connectToDnViaHostname);
+             }
+           });
+         } catch (InterruptedException e) {
+           LOG.warn("encountered exception ", e);
+         }
+       }
+       return proxy;
+     }
+     
+     private synchronized void resetDatanodeProxy() {
+       if (null != proxy) {
+         RPC.stopProxy(proxy);
+         proxy = null;
+       }
+     }
+ 
+     private BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b) {
+       return cache.get(b);
+     }
+ 
+     private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
+       cache.put(b, info);
+     }
+ 
+     private void removeBlockLocalPathInfo(ExtendedBlock b) {
+       cache.remove(b);
+     }
+   }
+   
+   // Multiple datanodes could be running on the local machine. Store proxies in
+   // a map keyed by the ipc port of the datanode.
+   private static final Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
+ 
+   private final FileInputStream dataIn; // reader for the data file
+   private final FileInputStream checksumIn;   // reader for the checksum file
+ 
+   /**
+    * Offset from the most recent chunk boundary at which the next read should
+    * take place. Is only set to non-zero at construction time, and is
+    * decremented (usually to 0) by subsequent reads. This avoids having to do a
+    * checksum read at construction to position the read cursor correctly.
+    */
+   private int offsetFromChunkBoundary;
+   
+   private byte[] skipBuf = null;
+ 
+   /**
+    * Used for checksummed reads that need to be staged before copying to their
+    * output buffer because they are either a) smaller than the checksum chunk
+    * size or b) issued by the slower read(byte[]...) path
+    */
+   private ByteBuffer slowReadBuff = null;
+   private ByteBuffer checksumBuff = null;
+   private DataChecksum checksum;
+   private final boolean verifyChecksum;
+ 
+   private static final DirectBufferPool bufferPool = new DirectBufferPool();
+ 
+   private final int bytesPerChecksum;
+   private final int checksumSize;
+ 
+   /** offset in block where reader wants to actually read */
+   private long startOffset;
+   private final String filename;
+   private long blockId;
+   
+   /**
+    * The only way this object can be instantiated.
+    */
+   static BlockReaderLocalLegacy newBlockReader(DfsClientConf conf,
+       UserGroupInformation userGroupInformation,
+       Configuration configuration, String file, ExtendedBlock blk,
+       Token<BlockTokenIdentifier> token, DatanodeInfo node, 
+       long startOffset, long length, StorageType storageType)
+       throws IOException {
+     final ShortCircuitConf scConf = conf.getShortCircuitConf();
+     LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
+         .getIpcPort());
+     // check the cache first
+     BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
+     if (pathinfo == null) {
+       if (userGroupInformation == null) {
+         userGroupInformation = UserGroupInformation.getCurrentUser();
+       }
+       pathinfo = getBlockPathInfo(userGroupInformation, blk, node,
+           configuration, conf.getSocketTimeout(), token,
+           conf.isConnectToDnViaHostname(), storageType);
+     }
+ 
+     // check to see if the file exists. It may so happen that the
+     // HDFS file has been deleted and this block-lookup is occurring
+     // on behalf of a new HDFS file. This time, the block file could
+     // be residing in a different portion of the fs.data.dir directory.
+     // In this case, we remove this entry from the cache. The next
+     // call to this method will re-populate the cache.
+     FileInputStream dataIn = null;
+     FileInputStream checksumIn = null;
+     BlockReaderLocalLegacy localBlockReader = null;
+     final boolean skipChecksumCheck = scConf.isSkipShortCircuitChecksums()
+         || storageType.isTransient();
+     try {
+       // get a local file system
+       File blkfile = new File(pathinfo.getBlockPath());
+       dataIn = new FileInputStream(blkfile);
+ 
+       if (LOG.isDebugEnabled()) {
+         LOG.debug("New BlockReaderLocalLegacy for file " + blkfile + " of size "
+             + blkfile.length() + " startOffset " + startOffset + " length "
+             + length + " short circuit checksum " + !skipChecksumCheck);
+       }
+ 
+       if (!skipChecksumCheck) {
+         // get the metadata file
+         File metafile = new File(pathinfo.getMetaPath());
+         checksumIn = new FileInputStream(metafile);
+ 
+         final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(
+             new DataInputStream(checksumIn), blk);
+         long firstChunkOffset = startOffset
+             - (startOffset % checksum.getBytesPerChecksum());
+         localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
+             startOffset, length, pathinfo, checksum, true, dataIn,
+             firstChunkOffset, checksumIn);
+       } else {
+         localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
+             startOffset, length, pathinfo, dataIn);
+       }
+     } catch (IOException e) {
+       // remove from cache
+       localDatanodeInfo.removeBlockLocalPathInfo(blk);
+       LOG.warn("BlockReaderLocalLegacy: Removing " + blk
+           + " from cache because local file " + pathinfo.getBlockPath()
+           + " could not be opened.");
+       throw e;
+     } finally {
+       if (localBlockReader == null) {
+         if (dataIn != null) {
+           dataIn.close();
+         }
+         if (checksumIn != null) {
+           checksumIn.close();
+         }
+       }
+     }
+     return localBlockReader;
+   }
+   
+   private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
+     LocalDatanodeInfo ldInfo = localDatanodeInfoMap.get(port);
+     if (ldInfo == null) {
+       ldInfo = new LocalDatanodeInfo();
+       localDatanodeInfoMap.put(port, ldInfo);
+     }
+     return ldInfo;
+   }
+   
+   private static BlockLocalPathInfo getBlockPathInfo(UserGroupInformation ugi,
+       ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
+       Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname,
+       StorageType storageType) throws IOException {
+     LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
+     BlockLocalPathInfo pathinfo = null;
+     ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
+         conf, timeout, connectToDnViaHostname);
+     try {
+       // make RPC to local datanode to find local pathnames of blocks
+       pathinfo = proxy.getBlockLocalPathInfo(blk, token);
+       // We cannot cache the path information for a replica on transient storage.
+       // If the replica gets evicted, then it moves to a different path.  Then,
+       // our next attempt to read from the cached path would fail to find the
+       // file.  Additionally, the failure would cause us to disable legacy
+       // short-circuit read for all subsequent use in the ClientContext.  Unlike
+       // the newer short-circuit read implementation, we have no communication
+       // channel for the DataNode to notify the client that the path has been
+       // invalidated.  Therefore, our only option is to skip caching.
+       if (pathinfo != null && !storageType.isTransient()) {
+         if (LOG.isDebugEnabled()) {
+           LOG.debug("Cached location of block " + blk + " as " + pathinfo);
+         }
+         localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
+       }
+     } catch (IOException e) {
+       localDatanodeInfo.resetDatanodeProxy(); // Reset proxy on error
+       throw e;
+     }
+     return pathinfo;
+   }
+   
+   private static int getSlowReadBufferNumChunks(int bufferSizeBytes,
+       int bytesPerChecksum) {
+     if (bufferSizeBytes < bytesPerChecksum) {
+       throw new IllegalArgumentException("Configured BlockReaderLocalLegacy " +
+           "buffer size (" + bufferSizeBytes + ") is not large enough to hold " +
+           "a single chunk (" + bytesPerChecksum +  "). Please configure " +
+           HdfsClientConfigKeys.Read.ShortCircuit.BUFFER_SIZE_KEY +
+           " appropriately");
+     }
+ 
+     // Round down to nearest chunk size
+     return bufferSizeBytes / bytesPerChecksum;
+   }
+ 
+   private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
+       ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
+       long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn)
+       throws IOException {
+     this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
+         DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
+         dataIn, startOffset, null);
+   }
+ 
+   private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
+       ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
+       long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
+       boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
+       FileInputStream checksumIn) throws IOException {
+     this.filename = hdfsfile;
+     this.checksum = checksum;
+     this.verifyChecksum = verifyChecksum;
+     this.startOffset = Math.max(startOffset, 0);
+     this.blockId = block.getBlockId();
+ 
+     bytesPerChecksum = this.checksum.getBytesPerChecksum();
+     checksumSize = this.checksum.getChecksumSize();
+ 
+     this.dataIn = dataIn;
+     this.checksumIn = checksumIn;
+     this.offsetFromChunkBoundary = (int) (startOffset-firstChunkOffset);
+ 
+     final int chunksPerChecksumRead = getSlowReadBufferNumChunks(
+         conf.getShortCircuitBufferSize(), bytesPerChecksum);
+     slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
+     checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
+     // Initially the buffers have nothing to read.
+     slowReadBuff.flip();
+     checksumBuff.flip();
+     boolean success = false;
+     try {
+       // Skip both input streams to beginning of the chunk containing startOffset
+       IOUtils.skipFully(dataIn, firstChunkOffset);
+       if (checksumIn != null) {
+         long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
+         IOUtils.skipFully(checksumIn, checkSumOffset);
+       }
+       success = true;
+     } finally {
+       if (!success) {
+         bufferPool.returnBuffer(slowReadBuff);
+         bufferPool.returnBuffer(checksumBuff);
+       }
+     }
+   }
+ 
+   /**
+    * Reads bytes into a buffer until EOF or the buffer's limit is reached
+    */
+   private int fillBuffer(FileInputStream stream, ByteBuffer buf)
+       throws IOException {
+     TraceScope scope = Trace.startSpan("BlockReaderLocalLegacy#fillBuffer(" +
+         blockId + ")", Sampler.NEVER);
+     try {
+       int bytesRead = stream.getChannel().read(buf);
+       if (bytesRead < 0) {
+         //EOF
+         return bytesRead;
+       }
+       while (buf.remaining() > 0) {
+         int n = stream.getChannel().read(buf);
+         if (n < 0) {
+           //EOF
+           return bytesRead;
+         }
+         bytesRead += n;
+       }
+       return bytesRead;
+     } finally {
+       scope.close();
+     }
+   }
+   
+   /**
+    * Utility method used by read(ByteBuffer) to partially copy a ByteBuffer into
+    * another.
+    */
+   private void writeSlice(ByteBuffer from, ByteBuffer to, int length) {
+     int oldLimit = from.limit();
+     from.limit(from.position() + length);
+     try {
+       to.put(from);
+     } finally {
+       from.limit(oldLimit);
+     }
+   }
+ 
+   @Override
+   public synchronized int read(ByteBuffer buf) throws IOException {
+     int nRead = 0;
+     if (verifyChecksum) {
+       // A 'direct' read actually has three phases. The first drains any
+       // remaining bytes from the slow read buffer. After this the read is
+       // guaranteed to be on a checksum chunk boundary. If there are still bytes
+       // to read, the fast direct path is used for as many remaining bytes as
+       // possible, up to a multiple of the checksum chunk size. Finally, any
+       // 'odd' bytes remaining at the end of the read cause another slow read to
+       // be issued, which involves an extra copy.
+ 
+       // Every 'slow' read tries to fill the slow read buffer in one go for
+       // efficiency's sake. As described above, all non-checksum-chunk-aligned
+       // reads will be served from the slower read path.
+ 
+       if (slowReadBuff.hasRemaining()) {
+         // There are remaining bytes from a small read available. This usually
+         // means this read is unaligned, which falls back to the slow path.
+         int fromSlowReadBuff = Math.min(buf.remaining(), slowReadBuff.remaining());
+         writeSlice(slowReadBuff, buf, fromSlowReadBuff);
+         nRead += fromSlowReadBuff;
+       }
+ 
+       if (buf.remaining() >= bytesPerChecksum && offsetFromChunkBoundary == 0) {
+         // Since we have drained the 'small read' buffer, we are guaranteed to
+         // be chunk-aligned
+         int len = buf.remaining() - (buf.remaining() % bytesPerChecksum);
+ 
+         // There's only enough checksum buffer space available to checksum one
+         // entire slow read buffer. This saves keeping the number of checksum
+         // chunks around.
+         len = Math.min(len, slowReadBuff.capacity());
+         int oldlimit = buf.limit();
+         buf.limit(buf.position() + len);
+         int readResult = 0;
+         try {
+           readResult = doByteBufferRead(buf);
+         } finally {
+           buf.limit(oldlimit);
+         }
+         if (readResult == -1) {
+           return nRead;
+         } else {
+           nRead += readResult;
+           buf.position(buf.position() + readResult);
+         }
+       }
+ 
+       // offsetFromChunkBoundary > 0 => unaligned read, use slow path to read
+       // until chunk boundary
+       if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) || offsetFromChunkBoundary > 0) {
+         int toRead = Math.min(buf.remaining(), bytesPerChecksum - offsetFromChunkBoundary);
+         int readResult = fillSlowReadBuffer(toRead);
+         if (readResult == -1) {
+           return nRead;
+         } else {
+           int fromSlowReadBuff = Math.min(readResult, buf.remaining());
+           writeSlice(slowReadBuff, buf, fromSlowReadBuff);
+           nRead += fromSlowReadBuff;
+         }
+       }
+     } else {
+       // Non-checksummed reads are much easier; we can just fill the buffer directly.
+       nRead = doByteBufferRead(buf);
+       if (nRead > 0) {
+         buf.position(buf.position() + nRead);
+       }
+     }
+     return nRead;
+   }
+ 
+   /**
+    * Tries to read as many bytes as possible into supplied buffer, checksumming
+    * each chunk if needed.
+    *
+    * <b>Preconditions:</b>
+    * <ul>
+    * <li>
+    * If checksumming is enabled, buf.remaining must be a multiple of
+    * bytesPerChecksum. Note that this is not a requirement for clients of
+    * read(ByteBuffer) - in the case of non-checksum-sized read requests,
+    * read(ByteBuffer) will substitute a suitably sized buffer to pass to this
+    * method.
+    * </li>
+    * </ul>
+    * <b>Postconditions:</b>
+    * <ul>
+    * <li>buf.limit and buf.mark are unchanged.</li>
+    * <li>buf.position += min(offsetFromChunkBoundary, totalBytesRead) - so the
+    * requested bytes can be read straight from the buffer</li>
+    * </ul>
+    *
+    * @param buf
+    *          byte buffer to write bytes to. If checksums are not required, buf
+    *          can have any number of bytes remaining, otherwise there must be a
+    *          multiple of the checksum chunk size remaining.
+    * @return <tt>max(min(totalBytesRead, len) - offsetFromChunkBoundary, 0)</tt>
+    *         that is, the the number of useful bytes (up to the amount
+    *         requested) readable from the buffer by the client.
+    */
+   private synchronized int doByteBufferRead(ByteBuffer buf) throws IOException {
+     if (verifyChecksum) {
+       assert buf.remaining() % bytesPerChecksum == 0;
+     }
+     int dataRead = -1;
+ 
+     int oldpos = buf.position();
+     // Read as much as we can into the buffer.
+     dataRead = fillBuffer(dataIn, buf);
+ 
+     if (dataRead == -1) {
+       return -1;
+     }
+ 
+     if (verifyChecksum) {
+       ByteBuffer toChecksum = buf.duplicate();
+       toChecksum.position(oldpos);
+       toChecksum.limit(oldpos + dataRead);
+ 
+       checksumBuff.clear();
+       // Equivalent to (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
+       int numChunks =
+         (toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
+       checksumBuff.limit(checksumSize * numChunks);
+ 
+       fillBuffer(checksumIn, checksumBuff);
+       checksumBuff.flip();
+ 
+       checksum.verifyChunkedSums(toChecksum, checksumBuff, filename,
+           this.startOffset);
+     }
+ 
+     if (dataRead >= 0) {
+       buf.position(oldpos + Math.min(offsetFromChunkBoundary, dataRead));
+     }
+ 
+     if (dataRead < offsetFromChunkBoundary) {
+       // yikes, didn't even get enough bytes to honour offset. This can happen
+       // even if we are verifying checksums if we are at EOF.
+       offsetFromChunkBoundary -= dataRead;
+       dataRead = 0;
+     } else {
+       dataRead -= offsetFromChunkBoundary;
+       offsetFromChunkBoundary = 0;
+     }
+ 
+     return dataRead;
+   }
+ 
+   /**
+    * Ensures that up to len bytes are available and checksummed in the slow read
+    * buffer. The number of bytes available to read is returned. If the buffer is
+    * not already empty, the number of remaining bytes is returned and no actual
+    * read happens.
+    *
+    * @param len
+    *          the maximum number of bytes to make available. After len bytes
+    *          are read, the underlying bytestream <b>must</b> be at a checksum
+    *          boundary, or EOF. That is, (len + currentPosition) %
+    *          bytesPerChecksum == 0.
+    * @return the number of bytes available to read, or -1 if EOF.
+    */
+   private synchronized int fillSlowReadBuffer(int len) throws IOException {
+     int nRead = -1;
+     if (slowReadBuff.hasRemaining()) {
+       // Already got data, good to go.
+       nRead = Math.min(len, slowReadBuff.remaining());
+     } else {
+       // Round a complete read of len bytes (plus any implicit offset) to the
+       // next chunk boundary, since we try and read in multiples of a chunk
+       int nextChunk = len + offsetFromChunkBoundary +
+           (bytesPerChecksum - ((len + offsetFromChunkBoundary) % bytesPerChecksum));
+       int limit = Math.min(nextChunk, slowReadBuff.capacity());
+       assert limit % bytesPerChecksum == 0;
+ 
+       slowReadBuff.clear();
+       slowReadBuff.limit(limit);
+ 
+       nRead = doByteBufferRead(slowReadBuff);
+ 
+       if (nRead > 0) {
+         // So that next time we call slowReadBuff.hasRemaining(), we don't get a
+         // false positive.
+         slowReadBuff.limit(nRead + slowReadBuff.position());
+       }
+     }
+     return nRead;
+   }
+ 
+   @Override
+   public synchronized int read(byte[] buf, int off, int len) throws IOException {
+     if (LOG.isTraceEnabled()) {
+       LOG.trace("read off " + off + " len " + len);
+     }
+     if (!verifyChecksum) {
+       return dataIn.read(buf, off, len);
+     }
+ 
+     int nRead = fillSlowReadBuffer(slowReadBuff.capacity());
+ 
+     if (nRead > 0) {
+       // Possible that buffer is filled with a larger read than we need, since
+       // we tried to read as much as possible at once
+       nRead = Math.min(len, nRead);
+       slowReadBuff.get(buf, off, nRead);
+     }
+ 
+     return nRead;
+   }
+ 
+   @Override
+   public synchronized long skip(long n) throws IOException {
+     if (LOG.isDebugEnabled()) {
+       LOG.debug("skip " + n);
+     }
+     if (n <= 0) {
+       return 0;
+     }
+     if (!verifyChecksum) {
+       return dataIn.skip(n);
+     }
+   
+     // caller made sure newPosition is not beyond EOF.
+     int remaining = slowReadBuff.remaining();
+     int position = slowReadBuff.position();
+     int newPosition = position + (int)n;
+   
+     // if the new offset is already read into dataBuff, just reposition
+     if (n <= remaining) {
+       assert offsetFromChunkBoundary == 0;
+       slowReadBuff.position(newPosition);
+       return n;
+     }
+   
+     // for small gap, read through to keep the data/checksum in sync
+     if (n - remaining <= bytesPerChecksum) {
+       slowReadBuff.position(position + remaining);
+       if (skipBuf == null) {
+         skipBuf = new byte[bytesPerChecksum];
+       }
+       int ret = read(skipBuf, 0, (int)(n - remaining));
+       return (remaining + ret);
+     }
+   
+     // optimize for big gap: discard the current buffer, skip to
+     // the beginning of the appropriate checksum chunk and then
+     // read to the middle of that chunk to be in sync with checksums.
+   
+     // We can't use this.offsetFromChunkBoundary because we need to know how
+     // many bytes of the offset were really read. Calling read(..) with a
+     // positive this.offsetFromChunkBoundary causes that many bytes to get
+     // silently skipped.
+     int myOffsetFromChunkBoundary = newPosition % bytesPerChecksum;
+     long toskip = n - remaining - myOffsetFromChunkBoundary;
+ 
+     slowReadBuff.position(slowReadBuff.limit());
+     checksumBuff.position(checksumBuff.limit());
+   
+     IOUtils.skipFully(dataIn, toskip);
+     long checkSumOffset = (toskip / bytesPerChecksum) * checksumSize;
+     IOUtils.skipFully(checksumIn, checkSumOffset);
+ 
+     // read into the middle of the chunk
+     if (skipBuf == null) {
+       skipBuf = new byte[bytesPerChecksum];
+     }
+     assert skipBuf.length == bytesPerChecksum;
+     assert myOffsetFromChunkBoundary < bytesPerChecksum;
+ 
+     int ret = read(skipBuf, 0, myOffsetFromChunkBoundary);
+ 
+     if (ret == -1) {  // EOS
+       return (toskip + remaining);
+     } else {
+       return (toskip + remaining + ret);
+     }
+   }
+ 
+   @Override
+   public synchronized void close() throws IOException {
+     IOUtilsClient.cleanup(LOG, dataIn, checksumIn);
+     if (slowReadBuff != null) {
+       bufferPool.returnBuffer(slowReadBuff);
+       slowReadBuff = null;
+     }
+     if (checksumBuff != null) {
+       bufferPool.returnBuffer(checksumBuff);
+       checksumBuff = null;
+     }
+     startOffset = -1;
+     checksum = null;
+   }
+ 
+   @Override
+   public int readAll(byte[] buf, int offset, int len) throws IOException {
+     return BlockReaderUtil.readAll(this, buf, offset, len);
+   }
+ 
+   @Override
+   public void readFully(byte[] buf, int off, int len) throws IOException {
+     BlockReaderUtil.readFully(this, buf, off, len);
+   }
+ 
+   @Override
+   public int available() throws IOException {
+     // We never do network I/O in BlockReaderLocalLegacy.
+     return Integer.MAX_VALUE;
+   }
+ 
+   @Override
+   public boolean isLocal() {
+     return true;
+   }
+   
+   @Override
+   public boolean isShortCircuit() {
+     return true;
+   }
+ 
+   @Override
+   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
+     return null;
+   }
++
++  @Override
++  public DataChecksum getDataChecksum() {
++    return checksum;
++  }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
index 0000000,e135d8e..f908dd3
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
@@@ -1,0 -1,120 +1,126 @@@
+ /**
+  * 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.hdfs;
+ 
+ import java.io.IOException;
+ import java.nio.ByteBuffer;
+ import java.util.EnumSet;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.fs.ReadOption;
+ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
++import org.apache.hadoop.util.DataChecksum;
+ 
+ /**
+  * An ExternalBlockReader uses pluggable ReplicaAccessor objects to read from
+  * replicas.
+  */
+ @InterfaceAudience.Private
+ public final class ExternalBlockReader implements BlockReader {
+   private final ReplicaAccessor accessor;
+   private final long visibleLength;
+   private long pos;
+ 
+   ExternalBlockReader(ReplicaAccessor accessor, long visibleLength,
+                       long startOffset) {
+     this.accessor = accessor;
+     this.visibleLength = visibleLength;
+     this.pos = startOffset;
+   }
+ 
+   @Override
+   public int read(byte[] buf, int off, int len) throws IOException {
+     int nread = accessor.read(pos, buf, off, len);
+     pos += nread;
+     return nread;
+   }
+ 
+   @Override
+   public int read(ByteBuffer buf) throws IOException {
+     int nread = accessor.read(pos, buf);
+     pos += nread;
+     return nread;
+   }
+ 
+   @Override
+   public long skip(long n) throws IOException {
+     // You cannot skip backwards
+     if (n <= 0) {
+       return 0;
+     }
+     // You can't skip past the end of the replica.
+     long oldPos = pos;
+     pos += n;
+     if (pos > visibleLength) {
+       pos = visibleLength;
+     }
+     return pos - oldPos;
+   }
+ 
+   @Override
+   public int available() throws IOException {
+     // We return the amount of bytes that we haven't read yet from the
+     // replica, based on our current position.  Some of the other block
+     // readers return a shorter length than that.  The only advantage to
+     // returning a shorter length is that the DFSInputStream will
+     // trash your block reader and create a new one if someone tries to
+     // seek() beyond the available() region.
+     long diff = visibleLength - pos;
+     if (diff > Integer.MAX_VALUE) {
+       return Integer.MAX_VALUE;
+     } else {
+       return (int)diff;
+     }
+   }
+ 
+   @Override
+   public void close() throws IOException {
+     accessor.close();
+   }
+ 
+   @Override
+   public void readFully(byte[] buf, int offset, int len) throws IOException {
+     BlockReaderUtil.readFully(this, buf, offset, len);
+   }
+ 
+   @Override
+   public int readAll(byte[] buf, int offset, int len) throws IOException {
+     return BlockReaderUtil.readAll(this, buf, offset, len);
+   }
+ 
+   @Override
+   public boolean isLocal() {
+     return accessor.isLocal();
+   }
+ 
+   @Override
+   public boolean isShortCircuit() {
+     return accessor.isShortCircuit();
+   }
+ 
+   @Override
+   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
+     // For now, pluggable ReplicaAccessors do not support zero-copy.
+     return null;
+   }
++
++  @Override
++  public DataChecksum getDataChecksum() {
++    return null;
++  }
+ }


[28/50] [abbrv] hadoop git commit: HDFS-8550. Erasure Coding: Fix FindBugs Multithreaded correctness Warning. Contributed by Rakesh R.

Posted by wa...@apache.org.
HDFS-8550. Erasure Coding: Fix FindBugs Multithreaded correctness Warning. Contributed by Rakesh R.

Change-Id: Ic248999a7f8e5e740d49c9b10abcf16f66dd0f98


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

Branch: refs/heads/trunk
Commit: c457095206e5093c577b0124ad6fc7eef6f77b0a
Parents: 82a88b9
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Fri Sep 18 11:04:06 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Fri Sep 18 11:04:06 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt              |  3 +++
 .../org/apache/hadoop/hdfs/DFSStripedInputStream.java | 14 +++++++-------
 .../hdfs/server/blockmanagement/BlockManager.java     |  1 +
 .../org/apache/hadoop/hdfs/util/StripedBlockUtil.java |  2 +-
 4 files changed, 12 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4570952/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 8ff696b..468cc56 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -435,3 +435,6 @@
 
     HDFS-9086. Rename dfs.datanode.stripedread.threshold.millis to 
     dfs.datanode.stripedread.timeout.millis. (wang via zhz)
+
+    HDFS-8550. Erasure Coding: Fix FindBugs Multithreaded correctness Warning.
+    (Rakesh R via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4570952/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
index 2ad63b8..b7c22c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@ -112,7 +112,6 @@ public class DFSStripedInputStream extends DFSInputStream {
      * offsets for all the block readers so that we can skip data if necessary.
      */
     long blockReaderOffset;
-    LocatedBlock targetBlock;
     /**
      * We use this field to indicate whether we should use this reader. In case
      * we hit any issue with this reader, we set this field to true and avoid
@@ -120,10 +119,8 @@ public class DFSStripedInputStream extends DFSInputStream {
      */
     boolean shouldSkip = false;
 
-    BlockReaderInfo(BlockReader reader, LocatedBlock targetBlock,
-        DatanodeInfo dn, long offset) {
+    BlockReaderInfo(BlockReader reader, DatanodeInfo dn, long offset) {
       this.reader = reader;
-      this.targetBlock = targetBlock;
       this.datanode = dn;
       this.blockReaderOffset = offset;
     }
@@ -649,8 +646,8 @@ public class DFSStripedInputStream extends DFSInputStream {
           }
         }
         if (reader != null) {
-          readerInfos[chunkIndex] = new BlockReaderInfo(reader, block,
-              dnInfo.info, alignedStripe.getOffsetInBlock());
+          readerInfos[chunkIndex] = new BlockReaderInfo(reader, dnInfo.info,
+              alignedStripe.getOffsetInBlock());
           return true;
         }
       }
@@ -826,7 +823,10 @@ public class DFSStripedInputStream extends DFSInputStream {
     void prepareDecodeInputs() {
       if (decodeInputs == null) {
         decodeInputs = new ByteBuffer[dataBlkNum + parityBlkNum];
-        ByteBuffer cur = curStripeBuf.duplicate();
+        final ByteBuffer cur;
+        synchronized (DFSStripedInputStream.this) {
+          cur = curStripeBuf.duplicate();
+        }
         StripedBlockUtil.VerticalRange range = alignedStripe.range;
         for (int i = 0; i < dataBlkNum; i++) {
           cur.limit(cur.capacity());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4570952/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 3c1c461..1211169 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -971,6 +971,7 @@ public class BlockManager implements BlockStatsMXBean {
     if (isBlockTokenEnabled()) {
       // Use cached UGI if serving RPC calls.
       if (b.isStriped()) {
+        Preconditions.checkState(b instanceof LocatedStripedBlock);
         LocatedStripedBlock sb = (LocatedStripedBlock) b;
         int[] indices = sb.getBlockIndices();
         Token<BlockTokenIdentifier>[] blockTokens = new Token[indices.length];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4570952/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
index 897b092..5af3585 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@ -372,7 +372,7 @@ public class StripedBlockUtil {
 
     // Step 4: calculate each chunk's position in destination buffer. Since the
     // whole read range is within a single stripe, the logic is simpler here.
-    int bufOffset = (int) (rangeStartInBlockGroup % (cellSize * dataBlkNum));
+    int bufOffset = (int) (rangeStartInBlockGroup % ((long) cellSize * dataBlkNum));
     for (StripingCell cell : cells) {
       long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
       long cellEnd = cellStart + cell.size - 1;


[50/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java

Change-Id: I53ec1c426dc988d6c4a2c87b00caef49c4057010


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

Branch: refs/heads/trunk
Commit: 6c17d315287020368689fa078a40a1eaedf89d5b
Parents: 8fd5520 854d25b
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Wed Sep 30 08:39:13 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Wed Sep 30 08:39:13 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   5 +
 .../org/apache/hadoop/fs/FSOutputSummer.java    |   3 -
 .../main/java/org/apache/hadoop/util/GSet.java  |  14 ++
 .../org/apache/hadoop/util/GSetByHashMap.java   |   6 +
 .../org/apache/hadoop/util/LightWeightGSet.java |  82 ++++--
 .../hadoop/util/LightWeightResizableGSet.java   | 129 ++++++++++
 .../hadoop-common/src/site/markdown/Tracing.md  |  15 +-
 .../apache/hadoop/fs/shell/TestTextCommand.java |  15 +-
 .../java/org/apache/hadoop/util/TestGSet.java   |  69 ++++-
 .../hadoop/util/TestLightWeightCache.java       |   6 +
 .../util/TestLightWeightResizableGSet.java      | 252 +++++++++++++++++++
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 114 +++------
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |  77 ++----
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |  20 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  36 +--
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  85 +++----
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  36 +--
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  10 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  40 +--
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |   4 +-
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |  25 +-
 .../hdfs/client/HdfsClientConfigKeys.java       |   2 -
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  18 +-
 .../protocol/datatransfer/PacketReceiver.java   |   8 +-
 .../hdfs/protocol/datatransfer/Sender.java      |   6 +-
 .../datatransfer/sasl/DataTransferSaslUtil.java |   8 +-
 .../ClientDatanodeProtocolTranslatorPB.java     |  10 +-
 .../hdfs/shortcircuit/DfsClientShmManager.java  |  45 +---
 .../hdfs/shortcircuit/ShortCircuitCache.java    | 101 ++------
 .../hdfs/shortcircuit/ShortCircuitReplica.java  |  33 +--
 .../hdfs/shortcircuit/ShortCircuitShm.java      |  14 +-
 .../hadoop/hdfs/util/ByteArrayManager.java      |  63 +----
 .../org/apache/hadoop/hdfs/web/TokenAspect.java |   8 +-
 .../hadoop/hdfs/web/URLConnectionFactory.java   |   4 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  28 +--
 .../services/org.apache.hadoop.fs.FileSystem    |   1 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  21 ++
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   9 +-
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  37 +--
 .../hadoop/hdfs/server/datanode/DataNode.java   |  11 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |   8 -
 .../hdfs/server/datanode/ReplicaInfo.java       |  27 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |   7 +-
 .../datanode/fsdataset/impl/ReplicaMap.java     |  38 +--
 .../server/datanode/web/DatanodeHttpServer.java |  15 ++
 .../hdfs/server/namenode/NameNodeRpcServer.java |   2 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   5 +-
 .../services/org.apache.hadoop.fs.FileSystem    |  16 --
 .../src/main/resources/hdfs-default.xml         |   8 -
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     | 144 ++++++++---
 .../blockmanagement/TestBlockTokenWithDFS.java  |   2 +
 .../datanode/TestDataNodeVolumeFailure.java     |   2 +
 .../apache/hadoop/tracing/TestTraceAdmin.java   |   8 +-
 hadoop-yarn-project/CHANGES.txt                 |   6 +
 .../localizer/LocalResourcesTracker.java        |   3 +-
 .../localizer/LocalResourcesTrackerImpl.java    |  24 +-
 .../localizer/ResourceLocalizationService.java  |   5 +-
 .../TestLocalResourcesTrackerImpl.java          |  65 ++++-
 .../TestResourceLocalizationService.java        |  16 +-
 .../scheduler/fair/QueueManager.java            |  80 ++++--
 60 files changed, 1157 insertions(+), 794 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index be346a4,8f3df81..183602a
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@@ -3103,48 -3049,9 +3091,46 @@@ public class DFSClient implements java.
        }
      });
      HEDGED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
-     if (LOG.isDebugEnabled()) {
-       LOG.debug("Using hedged reads; pool threads=" + num);
-     }
+     LOG.debug("Using hedged reads; pool threads={}", num);
    }
  
 +  /**
 +   * Create thread pool for parallel reading in striped layout,
 +   * STRIPED_READ_THREAD_POOL, if it does not already exist.
 +   * @param num Number of threads for striped reads thread pool.
 +   */
 +  private void initThreadsNumForStripedReads(int num) {
 +    assert num > 0;
 +    if (STRIPED_READ_THREAD_POOL != null) {
 +      return;
 +    }
 +    synchronized (DFSClient.class) {
 +      if (STRIPED_READ_THREAD_POOL == null) {
 +        STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
 +            TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
 +            new Daemon.DaemonFactory() {
 +          private final AtomicInteger threadIndex = new AtomicInteger(0);
 +
 +          @Override
 +          public Thread newThread(Runnable r) {
 +            Thread t = super.newThread(r);
 +            t.setName("stripedRead-" + threadIndex.getAndIncrement());
 +            return t;
 +          }
 +        }, new ThreadPoolExecutor.CallerRunsPolicy() {
 +          @Override
 +          public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {
 +            LOG.info("Execution for striped reading rejected, "
 +                + "Executing in current thread");
 +            // will run in the current thread
 +            super.rejectedExecution(runnable, e);
 +          }
 +        });
 +        STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
 +      }
 +    }
 +  }
 +
    ThreadPoolExecutor getHedgedReadsThreadPool() {
      return HEDGED_READ_THREAD_POOL;
    }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 78eaa6c,47c4b7e..6039177
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@@ -191,11 -190,11 +191,11 @@@ public class DFSOutputStream extends FS
      this.fileEncryptionInfo = stat.getFileEncryptionInfo();
      this.cachingStrategy = new AtomicReference<CachingStrategy>(
          dfsClient.getDefaultWriteCachingStrategy());
-     if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
-       DFSClient.LOG.debug(
-           "Set non-null progress callback on DFSOutputStream " + src);
+     if (progress != null) {
+       DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
+           +"{}", src);
      }
 -    
 +
      this.bytesPerChecksum = checksum.getBytesPerChecksum();
      if (bytesPerChecksum <= 0) {
        throw new HadoopIllegalArgumentException(
@@@ -410,14 -394,10 +407,10 @@@
      if (currentPacket == null) {
        currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
            .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
-       if (LOG.isDebugEnabled()) {
-         LOG.debug("WriteChunk allocating new packet seqno=" +
-             currentPacket.getSeqno() +
-             ", src=" + src +
-             ", packetSize=" + packetSize +
-             ", chunksPerPacket=" + chunksPerPacket +
-             ", bytesCurBlock=" + getStreamer().getBytesCurBlock() + ", " + this);
-       }
 -      DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno={},"
++      DFSClient.LOG.debug("WriteChunk allocating new packet seqno={},"
+               + " src={}, packetSize={}, chunksPerPacket={}, bytesCurBlock={}",
+           currentPacket.getSeqno(), src, packetSize, chunksPerPacket,
 -          getStreamer().getBytesCurBlock());
++          getStreamer().getBytesCurBlock() + ", " + this);
      }
  
      currentPacket.writeChecksum(checksum, ckoff, cklen);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index af7a61e,4ea1f41..683d98d
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@@ -507,10 -479,8 +503,8 @@@ class DataStreamer extends Daemon 
      stage = BlockConstructionStage.DATA_STREAMING;
    }
  
 -  private void endBlock() {
 -    LOG.debug("Closing old block {}", block);
 +  protected void endBlock() {
-     if(LOG.isDebugEnabled()) {
-       LOG.debug("Closing old block " + block);
-     }
++    LOG.debug("Closing old block " + block);
      this.setName("DataStreamer for file " + src);
      closeResponder();
      closeStream();
@@@ -590,10 -560,8 +584,11 @@@
          }
  
          // get new block from namenode.
 +        if (LOG.isDebugEnabled()) {
 +          LOG.debug("stage=" + stage + ", " + this);
 +        }
          if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
+           LOG.debug("Allocating new block");
            setPipeline(nextBlockOutputStream());
            initDataStreaming();
          } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
@@@ -645,9 -617,7 +641,7 @@@
            }
          }
  
-         if (LOG.isDebugEnabled()) {
-           LOG.debug(this + " sending " + one);
-         }
 -        LOG.debug("DataStreamer block {} sending packet {}", block, one);
++        LOG.debug(this + " sending " + one);
  
          // write out data to remote datanode
          TraceScope writeScope = dfsClient.getTracer().
@@@ -1785,9 -1768,7 +1773,7 @@@
        packet.addTraceParent(Tracer.getCurrentSpanId());
        dataQueue.addLast(packet);
        lastQueuedSeqno = packet.getSeqno();
-       if (LOG.isDebugEnabled()) {
-         LOG.debug("Queued " + packet + ", " + this);
-       }
 -      LOG.debug("Queued packet {}", packet.getSeqno());
++      LOG.debug("Queued " + packet + ", " + this);
        dataQueue.notifyAll();
      }
    }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c17d315/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------


[04/50] [abbrv] hadoop git commit: Merge commit '456e901a4c5c639267ee87b8e5f1319f256d20c2' (HDFS-6407. Add sorting and pagination in the datanode tab of the NN Web UI. Contributed by Haohui Mai.) into HDFS-7285-merge

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index b71e59e,0000000..4ca8fe6
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@@ -1,653 -1,0 +1,653 @@@
 +/**
 + * 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.hdfs;
 +
 +import java.io.IOException;
 +import java.io.InterruptedIOException;
 +import java.nio.ByteBuffer;
 +import java.nio.channels.ClosedChannelException;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.EnumSet;
 +import java.util.List;
 +import java.util.concurrent.BlockingQueue;
 +import java.util.concurrent.LinkedBlockingQueue;
 +
 +import org.apache.hadoop.HadoopIllegalArgumentException;
 +import org.apache.hadoop.classification.InterfaceAudience;
 +import org.apache.hadoop.fs.CreateFlag;
 +import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 +import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 +import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 +import org.apache.hadoop.io.MultipleIOException;
 +import org.apache.hadoop.io.erasurecode.CodecUtil;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 +import org.apache.hadoop.util.DataChecksum;
 +import org.apache.hadoop.util.Progressable;
 +import org.apache.htrace.Sampler;
 +import org.apache.htrace.Trace;
 +import org.apache.htrace.TraceScope;
 +
 +import com.google.common.base.Preconditions;
 +
 +
 +/**
 + * This class supports writing files in striped layout and erasure coded format.
 + * Each stripe contains a sequence of cells.
 + */
 +@InterfaceAudience.Private
 +public class DFSStripedOutputStream extends DFSOutputStream {
 +  static class MultipleBlockingQueue<T> {
 +    private final List<BlockingQueue<T>> queues;
 +
 +    MultipleBlockingQueue(int numQueue, int queueSize) {
 +      queues = new ArrayList<>(numQueue);
 +      for (int i = 0; i < numQueue; i++) {
 +        queues.add(new LinkedBlockingQueue<T>(queueSize));
 +      }
 +    }
 +
 +    boolean isEmpty() {
 +      for(int i = 0; i < queues.size(); i++) {
 +        if (!queues.get(i).isEmpty()) {
 +          return false;
 +        }
 +      }
 +      return true;
 +    }
 +
 +    int numQueues() {
 +      return queues.size();
 +    }
 +
 +    void offer(int i, T object) {
 +      final boolean b = queues.get(i).offer(object);
 +      Preconditions.checkState(b, "Failed to offer " + object
 +          + " to queue, i=" + i);
 +    }
 +
 +    T take(int i) throws InterruptedIOException {
 +      try {
 +        return queues.get(i).take();
 +      } catch(InterruptedException ie) {
 +        throw DFSUtil.toInterruptedIOException("take interrupted, i=" + i, ie);
 +      }
 +    }
 +
 +    T poll(int i) {
 +      return queues.get(i).poll();
 +    }
 +
 +    T peek(int i) {
 +      return queues.get(i).peek();
 +    }
 +  }
 +
 +  /** Coordinate the communication between the streamers. */
 +  class Coordinator {
 +    private final MultipleBlockingQueue<LocatedBlock> followingBlocks;
 +    private final MultipleBlockingQueue<ExtendedBlock> endBlocks;
 +
 +    private final MultipleBlockingQueue<LocatedBlock> newBlocks;
 +    private final MultipleBlockingQueue<ExtendedBlock> updateBlocks;
 +
 +    Coordinator(final DfsClientConf conf, final int numDataBlocks,
 +        final int numAllBlocks) {
 +      followingBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
 +      endBlocks = new MultipleBlockingQueue<>(numDataBlocks, 1);
 +
 +      newBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
 +      updateBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
 +    }
 +
 +    MultipleBlockingQueue<LocatedBlock> getFollowingBlocks() {
 +      return followingBlocks;
 +    }
 +
 +    MultipleBlockingQueue<LocatedBlock> getNewBlocks() {
 +      return newBlocks;
 +    }
 +
 +    MultipleBlockingQueue<ExtendedBlock> getUpdateBlocks() {
 +      return updateBlocks;
 +    }
 +
 +    StripedDataStreamer getStripedDataStreamer(int i) {
 +      return DFSStripedOutputStream.this.getStripedDataStreamer(i);
 +    }
 +
 +    void offerEndBlock(int i, ExtendedBlock block) {
 +      endBlocks.offer(i, block);
 +    }
 +
 +    ExtendedBlock takeEndBlock(int i) throws InterruptedIOException {
 +      return endBlocks.take(i);
 +    }
 +
 +    boolean hasAllEndBlocks() {
 +      for(int i = 0; i < endBlocks.numQueues(); i++) {
 +        if (endBlocks.peek(i) == null) {
 +          return false;
 +        }
 +      }
 +      return true;
 +    }
 +
 +    void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) {
 +      ExtendedBlock b = endBlocks.peek(i);
 +      if (b == null) {
 +        // streamer just has failed, put end block and continue
 +        b = block;
 +        offerEndBlock(i, b);
 +      }
 +      b.setNumBytes(newBytes);
 +    }
 +
 +    /** @return a block representing the entire block group. */
 +    ExtendedBlock getBlockGroup() {
 +      final StripedDataStreamer s0 = getStripedDataStreamer(0);
 +      final ExtendedBlock b0 = s0.getBlock();
 +      if (b0 == null) {
 +        return null;
 +      }
 +
 +      final boolean atBlockGroupBoundary = s0.getBytesCurBlock() == 0 && b0.getNumBytes() > 0;
 +      final ExtendedBlock block = new ExtendedBlock(b0);
 +      long numBytes = b0.getNumBytes();
 +      for (int i = 1; i < numDataBlocks; i++) {
 +        final StripedDataStreamer si = getStripedDataStreamer(i);
 +        final ExtendedBlock bi = si.getBlock();
 +        if (bi != null && bi.getGenerationStamp() > block.getGenerationStamp()) {
 +          block.setGenerationStamp(bi.getGenerationStamp());
 +        }
 +        numBytes += atBlockGroupBoundary? bi.getNumBytes(): si.getBytesCurBlock();
 +      }
 +      block.setNumBytes(numBytes);
 +      if (LOG.isDebugEnabled()) {
 +        LOG.debug("getBlockGroup: " + block + ", numBytes=" + block.getNumBytes());
 +      }
 +      return block;
 +    }
 +  }
 +
 +  /** Buffers for writing the data and parity cells of a stripe. */
 +  class CellBuffers {
 +    private final ByteBuffer[] buffers;
 +    private final byte[][] checksumArrays;
 +
 +    CellBuffers(int numParityBlocks) throws InterruptedException{
 +      if (cellSize % bytesPerChecksum != 0) {
 +        throw new HadoopIllegalArgumentException("Invalid values: "
 +            + DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (="
 +            + bytesPerChecksum + ") must divide cell size (=" + cellSize + ").");
 +      }
 +
 +      checksumArrays = new byte[numParityBlocks][];
 +      final int size = getChecksumSize() * (cellSize / bytesPerChecksum);
 +      for (int i = 0; i < checksumArrays.length; i++) {
 +        checksumArrays[i] = new byte[size];
 +      }
 +
 +      buffers = new ByteBuffer[numAllBlocks];
 +      for (int i = 0; i < buffers.length; i++) {
 +        buffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
 +      }
 +    }
 +
 +    private ByteBuffer[] getBuffers() {
 +      return buffers;
 +    }
 +
 +    byte[] getChecksumArray(int i) {
 +      return checksumArrays[i - numDataBlocks];
 +    }
 +
 +    private int addTo(int i, byte[] b, int off, int len) {
 +      final ByteBuffer buf = buffers[i];
 +      final int pos = buf.position() + len;
 +      Preconditions.checkState(pos <= cellSize);
 +      buf.put(b, off, len);
 +      return pos;
 +    }
 +
 +    private void clear() {
 +      for (int i = 0; i< numAllBlocks; i++) {
 +        buffers[i].clear();
 +        if (i >= numDataBlocks) {
 +          Arrays.fill(buffers[i].array(), (byte) 0);
 +        }
 +      }
 +    }
 +
 +    private void release() {
 +      for (int i = 0; i < numAllBlocks; i++) {
 +        byteArrayManager.release(buffers[i].array());
 +      }
 +    }
 +
 +    private void flipDataBuffers() {
 +      for (int i = 0; i < numDataBlocks; i++) {
 +        buffers[i].flip();
 +      }
 +    }
 +  }
 +
 +  private final Coordinator coordinator;
 +  private final CellBuffers cellBuffers;
 +  private final RawErasureEncoder encoder;
 +  private final List<StripedDataStreamer> streamers;
 +  private final DFSPacket[] currentPackets; // current Packet of each streamer
 +
 +  /** Size of each striping cell, must be a multiple of bytesPerChecksum */
 +  private final int cellSize;
 +  private final int numAllBlocks;
 +  private final int numDataBlocks;
 +
 +  @Override
 +  ExtendedBlock getBlock() {
 +    return coordinator.getBlockGroup();
 +  }
 +
 +  /** Construct a new output stream for creating a file. */
 +  DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
 +                         EnumSet<CreateFlag> flag, Progressable progress,
 +                         DataChecksum checksum, String[] favoredNodes)
 +                         throws IOException {
 +    super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Creating DFSStripedOutputStream for " + src);
 +    }
 +
 +    final ErasureCodingPolicy ecPolicy = stat.getErasureCodingPolicy();
 +    final int numParityBlocks = ecPolicy.getNumParityUnits();
 +    cellSize = ecPolicy.getCellSize();
 +    numDataBlocks = ecPolicy.getNumDataUnits();
 +    numAllBlocks = numDataBlocks + numParityBlocks;
 +
 +    encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(),
 +        numDataBlocks, numParityBlocks);
 +
 +    coordinator = new Coordinator(dfsClient.getConf(),
 +        numDataBlocks, numAllBlocks);
 +    try {
 +      cellBuffers = new CellBuffers(numParityBlocks);
 +    } catch (InterruptedException ie) {
 +      throw DFSUtil.toInterruptedIOException(
 +          "Failed to create cell buffers", ie);
 +    }
 +
 +    List<StripedDataStreamer> s = new ArrayList<>(numAllBlocks);
 +    for (short i = 0; i < numAllBlocks; i++) {
 +      StripedDataStreamer streamer = new StripedDataStreamer(stat,
 +          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
 +          favoredNodes, i, coordinator);
 +      s.add(streamer);
 +    }
 +    streamers = Collections.unmodifiableList(s);
 +    currentPackets = new DFSPacket[streamers.size()];
 +    setCurrentStreamer(0);
 +  }
 +
 +  StripedDataStreamer getStripedDataStreamer(int i) {
 +    return streamers.get(i);
 +  }
 +
 +  int getCurrentIndex() {
 +    return getCurrentStreamer().getIndex();
 +  }
 +
 +  private synchronized StripedDataStreamer getCurrentStreamer() {
 +    return (StripedDataStreamer)streamer;
 +  }
 +
 +  private synchronized StripedDataStreamer setCurrentStreamer(int newIdx)
 +      throws IOException {
 +    // backup currentPacket for current streamer
 +    int oldIdx = streamers.indexOf(streamer);
 +    if (oldIdx >= 0) {
 +      currentPackets[oldIdx] = currentPacket;
 +    }
 +
 +    streamer = streamers.get(newIdx);
 +    currentPacket = currentPackets[newIdx];
 +    adjustChunkBoundary();
 +
 +    return getCurrentStreamer();
 +  }
 +
 +  /**
 +   * Encode the buffers, i.e. compute parities.
 +   *
 +   * @param buffers data buffers + parity buffers
 +   */
 +  private static void encode(RawErasureEncoder encoder, int numData,
 +      ByteBuffer[] buffers) {
 +    final ByteBuffer[] dataBuffers = new ByteBuffer[numData];
 +    final ByteBuffer[] parityBuffers = new ByteBuffer[buffers.length - numData];
 +    System.arraycopy(buffers, 0, dataBuffers, 0, dataBuffers.length);
 +    System.arraycopy(buffers, numData, parityBuffers, 0, parityBuffers.length);
 +
 +    encoder.encode(dataBuffers, parityBuffers);
 +  }
 +
 +
 +  private void checkStreamers() throws IOException {
 +    int count = 0;
 +    for(StripedDataStreamer s : streamers) {
 +      if (!s.isFailed()) {
 +        if (s.getBlock() != null) {
 +          s.getErrorState().initExternalError();
 +        }
 +        count++;
 +      }
 +    }
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("checkStreamers: " + streamers);
 +      LOG.debug("count=" + count);
 +    }
 +    if (count < numDataBlocks) {
 +      throw new IOException("Failed: the number of remaining blocks = "
 +          + count + " < the number of data blocks = " + numDataBlocks);
 +    }
 +  }
 +
 +  private void handleStreamerFailure(String err,
 +                                     Exception e) throws IOException {
 +    LOG.warn("Failed: " + err + ", " + this, e);
 +    getCurrentStreamer().setFailed(true);
 +    checkStreamers();
 +    currentPacket = null;
 +  }
 +
 +  @Override
 +  protected synchronized void writeChunk(byte[] bytes, int offset, int len,
 +      byte[] checksum, int ckoff, int cklen) throws IOException {
 +    final int index = getCurrentIndex();
 +    final StripedDataStreamer current = getCurrentStreamer();
 +    final int pos = cellBuffers.addTo(index, bytes, offset, len);
 +    final boolean cellFull = pos == cellSize;
 +
 +    final long oldBytes = current.getBytesCurBlock();
 +    if (!current.isFailed()) {
 +      try {
 +        super.writeChunk(bytes, offset, len, checksum, ckoff, cklen);
 +      } catch(Exception e) {
 +        handleStreamerFailure("offset=" + offset + ", length=" + len, e);
 +      }
 +    }
 +
 +    if (current.isFailed()) {
 +      final long newBytes = oldBytes + len;
 +      coordinator.setBytesEndBlock(index, newBytes, current.getBlock());
 +      current.setBytesCurBlock(newBytes);
 +    }
 +
 +    // Two extra steps are needed when a striping cell is full:
 +    // 1. Forward the current index pointer
 +    // 2. Generate parity packets if a full stripe of data cells are present
 +    if (cellFull) {
 +      int next = index + 1;
 +      //When all data cells in a stripe are ready, we need to encode
 +      //them and generate some parity cells. These cells will be
 +      //converted to packets and put to their DataStreamer's queue.
 +      if (next == numDataBlocks) {
 +        cellBuffers.flipDataBuffers();
 +        writeParityCells();
 +        next = 0;
 +      }
 +      setCurrentStreamer(next);
 +    }
 +  }
 +
 +  private int stripeDataSize() {
 +    return numDataBlocks * cellSize;
 +  }
 +
 +  @Override
 +  public void hflush() {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  public void hsync() {
 +    throw new UnsupportedOperationException();
 +  }
 +
 +  @Override
 +  protected synchronized void start() {
 +    for (StripedDataStreamer streamer : streamers) {
 +      streamer.start();
 +    }
 +  }
 +
 +  @Override
 +  synchronized void abort() throws IOException {
 +    if (isClosed()) {
 +      return;
 +    }
 +    for (StripedDataStreamer streamer : streamers) {
 +      streamer.getLastException().set(new IOException("Lease timeout of "
 +          + (dfsClient.getConf().getHdfsTimeout()/1000) +
 +          " seconds expired."));
 +    }
 +    closeThreads(true);
 +    dfsClient.endFileLease(fileId);
 +  }
 +
 +  @Override
 +  boolean isClosed() {
 +    if (closed) {
 +      return true;
 +    }
 +    for(StripedDataStreamer s : streamers) {
 +      if (!s.streamerClosed()) {
 +        return false;
 +      }
 +    }
 +    return true;
 +  }
 +
 +  @Override
 +  protected void closeThreads(boolean force) throws IOException {
 +    final MultipleIOException.Builder b = new MultipleIOException.Builder();
 +    try {
 +      for (StripedDataStreamer streamer : streamers) {
 +        try {
 +          streamer.close(force);
 +          streamer.join();
 +          streamer.closeSocket();
 +        } catch (Exception e) {
 +          try {
 +            handleStreamerFailure("force=" + force, e);
 +          } catch (IOException ioe) {
 +            b.add(ioe);
 +          }
 +        } finally {
 +          streamer.setSocketToNull();
 +        }
 +      }
 +    } finally {
 +      setClosed();
 +    }
 +    final IOException ioe = b.build();
 +    if (ioe != null) {
 +      throw ioe;
 +    }
 +  }
 +
 +  /**
 +   * Simply add bytesCurBlock together. Note that this result is not accurately
 +   * the size of the block group.
 +   */
 +  private long getCurrentSumBytes() {
 +    long sum = 0;
 +    for (int i = 0; i < numDataBlocks; i++) {
 +      sum += streamers.get(i).getBytesCurBlock();
 +    }
 +    return sum;
 +  }
 +
 +  private void writeParityCellsForLastStripe() throws IOException {
 +    final long currentBlockGroupBytes = getCurrentSumBytes();
 +    if (currentBlockGroupBytes % stripeDataSize() == 0) {
 +      return;
 +    }
 +
 +    final int firstCellSize =
 +        (int)(getStripedDataStreamer(0).getBytesCurBlock() % cellSize);
 +    final int parityCellSize = firstCellSize > 0 && firstCellSize < cellSize?
 +        firstCellSize : cellSize;
 +    final ByteBuffer[] buffers = cellBuffers.getBuffers();
 +
 +    for (int i = 0; i < numAllBlocks; i++) {
 +      // Pad zero bytes to make all cells exactly the size of parityCellSize
 +      // If internal block is smaller than parity block, pad zero bytes.
 +      // Also pad zero bytes to all parity cells
 +      final int position = buffers[i].position();
 +      assert position <= parityCellSize : "If an internal block is smaller" +
 +          " than parity block, then its last cell should be small than last" +
 +          " parity cell";
 +      for (int j = 0; j < parityCellSize - position; j++) {
 +        buffers[i].put((byte) 0);
 +      }
 +      buffers[i].flip();
 +    }
 +
 +    writeParityCells();
 +  }
 +
 +  void writeParityCells() throws IOException {
 +    final ByteBuffer[] buffers = cellBuffers.getBuffers();
 +    //encode the data cells
 +    encode(encoder, numDataBlocks, buffers);
 +    for (int i = numDataBlocks; i < numAllBlocks; i++) {
 +      writeParity(i, buffers[i], cellBuffers.getChecksumArray(i));
 +    }
 +    cellBuffers.clear();
 +  }
 +
 +  void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf
 +      ) throws IOException {
 +    final StripedDataStreamer current = setCurrentStreamer(index);
 +    final int len = buffer.limit();
 +
 +    final long oldBytes = current.getBytesCurBlock();
 +    if (!current.isFailed()) {
 +      try {
 +        DataChecksum sum = getDataChecksum();
 +        sum.calculateChunkedSums(buffer.array(), 0, len, checksumBuf, 0);
 +        for (int i = 0; i < len; i += sum.getBytesPerChecksum()) {
 +          int chunkLen = Math.min(sum.getBytesPerChecksum(), len - i);
 +          int ckOffset = i / sum.getBytesPerChecksum() * getChecksumSize();
 +          super.writeChunk(buffer.array(), i, chunkLen, checksumBuf, ckOffset,
 +              getChecksumSize());
 +        }
 +      } catch(Exception e) {
 +        handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e);
 +      }
 +    }
 +
 +    if (current.isFailed()) {
 +      final long newBytes = oldBytes + len;
 +      current.setBytesCurBlock(newBytes);
 +    }
 +  }
 +
 +  @Override
 +  void setClosed() {
 +    super.setClosed();
 +    for (int i = 0; i < numAllBlocks; i++) {
 +      streamers.get(i).release();
 +    }
 +    cellBuffers.release();
 +  }
 +
 +  @Override
 +  protected synchronized void closeImpl() throws IOException {
 +    if (isClosed()) {
 +      final MultipleIOException.Builder b = new MultipleIOException.Builder();
 +      for(int i = 0; i < streamers.size(); i++) {
 +        final StripedDataStreamer si = getStripedDataStreamer(i);
 +        try {
 +          si.getLastException().check(true);
 +        } catch (IOException e) {
 +          b.add(e);
 +        }
 +      }
 +      final IOException ioe = b.build();
 +      if (ioe != null) {
 +        throw ioe;
 +      }
 +      return;
 +    }
 +
 +    try {
 +      // flush from all upper layers
 +      try {
 +        flushBuffer();
 +        // if the last stripe is incomplete, generate and write parity cells
 +        writeParityCellsForLastStripe();
 +        enqueueAllCurrentPackets();
 +      } catch(Exception e) {
 +        handleStreamerFailure("closeImpl", e);
 +      }
 +
 +      for (int i = 0; i < numAllBlocks; i++) {
 +        final StripedDataStreamer s = setCurrentStreamer(i);
 +        if (!s.isFailed()) {
 +          try {
 +            if (s.getBytesCurBlock() > 0) {
-               setCurrentPacket2Empty();
++              setCurrentPacketToEmpty();
 +            }
 +            // flush all data to Datanode
 +            flushInternal();
 +          } catch(Exception e) {
 +            handleStreamerFailure("closeImpl", e);
 +          }
 +        }
 +      }
 +
 +      closeThreads(false);
 +      final ExtendedBlock lastBlock = coordinator.getBlockGroup();
 +      TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
 +      try {
 +        completeFile(lastBlock);
 +      } finally {
 +        scope.close();
 +      }
 +      dfsClient.endFileLease(fileId);
 +    } catch (ClosedChannelException ignored) {
 +    } finally {
 +      setClosed();
 +    }
 +  }
 +
 +  private void enqueueAllCurrentPackets() throws IOException {
 +    int idx = streamers.indexOf(getCurrentStreamer());
 +    for(int i = 0; i < streamers.size(); i++) {
 +      setCurrentStreamer(i);
 +      if (currentPacket != null) {
 +        enqueueCurrentPacket();
 +      }
 +    }
 +    setCurrentStreamer(idx);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index d55d00b,8e81fdc..50a367a
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@@ -199,12 -200,7 +201,13 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
  import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto;
  import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index f2facd7,4ca5b26..c083b5e
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@@ -3124,193 -3067,7 +3141,194 @@@ public class PBHelper 
          setTotalRpcs(context.getTotalRpcs()).
          setCurRpc(context.getCurRpc()).
          setId(context.getReportId()).
+         setLeaseId(context.getLeaseId()).
          build();
    }
 +
 +  public static ECSchema convertECSchema(ECSchemaProto schema) {
 +    List<ECSchemaOptionEntryProto> optionsList = schema.getOptionsList();
 +    Map<String, String> options = new HashMap<>(optionsList.size());
 +    for (ECSchemaOptionEntryProto option : optionsList) {
 +      options.put(option.getKey(), option.getValue());
 +    }
 +    return new ECSchema(schema.getCodecName(), schema.getDataUnits(),
 +        schema.getParityUnits(), options);
 +  }
 +
 +  public static ECSchemaProto convertECSchema(ECSchema schema) {
 +    ECSchemaProto.Builder builder = ECSchemaProto.newBuilder()
 +        .setCodecName(schema.getCodecName())
 +        .setDataUnits(schema.getNumDataUnits())
 +        .setParityUnits(schema.getNumParityUnits());
 +    Set<Entry<String, String>> entrySet = schema.getExtraOptions().entrySet();
 +    for (Entry<String, String> entry : entrySet) {
 +      builder.addOptions(ECSchemaOptionEntryProto.newBuilder()
 +          .setKey(entry.getKey()).setValue(entry.getValue()).build());
 +    }
 +    return builder.build();
 +  }
 +
 +  public static ErasureCodingPolicy convertErasureCodingPolicy(
 +      ErasureCodingPolicyProto policy) {
 +    return new ErasureCodingPolicy(policy.getName(),
 +        convertECSchema(policy.getSchema()),
 +        policy.getCellSize());
 +  }
 +
 +  public static ErasureCodingPolicyProto convertErasureCodingPolicy(
 +      ErasureCodingPolicy policy) {
 +    ErasureCodingPolicyProto.Builder builder = ErasureCodingPolicyProto
 +        .newBuilder()
 +        .setName(policy.getName())
 +        .setSchema(convertECSchema(policy.getSchema()))
 +        .setCellSize(policy.getCellSize());
 +    return builder.build();
 +  }
 +
 +  public static ErasureCodingZoneProto convertErasureCodingZone(
 +      ErasureCodingZone ecZone) {
 +    return ErasureCodingZoneProto.newBuilder().setDir(ecZone.getDir())
 +        .setEcPolicy(convertErasureCodingPolicy(ecZone.getErasureCodingPolicy()))
 +        .build();
 +  }
 +
 +  public static ErasureCodingZone convertErasureCodingZone(
 +      ErasureCodingZoneProto ecZoneProto) {
 +    return new ErasureCodingZone(ecZoneProto.getDir(),
 +        convertErasureCodingPolicy(ecZoneProto.getEcPolicy()));
 +  }
 +  
 +  public static BlockECRecoveryInfo convertBlockECRecoveryInfo(
 +      BlockECRecoveryInfoProto blockEcRecoveryInfoProto) {
 +    ExtendedBlockProto blockProto = blockEcRecoveryInfoProto.getBlock();
 +    ExtendedBlock block = convert(blockProto);
 +
 +    DatanodeInfosProto sourceDnInfosProto = blockEcRecoveryInfoProto
 +        .getSourceDnInfos();
 +    DatanodeInfo[] sourceDnInfos = convert(sourceDnInfosProto);
 +
 +    DatanodeInfosProto targetDnInfosProto = blockEcRecoveryInfoProto
 +        .getTargetDnInfos();
 +    DatanodeInfo[] targetDnInfos = convert(targetDnInfosProto);
 +
 +    StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto
 +        .getTargetStorageUuids();
 +    String[] targetStorageUuids = convert(targetStorageUuidsProto);
 +
 +    StorageTypesProto targetStorageTypesProto = blockEcRecoveryInfoProto
 +        .getTargetStorageTypes();
 +    StorageType[] convertStorageTypes = convertStorageTypes(
 +        targetStorageTypesProto.getStorageTypesList(), targetStorageTypesProto
 +            .getStorageTypesList().size());
 +
 +    List<Integer> liveBlockIndicesList = blockEcRecoveryInfoProto
 +        .getLiveBlockIndicesList();
 +    short[] liveBlkIndices = new short[liveBlockIndicesList.size()];
 +    for (int i = 0; i < liveBlockIndicesList.size(); i++) {
 +      liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue();
 +    }
 +
 +    ErasureCodingPolicy ecPolicy =
 +        convertErasureCodingPolicy(blockEcRecoveryInfoProto.getEcPolicy());
 +
 +    return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos,
 +        targetStorageUuids, convertStorageTypes, liveBlkIndices, ecPolicy);
 +  }
 +
 +  public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo(
 +      BlockECRecoveryInfo blockEcRecoveryInfo) {
 +    BlockECRecoveryInfoProto.Builder builder = BlockECRecoveryInfoProto
 +        .newBuilder();
 +    builder.setBlock(convert(blockEcRecoveryInfo.getExtendedBlock()));
 +
 +    DatanodeInfo[] sourceDnInfos = blockEcRecoveryInfo.getSourceDnInfos();
 +    builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos));
 +
 +    DatanodeInfo[] targetDnInfos = blockEcRecoveryInfo.getTargetDnInfos();
 +    builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos));
 +
 +    String[] targetStorageIDs = blockEcRecoveryInfo.getTargetStorageIDs();
 +    builder.setTargetStorageUuids(convertStorageIDs(targetStorageIDs));
 +
 +    StorageType[] targetStorageTypes = blockEcRecoveryInfo
 +        .getTargetStorageTypes();
 +    builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes));
 +
 +    short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices();
 +    builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices));
 +
 +    builder.setEcPolicy(convertErasureCodingPolicy(blockEcRecoveryInfo
 +        .getErasureCodingPolicy()));
 +
 +    return builder.build();
 +  }
 +
 +  private static List<Integer> convertIntArray(short[] liveBlockIndices) {
 +    List<Integer> liveBlockIndicesList = new ArrayList<Integer>();
 +    for (short s : liveBlockIndices) {
 +      liveBlockIndicesList.add((int) s);
 +    }
 +    return liveBlockIndicesList;
 +  }
 +
 +  private static StorageTypesProto convertStorageTypesProto(
 +      StorageType[] targetStorageTypes) {
 +    StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
 +    for (StorageType storageType : targetStorageTypes) {
 +      builder.addStorageTypes(convertStorageType(storageType));
 +    }
 +    return builder.build();
 +  }
 +
 +  private static StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) {
 +    StorageUuidsProto.Builder builder = StorageUuidsProto.newBuilder();
 +    for (String storageUuid : targetStorageIDs) {
 +      builder.addStorageUuids(storageUuid);
 +    }
 +    return builder.build();
 +  }
 +
 +  private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) {
 +    DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder();
 +    for (DatanodeInfo datanodeInfo : dnInfos) {
 +      builder.addDatanodes(convert(datanodeInfo));
 +    }
 +    return builder.build();
 +  }
 +
 +  private static String[] convert(StorageUuidsProto targetStorageUuidsProto) {
 +    List<String> storageUuidsList = targetStorageUuidsProto
 +        .getStorageUuidsList();
 +    String[] storageUuids = new String[storageUuidsList.size()];
 +    for (int i = 0; i < storageUuidsList.size(); i++) {
 +      storageUuids[i] = storageUuidsList.get(i);
 +    }
 +    return storageUuids;
 +  }
 +  
 +  public static BlockECRecoveryCommandProto convert(
 +      BlockECRecoveryCommand blkECRecoveryCmd) {
 +    BlockECRecoveryCommandProto.Builder builder = BlockECRecoveryCommandProto
 +        .newBuilder();
 +    Collection<BlockECRecoveryInfo> blockECRecoveryInfos = blkECRecoveryCmd
 +        .getECTasks();
 +    for (BlockECRecoveryInfo blkECRecoveryInfo : blockECRecoveryInfos) {
 +      builder
 +          .addBlockECRecoveryinfo(convertBlockECRecoveryInfo(blkECRecoveryInfo));
 +    }
 +    return builder.build();
 +  }
 +  
 +  public static BlockECRecoveryCommand convert(
 +      BlockECRecoveryCommandProto blkECRecoveryCmdProto) {
 +    Collection<BlockECRecoveryInfo> blkECRecoveryInfos = new ArrayList<BlockECRecoveryInfo>();
 +    List<BlockECRecoveryInfoProto> blockECRecoveryinfoList = blkECRecoveryCmdProto
 +        .getBlockECRecoveryinfoList();
 +    for (BlockECRecoveryInfoProto blockECRecoveryInfoProto : blockECRecoveryinfoList) {
 +      blkECRecoveryInfos
 +          .add(convertBlockECRecoveryInfo(blockECRecoveryInfoProto));
 +    }
 +    return new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY,
 +        blkECRecoveryInfos);
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index 930001a,f9847ca..555f506
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@@ -291,10 -310,7 +316,8 @@@ public class Dispatcher 
  
      /** Dispatch the move to the proxy source & wait for the response. */
      private void dispatch() {
-       if (LOG.isDebugEnabled()) {
-         LOG.debug("Start moving " + this);
-       }
+       LOG.info("Start moving " + this);
 +      assert !(reportedBlock instanceof DBlockStriped);
  
        Socket sock = new Socket();
        DataOutputStream out = null;
@@@ -323,7 -339,8 +346,8 @@@
  
          sendRequest(out, eb, accessToken);
          receiveResponse(in);
 -        nnc.getBytesMoved().addAndGet(block.getNumBytes());
 +        nnc.getBytesMoved().addAndGet(reportedBlock.getNumBytes());
+         target.getDDatanode().setHasSuccess();
          LOG.info("Successfully moved " + this);
        } catch (IOException e) {
          LOG.warn("Failed to move " + this + ": " + e.getMessage());
@@@ -656,29 -650,25 +695,39 @@@
       * @return the total size of the received blocks in the number of bytes.
       */
      private long getBlockList() throws IOException {
-       final long size = Math.min(MAX_BLOCKS_SIZE_TO_FETCH, blocksToReceive);
+       final long size = Math.min(getBlocksSize, blocksToReceive);
 -      final BlocksWithLocations newBlocks = nnc.getBlocks(getDatanodeInfo(), size);
 +      final BlocksWithLocations newBlksLocs =
 +          nnc.getBlocks(getDatanodeInfo(), size);
  
+       if (LOG.isTraceEnabled()) {
+         LOG.trace("getBlocks(" + getDatanodeInfo() + ", "
+             + StringUtils.TraditionalBinaryPrefix.long2String(size, "B", 2)
 -            + ") returns " + newBlocks.getBlocks().length + " blocks.");
++            + ") returns " + newBlksLocs.getBlocks().length + " blocks.");
+       }
+ 
        long bytesReceived = 0;
 -      for (BlockWithLocations blk : newBlocks.getBlocks()) {
 +      for (BlockWithLocations blkLocs : newBlksLocs.getBlocks()) {
+         // Skip small blocks.
 -        if (blk.getBlock().getNumBytes() < getBlocksMinBlockSize) {
++        if (blkLocs.getBlock().getNumBytes() < getBlocksMinBlockSize) {
+           continue;
+         }
  
 -        bytesReceived += blk.getBlock().getNumBytes();
 +        DBlock block;
 +        if (blkLocs instanceof StripedBlockWithLocations) {
 +          StripedBlockWithLocations sblkLocs =
 +              (StripedBlockWithLocations) blkLocs;
 +          // approximate size
 +          bytesReceived += sblkLocs.getBlock().getNumBytes() /
 +              sblkLocs.getDataBlockNum();
 +          block = new DBlockStriped(sblkLocs.getBlock(), sblkLocs.getIndices(),
 +              sblkLocs.getDataBlockNum());
 +        } else{
 +          bytesReceived += blkLocs.getBlock().getNumBytes();
 +          block = new DBlock(blkLocs.getBlock());
 +        }
 +
          synchronized (globalBlocks) {
 -          final DBlock block = globalBlocks.get(blk.getBlock());
 +          block = globalBlocks.putIfAbsent(blkLocs.getBlock(), block);
            synchronized (block) {
              block.clearLocations();
  
@@@ -944,8 -954,22 +1015,21 @@@
      return new DDatanode(datanode, maxConcurrentMovesPerNode);
    }
  
+ 
    public void executePendingMove(final PendingMove p) {
 -    // move the block
 +    // move the reportedBlock
+     final DDatanode targetDn = p.target.getDDatanode();
+     ExecutorService moveExecutor = targetDn.getMoveExecutor();
+     if (moveExecutor == null) {
+       final int nThreads = moverThreadAllocator.allocate(maxConcurrentMovesPerNode);
+       if (nThreads > 0) {
+         moveExecutor = targetDn.initMoveExecutor(nThreads);
+       }
+     }
+     if (moveExecutor == null) {
+       LOG.warn("No mover threads available: skip moving " + p);
+       return;
+     }
 -
      moveExecutor.execute(new Runnable() {
        @Override
        public void run() {
@@@ -996,11 -1020,8 +1080,8 @@@
      return getBytesMoved() - bytesLastMoved;
    }
  
-   /** The sleeping period before checking if reportedBlock move is completed again */
-   static private long blockMoveWaitTime = 30000L;
- 
    /**
 -   * Wait for all block move confirmations.
 +   * Wait for all reportedBlock move confirmations.
     * @return true if there is failed move execution
     */
    public static boolean waitForMoveCompletion(
@@@ -1027,10 -1048,22 +1108,22 @@@
    }
  
    /**
+    * @return true if some moves are success.
+    */
+   public static boolean checkForSuccess(
+       Iterable<? extends StorageGroup> targets) {
+     boolean hasSuccess = false;
+     for (StorageGroup t : targets) {
+       hasSuccess |= t.getDDatanode().hasSuccess;
+     }
+     return hasSuccess;
+   }
+ 
+   /**
 -   * Decide if the block is a good candidate to be moved from source to target.
 -   * A block is a good candidate if
 +   * Decide if the block/blockGroup is a good candidate to be moved from source
 +   * to target. A block is a good candidate if
     * 1. the block is not in the process of being moved/has not been moved;
 -   * 2. the block does not have a replica on the target;
 +   * 2. the block does not have a replica/internalBlock on the target;
     * 3. doing the move does not reduce the number of racks that the block has
     */
    private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index 4308278,dea31c4..bf11914
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@@ -19,19 -19,21 +19,20 @@@ package org.apache.hadoop.hdfs.server.b
  
  import java.util.LinkedList;
  
 -import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.hdfs.protocol.Block;
- import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
  import org.apache.hadoop.util.LightWeightGSet;
  
  /**
 - * BlockInfo class maintains for a given block
 - * the {@link BlockCollection} it is part of and datanodes where the replicas of
 - * the block are stored.
 + * For a given block (or an erasure coding block group), BlockInfo class
 + * maintains 1) the {@link BlockCollection} it is part of, and 2) datanodes
 + * where the replicas of the block, or blocks belonging to the erasure coding
 + * block group, are stored.
   */
 -@InterfaceAudience.Private
 -public abstract class  BlockInfo extends Block
 +public abstract class BlockInfo extends Block
      implements LightWeightGSet.LinkedElement {
    public static final BlockInfo[] EMPTY_ARRAY = {};
+ 
    private BlockCollection bc;
  
    /** For implementing {@link LightWeightGSet.LinkedElement} interface */
@@@ -177,27 -188,7 +178,12 @@@
     */
    abstract void replaceBlock(BlockInfo newBlock);
  
 +  public abstract boolean isStriped();
 +
 +  /** @return true if there is no datanode storage associated with the block */
 +  abstract boolean hasNoStorage();
 +
    /**
-    * Find specified DatanodeDescriptor.
-    * @return index or -1 if not found.
-    */
-   boolean findDatanode(DatanodeDescriptor dn) {
-     int len = getCapacity();
-     for (int idx = 0; idx < len; idx++) {
-       DatanodeDescriptor cur = getDatanode(idx);
-       if(cur == dn) {
-         return true;
-       }
-     }
-     return false;
-   }
- 
-   /**
     * Find specified DatanodeStorageInfo.
     * @return DatanodeStorageInfo or null if not found.
     */
@@@ -303,27 -294,43 +289,26 @@@
  
    /**
     * BlockInfo represents a block that is not being constructed.
 -   * In order to start modifying the block, the BlockInfo should be converted
 -   * to {@link BlockInfoContiguousUnderConstruction}.
 +   * In order to start modifying the block, the BlockInfo should be converted to
-    * {@link BlockInfoUnderConstructionContiguous} or
-    * {@link BlockInfoUnderConstructionStriped}.
-    * @return {@link HdfsServerConstants.BlockUCState#COMPLETE}
++   * {@link BlockInfoContiguousUnderConstruction} or
++   * {@link BlockInfoStripedUnderConstruction}.
+    * @return {@link BlockUCState#COMPLETE}
     */
-   public HdfsServerConstants.BlockUCState getBlockUCState() {
-     return HdfsServerConstants.BlockUCState.COMPLETE;
+   public BlockUCState getBlockUCState() {
+     return BlockUCState.COMPLETE;
    }
  
    /**
     * Is this block complete?
     *
-    * @return true if the state of the block is
-    *         {@link HdfsServerConstants.BlockUCState#COMPLETE}
+    * @return true if the state of the block is {@link BlockUCState#COMPLETE}
     */
    public boolean isComplete() {
-     return getBlockUCState().equals(HdfsServerConstants.BlockUCState.COMPLETE);
+     return getBlockUCState().equals(BlockUCState.COMPLETE);
    }
  
 -  /**
 -   * Convert a complete block to an under construction block.
 -   * @return BlockInfoUnderConstruction -  an under construction block.
 -   */
 -  public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction(
 -      BlockUCState s, DatanodeStorageInfo[] targets) {
 -    if(isComplete()) {
 -      BlockInfoContiguousUnderConstruction ucBlock =
 -          new BlockInfoContiguousUnderConstruction(this,
 -          getBlockCollection().getPreferredBlockReplication(), s, targets);
 -      ucBlock.setBlockCollection(getBlockCollection());
 -      return ucBlock;
 -    }
 -    // the block is already under construction
 -    BlockInfoContiguousUnderConstruction ucBlock =
 -        (BlockInfoContiguousUnderConstruction)this;
 -    ucBlock.setBlockUCState(s);
 -    ucBlock.setExpectedLocations(targets);
 -    ucBlock.setBlockCollection(getBlockCollection());
 -    return ucBlock;
 +  public boolean isDeleted() {
 +    return (bc == null);
    }
  
    @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index d9adccc,eff89a8..bb9bf5b
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@@ -122,36 -120,4 +122,36 @@@ public class BlockInfoContiguous extend
            "newBlock already exists.";
      }
    }
 +
 +  /**
 +   * Convert a complete block to an under construction block.
 +   * @return BlockInfoUnderConstruction -  an under construction block.
 +   */
-   public BlockInfoUnderConstructionContiguous convertToBlockUnderConstruction(
++  public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction(
 +      BlockUCState s, DatanodeStorageInfo[] targets) {
 +    if(isComplete()) {
-       BlockInfoUnderConstructionContiguous ucBlock =
-           new BlockInfoUnderConstructionContiguous(this,
++      BlockInfoContiguousUnderConstruction ucBlock =
++          new BlockInfoContiguousUnderConstruction(this,
 +          getBlockCollection().getPreferredBlockReplication(), s, targets);
 +      ucBlock.setBlockCollection(getBlockCollection());
 +      return ucBlock;
 +    }
 +    // the block is already under construction
-     BlockInfoUnderConstructionContiguous ucBlock =
-         (BlockInfoUnderConstructionContiguous) this;
++    BlockInfoContiguousUnderConstruction ucBlock =
++        (BlockInfoContiguousUnderConstruction) this;
 +    ucBlock.setBlockUCState(s);
 +    ucBlock.setExpectedLocations(targets);
 +    ucBlock.setBlockCollection(getBlockCollection());
 +    return ucBlock;
 +  }
 +
 +  @Override
 +  public final boolean isStriped() {
 +    return false;
 +  }
 +
 +  @Override
 +  final boolean hasNoStorage() {
 +    return getStorageInfo(0) == null;
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
index 0000000,7ca6419..96b209d
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
@@@ -1,0 -1,403 +1,281 @@@
+ /**
+  * 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.hdfs.server.blockmanagement;
+ 
+ import java.io.IOException;
+ import java.util.ArrayList;
+ import java.util.Iterator;
+ import java.util.List;
+ 
+ import org.apache.hadoop.hdfs.protocol.Block;
+ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+ import org.apache.hadoop.hdfs.server.namenode.NameNode;
+ 
+ /**
+  * Represents a block that is currently being constructed.<br>
+  * This is usually the last block of a file opened for write or append.
+  */
 -public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
++public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous
++    implements BlockInfoUnderConstruction{
+   /** Block state. See {@link BlockUCState} */
+   private BlockUCState blockUCState;
+ 
+   /**
+    * Block replicas as assigned when the block was allocated.
+    * This defines the pipeline order.
+    */
+   private List<ReplicaUnderConstruction> replicas;
+ 
+   /**
+    * Index of the primary data node doing the recovery. Useful for log
+    * messages.
+    */
+   private int primaryNodeIndex = -1;
+ 
+   /**
+    * The new generation stamp, which this block will have
+    * after the recovery succeeds. Also used as a recovery id to identify
+    * the right recovery if any of the abandoned recoveries re-appear.
+    */
+   private long blockRecoveryId = 0;
+ 
+   /**
+    * The block source to use in the event of copy-on-write truncate.
+    */
+   private Block truncateBlock;
+ 
+   /**
 -   * ReplicaUnderConstruction contains information about replicas while
 -   * they are under construction.
 -   * The GS, the length and the state of the replica is as reported by 
 -   * the data-node.
 -   * It is not guaranteed, but expected, that data-nodes actually have
 -   * corresponding replicas.
 -   */
 -  static class ReplicaUnderConstruction extends Block {
 -    private final DatanodeStorageInfo expectedLocation;
 -    private ReplicaState state;
 -    private boolean chosenAsPrimary;
 -
 -    ReplicaUnderConstruction(Block block,
 -                             DatanodeStorageInfo target,
 -                             ReplicaState state) {
 -      super(block);
 -      this.expectedLocation = target;
 -      this.state = state;
 -      this.chosenAsPrimary = false;
 -    }
 -
 -    /**
 -     * Expected block replica location as assigned when the block was allocated.
 -     * This defines the pipeline order.
 -     * It is not guaranteed, but expected, that the data-node actually has
 -     * the replica.
 -     */
 -    private DatanodeStorageInfo getExpectedStorageLocation() {
 -      return expectedLocation;
 -    }
 -
 -    /**
 -     * Get replica state as reported by the data-node.
 -     */
 -    ReplicaState getState() {
 -      return state;
 -    }
 -
 -    /**
 -     * Whether the replica was chosen for recovery.
 -     */
 -    boolean getChosenAsPrimary() {
 -      return chosenAsPrimary;
 -    }
 -
 -    /**
 -     * Set replica state.
 -     */
 -    void setState(ReplicaState s) {
 -      state = s;
 -    }
 -
 -    /**
 -     * Set whether this replica was chosen for recovery.
 -     */
 -    void setChosenAsPrimary(boolean chosenAsPrimary) {
 -      this.chosenAsPrimary = chosenAsPrimary;
 -    }
 -
 -    /**
 -     * Is data-node the replica belongs to alive.
 -     */
 -    boolean isAlive() {
 -      return expectedLocation.getDatanodeDescriptor().isAlive;
 -    }
 -
 -    @Override // Block
 -    public int hashCode() {
 -      return super.hashCode();
 -    }
 -
 -    @Override // Block
 -    public boolean equals(Object obj) {
 -      // Sufficient to rely on super's implementation
 -      return (this == obj) || super.equals(obj);
 -    }
 -
 -    @Override
 -    public String toString() {
 -      final StringBuilder b = new StringBuilder(50);
 -      appendStringTo(b);
 -      return b.toString();
 -    }
 -    
 -    @Override
 -    public void appendStringTo(StringBuilder sb) {
 -      sb.append("ReplicaUC[")
 -        .append(expectedLocation)
 -        .append("|")
 -        .append(state)
 -        .append("]");
 -    }
 -  }
 -
 -  /**
+    * Create block and set its state to
+    * {@link BlockUCState#UNDER_CONSTRUCTION}.
+    */
+   public BlockInfoContiguousUnderConstruction(Block blk, short replication) {
+     this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
+   }
+ 
+   /**
+    * Create a block that is currently being constructed.
+    */
+   public BlockInfoContiguousUnderConstruction(Block blk, short replication,
+       BlockUCState state, DatanodeStorageInfo[] targets) {
+     super(blk, replication);
+     assert getBlockUCState() != BlockUCState.COMPLETE :
 -      "BlockInfoUnderConstruction cannot be in COMPLETE state";
++      "BlockInfoContiguousUnderConstruction cannot be in COMPLETE state";
+     this.blockUCState = state;
+     setExpectedLocations(targets);
+   }
+ 
 -  /**
 -   * Convert an under construction block to a complete block.
 -   * 
 -   * @return BlockInfo - a complete block.
 -   * @throws IOException if the state of the block 
 -   * (the generation stamp and the length) has not been committed by 
 -   * the client or it does not have at least a minimal number of replicas 
 -   * reported from data-nodes. 
 -   */
 -  BlockInfo convertToCompleteBlock() throws IOException {
++  @Override
++  public BlockInfoContiguous convertToCompleteBlock() throws IOException {
+     assert getBlockUCState() != BlockUCState.COMPLETE :
+       "Trying to convert a COMPLETE block";
+     return new BlockInfoContiguous(this);
+   }
+ 
 -  /** Set expected locations */
++  @Override
+   public void setExpectedLocations(DatanodeStorageInfo[] targets) {
+     int numLocations = targets == null ? 0 : targets.length;
 -    this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
 -    for(int i = 0; i < numLocations; i++)
 -      replicas.add(
 -        new ReplicaUnderConstruction(this, targets[i], ReplicaState.RBW));
++    this.replicas = new ArrayList<>(numLocations);
++    for(int i = 0; i < numLocations; i++) {
++      replicas.add(new ReplicaUnderConstruction(this, targets[i],
++          ReplicaState.RBW));
++    }
+   }
+ 
 -  /**
 -   * Create array of expected replica locations
 -   * (as has been assigned by chooseTargets()).
 -   */
++  @Override
+   public DatanodeStorageInfo[] getExpectedStorageLocations() {
+     int numLocations = replicas == null ? 0 : replicas.size();
+     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
 -    for(int i = 0; i < numLocations; i++)
++    for (int i = 0; i < numLocations; i++) {
+       storages[i] = replicas.get(i).getExpectedStorageLocation();
++    }
+     return storages;
+   }
+ 
 -  /** Get the number of expected locations */
++  @Override
+   public int getNumExpectedLocations() {
+     return replicas == null ? 0 : replicas.size();
+   }
+ 
+   /**
+    * Return the state of the block under construction.
+    * @see BlockUCState
+    */
+   @Override // BlockInfo
+   public BlockUCState getBlockUCState() {
+     return blockUCState;
+   }
+ 
+   void setBlockUCState(BlockUCState s) {
+     blockUCState = s;
+   }
+ 
 -  /** Get block recovery ID */
++  @Override
+   public long getBlockRecoveryId() {
+     return blockRecoveryId;
+   }
+ 
 -  /** Get recover block */
++  @Override
+   public Block getTruncateBlock() {
+     return truncateBlock;
+   }
+ 
++  @Override
++  public Block toBlock(){
++    return this;
++  }
++
+   public void setTruncateBlock(Block recoveryBlock) {
+     this.truncateBlock = recoveryBlock;
+   }
+ 
 -  /**
 -   * Process the recorded replicas. When about to commit or finish the
 -   * pipeline recovery sort out bad replicas.
 -   * @param genStamp  The final generation stamp for the block.
 -   */
++  @Override
+   public void setGenerationStampAndVerifyReplicas(long genStamp) {
+     // Set the generation stamp for the block.
+     setGenerationStamp(genStamp);
+     if (replicas == null)
+       return;
+ 
+     // Remove the replicas with wrong gen stamp.
+     // The replica list is unchanged.
+     for (ReplicaUnderConstruction r : replicas) {
+       if (genStamp != r.getGenerationStamp()) {
+         r.getExpectedStorageLocation().removeBlock(this);
+         NameNode.blockStateChangeLog.debug("BLOCK* Removing stale replica "
+             + "from location: {}", r.getExpectedStorageLocation());
+       }
+     }
+   }
+ 
 -  /**
 -   * Commit block's length and generation stamp as reported by the client.
 -   * Set block state to {@link BlockUCState#COMMITTED}.
 -   * @param block - contains client reported block length and generation 
 -   * @throws IOException if block ids are inconsistent.
 -   */
 -  void commitBlock(Block block) throws IOException {
++  @Override
++  public void commitBlock(Block block) throws IOException {
+     if(getBlockId() != block.getBlockId())
+       throw new IOException("Trying to commit inconsistent block: id = "
+           + block.getBlockId() + ", expected id = " + getBlockId());
+     blockUCState = BlockUCState.COMMITTED;
+     this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
+     // Sort out invalid replicas.
+     setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
+   }
+ 
 -  /**
 -   * Initialize lease recovery for this block.
 -   * Find the first alive data-node starting from the previous primary and
 -   * make it primary.
 -   */
++  @Override
+   public void initializeBlockRecovery(long recoveryId) {
+     setBlockUCState(BlockUCState.UNDER_RECOVERY);
+     blockRecoveryId = recoveryId;
+     if (replicas.size() == 0) {
+       NameNode.blockStateChangeLog.warn("BLOCK*"
 -        + " BlockInfoUnderConstruction.initLeaseRecovery:"
++        + " BlockInfoContiguousUnderConstruction.initLeaseRecovery:"
+         + " No blocks found, lease removed.");
+     }
+     boolean allLiveReplicasTriedAsPrimary = true;
 -    for (int i = 0; i < replicas.size(); i++) {
++    for (ReplicaUnderConstruction replica : replicas) {
+       // Check if all replicas have been tried or not.
 -      if (replicas.get(i).isAlive()) {
 -        allLiveReplicasTriedAsPrimary =
 -            (allLiveReplicasTriedAsPrimary && replicas.get(i).getChosenAsPrimary());
++      if (replica.isAlive()) {
++        allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary &&
++            replica.getChosenAsPrimary());
+       }
+     }
+     if (allLiveReplicasTriedAsPrimary) {
+       // Just set all the replicas to be chosen whether they are alive or not.
 -      for (int i = 0; i < replicas.size(); i++) {
 -        replicas.get(i).setChosenAsPrimary(false);
++      for (ReplicaUnderConstruction replica : replicas) {
++        replica.setChosenAsPrimary(false);
+       }
+     }
+     long mostRecentLastUpdate = 0;
+     ReplicaUnderConstruction primary = null;
+     primaryNodeIndex = -1;
+     for(int i = 0; i < replicas.size(); i++) {
+       // Skip alive replicas which have been chosen for recovery.
+       if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
+         continue;
+       }
+       final ReplicaUnderConstruction ruc = replicas.get(i);
+       final long lastUpdate = ruc.getExpectedStorageLocation()
+           .getDatanodeDescriptor().getLastUpdateMonotonic();
+       if (lastUpdate > mostRecentLastUpdate) {
+         primaryNodeIndex = i;
+         primary = ruc;
+         mostRecentLastUpdate = lastUpdate;
+       }
+     }
+     if (primary != null) {
 -      primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
++      primary.getExpectedStorageLocation().getDatanodeDescriptor()
++          .addBlockToBeRecovered(this);
+       primary.setChosenAsPrimary(true);
+       NameNode.blockStateChangeLog.debug(
+           "BLOCK* {} recovery started, primary={}", this, primary);
+     }
+   }
+ 
 -  void addReplicaIfNotPresent(DatanodeStorageInfo storage,
 -                     Block block,
 -                     ReplicaState rState) {
++  @Override
++  public void addReplicaIfNotPresent(DatanodeStorageInfo storage,
++      Block block, ReplicaState rState) {
+     Iterator<ReplicaUnderConstruction> it = replicas.iterator();
+     while (it.hasNext()) {
+       ReplicaUnderConstruction r = it.next();
+       DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
+       if(expectedLocation == storage) {
+         // Record the gen stamp from the report
+         r.setGenerationStamp(block.getGenerationStamp());
+         return;
+       } else if (expectedLocation != null &&
+                  expectedLocation.getDatanodeDescriptor() ==
+                      storage.getDatanodeDescriptor()) {
+ 
+         // The Datanode reported that the block is on a different storage
+         // than the one chosen by BlockPlacementPolicy. This can occur as
+         // we allow Datanodes to choose the target storage. Update our
+         // state by removing the stale entry and adding a new one.
+         it.remove();
+         break;
+       }
+     }
+     replicas.add(new ReplicaUnderConstruction(block, storage, rState));
+   }
+ 
 -  @Override // BlockInfo
 -  // BlockInfoUnderConstruction participates in maps the same way as BlockInfo
 -  public int hashCode() {
 -    return super.hashCode();
 -  }
 -
 -  @Override // BlockInfo
 -  public boolean equals(Object obj) {
 -    // Sufficient to rely on super's implementation
 -    return (this == obj) || super.equals(obj);
 -  }
 -
+   @Override
+   public String toString() {
+     final StringBuilder b = new StringBuilder(100);
+     appendStringTo(b);
+     return b.toString();
+   }
+ 
+   @Override
+   public void appendStringTo(StringBuilder sb) {
+     super.appendStringTo(sb);
+     appendUCParts(sb);
+   }
+ 
+   private void appendUCParts(StringBuilder sb) {
+     sb.append("{UCState=").append(blockUCState)
+       .append(", truncateBlock=" + truncateBlock)
+       .append(", primaryNodeIndex=").append(primaryNodeIndex)
+       .append(", replicas=[");
+     if (replicas != null) {
+       Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
+       if (iter.hasNext()) {
+         iter.next().appendStringTo(sb);
+         while (iter.hasNext()) {
+           sb.append(", ");
+           iter.next().appendStringTo(sb);
+         }
+       }
+     }
+     sb.append("]}");
+   }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index b88b554,0000000..14d2fcc
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@@ -1,279 -1,0 +1,279 @@@
 +/**
 + * 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.hdfs.server.blockmanagement;
 +
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +
 +/**
 + * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
 + *
 + * We still use triplets to store DatanodeStorageInfo for each block in the
 + * block group, as well as the previous/next block in the corresponding
 + * DatanodeStorageInfo. For a (m+k) block group, the first (m+k) triplet units
 + * are sorted and strictly mapped to the corresponding block.
 + *
 + * Normally each block belonging to group is stored in only one DataNode.
 + * However, it is possible that some block is over-replicated. Thus the triplet
 + * array's size can be larger than (m+k). Thus currently we use an extra byte
 + * array to record the block index for each triplet.
 + */
 +public class BlockInfoStriped extends BlockInfo {
 +  private final ErasureCodingPolicy ecPolicy;
 +  /**
 +   * Always the same size with triplets. Record the block index for each triplet
 +   * TODO: actually this is only necessary for over-replicated block. Thus can
 +   * be further optimized to save memory usage.
 +   */
 +  private byte[] indices;
 +
 +  public BlockInfoStriped(Block blk, ErasureCodingPolicy ecPolicy) {
 +    super(blk, (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()));
 +    indices = new byte[ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()];
 +    initIndices();
 +    this.ecPolicy = ecPolicy;
 +  }
 +
 +  BlockInfoStriped(BlockInfoStriped b) {
 +    this(b, b.getErasureCodingPolicy());
 +    this.setBlockCollection(b.getBlockCollection());
 +  }
 +
 +  public short getTotalBlockNum() {
 +    return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits());
 +  }
 +
 +  public short getDataBlockNum() {
 +    return (short) ecPolicy.getNumDataUnits();
 +  }
 +
 +  public short getParityBlockNum() {
 +    return (short) ecPolicy.getNumParityUnits();
 +  }
 +
 +  /**
 +   * If the block is committed/completed and its length is less than a full
 +   * stripe, it returns the the number of actual data blocks.
 +   * Otherwise it returns the number of data units specified by erasure coding policy.
 +   */
 +  public short getRealDataBlockNum() {
 +    if (isComplete() || getBlockUCState() == BlockUCState.COMMITTED) {
 +      return (short) Math.min(getDataBlockNum(),
 +          (getNumBytes() - 1) / BLOCK_STRIPED_CELL_SIZE + 1);
 +    } else {
 +      return getDataBlockNum();
 +    }
 +  }
 +
 +  public short getRealTotalBlockNum() {
 +    return (short) (getRealDataBlockNum() + getParityBlockNum());
 +  }
 +
 +  public ErasureCodingPolicy getErasureCodingPolicy() {
 +    return ecPolicy;
 +  }
 +
 +  private void initIndices() {
 +    for (int i = 0; i < indices.length; i++) {
 +      indices[i] = -1;
 +    }
 +  }
 +
 +  private int findSlot() {
 +    int i = getTotalBlockNum();
 +    for (; i < getCapacity(); i++) {
 +      if (getStorageInfo(i) == null) {
 +        return i;
 +      }
 +    }
 +    // need to expand the triplet size
 +    ensureCapacity(i + 1, true);
 +    return i;
 +  }
 +
 +  @Override
 +  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
 +    int blockIndex = BlockIdManager.getBlockIndex(reportedBlock);
 +    int index = blockIndex;
 +    DatanodeStorageInfo old = getStorageInfo(index);
 +    if (old != null && !old.equals(storage)) { // over replicated
 +      // check if the storage has been stored
 +      int i = findStorageInfo(storage);
 +      if (i == -1) {
 +        index = findSlot();
 +      } else {
 +        return true;
 +      }
 +    }
 +    addStorage(storage, index, blockIndex);
 +    return true;
 +  }
 +
 +  private void addStorage(DatanodeStorageInfo storage, int index,
 +      int blockIndex) {
 +    setStorageInfo(index, storage);
 +    setNext(index, null);
 +    setPrevious(index, null);
 +    indices[index] = (byte) blockIndex;
 +  }
 +
 +  private int findStorageInfoFromEnd(DatanodeStorageInfo storage) {
 +    final int len = getCapacity();
 +    for(int idx = len - 1; idx >= 0; idx--) {
 +      DatanodeStorageInfo cur = getStorageInfo(idx);
 +      if (storage.equals(cur)) {
 +        return idx;
 +      }
 +    }
 +    return -1;
 +  }
 +
 +  int getStorageBlockIndex(DatanodeStorageInfo storage) {
 +    int i = this.findStorageInfo(storage);
 +    return i == -1 ? -1 : indices[i];
 +  }
 +
 +  /**
 +   * Identify the block stored in the given datanode storage. Note that
 +   * the returned block has the same block Id with the one seen/reported by the
 +   * DataNode.
 +   */
 +  Block getBlockOnStorage(DatanodeStorageInfo storage) {
 +    int index = getStorageBlockIndex(storage);
 +    if (index < 0) {
 +      return null;
 +    } else {
 +      Block block = new Block(this);
 +      block.setBlockId(this.getBlockId() + index);
 +      return block;
 +    }
 +  }
 +
 +  @Override
 +  boolean removeStorage(DatanodeStorageInfo storage) {
 +    int dnIndex = findStorageInfoFromEnd(storage);
 +    if (dnIndex < 0) { // the node is not found
 +      return false;
 +    }
 +    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
 +        "Block is still in the list and must be removed first.";
 +    // set the triplet to null
 +    setStorageInfo(dnIndex, null);
 +    setNext(dnIndex, null);
 +    setPrevious(dnIndex, null);
 +    indices[dnIndex] = -1;
 +    return true;
 +  }
 +
 +  private void ensureCapacity(int totalSize, boolean keepOld) {
 +    if (getCapacity() < totalSize) {
 +      Object[] old = triplets;
 +      byte[] oldIndices = indices;
 +      triplets = new Object[totalSize * 3];
 +      indices = new byte[totalSize];
 +      initIndices();
 +
 +      if (keepOld) {
 +        System.arraycopy(old, 0, triplets, 0, old.length);
 +        System.arraycopy(oldIndices, 0, indices, 0, oldIndices.length);
 +      }
 +    }
 +  }
 +
 +  @Override
 +  void replaceBlock(BlockInfo newBlock) {
 +    assert newBlock instanceof BlockInfoStriped;
 +    BlockInfoStriped newBlockGroup = (BlockInfoStriped) newBlock;
 +    final int size = getCapacity();
 +    newBlockGroup.ensureCapacity(size, false);
 +    for (int i = 0; i < size; i++) {
 +      final DatanodeStorageInfo storage = this.getStorageInfo(i);
 +      if (storage != null) {
 +        final int blockIndex = indices[i];
 +        final boolean removed = storage.removeBlock(this);
 +        assert removed : "currentBlock not found.";
 +
 +        newBlockGroup.addStorage(storage, i, blockIndex);
 +        storage.insertToList(newBlockGroup);
 +      }
 +    }
 +  }
 +
 +  public long spaceConsumed() {
 +    // In case striped blocks, total usage by this striped blocks should
 +    // be the total of data blocks and parity blocks because
 +    // `getNumBytes` is the total of actual data block size.
 +    return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(),
 +        ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits(),
 +        BLOCK_STRIPED_CELL_SIZE);
 +    }
 +
 +  @Override
 +  public final boolean isStriped() {
 +    return true;
 +  }
 +
 +  @Override
 +  public int numNodes() {
 +    assert this.triplets != null : "BlockInfo is not initialized";
 +    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
 +    int num = 0;
 +    for (int idx = getCapacity()-1; idx >= 0; idx--) {
 +      if (getStorageInfo(idx) != null) {
 +        num++;
 +      }
 +    }
 +    return num;
 +  }
 +
 +  /**
 +   * Convert a complete block to an under construction block.
 +   * @return BlockInfoUnderConstruction -  an under construction block.
 +   */
-   public BlockInfoUnderConstructionStriped convertToBlockUnderConstruction(
++  public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction(
 +      BlockUCState s, DatanodeStorageInfo[] targets) {
-     final BlockInfoUnderConstructionStriped ucBlock;
++    final BlockInfoStripedUnderConstruction ucBlock;
 +    if(isComplete()) {
-       ucBlock = new BlockInfoUnderConstructionStriped(this, ecPolicy,
++      ucBlock = new BlockInfoStripedUnderConstruction(this, ecPolicy,
 +          s, targets);
 +      ucBlock.setBlockCollection(getBlockCollection());
 +    } else {
 +      // the block is already under construction
-       ucBlock = (BlockInfoUnderConstructionStriped) this;
++      ucBlock = (BlockInfoStripedUnderConstruction) this;
 +      ucBlock.setBlockUCState(s);
 +      ucBlock.setExpectedLocations(targets);
 +      ucBlock.setBlockCollection(getBlockCollection());
 +    }
 +    return ucBlock;
 +  }
 +
 +  @Override
 +  final boolean hasNoStorage() {
 +    final int len = getCapacity();
 +    for(int idx = 0; idx < len; idx++) {
 +      if (getStorageInfo(idx) != null) {
 +        return false;
 +      }
 +    }
 +    return true;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
index 0000000,0000000..9de8294
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
@@@ -1,0 -1,0 +1,297 @@@
++/**
++ * 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.hdfs.server.blockmanagement;
++
++import org.apache.hadoop.hdfs.protocol.Block;
++import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
++import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
++import org.apache.hadoop.hdfs.server.namenode.NameNode;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
++
++import java.io.IOException;
++
++import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE;
++import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
++
++/**
++ * Represents a striped block that is currently being constructed.
++ * This is usually the last block of a file opened for write or append.
++ */
++public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
++    implements BlockInfoUnderConstruction{
++  private BlockUCState blockUCState;
++
++  /**
++   * Block replicas as assigned when the block was allocated.
++   */
++  private ReplicaUnderConstruction[] replicas;
++
++  /**
++   * Index of the primary data node doing the recovery. Useful for log
++   * messages.
++   */
++  private int primaryNodeIndex = -1;
++
++  /**
++   * The new generation stamp, which this block will have
++   * after the recovery succeeds. Also used as a recovery id to identify
++   * the right recovery if any of the abandoned recoveries re-appear.
++   */
++  private long blockRecoveryId = 0;
++
++  /**
++   * Constructor with null storage targets.
++   */
++  public BlockInfoStripedUnderConstruction(Block blk, ErasureCodingPolicy ecPolicy) {
++    this(blk, ecPolicy, UNDER_CONSTRUCTION, null);
++  }
++
++  /**
++   * Create a striped block that is currently being constructed.
++   */
++  public BlockInfoStripedUnderConstruction(Block blk, ErasureCodingPolicy ecPolicy,
++      BlockUCState state, DatanodeStorageInfo[] targets) {
++    super(blk, ecPolicy);
++    assert getBlockUCState() != COMPLETE :
++      "BlockInfoStripedUnderConstruction cannot be in COMPLETE state";
++    this.blockUCState = state;
++    setExpectedLocations(targets);
++  }
++
++  @Override
++  public BlockInfoStriped convertToCompleteBlock() throws IOException {
++    assert getBlockUCState() != COMPLETE :
++      "Trying to convert a COMPLETE block";
++    return new BlockInfoStriped(this);
++  }
++
++  /** Set expected locations */
++  @Override
++  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
++    int numLocations = targets == null ? 0 : targets.length;
++    this.replicas = new ReplicaUnderConstruction[numLocations];
++    for(int i = 0; i < numLocations; i++) {
++      // when creating a new block we simply sequentially assign block index to
++      // each storage
++      Block blk = new Block(this.getBlockId() + i, 0, this.getGenerationStamp());
++      replicas[i] = new ReplicaUnderConstruction(blk, targets[i],
++          ReplicaState.RBW);
++    }
++  }
++
++  /**
++   * Create array of expected replica locations
++   * (as has been assigned by chooseTargets()).
++   */
++  @Override
++  public DatanodeStorageInfo[] getExpectedStorageLocations() {
++    int numLocations = getNumExpectedLocations();
++    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
++    for (int i = 0; i < numLocations; i++) {
++      storages[i] = replicas[i].getExpectedStorageLocation();
++    }
++    return storages;
++  }
++
++  /** @return the index array indicating the block index in each storage */
++  public int[] getBlockIndices() {
++    int numLocations = getNumExpectedLocations();
++    int[] indices = new int[numLocations];
++    for (int i = 0; i < numLocations; i++) {
++      indices[i] = BlockIdManager.getBlockIndex(replicas[i]);
++    }
++    return indices;
++  }
++
++  @Override
++  public int getNumExpectedLocations() {
++    return replicas == null ? 0 : replicas.length;
++  }
++
++  /**
++   * Return the state of the block under construction.
++   * @see BlockUCState
++   */
++  @Override // BlockInfo
++  public BlockUCState getBlockUCState() {
++    return blockUCState;
++  }
++
++  void setBlockUCState(BlockUCState s) {
++    blockUCState = s;
++  }
++
++  @Override
++  public long getBlockRecoveryId() {
++    return blockRecoveryId;
++  }
++
++  @Override
++  public Block getTruncateBlock() {
++    return null;
++  }
++
++  @Override
++  public Block toBlock(){
++    return this;
++  }
++
++  @Override
++  public void setGenerationStampAndVerifyReplicas(long genStamp) {
++    // Set the generation stamp for the block.
++    setGenerationStamp(genStamp);
++    if (replicas == null)
++      return;
++
++    // Remove the replicas with wrong gen stamp.
++    // The replica list is unchanged.
++    for (ReplicaUnderConstruction r : replicas) {
++      if (genStamp != r.getGenerationStamp()) {
++        r.getExpectedStorageLocation().removeBlock(this);
++        NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
++            + "from location: {}", r.getExpectedStorageLocation());
++      }
++    }
++  }
++
++  @Override
++  public void commitBlock(Block block) throws IOException {
++    if (getBlockId() != block.getBlockId()) {
++      throw new IOException("Trying to commit inconsistent block: id = "
++          + block.getBlockId() + ", expected id = " + getBlockId());
++    }
++    blockUCState = BlockUCState.COMMITTED;
++    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
++    // Sort out invalid replicas.
++    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
++  }
++
++  @Override
++  public void initializeBlockRecovery(long recoveryId) {
++    setBlockUCState(BlockUCState.UNDER_RECOVERY);
++    blockRecoveryId = recoveryId;
++    if (replicas == null || replicas.length == 0) {
++      NameNode.blockStateChangeLog.warn("BLOCK*" +
++          " BlockInfoStripedUnderConstruction.initLeaseRecovery:" +
++          " No blocks found, lease removed.");
++      // sets primary node index and return.
++      primaryNodeIndex = -1;
++      return;
++    }
++    boolean allLiveReplicasTriedAsPrimary = true;
++    for (ReplicaUnderConstruction replica : replicas) {
++      // Check if all replicas have been tried or not.
++      if (replica.isAlive()) {
++        allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary &&
++            replica.getChosenAsPrimary());
++      }
++    }
++    if (allLiveReplicasTriedAsPrimary) {
++      // Just set all the replicas to be chosen whether they are alive or not.
++      for (ReplicaUnderConstruction replica : replicas) {
++        replica.setChosenAsPrimary(false);
++      }
++    }
++    long mostRecentLastUpdate = 0;
++    ReplicaUnderConstruction primary = null;
++    primaryNodeIndex = -1;
++    for(int i = 0; i < replicas.length; i++) {
++      // Skip alive replicas which have been chosen for recovery.
++      if (!(replicas[i].isAlive() && !replicas[i].getChosenAsPrimary())) {
++        continue;
++      }
++      final ReplicaUnderConstruction ruc = replicas[i];
++      final long lastUpdate = ruc.getExpectedStorageLocation()
++          .getDatanodeDescriptor().getLastUpdateMonotonic();
++      if (lastUpdate > mostRecentLastUpdate) {
++        primaryNodeIndex = i;
++        primary = ruc;
++        mostRecentLastUpdate = lastUpdate;
++      }
++    }
++    if (primary != null) {
++      primary.getExpectedStorageLocation().getDatanodeDescriptor()
++          .addBlockToBeRecovered(this);
++      primary.setChosenAsPrimary(true);
++      NameNode.blockStateChangeLog.info(
++          "BLOCK* {} recovery started, primary={}", this, primary);
++    }
++  }
++
++  @Override
++  public void addReplicaIfNotPresent(DatanodeStorageInfo storage,
++      Block reportedBlock, ReplicaState rState) {
++    if (replicas == null) {
++      replicas = new ReplicaUnderConstruction[1];
++      replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage, rState);
++    } else {
++      for (int i = 0; i < replicas.length; i++) {
++        DatanodeStorageInfo expected = replicas[i].getExpectedStorageLocation();
++        if (expected == storage) {
++          replicas[i].setBlockId(reportedBlock.getBlockId());
++          replicas[i].setGenerationStamp(reportedBlock.getGenerationStamp());
++          return;
++        } else if (expected != null && expected.getDatanodeDescriptor() ==
++            storage.getDatanodeDescriptor()) {
++          // The Datanode reported that the block is on a different storage
++          // than the one chosen by BlockPlacementPolicy. This can occur as
++          // we allow Datanodes to choose the target storage. Update our
++          // state by removing the stale entry and adding a new one.
++          replicas[i] = new ReplicaUnderConstruction(reportedBlock, storage,
++              rState);
++          return;
++        }
++      }
++      ReplicaUnderConstruction[] newReplicas =
++          new ReplicaUnderConstruction[replicas.length + 1];
++      System.arraycopy(replicas, 0, newReplicas, 0, replicas.length);
++      newReplicas[newReplicas.length - 1] = new ReplicaUnderConstruction(
++          reportedBlock, storage, rState);
++      replicas = newReplicas;
++    }
++  }
++
++  @Override
++  public String toString() {
++    final StringBuilder b = new StringBuilder(100);
++    appendStringTo(b);
++    return b.toString();
++  }
++
++  @Override
++  public void appendStringTo(StringBuilder sb) {
++    super.appendStringTo(sb);
++    appendUCParts(sb);
++  }
++
++  private void appendUCParts(StringBuilder sb) {
++    sb.append("{UCState=").append(blockUCState).
++        append(", primaryNodeIndex=").append(primaryNodeIndex).
++        append(", replicas=[");
++    if (replicas != null) {
++      int i = 0;
++      for (ReplicaUnderConstruction r : replicas) {
++        r.appendStringTo(sb);
++        if (++i < replicas.length) {
++          sb.append(", ");
++        }
++      }
++    }
++    sb.append("]}");
++  }
++}


[44/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 0000000,d1d8d37..af7a61e
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@@ -1,0 -1,1947 +1,1944 @@@
+ /**
+  * 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.hdfs;
+ 
+ import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+ 
+ import java.io.BufferedOutputStream;
+ import java.io.DataInputStream;
+ import java.io.DataOutputStream;
 -import java.io.FileNotFoundException;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.InterruptedIOException;
+ import java.io.OutputStream;
+ import java.net.InetAddress;
+ import java.net.InetSocketAddress;
+ import java.net.Socket;
+ import java.nio.channels.ClosedChannelException;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.HashSet;
+ import java.util.LinkedList;
+ import java.util.List;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicReference;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
+ import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 -import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 -import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 -import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
+ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 -import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+ import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 -import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+ import org.apache.hadoop.hdfs.util.ByteArrayManager;
+ import org.apache.hadoop.io.IOUtils;
+ import org.apache.hadoop.io.MultipleIOException;
 -import org.apache.hadoop.ipc.RemoteException;
+ import org.apache.hadoop.net.NetUtils;
 -import org.apache.hadoop.security.AccessControlException;
+ import org.apache.hadoop.security.token.Token;
+ import org.apache.hadoop.util.Daemon;
+ import org.apache.hadoop.util.DataChecksum;
+ import org.apache.hadoop.util.Progressable;
+ import org.apache.hadoop.util.Time;
+ import org.apache.htrace.core.Sampler;
+ import org.apache.htrace.core.Span;
+ import org.apache.htrace.core.SpanId;
+ import org.apache.htrace.core.TraceScope;
+ import org.apache.htrace.core.Tracer;
+ 
+ import com.google.common.cache.CacheBuilder;
+ import com.google.common.cache.CacheLoader;
+ import com.google.common.cache.LoadingCache;
+ import com.google.common.cache.RemovalListener;
+ import com.google.common.cache.RemovalNotification;
+ 
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ /*********************************************************************
+  *
+  * The DataStreamer class is responsible for sending data packets to the
+  * datanodes in the pipeline. It retrieves a new blockid and block locations
+  * from the namenode, and starts streaming packets to the pipeline of
+  * Datanodes. Every packet has a sequence number associated with
+  * it. When all the packets for a block are sent out and acks for each
+  * if them are received, the DataStreamer closes the current block.
+  *
+  * The DataStreamer thread picks up packets from the dataQueue, sends it to
+  * the first datanode in the pipeline and moves it from the dataQueue to the
+  * ackQueue. The ResponseProcessor receives acks from the datanodes. When an
+  * successful ack for a packet is received from all datanodes, the
+  * ResponseProcessor removes the corresponding packet from the ackQueue.
+  *
+  * In case of error, all outstanding packets are moved from ackQueue. A new
+  * pipeline is setup by eliminating the bad datanode from the original
+  * pipeline. The DataStreamer now starts sending packets from the dataQueue.
+  *
+  *********************************************************************/
+ 
+ @InterfaceAudience.Private
+ class DataStreamer extends Daemon {
+   static final Logger LOG = LoggerFactory.getLogger(DataStreamer.class);
+ 
+   /**
+    * Create a socket for a write pipeline
+    *
+    * @param first the first datanode
+    * @param length the pipeline length
+    * @param client client
+    * @return the socket connected to the first datanode
+    */
+   static Socket createSocketForPipeline(final DatanodeInfo first,
+       final int length, final DFSClient client) throws IOException {
+     final DfsClientConf conf = client.getConf();
+     final String dnAddr = first.getXferAddr(conf.isConnectToDnViaHostname());
+     if (LOG.isDebugEnabled()) {
+       LOG.debug("Connecting to datanode " + dnAddr);
+     }
+     final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
+     final Socket sock = client.socketFactory.createSocket();
+     final int timeout = client.getDatanodeReadTimeout(length);
+     NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
+     sock.setSoTimeout(timeout);
+     sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
+     if (LOG.isDebugEnabled()) {
+       LOG.debug("Send buf size " + sock.getSendBufferSize());
+     }
+     return sock;
+   }
+ 
+   /**
+    * if this file is lazy persist
+    *
+    * @param stat the HdfsFileStatus of a file
+    * @return if this file is lazy persist
+    */
+   static boolean isLazyPersist(HdfsFileStatus stat) {
+     return stat.getStoragePolicy() == HdfsConstants.MEMORY_STORAGE_POLICY_ID;
+   }
+ 
+   /**
+    * release a list of packets to ByteArrayManager
+    *
+    * @param packets packets to be release
+    * @param bam ByteArrayManager
+    */
+   private static void releaseBuffer(List<DFSPacket> packets, ByteArrayManager bam) {
+     for(DFSPacket p : packets) {
+       p.releaseBuffer(bam);
+     }
+     packets.clear();
+   }
+   
 -  static class LastExceptionInStreamer {
++  class LastExceptionInStreamer {
+     private IOException thrown;
+ 
+     synchronized void set(Throwable t) {
+       assert t != null;
+       this.thrown = t instanceof IOException ?
+           (IOException) t : new IOException(t);
+     }
+ 
+     synchronized void clear() {
+       thrown = null;
+     }
+ 
+     /** Check if there already is an exception. */
+     synchronized void check(boolean resetToNull) throws IOException {
+       if (thrown != null) {
+         if (LOG.isTraceEnabled()) {
+           // wrap and print the exception to know when the check is called
 -          LOG.trace("Got Exception while checking", new Throwable(thrown));
++          LOG.trace("Got Exception while checking, " + DataStreamer.this,
++              new Throwable(thrown));
+         }
+         final IOException e = thrown;
+         if (resetToNull) {
+           thrown = null;
+         }
+         throw e;
+       }
+     }
+ 
+     synchronized void throwException4Close() throws IOException {
+       check(false);
+       throw new ClosedChannelException();
+     }
+   }
+ 
++  enum ErrorType {
++    NONE, INTERNAL, EXTERNAL
++  }
++
+   static class ErrorState {
 -    private boolean error = false;
++    ErrorType error = ErrorType.NONE;
+     private int badNodeIndex = -1;
+     private int restartingNodeIndex = -1;
+     private long restartingNodeDeadline = 0;
+     private final long datanodeRestartTimeout;
+ 
+     ErrorState(long datanodeRestartTimeout) {
+       this.datanodeRestartTimeout = datanodeRestartTimeout;
+     }
+ 
++    synchronized void resetInternalError() {
++      if (hasInternalError()) {
++        error = ErrorType.NONE;
++      }
++      badNodeIndex = -1;
++      restartingNodeIndex = -1;
++      restartingNodeDeadline = 0;
++    }
++
+     synchronized void reset() {
 -      error = false;
++      error = ErrorType.NONE;
+       badNodeIndex = -1;
+       restartingNodeIndex = -1;
+       restartingNodeDeadline = 0;
+     }
+ 
++    synchronized boolean hasInternalError() {
++      return error == ErrorType.INTERNAL;
++    }
++
++    synchronized boolean hasExternalError() {
++      return error == ErrorType.EXTERNAL;
++    }
++
+     synchronized boolean hasError() {
 -      return error;
++      return error != ErrorType.NONE;
+     }
+ 
+     synchronized boolean hasDatanodeError() {
 -      return error && isNodeMarked();
++      return error == ErrorType.INTERNAL && isNodeMarked();
+     }
+ 
 -    synchronized void setError(boolean err) {
 -      this.error = err;
++    synchronized void setInternalError() {
++      this.error = ErrorType.INTERNAL;
++    }
++
++    synchronized void setExternalError() {
++      if (!hasInternalError()) {
++        this.error = ErrorType.EXTERNAL;
++      }
+     }
+ 
+     synchronized void setBadNodeIndex(int index) {
+       this.badNodeIndex = index;
+     }
+ 
+     synchronized int getBadNodeIndex() {
+       return badNodeIndex;
+     }
+ 
+     synchronized int getRestartingNodeIndex() {
+       return restartingNodeIndex;
+     }
+ 
+     synchronized void initRestartingNode(int i, String message) {
+       restartingNodeIndex = i;
+       restartingNodeDeadline =  Time.monotonicNow() + datanodeRestartTimeout;
+       // If the data streamer has already set the primary node
+       // bad, clear it. It is likely that the write failed due to
+       // the DN shutdown. Even if it was a real failure, the pipeline
+       // recovery will take care of it.
+       badNodeIndex = -1;
+       LOG.info(message);
+     }
+ 
+     synchronized boolean isRestartingNode() {
+       return restartingNodeIndex >= 0;
+     }
+ 
+     synchronized boolean isNodeMarked() {
+       return badNodeIndex >= 0 || isRestartingNode();
+     }
+ 
+     /**
+      * This method is used when no explicit error report was received, but
+      * something failed. The first node is a suspect or unsure about the cause
+      * so that it is marked as failed.
+      */
+     synchronized void markFirstNodeIfNotMarked() {
+       // There should be no existing error and no ongoing restart.
+       if (!isNodeMarked()) {
+         badNodeIndex = 0;
+       }
+     }
+ 
+     synchronized void adjustState4RestartingNode() {
+       // Just took care of a node error while waiting for a node restart
+       if (restartingNodeIndex >= 0) {
+         // If the error came from a node further away than the restarting
+         // node, the restart must have been complete.
+         if (badNodeIndex > restartingNodeIndex) {
+           restartingNodeIndex = -1;
+         } else if (badNodeIndex < restartingNodeIndex) {
+           // the node index has shifted.
+           restartingNodeIndex--;
+         } else {
+           throw new IllegalStateException("badNodeIndex = " + badNodeIndex
+               + " = restartingNodeIndex = " + restartingNodeIndex);
+         }
+       }
+ 
+       if (!isRestartingNode()) {
 -        error = false;
++        error = ErrorType.NONE;
+       }
+       badNodeIndex = -1;
+     }
+ 
+     synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) {
+       if (restartingNodeIndex >= 0) {
 -        if (!error) {
++        if (error == ErrorType.NONE) {
+           throw new IllegalStateException("error=false while checking" +
+               " restarting node deadline");
+         }
+ 
+         // check badNodeIndex
+         if (badNodeIndex == restartingNodeIndex) {
+           // ignore, if came from the restarting node
+           badNodeIndex = -1;
+         }
+         // not within the deadline
+         if (Time.monotonicNow() >= restartingNodeDeadline) {
+           // expired. declare the restarting node dead
+           restartingNodeDeadline = 0;
+           final int i = restartingNodeIndex;
+           restartingNodeIndex = -1;
+           LOG.warn("Datanode " + i + " did not restart within "
+               + datanodeRestartTimeout + "ms: " + nodes[i]);
+           // Mark the restarting node as failed. If there is any other failed
+           // node during the last pipeline construction attempt, it will not be
+           // overwritten/dropped. In this case, the restarting node will get
+           // excluded in the following attempt, if it still does not come up.
+           if (badNodeIndex == -1) {
+             badNodeIndex = i;
+           }
+         }
+       }
+     }
+   }
+ 
+   private volatile boolean streamerClosed = false;
 -  private ExtendedBlock block; // its length is number of bytes acked
 -  private Token<BlockTokenIdentifier> accessToken;
++  protected ExtendedBlock block; // its length is number of bytes acked
++  protected Token<BlockTokenIdentifier> accessToken;
+   private DataOutputStream blockStream;
+   private DataInputStream blockReplyStream;
+   private ResponseProcessor response = null;
+   private volatile DatanodeInfo[] nodes = null; // list of targets for current block
+   private volatile StorageType[] storageTypes = null;
+   private volatile String[] storageIDs = null;
+   private final ErrorState errorState;
+ 
+   private BlockConstructionStage stage;  // block construction stage
 -  private long bytesSent = 0; // number of bytes that've been sent
++  protected long bytesSent = 0; // number of bytes that've been sent
+   private final boolean isLazyPersistFile;
+ 
+   /** Nodes have been used in the pipeline before and have failed. */
+   private final List<DatanodeInfo> failed = new ArrayList<>();
+   /** The last ack sequence number before pipeline failure. */
+   private long lastAckedSeqnoBeforeFailure = -1;
+   private int pipelineRecoveryCount = 0;
+   /** Has the current block been hflushed? */
+   private boolean isHflushed = false;
+   /** Append on an existing block? */
+   private final boolean isAppend;
+ 
+   private long currentSeqno = 0;
+   private long lastQueuedSeqno = -1;
+   private long lastAckedSeqno = -1;
+   private long bytesCurBlock = 0; // bytes written in current block
+   private final LastExceptionInStreamer lastException = new LastExceptionInStreamer();
+   private Socket s;
+ 
 -  private final DFSClient dfsClient;
 -  private final String src;
++  protected final DFSClient dfsClient;
++  protected final String src;
+   /** Only for DataTransferProtocol.writeBlock(..) */
 -  private final DataChecksum checksum4WriteBlock;
 -  private final Progressable progress;
 -  private final HdfsFileStatus stat;
++  final DataChecksum checksum4WriteBlock;
++  final Progressable progress;
++  protected final HdfsFileStatus stat;
+   // appending to existing partial block
+   private volatile boolean appendChunk = false;
+   // both dataQueue and ackQueue are protected by dataQueue lock
 -  private final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
++  protected final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
+   private final LinkedList<DFSPacket> ackQueue = new LinkedList<>();
+   private final AtomicReference<CachingStrategy> cachingStrategy;
+   private final ByteArrayManager byteArrayManager;
+   //persist blocks on namenode
+   private final AtomicBoolean persistBlocks = new AtomicBoolean(false);
+   private boolean failPacket = false;
+   private final long dfsclientSlowLogThresholdMs;
+   private long artificialSlowdown = 0;
+   // List of congested data nodes. The stream will back off if the DataNodes
+   // are congested
+   private final List<DatanodeInfo> congestedNodes = new ArrayList<>();
+   private static final int CONGESTION_BACKOFF_MEAN_TIME_IN_MS = 5000;
+   private static final int CONGESTION_BACK_OFF_MAX_TIME_IN_MS =
+       CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10;
+   private int lastCongestionBackoffTime;
+ 
 -  private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
++  protected final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
+   private final String[] favoredNodes;
+ 
 -  private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src,
++  private DataStreamer(HdfsFileStatus stat, ExtendedBlock block,
++                       DFSClient dfsClient, String src,
+                        Progressable progress, DataChecksum checksum,
+                        AtomicReference<CachingStrategy> cachingStrategy,
+                        ByteArrayManager byteArrayManage,
+                        boolean isAppend, String[] favoredNodes) {
++    this.block = block;
+     this.dfsClient = dfsClient;
+     this.src = src;
+     this.progress = progress;
+     this.stat = stat;
+     this.checksum4WriteBlock = checksum;
+     this.cachingStrategy = cachingStrategy;
+     this.byteArrayManager = byteArrayManage;
+     this.isLazyPersistFile = isLazyPersist(stat);
+     this.isAppend = isAppend;
+     this.favoredNodes = favoredNodes;
+ 
+     final DfsClientConf conf = dfsClient.getConf();
+     this.dfsclientSlowLogThresholdMs = conf.getSlowIoWarningThresholdMs();
+     this.excludedNodes = initExcludedNodes(conf.getExcludedNodesCacheExpiry());
+     this.errorState = new ErrorState(conf.getDatanodeRestartTimeout());
+   }
+ 
+   /**
+    * construction with tracing info
+    */
+   DataStreamer(HdfsFileStatus stat, ExtendedBlock block, DFSClient dfsClient,
+                String src, Progressable progress, DataChecksum checksum,
+                AtomicReference<CachingStrategy> cachingStrategy,
+                ByteArrayManager byteArrayManage, String[] favoredNodes) {
 -    this(stat, dfsClient, src, progress, checksum, cachingStrategy,
++    this(stat, block, dfsClient, src, progress, checksum, cachingStrategy,
+         byteArrayManage, false, favoredNodes);
 -    this.block = block;
+     stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
+   }
+ 
+   /**
+    * Construct a data streamer for appending to the last partial block
+    * @param lastBlock last block of the file to be appended
+    * @param stat status of the file to be appended
+    * @throws IOException if error occurs
+    */
+   DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
+                String src, Progressable progress, DataChecksum checksum,
+                AtomicReference<CachingStrategy> cachingStrategy,
+                ByteArrayManager byteArrayManage) throws IOException {
 -    this(stat, dfsClient, src, progress, checksum, cachingStrategy,
++    this(stat, lastBlock.getBlock(), dfsClient, src, progress, checksum, cachingStrategy,
+         byteArrayManage, true, null);
+     stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
 -    block = lastBlock.getBlock();
+     bytesSent = block.getNumBytes();
+     accessToken = lastBlock.getBlockToken();
+   }
+ 
+   /**
+    * Set pipeline in construction
+    *
+    * @param lastBlock the last block of a file
+    * @throws IOException
+    */
+   void setPipelineInConstruction(LocatedBlock lastBlock) throws IOException{
+     // setup pipeline to append to the last block XXX retries??
+     setPipeline(lastBlock);
+     if (nodes.length < 1) {
+       throw new IOException("Unable to retrieve blocks locations " +
+           " for last block " + block +
+           "of file " + src);
+     }
+   }
+ 
++  void setAccessToken(Token<BlockTokenIdentifier> t) {
++    this.accessToken = t;
++  }
++
+   private void setPipeline(LocatedBlock lb) {
+     setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
+   }
+ 
+   private void setPipeline(DatanodeInfo[] nodes, StorageType[] storageTypes,
+                            String[] storageIDs) {
+     this.nodes = nodes;
+     this.storageTypes = storageTypes;
+     this.storageIDs = storageIDs;
+   }
+ 
+   /**
+    * Initialize for data streaming
+    */
+   private void initDataStreaming() {
+     this.setName("DataStreamer for file " + src +
+         " block " + block);
+     response = new ResponseProcessor(nodes);
+     response.start();
+     stage = BlockConstructionStage.DATA_STREAMING;
+   }
+ 
 -  private void endBlock() {
++  protected void endBlock() {
+     if(LOG.isDebugEnabled()) {
+       LOG.debug("Closing old block " + block);
+     }
+     this.setName("DataStreamer for file " + src);
+     closeResponder();
+     closeStream();
+     setPipeline(null, null, null);
+     stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
+   }
+ 
+   private boolean shouldStop() {
+     return streamerClosed || errorState.hasError() || !dfsClient.clientRunning;
+   }
+ 
+   /*
+    * streamer thread is the only thread that opens streams to datanode,
+    * and closes them. Any error recovery is also done by this thread.
+    */
+   @Override
+   public void run() {
+     long lastPacket = Time.monotonicNow();
+     TraceScope scope = null;
+     while (!streamerClosed && dfsClient.clientRunning) {
+       // if the Responder encountered an error, shutdown Responder
+       if (errorState.hasError() && response != null) {
+         try {
+           response.close();
+           response.join();
+           response = null;
+         } catch (InterruptedException  e) {
+           LOG.warn("Caught exception", e);
+         }
+       }
+ 
+       DFSPacket one;
+       try {
+         // process datanode IO errors if any
 -        boolean doSleep = processDatanodeError();
++        boolean doSleep = processDatanodeOrExternalError();
+ 
+         final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2; 
+         synchronized (dataQueue) {
+           // wait for a packet to be sent.
+           long now = Time.monotonicNow();
+           while ((!shouldStop() && dataQueue.size() == 0 &&
+               (stage != BlockConstructionStage.DATA_STREAMING ||
+                   stage == BlockConstructionStage.DATA_STREAMING &&
+                       now - lastPacket < halfSocketTimeout)) || doSleep ) {
+             long timeout = halfSocketTimeout - (now-lastPacket);
+             timeout = timeout <= 0 ? 1000 : timeout;
+             timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
+                 timeout : 1000;
+             try {
+               dataQueue.wait(timeout);
+             } catch (InterruptedException  e) {
+               LOG.warn("Caught exception", e);
+             }
+             doSleep = false;
+             now = Time.monotonicNow();
+           }
+           if (shouldStop()) {
+             continue;
+           }
+           // get packet to be sent.
+           if (dataQueue.isEmpty()) {
+             one = createHeartbeatPacket();
+           } else {
+             try {
+               backOffIfNecessary();
+             } catch (InterruptedException e) {
+               LOG.warn("Caught exception", e);
+             }
+             one = dataQueue.getFirst(); // regular data packet
+             SpanId[] parents = one.getTraceParents();
+             if (parents.length > 0) {
+               scope = dfsClient.getTracer().
+                   newScope("dataStreamer", parents[0]);
+               scope.getSpan().setParents(parents);
+             }
+           }
+         }
+ 
+         // get new block from namenode.
++        if (LOG.isDebugEnabled()) {
++          LOG.debug("stage=" + stage + ", " + this);
++        }
+         if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
 -          if(LOG.isDebugEnabled()) {
 -            LOG.debug("Allocating new block");
 -          }
+           setPipeline(nextBlockOutputStream());
+           initDataStreaming();
+         } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
 -          if(LOG.isDebugEnabled()) {
 -            LOG.debug("Append to block " + block);
 -          }
+           setupPipelineForAppendOrRecovery();
+           if (streamerClosed) {
+             continue;
+           }
+           initDataStreaming();
+         }
+ 
+         long lastByteOffsetInBlock = one.getLastByteOffsetBlock();
+         if (lastByteOffsetInBlock > stat.getBlockSize()) {
+           throw new IOException("BlockSize " + stat.getBlockSize() +
 -              " is smaller than data size. " +
 -              " Offset of packet in block " +
 -              lastByteOffsetInBlock +
 -              " Aborting file " + src);
++              " < lastByteOffsetInBlock, " + this + ", " + one);
+         }
+ 
+         if (one.isLastPacketInBlock()) {
+           // wait for all data packets have been successfully acked
+           synchronized (dataQueue) {
+             while (!shouldStop() && ackQueue.size() != 0) {
+               try {
+                 // wait for acks to arrive from datanodes
+                 dataQueue.wait(1000);
+               } catch (InterruptedException  e) {
+                 LOG.warn("Caught exception", e);
+               }
+             }
+           }
+           if (shouldStop()) {
+             continue;
+           }
+           stage = BlockConstructionStage.PIPELINE_CLOSE;
+         }
+ 
+         // send the packet
+         SpanId spanId = SpanId.INVALID;
+         synchronized (dataQueue) {
+           // move packet from dataQueue to ackQueue
+           if (!one.isHeartbeatPacket()) {
+             if (scope != null) {
+               spanId = scope.getSpanId();
+               scope.detach();
+               one.setTraceScope(scope);
+             }
+             scope = null;
+             dataQueue.removeFirst();
+             ackQueue.addLast(one);
+             dataQueue.notifyAll();
+           }
+         }
+ 
+         if (LOG.isDebugEnabled()) {
 -          LOG.debug("DataStreamer block " + block +
 -              " sending packet " + one);
++          LOG.debug(this + " sending " + one);
+         }
+ 
+         // write out data to remote datanode
+         TraceScope writeScope = dfsClient.getTracer().
+             newScope("DataStreamer#writeTo", spanId);
+         try {
+           one.writeTo(blockStream);
+           blockStream.flush();
+         } catch (IOException e) {
+           // HDFS-3398 treat primary DN is down since client is unable to
+           // write to primary DN. If a failed or restarting node has already
+           // been recorded by the responder, the following call will have no
+           // effect. Pipeline recovery can handle only one node error at a
+           // time. If the primary node fails again during the recovery, it
+           // will be taken out then.
+           errorState.markFirstNodeIfNotMarked();
+           throw e;
+         } finally {
+           writeScope.close();
+         }
+         lastPacket = Time.monotonicNow();
+ 
+         // update bytesSent
+         long tmpBytesSent = one.getLastByteOffsetBlock();
+         if (bytesSent < tmpBytesSent) {
+           bytesSent = tmpBytesSent;
+         }
+ 
+         if (shouldStop()) {
+           continue;
+         }
+ 
+         // Is this block full?
+         if (one.isLastPacketInBlock()) {
+           // wait for the close packet has been acked
+           synchronized (dataQueue) {
+             while (!shouldStop() && ackQueue.size() != 0) {
+               dataQueue.wait(1000);// wait for acks to arrive from datanodes
+             }
+           }
+           if (shouldStop()) {
+             continue;
+           }
+ 
+           endBlock();
+         }
+         if (progress != null) { progress.progress(); }
+ 
+         // This is used by unit test to trigger race conditions.
+         if (artificialSlowdown != 0 && dfsClient.clientRunning) {
+           Thread.sleep(artificialSlowdown);
+         }
+       } catch (Throwable e) {
+         // Log warning if there was a real error.
+         if (!errorState.isRestartingNode()) {
+           // Since their messages are descriptive enough, do not always
+           // log a verbose stack-trace WARN for quota exceptions.
+           if (e instanceof QuotaExceededException) {
+             LOG.debug("DataStreamer Quota Exception", e);
+           } else {
+             LOG.warn("DataStreamer Exception", e);
+           }
+         }
+         lastException.set(e);
+         assert !(e instanceof NullPointerException);
 -        errorState.setError(true);
++        errorState.setInternalError();
+         if (!errorState.isNodeMarked()) {
+           // Not a datanode issue
+           streamerClosed = true;
+         }
+       } finally {
+         if (scope != null) {
+           scope.close();
+           scope = null;
+         }
+       }
+     }
+     closeInternal();
+   }
+ 
+   private void closeInternal() {
+     closeResponder();       // close and join
+     closeStream();
+     streamerClosed = true;
+     release();
+     synchronized (dataQueue) {
+       dataQueue.notifyAll();
+     }
+   }
+ 
+   /**
+    * release the DFSPackets in the two queues
+    *
+    */
+   void release() {
+     synchronized (dataQueue) {
+       releaseBuffer(dataQueue, byteArrayManager);
+       releaseBuffer(ackQueue, byteArrayManager);
+     }
+   }
+ 
+   /**
+    * wait for the ack of seqno
+    *
+    * @param seqno the sequence number to be acked
+    * @throws IOException
+    */
+   void waitForAckedSeqno(long seqno) throws IOException {
+     TraceScope scope = dfsClient.getTracer().
+         newScope("waitForAckedSeqno");
+     try {
+       if (LOG.isDebugEnabled()) {
+         LOG.debug("Waiting for ack for: " + seqno);
+       }
+       long begin = Time.monotonicNow();
+       try {
+         synchronized (dataQueue) {
+           while (!streamerClosed) {
+             checkClosed();
+             if (lastAckedSeqno >= seqno) {
+               break;
+             }
+             try {
+               dataQueue.wait(1000); // when we receive an ack, we notify on
+               // dataQueue
+             } catch (InterruptedException ie) {
+               throw new InterruptedIOException(
+                   "Interrupted while waiting for data to be acknowledged by pipeline");
+             }
+           }
+         }
+         checkClosed();
+       } catch (ClosedChannelException e) {
+       }
+       long duration = Time.monotonicNow() - begin;
+       if (duration > dfsclientSlowLogThresholdMs) {
+         LOG.warn("Slow waitForAckedSeqno took " + duration
+             + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
+       }
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * wait for space of dataQueue and queue the packet
+    *
+    * @param packet  the DFSPacket to be queued
+    * @throws IOException
+    */
+   void waitAndQueuePacket(DFSPacket packet) throws IOException {
+     synchronized (dataQueue) {
+       try {
+         // If queue is full, then wait till we have enough space
+         boolean firstWait = true;
+         try {
+           while (!streamerClosed && dataQueue.size() + ackQueue.size() >
+               dfsClient.getConf().getWriteMaxPackets()) {
+             if (firstWait) {
+               Span span = Tracer.getCurrentSpan();
+               if (span != null) {
+                 span.addTimelineAnnotation("dataQueue.wait");
+               }
+               firstWait = false;
+             }
+             try {
+               dataQueue.wait();
+             } catch (InterruptedException e) {
+               // If we get interrupted while waiting to queue data, we still need to get rid
+               // of the current packet. This is because we have an invariant that if
+               // currentPacket gets full, it will get queued before the next writeChunk.
+               //
+               // Rather than wait around for space in the queue, we should instead try to
+               // return to the caller as soon as possible, even though we slightly overrun
+               // the MAX_PACKETS length.
+               Thread.currentThread().interrupt();
+               break;
+             }
+           }
+         } finally {
+           Span span = Tracer.getCurrentSpan();
+           if ((span != null) && (!firstWait)) {
+             span.addTimelineAnnotation("end.wait");
+           }
+         }
+         checkClosed();
+         queuePacket(packet);
+       } catch (ClosedChannelException e) {
+       }
+     }
+   }
+ 
+   /*
+    * close the streamer, should be called only by an external thread
+    * and only after all data to be sent has been flushed to datanode.
+    *
+    * Interrupt this data streamer if force is true
+    *
+    * @param force if this data stream is forced to be closed
+    */
+   void close(boolean force) {
+     streamerClosed = true;
+     synchronized (dataQueue) {
+       dataQueue.notifyAll();
+     }
+     if (force) {
+       this.interrupt();
+     }
+   }
+ 
++  void setStreamerAsClosed() {
++    streamerClosed = true;
++  }
+ 
+   private void checkClosed() throws IOException {
+     if (streamerClosed) {
+       lastException.throwException4Close();
+     }
+   }
+ 
+   private void closeResponder() {
+     if (response != null) {
+       try {
+         response.close();
+         response.join();
+       } catch (InterruptedException  e) {
+         LOG.warn("Caught exception", e);
+       } finally {
+         response = null;
+       }
+     }
+   }
+ 
 -  private void closeStream() {
++  void closeStream() {
+     final MultipleIOException.Builder b = new MultipleIOException.Builder();
+ 
+     if (blockStream != null) {
+       try {
+         blockStream.close();
+       } catch (IOException e) {
+         b.add(e);
+       } finally {
+         blockStream = null;
+       }
+     }
+     if (blockReplyStream != null) {
+       try {
+         blockReplyStream.close();
+       } catch (IOException e) {
+         b.add(e);
+       } finally {
+         blockReplyStream = null;
+       }
+     }
+     if (null != s) {
+       try {
+         s.close();
+       } catch (IOException e) {
+         b.add(e);
+       } finally {
+         s = null;
+       }
+     }
+ 
+     final IOException ioe = b.build();
+     if (ioe != null) {
+       lastException.set(ioe);
+     }
+   }
+ 
+   /**
+    * Examine whether it is worth waiting for a node to restart.
+    * @param index the node index
+    */
+   boolean shouldWaitForRestart(int index) {
+     // Only one node in the pipeline.
+     if (nodes.length == 1) {
+       return true;
+     }
+ 
+     // Is it a local node?
+     InetAddress addr = null;
+     try {
+       addr = InetAddress.getByName(nodes[index].getIpAddr());
+     } catch (java.net.UnknownHostException e) {
+       // we are passing an ip address. this should not happen.
+       assert false;
+     }
+ 
+     if (addr != null && NetUtils.isLocalAddress(addr)) {
+       return true;
+     }
+     return false;
+   }
+ 
+   //
+   // Processes responses from the datanodes.  A packet is removed
+   // from the ackQueue when its response arrives.
+   //
+   private class ResponseProcessor extends Daemon {
+ 
+     private volatile boolean responderClosed = false;
+     private DatanodeInfo[] targets = null;
+     private boolean isLastPacketInBlock = false;
+ 
+     ResponseProcessor (DatanodeInfo[] targets) {
+       this.targets = targets;
+     }
+ 
+     @Override
+     public void run() {
+ 
+       setName("ResponseProcessor for block " + block);
+       PipelineAck ack = new PipelineAck();
+ 
+       TraceScope scope = null;
+       while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) {
+         // process responses from datanodes.
+         try {
+           // read an ack from the pipeline
+           long begin = Time.monotonicNow();
+           ack.readFields(blockReplyStream);
+           long duration = Time.monotonicNow() - begin;
+           if (duration > dfsclientSlowLogThresholdMs
+               && ack.getSeqno() != DFSPacket.HEART_BEAT_SEQNO) {
+             LOG.warn("Slow ReadProcessor read fields took " + duration
+                 + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
+                 + ack + ", targets: " + Arrays.asList(targets));
+           } else if (LOG.isDebugEnabled()) {
+             LOG.debug("DFSClient " + ack);
+           }
+ 
+           long seqno = ack.getSeqno();
+           // processes response status from datanodes.
+           ArrayList<DatanodeInfo> congestedNodesFromAck = new ArrayList<>();
+           for (int i = ack.getNumOfReplies()-1; i >=0  && dfsClient.clientRunning; i--) {
+             final Status reply = PipelineAck.getStatusFromHeader(ack
+                 .getHeaderFlag(i));
+             if (PipelineAck.getECNFromHeader(ack.getHeaderFlag(i)) ==
+                 PipelineAck.ECN.CONGESTED) {
+               congestedNodesFromAck.add(targets[i]);
+             }
+             // Restart will not be treated differently unless it is
+             // the local node or the only one in the pipeline.
+             if (PipelineAck.isRestartOOBStatus(reply) &&
+                 shouldWaitForRestart(i)) {
+               final String message = "Datanode " + i + " is restarting: "
+                   + targets[i];
+               errorState.initRestartingNode(i, message);
+               throw new IOException(message);
+             }
+             // node error
+             if (reply != SUCCESS) {
+               errorState.setBadNodeIndex(i); // mark bad datanode
+               throw new IOException("Bad response " + reply +
+                   " for " + block + " from datanode " + targets[i]);
+             }
+           }
+ 
+           if (!congestedNodesFromAck.isEmpty()) {
+             synchronized (congestedNodes) {
+               congestedNodes.clear();
+               congestedNodes.addAll(congestedNodesFromAck);
+             }
+           } else {
+             synchronized (congestedNodes) {
+               congestedNodes.clear();
+               lastCongestionBackoffTime = 0;
+             }
+           }
+ 
+           assert seqno != PipelineAck.UNKOWN_SEQNO :
+               "Ack for unknown seqno should be a failed ack: " + ack;
+           if (seqno == DFSPacket.HEART_BEAT_SEQNO) {  // a heartbeat ack
+             continue;
+           }
+ 
+           // a success ack for a data packet
+           DFSPacket one;
+           synchronized (dataQueue) {
+             one = ackQueue.getFirst();
+           }
+           if (one.getSeqno() != seqno) {
+             throw new IOException("ResponseProcessor: Expecting seqno " +
+                 " for block " + block +
+                 one.getSeqno() + " but received " + seqno);
+           }
+           isLastPacketInBlock = one.isLastPacketInBlock();
+ 
+           // Fail the packet write for testing in order to force a
+           // pipeline recovery.
+           if (DFSClientFaultInjector.get().failPacket() &&
+               isLastPacketInBlock) {
+             failPacket = true;
+             throw new IOException(
+                 "Failing the last packet for testing.");
+           }
+ 
+           // update bytesAcked
+           block.setNumBytes(one.getLastByteOffsetBlock());
+ 
+           synchronized (dataQueue) {
+             scope = one.getTraceScope();
+             if (scope != null) {
+               scope.reattach();
+               one.setTraceScope(null);
+             }
+             lastAckedSeqno = seqno;
+             ackQueue.removeFirst();
+             dataQueue.notifyAll();
+ 
+             one.releaseBuffer(byteArrayManager);
+           }
+         } catch (Exception e) {
+           if (!responderClosed) {
+             lastException.set(e);
 -            errorState.setError(true);
++            errorState.setInternalError();
+             errorState.markFirstNodeIfNotMarked();
+             synchronized (dataQueue) {
+               dataQueue.notifyAll();
+             }
+             if (!errorState.isRestartingNode()) {
+               LOG.warn("Exception for " + block, e);
+             }
+             responderClosed = true;
+           }
+         } finally {
+           if (scope != null) {
+             scope.close();
+           }
+           scope = null;
+         }
+       }
+     }
+ 
+     void close() {
+       responderClosed = true;
+       this.interrupt();
+     }
+   }
+ 
++  private boolean shouldHandleExternalError(){
++    return errorState.hasExternalError() && blockStream != null;
++  }
++
+   /**
+    * If this stream has encountered any errors, shutdown threads
+    * and mark the stream as closed.
+    *
+    * @return true if it should sleep for a while after returning.
+    */
 -  private boolean processDatanodeError() throws IOException {
 -    if (!errorState.hasDatanodeError()) {
++  private boolean processDatanodeOrExternalError() throws IOException {
++    if (!errorState.hasDatanodeError() && !shouldHandleExternalError()) {
+       return false;
+     }
+     if (response != null) {
+       LOG.info("Error Recovery for " + block +
+           " waiting for responder to exit. ");
+       return true;
+     }
+     closeStream();
+ 
+     // move packets from ack queue to front of the data queue
+     synchronized (dataQueue) {
+       dataQueue.addAll(0, ackQueue);
+       ackQueue.clear();
+     }
+ 
+     // Record the new pipeline failure recovery.
+     if (lastAckedSeqnoBeforeFailure != lastAckedSeqno) {
+       lastAckedSeqnoBeforeFailure = lastAckedSeqno;
+       pipelineRecoveryCount = 1;
+     } else {
+       // If we had to recover the pipeline five times in a row for the
+       // same packet, this client likely has corrupt data or corrupting
+       // during transmission.
+       if (++pipelineRecoveryCount > 5) {
+         LOG.warn("Error recovering pipeline for writing " +
+             block + ". Already retried 5 times for the same packet.");
+         lastException.set(new IOException("Failing write. Tried pipeline " +
+             "recovery 5 times without success."));
+         streamerClosed = true;
+         return false;
+       }
+     }
 -    boolean doSleep = setupPipelineForAppendOrRecovery();
++
++    setupPipelineForAppendOrRecovery();
+ 
+     if (!streamerClosed && dfsClient.clientRunning) {
+       if (stage == BlockConstructionStage.PIPELINE_CLOSE) {
+ 
+         // If we had an error while closing the pipeline, we go through a fast-path
+         // where the BlockReceiver does not run. Instead, the DataNode just finalizes
+         // the block immediately during the 'connect ack' process. So, we want to pull
+         // the end-of-block packet from the dataQueue, since we don't actually have
+         // a true pipeline to send it over.
+         //
+         // We also need to set lastAckedSeqno to the end-of-block Packet's seqno, so that
+         // a client waiting on close() will be aware that the flush finished.
+         synchronized (dataQueue) {
+           DFSPacket endOfBlockPacket = dataQueue.remove();  // remove the end of block packet
+           // Close any trace span associated with this Packet
+           TraceScope scope = endOfBlockPacket.getTraceScope();
+           if (scope != null) {
+             scope.reattach();
+             scope.close();
+             endOfBlockPacket.setTraceScope(null);
+           }
+           assert endOfBlockPacket.isLastPacketInBlock();
+           assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
+           lastAckedSeqno = endOfBlockPacket.getSeqno();
+           dataQueue.notifyAll();
+         }
+         endBlock();
+       } else {
+         initDataStreaming();
+       }
+     }
+ 
 -    return doSleep;
++    return false;
+   }
+ 
+   void setHflush() {
+     isHflushed = true;
+   }
+ 
+   private int findNewDatanode(final DatanodeInfo[] original
+   ) throws IOException {
+     if (nodes.length != original.length + 1) {
+       throw new IOException(
+           new StringBuilder()
+               .append("Failed to replace a bad datanode on the existing pipeline ")
+               .append("due to no more good datanodes being available to try. ")
+               .append("(Nodes: current=").append(Arrays.asList(nodes))
+               .append(", original=").append(Arrays.asList(original)).append("). ")
+               .append("The current failed datanode replacement policy is ")
+               .append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
+               .append("a client may configure this via '")
+               .append(BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY)
+               .append("' in its configuration.")
+               .toString());
+     }
+     for(int i = 0; i < nodes.length; i++) {
+       int j = 0;
+       for(; j < original.length && !nodes[i].equals(original[j]); j++);
+       if (j == original.length) {
+         return i;
+       }
+     }
+     throw new IOException("Failed: new datanode not found: nodes="
+         + Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
+   }
+ 
+   private void addDatanode2ExistingPipeline() throws IOException {
+     if (DataTransferProtocol.LOG.isDebugEnabled()) {
+       DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
+     }
+       /*
+        * Is data transfer necessary?  We have the following cases.
+        *
+        * Case 1: Failure in Pipeline Setup
+        * - Append
+        *    + Transfer the stored replica, which may be a RBW or a finalized.
+        * - Create
+        *    + If no data, then no transfer is required.
+        *    + If there are data written, transfer RBW. This case may happens
+        *      when there are streaming failure earlier in this pipeline.
+        *
+        * Case 2: Failure in Streaming
+        * - Append/Create:
+        *    + transfer RBW
+        *
+        * Case 3: Failure in Close
+        * - Append/Create:
+        *    + no transfer, let NameNode replicates the block.
+        */
+     if (!isAppend && lastAckedSeqno < 0
+         && stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
+       //no data have been written
+       return;
+     } else if (stage == BlockConstructionStage.PIPELINE_CLOSE
+         || stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
+       //pipeline is closing
+       return;
+     }
+ 
+     int tried = 0;
+     final DatanodeInfo[] original = nodes;
+     final StorageType[] originalTypes = storageTypes;
+     final String[] originalIDs = storageIDs;
+     IOException caughtException = null;
+     ArrayList<DatanodeInfo> exclude = new ArrayList<DatanodeInfo>(failed);
+     while (tried < 3) {
+       LocatedBlock lb;
+       //get a new datanode
+       lb = dfsClient.namenode.getAdditionalDatanode(
+           src, stat.getFileId(), block, nodes, storageIDs,
+           exclude.toArray(new DatanodeInfo[exclude.size()]),
+           1, dfsClient.clientName);
+       // a new node was allocated by the namenode. Update nodes.
+       setPipeline(lb);
+ 
+       //find the new datanode
+       final int d = findNewDatanode(original);
+       //transfer replica. pick a source from the original nodes
+       final DatanodeInfo src = original[tried % original.length];
+       final DatanodeInfo[] targets = {nodes[d]};
+       final StorageType[] targetStorageTypes = {storageTypes[d]};
+ 
+       try {
+         transfer(src, targets, targetStorageTypes, lb.getBlockToken());
+       } catch (IOException ioe) {
+         DFSClient.LOG.warn("Error transferring data from " + src + " to " +
+             nodes[d] + ": " + ioe.getMessage());
+         caughtException = ioe;
+         // add the allocated node to the exclude list.
+         exclude.add(nodes[d]);
+         setPipeline(original, originalTypes, originalIDs);
+         tried++;
+         continue;
+       }
+       return; // finished successfully
+     }
+     // All retries failed
+     throw (caughtException != null) ? caughtException :
+         new IOException("Failed to add a node");
+   }
+ 
+   private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
+                         final StorageType[] targetStorageTypes,
+                         final Token<BlockTokenIdentifier> blockToken) throws IOException {
+     //transfer replica to the new datanode
+     Socket sock = null;
+     DataOutputStream out = null;
+     DataInputStream in = null;
+     try {
+       sock = createSocketForPipeline(src, 2, dfsClient);
+       final long writeTimeout = dfsClient.getDatanodeWriteTimeout(2);
+ 
+       // transfer timeout multiplier based on the transfer size
+       // One per 200 packets = 12.8MB. Minimum is 2.
+       int multi = 2 + (int)(bytesSent/dfsClient.getConf().getWritePacketSize())/200;
+       final long readTimeout = dfsClient.getDatanodeReadTimeout(multi);
+ 
+       OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
+       InputStream unbufIn = NetUtils.getInputStream(sock, readTimeout);
+       IOStreamPair saslStreams = dfsClient.saslClient.socketSend(sock,
+           unbufOut, unbufIn, dfsClient, blockToken, src);
+       unbufOut = saslStreams.out;
+       unbufIn = saslStreams.in;
+       out = new DataOutputStream(new BufferedOutputStream(unbufOut,
+           DFSUtilClient.getSmallBufferSize(dfsClient.getConfiguration())));
+       in = new DataInputStream(unbufIn);
+ 
+       //send the TRANSFER_BLOCK request
+       new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
+           targets, targetStorageTypes);
+       out.flush();
+ 
+       //ack
+       BlockOpResponseProto response =
+           BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
+       if (SUCCESS != response.getStatus()) {
+         throw new IOException("Failed to add a datanode");
+       }
+     } finally {
+       IOUtils.closeStream(in);
+       IOUtils.closeStream(out);
+       IOUtils.closeSocket(sock);
+     }
+   }
+ 
+   /**
+    * Open a DataStreamer to a DataNode pipeline so that
+    * it can be written to.
+    * This happens when a file is appended or data streaming fails
+    * It keeps on trying until a pipeline is setup
+    */
 -  private boolean setupPipelineForAppendOrRecovery() throws IOException {
++  private void setupPipelineForAppendOrRecovery() throws IOException {
+     // check number of datanodes
+     if (nodes == null || nodes.length == 0) {
+       String msg = "Could not get block locations. " + "Source file \""
+           + src + "\" - Aborting...";
+       LOG.warn(msg);
+       lastException.set(new IOException(msg));
+       streamerClosed = true;
 -      return false;
++      return;
+     }
++    setupPipelineInternal(nodes, storageTypes);
++  }
+ 
++  protected void setupPipelineInternal(DatanodeInfo[] datanodes,
++      StorageType[] nodeStorageTypes) throws IOException {
+     boolean success = false;
+     long newGS = 0L;
+     while (!success && !streamerClosed && dfsClient.clientRunning) {
+       if (!handleRestartingDatanode()) {
 -        return false;
++        return;
+       }
+ 
 -      final boolean isRecovery = errorState.hasError();
++      final boolean isRecovery = errorState.hasInternalError();
+       if (!handleBadDatanode()) {
 -        return false;
++        return;
+       }
+ 
+       handleDatanodeReplacement();
+ 
+       // get a new generation stamp and an access token
+       final LocatedBlock lb = updateBlockForPipeline();
+       newGS = lb.getBlock().getGenerationStamp();
+       accessToken = lb.getBlockToken();
+ 
+       // set up the pipeline again with the remaining nodes
+       success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery);
+ 
+       failPacket4Testing();
+ 
+       errorState.checkRestartingNodeDeadline(nodes);
+     } // while
+ 
+     if (success) {
+       block = updatePipeline(newGS);
+     }
 -    return false; // do not sleep, continue processing
+   }
+ 
+   /**
+    * Sleep if a node is restarting.
+    * This process is repeated until the deadline or the node starts back up.
+    * @return true if it should continue.
+    */
 -  private boolean handleRestartingDatanode() {
++  boolean handleRestartingDatanode() {
+     if (errorState.isRestartingNode()) {
+       // 4 seconds or the configured deadline period, whichever is shorter.
+       // This is the retry interval and recovery will be retried in this
+       // interval until timeout or success.
+       final long delay = Math.min(errorState.datanodeRestartTimeout, 4000L);
+       try {
+         Thread.sleep(delay);
+       } catch (InterruptedException ie) {
+         lastException.set(new IOException(
+             "Interrupted while waiting for restarting "
+             + nodes[errorState.getRestartingNodeIndex()]));
+         streamerClosed = true;
+         return false;
+       }
+     }
+     return true;
+   }
+ 
+   /**
+    * Remove bad node from list of nodes if badNodeIndex was set.
+    * @return true if it should continue.
+    */
 -  private boolean handleBadDatanode() {
++  boolean handleBadDatanode() {
+     final int badNodeIndex = errorState.getBadNodeIndex();
+     if (badNodeIndex >= 0) {
+       if (nodes.length <= 1) {
+         lastException.set(new IOException("All datanodes "
+             + Arrays.toString(nodes) + " are bad. Aborting..."));
+         streamerClosed = true;
+         return false;
+       }
+ 
+       LOG.warn("Error Recovery for " + block + " in pipeline "
+           + Arrays.toString(nodes) + ": datanode " + badNodeIndex
+           + "("+ nodes[badNodeIndex] + ") is bad.");
+       failed.add(nodes[badNodeIndex]);
+ 
+       DatanodeInfo[] newnodes = new DatanodeInfo[nodes.length-1];
+       arraycopy(nodes, newnodes, badNodeIndex);
+ 
+       final StorageType[] newStorageTypes = new StorageType[newnodes.length];
+       arraycopy(storageTypes, newStorageTypes, badNodeIndex);
+ 
+       final String[] newStorageIDs = new String[newnodes.length];
+       arraycopy(storageIDs, newStorageIDs, badNodeIndex);
+ 
+       setPipeline(newnodes, newStorageTypes, newStorageIDs);
+ 
+       errorState.adjustState4RestartingNode();
+       lastException.clear();
+     }
+     return true;
+   }
+ 
+   /** Add a datanode if replace-datanode policy is satisfied. */
+   private void handleDatanodeReplacement() throws IOException {
+     if (dfsClient.dtpReplaceDatanodeOnFailure.satisfy(stat.getReplication(),
+         nodes, isAppend, isHflushed)) {
+       try {
+         addDatanode2ExistingPipeline();
+       } catch(IOException ioe) {
+         if (!dfsClient.dtpReplaceDatanodeOnFailure.isBestEffort()) {
+           throw ioe;
+         }
+         LOG.warn("Failed to replace datanode."
+             + " Continue with the remaining datanodes since "
+             + BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY
+             + " is set to true.", ioe);
+       }
+     }
+   }
+ 
 -  private void failPacket4Testing() {
++  void failPacket4Testing() {
+     if (failPacket) { // for testing
+       failPacket = false;
+       try {
+         // Give DNs time to send in bad reports. In real situations,
+         // good reports should follow bad ones, if client committed
+         // with those nodes.
+         Thread.sleep(2000);
+       } catch (InterruptedException ie) {}
+     }
+   }
+ 
 -  LocatedBlock updateBlockForPipeline() throws IOException {
 -    return dfsClient.namenode.updateBlockForPipeline(
 -        block, dfsClient.clientName);
++  private LocatedBlock updateBlockForPipeline() throws IOException {
++    return dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName);
++  }
++
++  static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) {
++    return new ExtendedBlock(b.getBlockPoolId(), b.getBlockId(),
++        b.getNumBytes(), newGS);
+   }
+ 
+   /** update pipeline at the namenode */
+   ExtendedBlock updatePipeline(long newGS) throws IOException {
 -    final ExtendedBlock newBlock = new ExtendedBlock(
 -        block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
++    final ExtendedBlock newBlock = newBlock(block, newGS);
+     dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
+         nodes, storageIDs);
+     return newBlock;
+   }
+ 
++  private int getNumBlockWriteRetry() {
++    return dfsClient.getConf().getNumBlockWriteRetry();
++  }
++
+   /**
+    * Open a DataStreamer to a DataNode so that it can be written to.
+    * This happens when a file is created and each time a new block is allocated.
+    * Must get block ID and the IDs of the destinations from the namenode.
+    * Returns the list of target datanodes.
+    */
 -  private LocatedBlock nextBlockOutputStream() throws IOException {
++  protected LocatedBlock nextBlockOutputStream() throws IOException {
+     LocatedBlock lb = null;
+     DatanodeInfo[] nodes = null;
+     StorageType[] storageTypes = null;
 -    int count = dfsClient.getConf().getNumBlockWriteRetry();
++    int count = getNumBlockWriteRetry();
+     boolean success = false;
+     ExtendedBlock oldBlock = block;
+     do {
 -      errorState.reset();
++      errorState.resetInternalError();
+       lastException.clear();
 -      success = false;
+ 
+       DatanodeInfo[] excluded =
+           excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
+               .keySet()
+               .toArray(new DatanodeInfo[0]);
+       block = oldBlock;
+       lb = locateFollowingBlock(excluded.length > 0 ? excluded : null);
+       block = lb.getBlock();
+       block.setNumBytes(0);
+       bytesSent = 0;
+       accessToken = lb.getBlockToken();
+       nodes = lb.getLocations();
+       storageTypes = lb.getStorageTypes();
+ 
+       //
+       // Connect to first DataNode in the list.
+       //
+       success = createBlockOutputStream(nodes, storageTypes, 0L, false);
+ 
+       if (!success) {
+         LOG.info("Abandoning " + block);
+         dfsClient.namenode.abandonBlock(block, stat.getFileId(), src,
+             dfsClient.clientName);
+         block = null;
+         final DatanodeInfo badNode = nodes[errorState.getBadNodeIndex()];
+         LOG.info("Excluding datanode " + badNode);
+         excludedNodes.put(badNode, badNode);
+       }
+     } while (!success && --count >= 0);
+ 
+     if (!success) {
+       throw new IOException("Unable to create new block.");
+     }
+     return lb;
+   }
+ 
+   // connects to the first datanode in the pipeline
+   // Returns true if success, otherwise return failure.
+   //
 -  private boolean createBlockOutputStream(DatanodeInfo[] nodes,
++  boolean createBlockOutputStream(DatanodeInfo[] nodes,
+       StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
+     if (nodes.length == 0) {
+       LOG.info("nodes are empty for write pipeline of " + block);
+       return false;
+     }
+     Status pipelineStatus = SUCCESS;
+     String firstBadLink = "";
+     boolean checkRestart = false;
+     if (LOG.isDebugEnabled()) {
 -      LOG.debug("pipeline = " + Arrays.asList(nodes));
++      LOG.debug("pipeline = " + Arrays.toString(nodes) + ", " + this);
+     }
+ 
+     // persist blocks on namenode on next flush
+     persistBlocks.set(true);
+ 
+     int refetchEncryptionKey = 1;
+     while (true) {
+       boolean result = false;
+       DataOutputStream out = null;
+       try {
+         assert null == s : "Previous socket unclosed";
+         assert null == blockReplyStream : "Previous blockReplyStream unclosed";
+         s = createSocketForPipeline(nodes[0], nodes.length, dfsClient);
+         long writeTimeout = dfsClient.getDatanodeWriteTimeout(nodes.length);
+         long readTimeout = dfsClient.getDatanodeReadTimeout(nodes.length);
+ 
+         OutputStream unbufOut = NetUtils.getOutputStream(s, writeTimeout);
+         InputStream unbufIn = NetUtils.getInputStream(s, readTimeout);
+         IOStreamPair saslStreams = dfsClient.saslClient.socketSend(s,
+             unbufOut, unbufIn, dfsClient, accessToken, nodes[0]);
+         unbufOut = saslStreams.out;
+         unbufIn = saslStreams.in;
+         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
+             DFSUtilClient.getSmallBufferSize(dfsClient.getConfiguration())));
+         blockReplyStream = new DataInputStream(unbufIn);
+ 
+         //
+         // Xmit header info to datanode
+         //
+ 
+         BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
+ 
+         // We cannot change the block length in 'block' as it counts the number
+         // of bytes ack'ed.
+         ExtendedBlock blockCopy = new ExtendedBlock(block);
+         blockCopy.setNumBytes(stat.getBlockSize());
+ 
+         boolean[] targetPinnings = getPinnings(nodes, true);
+         // send the request
+         new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
+             dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
+             nodes.length, block.getNumBytes(), bytesSent, newGS,
+             checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
+             (targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
+ 
+         // receive ack for connect
+         BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
+             PBHelperClient.vintPrefixed(blockReplyStream));
+         pipelineStatus = resp.getStatus();
+         firstBadLink = resp.getFirstBadLink();
+ 
+         // Got an restart OOB ack.
+         // If a node is already restarting, this status is not likely from
+         // the same node. If it is from a different node, it is not
+         // from the local datanode. Thus it is safe to treat this as a
+         // regular node error.
+         if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
+             !errorState.isRestartingNode()) {
+           checkRestart = true;
+           throw new IOException("A datanode is restarting.");
+         }
+ 		
+         String logInfo = "ack with firstBadLink as " + firstBadLink;
+         DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
+ 
+         assert null == blockStream : "Previous blockStream unclosed";
+         blockStream = out;
+         result =  true; // success
 -        errorState.reset();
++        errorState.resetInternalError();
+       } catch (IOException ie) {
+         if (!errorState.isRestartingNode()) {
 -          LOG.info("Exception in createBlockOutputStream", ie);
++          LOG.info("Exception in createBlockOutputStream " + this, ie);
+         }
+         if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+           LOG.info("Will fetch a new encryption key and retry, "
+               + "encryption key was invalid when connecting to "
+               + nodes[0] + " : " + ie);
+           // The encryption key used is invalid.
+           refetchEncryptionKey--;
+           dfsClient.clearDataEncryptionKey();
+           // Don't close the socket/exclude this node just yet. Try again with
+           // a new encryption key.
+           continue;
+         }
+ 
+         // find the datanode that matches
+         if (firstBadLink.length() != 0) {
+           for (int i = 0; i < nodes.length; i++) {
+             // NB: Unconditionally using the xfer addr w/o hostname
+             if (firstBadLink.equals(nodes[i].getXferAddr())) {
+               errorState.setBadNodeIndex(i);
+               break;
+             }
+           }
+         } else {
+           assert checkRestart == false;
+           errorState.setBadNodeIndex(0);
+         }
+ 
+         final int i = errorState.getBadNodeIndex();
+         // Check whether there is a restart worth waiting for.
+         if (checkRestart && shouldWaitForRestart(i)) {
+           errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
+         }
 -        errorState.setError(true);
++        errorState.setInternalError();
+         lastException.set(ie);
+         result =  false;  // error
+       } finally {
+         if (!result) {
+           IOUtils.closeSocket(s);
+           s = null;
+           IOUtils.closeStream(out);
+           out = null;
+           IOUtils.closeStream(blockReplyStream);
+           blockReplyStream = null;
+         }
+       }
+       return result;
+     }
+   }
+ 
+   private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
+     if (favoredNodes == null) {
+       return null;
+     } else {
+       boolean[] pinnings = new boolean[nodes.length];
+       HashSet<String> favoredSet =
+           new HashSet<String>(Arrays.asList(favoredNodes));
+       for (int i = 0; i < nodes.length; i++) {
+         pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
+         if (LOG.isDebugEnabled()) {
+           LOG.debug(nodes[i].getXferAddrWithHostname() +
+               " was chosen by name node (favored=" + pinnings[i] + ").");
+         }
+       }
+       if (shouldLog && !favoredSet.isEmpty()) {
+         // There is one or more favored nodes that were not allocated.
+         LOG.warn("These favored nodes were specified but not chosen: "
+             + favoredSet + " Specified favored nodes: "
+             + Arrays.toString(favoredNodes));
+ 
+       }
+       return pinnings;
+     }
+   }
+ 
 -  protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
++  private LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
+       throws IOException {
 -    final DfsClientConf conf = dfsClient.getConf(); 
 -    int retries = conf.getNumBlockWriteLocateFollowingRetry();
 -    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
 -    while (true) {
 -      long localstart = Time.monotonicNow();
 -      while (true) {
 -        try {
 -          return dfsClient.namenode.addBlock(src, dfsClient.clientName,
 -              block, excludedNodes, stat.getFileId(), favoredNodes);
 -        } catch (RemoteException e) {
 -          IOException ue =
 -              e.unwrapRemoteException(FileNotFoundException.class,
 -                  AccessControlException.class,
 -                  NSQuotaExceededException.class,
 -                  DSQuotaExceededException.class,
 -                  QuotaByStorageTypeExceededException.class,
 -                  UnresolvedPathException.class);
 -          if (ue != e) {
 -            throw ue; // no need to retry these exceptions
 -          }
 -
 -
 -          if (NotReplicatedYetException.class.getName().
 -              equals(e.getClassName())) {
 -            if (retries == 0) {
 -              throw e;
 -            } else {
 -              --retries;
 -              LOG.info("Exception while adding a block", e);
 -              long elapsed = Time.monotonicNow() - localstart;
 -              if (elapsed > 5000) {
 -                LOG.info("Waiting for replication for "
 -                    + (elapsed / 1000) + " seconds");
 -              }
 -              try {
 -                LOG.warn("NotReplicatedYetException sleeping " + src
 -                    + " retries left " + retries);
 -                Thread.sleep(sleeptime);
 -                sleeptime *= 2;
 -              } catch (InterruptedException ie) {
 -                LOG.warn("Caught exception", ie);
 -              }
 -            }
 -          } else {
 -            throw e;
 -          }
 -
 -        }
 -      }
 -    }
++    return DFSOutputStream.addBlock(excludedNodes, dfsClient, src, block,
++        stat.getFileId(), favoredNodes);
+   }
+ 
+   /**
+    * This function sleeps for a certain amount of time when the writing
+    * pipeline is congested. The function calculates the time based on a
+    * decorrelated filter.
+    *
+    * @see
+    * <a href="http://www.awsarchitectureblog.com/2015/03/backoff.html">
+    *   http://www.awsarchitectureblog.com/2015/03/backoff.html</a>
+    */
+   private void backOffIfNecessary() throws InterruptedException {
+     int t = 0;
+     synchronized (congestedNodes) {
+       if (!congestedNodes.isEmpty()) {
+         StringBuilder sb = new StringBuilder("DataNode");
+         for (DatanodeInfo i : congestedNodes) {
+           sb.append(' ').append(i);
+         }
+         int range = Math.abs(lastCongestionBackoffTime * 3 -
+                                 CONGESTION_BACKOFF_MEAN_TIME_IN_MS);
+         int base = Math.min(lastCongestionBackoffTime * 3,
+                             CONGESTION_BACKOFF_MEAN_TIME_IN_MS);
+         t = Math.min(CONGESTION_BACK_OFF_MAX_TIME_IN_MS,
+                      (int)(base + Math.random() * range));
+         lastCongestionBackoffTime = t;
+         sb.append(" are congested. Backing off for ").append(t).append(" ms");
+         LOG.info(sb.toString());
+         congestedNodes.clear();
+       }
+     }
+     if (t != 0) {
+       Thread.sleep(t);
+     }
+   }
+ 
+   /**
+    * get the block this streamer is writing to
+    *
+    * @return the block this streamer is writing to
+    */
+   ExtendedBlock getBlock() {
+     return block;
+   }
+ 
+   /**
+    * return the target datanodes in the pipeline
+    *
+    * @return the target datanodes in the pipeline
+    */
+   DatanodeInfo[] getNodes() {
+     return nodes;
+   }
+ 
++  String[] getStorageIDs() {
++    return storageIDs;
++  }
++
++  BlockConstructionStage getStage() {
++    return stage;
++  }
++
+   /**
+    * return the token of the block
+    *
+    * @return the token of the block
+    */
+   Token<BlockTokenIdentifier> getBlockToken() {
+     return accessToken;
+   }
+ 
++  ErrorState getErrorState() {
++    return errorState;
++  }
++
+   /**
+    * Put a packet to the data queue
+    *
+    * @param packet the packet to be put into the data queued
+    */
+   void queuePacket(DFSPacket packet) {
+     synchronized (dataQueue) {
+       if (packet == null) return;
+       packet.addTraceParent(Tracer.getCurrentSpanId());
+       dataQueue.addLast(packet);
+       lastQueuedSeqno = packet.getSeqno();
+       if (LOG.isDebugEnabled()) {
 -        LOG.debug("Queued packet " + packet.getSeqno());
++        LOG.debug("Queued " + packet + ", " + this);
+       }
+       dataQueue.notifyAll();
+     }
+   }
+ 
+   /**
+    * For heartbeat packets, create buffer directly by new byte[]
+    * since heartbeats should not be blocked.
+    */
+   private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
+     final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
+     return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
+   }
+ 
+   private static LoadingCache<DatanodeInfo, DatanodeInfo> initExcludedNodes(
+       long excludedNodesCacheExpiry) {
+     return CacheBuilder.newBuilder()
+         .expireAfterWrite(excludedNodesCacheExpiry, TimeUnit.MILLISECONDS)
+         .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
+           @Override
+           public void onRemoval(
+               RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
+             LOG.info("Removing node " + notification.getKey()
+                 + " from the excluded nodes list");
+           }
+         }).build(new CacheLoader<DatanodeInfo, DatanodeInfo>() {
+           @Override
+           public DatanodeInfo load(DatanodeInfo key) throws Exception {
+             return key;
+           }
+         });
+   }
+ 
+   private static <T> void arraycopy(T[] srcs, T[] dsts, int skipIndex) {
+     System.arraycopy(srcs, 0, dsts, 0, skipIndex);
+     System.arraycopy(srcs, skipIndex+1, dsts, skipIndex, dsts.length-skipIndex);
+   }
+ 
+   /**
+    * check if to persist blocks on namenode
+    *
+    * @return if to persist blocks on namenode
+    */
+   AtomicBoolean getPersistBlocks(){
+     return persistBlocks;
+   }
+ 
+   /**
+    * check if to append a chunk
+    *
+    * @param appendChunk if to append a chunk
+    */
+   void setAppendChunk(boolean appendChunk){
+     this.appendChunk = appendChunk;
+   }
+ 
+   /**
+    * get if to append a chunk
+    *
+    * @return if to append a chunk
+    */
+   boolean getAppendChunk(){
+     return appendChunk;
+   }
+ 
+   /**
+    * @return the last exception
+    */
+   LastExceptionInStreamer getLastException(){
+     return lastException;
+   }
+ 
+   /**
+    * set socket to null
+    */
+   void setSocketToNull() {
+     this.s = null;
+   }
+ 
+   /**
+    * return current sequence number and then increase it by 1
+    *
+    * @return current sequence number before increasing
+    */
+   long getAndIncCurrentSeqno() {
+     long old = this.currentSeqno;
+     this.currentSeqno++;
+     return old;
+   }
+ 
+   /**
+    * get last queued sequence number
+    *
+    * @return last queued sequence number
+    */
+   long getLastQueuedSeqno() {
+     return lastQueuedSeqno;
+   }
+ 
+   /**
+    * get the number of bytes of current block
+    *
+    * @return the number of bytes of current block
+    */
+   long getBytesCurBlock() {
+     return bytesCurBlock;
+   }
+ 
+   /**
+    * set the bytes of current block that have been written
+    *
+    * @param bytesCurBlock bytes of current block that have been written
+    */
+   void setBytesCurBlock(long bytesCurBlock) {
+     this.bytesCurBlock = bytesCurBlock;
+   }
+ 
+   /**
+    * increase bytes of current block by len.
+    *
+    * @param len how many bytes to increase to current block
+    */
+   void incBytesCurBlock(long len) {
+     this.bytesCurBlock += len;
+   }
+ 
+   /**
+    * set artificial slow down for unit test
+    *
+    * @param period artificial slow down
+    */
+   void setArtificialSlowdown(long period) {
+     this.artificialSlowdown = period;
+   }
+ 
+   /**
+    * if this streamer is to terminate
+    *
+    * @return if this streamer is to terminate
+    */
+   boolean streamerClosed(){
+     return streamerClosed;
+   }
+ 
+   void closeSocket() throws IOException {
+     if (s != null) {
+       s.close();
+     }
+   }
+ 
+   @Override
+   public String toString() {
 -    return  (block == null? null: block.getLocalBlock())
 -        + "@" + Arrays.toString(getNodes());
++    return block == null? "block==null": "" + block.getLocalBlock();
+   }
+ }


[10/50] [abbrv] hadoop git commit: HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC to use BlockUnderConstructionFeature. Contributed by Jing Zhao.

Posted by wa...@apache.org.
HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC to use BlockUnderConstructionFeature. Contributed by Jing Zhao.


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

Branch: refs/heads/trunk
Commit: 164cbe643988f878f0f4100a4de51783e5b6738e
Parents: 067ec8c
Author: Walter Su <wa...@apache.org>
Authored: Thu Aug 27 16:02:30 2015 +0800
Committer: Walter Su <wa...@apache.org>
Committed: Thu Aug 27 16:02:30 2015 +0800

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../hdfs/server/blockmanagement/BlockInfo.java  | 105 +++++--
 .../blockmanagement/BlockInfoContiguous.java    |  23 --
 .../BlockInfoContiguousUnderConstruction.java   | 281 ------------------
 .../blockmanagement/BlockInfoStriped.java       |  21 --
 .../BlockInfoStripedUnderConstruction.java      | 297 -------------------
 .../BlockInfoUnderConstruction.java             |  84 ------
 .../server/blockmanagement/BlockManager.java    |  96 +++---
 .../BlockUnderConstructionFeature.java          | 269 +++++++++++++++++
 .../blockmanagement/DatanodeDescriptor.java     |  10 +-
 .../server/blockmanagement/DatanodeManager.java |  24 +-
 .../hdfs/server/namenode/FSDirTruncateOp.java   |  42 +--
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  31 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  15 +-
 .../hdfs/server/namenode/FSImageFormat.java     |   9 +-
 .../server/namenode/FSImageFormatPBINode.java   |   9 +-
 .../server/namenode/FSImageSerialization.java   |   6 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  40 ++-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  33 +--
 .../server/namenode/snapshot/FileDiffList.java  |   3 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   9 +-
 .../TestBlockInfoUnderConstruction.java         |  17 +-
 .../blockmanagement/TestBlockManager.java       |   9 +-
 .../blockmanagement/TestHeartbeatHandling.java  |  21 +-
 .../blockmanagement/TestReplicationPolicy.java  |  12 +-
 .../server/namenode/TestAddStripedBlocks.java   |  38 ++-
 .../namenode/TestBlockUnderConstruction.java    |   6 +-
 .../TestCommitBlockSynchronization.java         |   9 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |   5 +-
 .../server/namenode/TestStripedINodeFile.java   |  11 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |  11 +-
 .../namenode/snapshot/SnapshotTestHelper.java   |   6 +-
 32 files changed, 577 insertions(+), 978 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 8b25e68..28cc34a 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -403,3 +403,6 @@
 
     HDFS-8838. Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream
     when the data length is small. (szetszwo via waltersu4549)
+
+    HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC
+    to use BlockUnderConstructionFeature. (Jing Zhao via waltersu4549)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index bf11914..f440e14 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import java.io.IOException;
 import java.util.LinkedList;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -52,6 +54,8 @@ public abstract class BlockInfo extends Block
    */
   protected Object[] triplets;
 
+  private BlockUnderConstructionFeature uc;
+
   /**
    * Construct an entry for blocksmap
    * @param size the block's replication factor, or the total number of blocks
@@ -287,26 +291,6 @@ public abstract class BlockInfo extends Block
     return this;
   }
 
-  /**
-   * BlockInfo represents a block that is not being constructed.
-   * In order to start modifying the block, the BlockInfo should be converted to
-   * {@link BlockInfoContiguousUnderConstruction} or
-   * {@link BlockInfoStripedUnderConstruction}.
-   * @return {@link BlockUCState#COMPLETE}
-   */
-  public BlockUCState getBlockUCState() {
-    return BlockUCState.COMPLETE;
-  }
-
-  /**
-   * Is this block complete?
-   *
-   * @return true if the state of the block is {@link BlockUCState#COMPLETE}
-   */
-  public boolean isComplete() {
-    return getBlockUCState().equals(BlockUCState.COMPLETE);
-  }
-
   public boolean isDeleted() {
     return (bc == null);
   }
@@ -332,4 +316,85 @@ public abstract class BlockInfo extends Block
   public void setNext(LightWeightGSet.LinkedElement next) {
     this.nextLinkedElement = next;
   }
+
+  /* UnderConstruction Feature related */
+
+  public BlockUnderConstructionFeature getUnderConstructionFeature() {
+    return uc;
+  }
+
+  public BlockUCState getBlockUCState() {
+    return uc == null ? BlockUCState.COMPLETE : uc.getBlockUCState();
+  }
+
+  /**
+   * Is this block complete?
+   *
+   * @return true if the state of the block is {@link BlockUCState#COMPLETE}
+   */
+  public boolean isComplete() {
+    return getBlockUCState().equals(BlockUCState.COMPLETE);
+  }
+
+  /**
+   * Add/Update the under construction feature.
+   */
+  public void convertToBlockUnderConstruction(BlockUCState s,
+      DatanodeStorageInfo[] targets) {
+    if (isComplete()) {
+      uc = new BlockUnderConstructionFeature(this, s, targets, this.isStriped());
+    } else {
+      // the block is already under construction
+      uc.setBlockUCState(s);
+      uc.setExpectedLocations(this, targets, this.isStriped());
+    }
+  }
+
+  /**
+   * Convert an under construction block to a complete block.
+   *
+   * @return BlockInfo - a complete block.
+   * @throws IOException if the state of the block
+   * (the generation stamp and the length) has not been committed by
+   * the client or it does not have at least a minimal number of replicas
+   * reported from data-nodes.
+   */
+  BlockInfo convertToCompleteBlock() throws IOException {
+    assert getBlockUCState() != BlockUCState.COMPLETE :
+        "Trying to convert a COMPLETE block";
+    uc = null;
+    return this;
+  }
+
+  /**
+   * Process the recorded replicas. When about to commit or finish the
+   * pipeline recovery sort out bad replicas.
+   * @param genStamp  The final generation stamp for the block.
+   */
+  public void setGenerationStampAndVerifyReplicas(long genStamp) {
+    Preconditions.checkState(uc != null && !isComplete());
+    // Set the generation stamp for the block.
+    setGenerationStamp(genStamp);
+
+    // Remove the replicas with wrong gen stamp
+    uc.removeStaleReplicas(this);
+  }
+
+  /**
+   * Commit block's length and generation stamp as reported by the client.
+   * Set block state to {@link BlockUCState#COMMITTED}.
+   * @param block - contains client reported block length and generation
+   * @throws IOException if block ids are inconsistent.
+   */
+  void commitBlock(Block block) throws IOException {
+    if (getBlockId() != block.getBlockId()) {
+      throw new IOException("Trying to commit inconsistent block: id = "
+          + block.getBlockId() + ", expected id = " + getBlockId());
+    }
+    Preconditions.checkState(!isComplete());
+    uc.commit();
+    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
+    // Sort out invalid replicas.
+    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index bb9bf5b..12b4fd3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
 /**
  * Subclass of {@link BlockInfo}, used for a block with replication scheme.
@@ -123,28 +122,6 @@ public class BlockInfoContiguous extends BlockInfo {
     }
   }
 
-  /**
-   * Convert a complete block to an under construction block.
-   * @return BlockInfoUnderConstruction -  an under construction block.
-   */
-  public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction(
-      BlockUCState s, DatanodeStorageInfo[] targets) {
-    if(isComplete()) {
-      BlockInfoContiguousUnderConstruction ucBlock =
-          new BlockInfoContiguousUnderConstruction(this,
-          getBlockCollection().getPreferredBlockReplication(), s, targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-      return ucBlock;
-    }
-    // the block is already under construction
-    BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction) this;
-    ucBlock.setBlockUCState(s);
-    ucBlock.setExpectedLocations(targets);
-    ucBlock.setBlockCollection(getBlockCollection());
-    return ucBlock;
-  }
-
   @Override
   public final boolean isStriped() {
     return false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
deleted file mode 100644
index 96b209d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
+++ /dev/null
@@ -1,281 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-
-/**
- * Represents a block that is currently being constructed.<br>
- * This is usually the last block of a file opened for write or append.
- */
-public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous
-    implements BlockInfoUnderConstruction{
-  /** Block state. See {@link BlockUCState} */
-  private BlockUCState blockUCState;
-
-  /**
-   * Block replicas as assigned when the block was allocated.
-   * This defines the pipeline order.
-   */
-  private List<ReplicaUnderConstruction> replicas;
-
-  /**
-   * Index of the primary data node doing the recovery. Useful for log
-   * messages.
-   */
-  private int primaryNodeIndex = -1;
-
-  /**
-   * The new generation stamp, which this block will have
-   * after the recovery succeeds. Also used as a recovery id to identify
-   * the right recovery if any of the abandoned recoveries re-appear.
-   */
-  private long blockRecoveryId = 0;
-
-  /**
-   * The block source to use in the event of copy-on-write truncate.
-   */
-  private Block truncateBlock;
-
-  /**
-   * Create block and set its state to
-   * {@link BlockUCState#UNDER_CONSTRUCTION}.
-   */
-  public BlockInfoContiguousUnderConstruction(Block blk, short replication) {
-    this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
-  }
-
-  /**
-   * Create a block that is currently being constructed.
-   */
-  public BlockInfoContiguousUnderConstruction(Block blk, short replication,
-      BlockUCState state, DatanodeStorageInfo[] targets) {
-    super(blk, replication);
-    assert getBlockUCState() != BlockUCState.COMPLETE :
-      "BlockInfoContiguousUnderConstruction cannot be in COMPLETE state";
-    this.blockUCState = state;
-    setExpectedLocations(targets);
-  }
-
-  @Override
-  public BlockInfoContiguous convertToCompleteBlock() throws IOException {
-    assert getBlockUCState() != BlockUCState.COMPLETE :
-      "Trying to convert a COMPLETE block";
-    return new BlockInfoContiguous(this);
-  }
-
-  @Override
-  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
-    int numLocations = targets == null ? 0 : targets.length;
-    this.replicas = new ArrayList<>(numLocations);
-    for(int i = 0; i < numLocations; i++) {
-      replicas.add(new ReplicaUnderConstruction(this, targets[i],
-          ReplicaState.RBW));
-    }
-  }
-
-  @Override
-  public DatanodeStorageInfo[] getExpectedStorageLocations() {
-    int numLocations = replicas == null ? 0 : replicas.size();
-    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
-    for (int i = 0; i < numLocations; i++) {
-      storages[i] = replicas.get(i).getExpectedStorageLocation();
-    }
-    return storages;
-  }
-
-  @Override
-  public int getNumExpectedLocations() {
-    return replicas == null ? 0 : replicas.size();
-  }
-
-  /**
-   * Return the state of the block under construction.
-   * @see BlockUCState
-   */
-  @Override // BlockInfo
-  public BlockUCState getBlockUCState() {
-    return blockUCState;
-  }
-
-  void setBlockUCState(BlockUCState s) {
-    blockUCState = s;
-  }
-
-  @Override
-  public long getBlockRecoveryId() {
-    return blockRecoveryId;
-  }
-
-  @Override
-  public Block getTruncateBlock() {
-    return truncateBlock;
-  }
-
-  @Override
-  public Block toBlock(){
-    return this;
-  }
-
-  public void setTruncateBlock(Block recoveryBlock) {
-    this.truncateBlock = recoveryBlock;
-  }
-
-  @Override
-  public void setGenerationStampAndVerifyReplicas(long genStamp) {
-    // Set the generation stamp for the block.
-    setGenerationStamp(genStamp);
-    if (replicas == null)
-      return;
-
-    // Remove the replicas with wrong gen stamp.
-    // The replica list is unchanged.
-    for (ReplicaUnderConstruction r : replicas) {
-      if (genStamp != r.getGenerationStamp()) {
-        r.getExpectedStorageLocation().removeBlock(this);
-        NameNode.blockStateChangeLog.debug("BLOCK* Removing stale replica "
-            + "from location: {}", r.getExpectedStorageLocation());
-      }
-    }
-  }
-
-  @Override
-  public void commitBlock(Block block) throws IOException {
-    if(getBlockId() != block.getBlockId())
-      throw new IOException("Trying to commit inconsistent block: id = "
-          + block.getBlockId() + ", expected id = " + getBlockId());
-    blockUCState = BlockUCState.COMMITTED;
-    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
-    // Sort out invalid replicas.
-    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
-  }
-
-  @Override
-  public void initializeBlockRecovery(long recoveryId) {
-    setBlockUCState(BlockUCState.UNDER_RECOVERY);
-    blockRecoveryId = recoveryId;
-    if (replicas.size() == 0) {
-      NameNode.blockStateChangeLog.warn("BLOCK*"
-        + " BlockInfoContiguousUnderConstruction.initLeaseRecovery:"
-        + " No blocks found, lease removed.");
-    }
-    boolean allLiveReplicasTriedAsPrimary = true;
-    for (ReplicaUnderConstruction replica : replicas) {
-      // Check if all replicas have been tried or not.
-      if (replica.isAlive()) {
-        allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary &&
-            replica.getChosenAsPrimary());
-      }
-    }
-    if (allLiveReplicasTriedAsPrimary) {
-      // Just set all the replicas to be chosen whether they are alive or not.
-      for (ReplicaUnderConstruction replica : replicas) {
-        replica.setChosenAsPrimary(false);
-      }
-    }
-    long mostRecentLastUpdate = 0;
-    ReplicaUnderConstruction primary = null;
-    primaryNodeIndex = -1;
-    for(int i = 0; i < replicas.size(); i++) {
-      // Skip alive replicas which have been chosen for recovery.
-      if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
-        continue;
-      }
-      final ReplicaUnderConstruction ruc = replicas.get(i);
-      final long lastUpdate = ruc.getExpectedStorageLocation()
-          .getDatanodeDescriptor().getLastUpdateMonotonic();
-      if (lastUpdate > mostRecentLastUpdate) {
-        primaryNodeIndex = i;
-        primary = ruc;
-        mostRecentLastUpdate = lastUpdate;
-      }
-    }
-    if (primary != null) {
-      primary.getExpectedStorageLocation().getDatanodeDescriptor()
-          .addBlockToBeRecovered(this);
-      primary.setChosenAsPrimary(true);
-      NameNode.blockStateChangeLog.debug(
-          "BLOCK* {} recovery started, primary={}", this, primary);
-    }
-  }
-
-  @Override
-  public void addReplicaIfNotPresent(DatanodeStorageInfo storage,
-      Block block, ReplicaState rState) {
-    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
-    while (it.hasNext()) {
-      ReplicaUnderConstruction r = it.next();
-      DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
-      if(expectedLocation == storage) {
-        // Record the gen stamp from the report
-        r.setGenerationStamp(block.getGenerationStamp());
-        return;
-      } else if (expectedLocation != null &&
-                 expectedLocation.getDatanodeDescriptor() ==
-                     storage.getDatanodeDescriptor()) {
-
-        // The Datanode reported that the block is on a different storage
-        // than the one chosen by BlockPlacementPolicy. This can occur as
-        // we allow Datanodes to choose the target storage. Update our
-        // state by removing the stale entry and adding a new one.
-        it.remove();
-        break;
-      }
-    }
-    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder b = new StringBuilder(100);
-    appendStringTo(b);
-    return b.toString();
-  }
-
-  @Override
-  public void appendStringTo(StringBuilder sb) {
-    super.appendStringTo(sb);
-    appendUCParts(sb);
-  }
-
-  private void appendUCParts(StringBuilder sb) {
-    sb.append("{UCState=").append(blockUCState)
-      .append(", truncateBlock=" + truncateBlock)
-      .append(", primaryNodeIndex=").append(primaryNodeIndex)
-      .append(", replicas=[");
-    if (replicas != null) {
-      Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
-      if (iter.hasNext()) {
-        iter.next().appendStringTo(sb);
-        while (iter.hasNext()) {
-          sb.append(", ");
-          iter.next().appendStringTo(sb);
-        }
-      }
-    }
-    sb.append("]}");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 14d2fcc..6093776 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -245,27 +245,6 @@ public class BlockInfoStriped extends BlockInfo {
     return num;
   }
 
-  /**
-   * Convert a complete block to an under construction block.
-   * @return BlockInfoUnderConstruction -  an under construction block.
-   */
-  public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction(
-      BlockUCState s, DatanodeStorageInfo[] targets) {
-    final BlockInfoStripedUnderConstruction ucBlock;
-    if(isComplete()) {
-      ucBlock = new BlockInfoStripedUnderConstruction(this, ecPolicy,
-          s, targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-    } else {
-      // the block is already under construction
-      ucBlock = (BlockInfoStripedUnderConstruction) this;
-      ucBlock.setBlockUCState(s);
-      ucBlock.setExpectedLocations(targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-    }
-    return ucBlock;
-  }
-
   @Override
   final boolean hasNoStorage() {
     final int len = getCapacity();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
deleted file mode 100644
index 9de8294..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
+++ /dev/null
@@ -1,297 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE;
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
-
-/**
- * Represents a striped block that is currently being constructed.
- * This is usually the last block of a file opened for write or append.
- */
-public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
-    implements BlockInfoUnderConstruction{
-  private BlockUCState blockUCState;
-
-  /**
-   * Block replicas as assigned when the block was allocated.
-   */
-  private ReplicaUnderConstruction[] replicas;
-
-  /**
-   * Index of the primary data node doing the recovery. Useful for log
-   * messages.
-   */
-  private int primaryNodeIndex = -1;
-
-  /**
-   * The new generation stamp, which this block will have
-   * after the recovery succeeds. Also used as a recovery id to identify
-   * the right recovery if any of the abandoned recoveries re-appear.
-   */
-  private long blockRecoveryId = 0;
-
-  /**
-   * Constructor with null storage targets.
-   */
-  public BlockInfoStripedUnderConstruction(Block blk, ErasureCodingPolicy ecPolicy) {
-    this(blk, ecPolicy, UNDER_CONSTRUCTION, null);
-  }
-
-  /**
-   * Create a striped block that is currently being constructed.
-   */
-  public BlockInfoStripedUnderConstruction(Block blk, ErasureCodingPolicy ecPolicy,
-      BlockUCState state, DatanodeStorageInfo[] targets) {
-    super(blk, ecPolicy);
-    assert getBlockUCState() != COMPLETE :
-      "BlockInfoStripedUnderConstruction cannot be in COMPLETE state";
-    this.blockUCState = state;
-    setExpectedLocations(targets);
-  }
-
-  @Override
-  public BlockInfoStriped convertToCompleteBlock() throws IOException {
-    assert getBlockUCState() != COMPLETE :
-      "Trying to convert a COMPLETE block";
-    return new BlockInfoStriped(this);
-  }
-
-  /** Set expected locations */
-  @Override
-  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
-    int numLocations = targets == null ? 0 : targets.length;
-    this.replicas = new ReplicaUnderConstruction[numLocations];
-    for(int i = 0; i < numLocations; i++) {
-      // when creating a new block we simply sequentially assign block index to
-      // each storage
-      Block blk = new Block(this.getBlockId() + i, 0, this.getGenerationStamp());
-      replicas[i] = new ReplicaUnderConstruction(blk, targets[i],
-          ReplicaState.RBW);
-    }
-  }
-
-  /**
-   * Create array of expected replica locations
-   * (as has been assigned by chooseTargets()).
-   */
-  @Override
-  public DatanodeStorageInfo[] getExpectedStorageLocations() {
-    int numLocations = getNumExpectedLocations();
-    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
-    for (int i = 0; i < numLocations; i++) {
-      storages[i] = replicas[i].getExpectedStorageLocation();
-    }
-    return storages;
-  }
-
-  /** @return the index array indicating the block index in each storage */
-  public int[] getBlockIndices() {
-    int numLocations = getNumExpectedLocations();
-    int[] indices = new int[numLocations];
-    for (int i = 0; i < numLocations; i++) {
-      indices[i] = BlockIdManager.getBlockIndex(replicas[i]);
-    }
-    return indices;
-  }
-
-  @Override
-  public int getNumExpectedLocations() {
-    return replicas == null ? 0 : replicas.length;
-  }
-
-  /**
-   * Return the state of the block under construction.
-   * @see BlockUCState
-   */
-  @Override // BlockInfo
-  public BlockUCState getBlockUCState() {
-    return blockUCState;
-  }
-
-  void setBlockUCState(BlockUCState s) {
-    blockUCState = s;
-  }
-
-  @Override
-  public long getBlockRecoveryId() {
-    return blockRecoveryId;
-  }
-
-  @Override
-  public Block getTruncateBlock() {
-    return null;
-  }
-
-  @Override
-  public Block toBlock(){
-    return this;
-  }
-
-  @Override
-  public void setGenerationStampAndVerifyReplicas(long genStamp) {
-    // Set the generation stamp for the block.
-    setGenerationStamp(genStamp);
-    if (replicas == null)
-      return;
-
-    // Remove the replicas with wrong gen stamp.
-    // The replica list is unchanged.
-    for (ReplicaUnderConstruction r : replicas) {
-      if (genStamp != r.getGenerationStamp()) {
-        r.getExpectedStorageLocation().removeBlock(this);
-        NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
-            + "from location: {}", r.getExpectedStorageLocation());
-      }
-    }
-  }
-
-  @Override
-  public void commitBlock(Block block) throws IOException {
-    if (getBlockId() != block.getBlockId()) {
-      throw new IOException("Trying to commit inconsistent block: id = "
-          + block.getBlockId() + ", expected id = " + getBlockId());
-    }
-    blockUCState = BlockUCState.COMMITTED;
-    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
-    // Sort out invalid replicas.
-    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
-  }
-
-  @Override
-  public void initializeBlockRecovery(long recoveryId) {
-    setBlockUCState(BlockUCState.UNDER_RECOVERY);
-    blockRecoveryId = recoveryId;
-    if (replicas == null || replicas.length == 0) {
-      NameNode.blockStateChangeLog.warn("BLOCK*" +
-          " BlockInfoStripedUnderConstruction.initLeaseRecovery:" +
-          " No blocks found, lease removed.");
-      // sets primary node index and return.
-      primaryNodeIndex = -1;
-      return;
-    }
-    boolean allLiveReplicasTriedAsPrimary = true;
-    for (ReplicaUnderConstruction replica : replicas) {
-      // Check if all replicas have been tried or not.
-      if (replica.isAlive()) {
-        allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary &&
-            replica.getChosenAsPrimary());
-      }
-    }
-    if (allLiveReplicasTriedAsPrimary) {
-      // Just set all the replicas to be chosen whether they are alive or not.
-      for (ReplicaUnderConstruction replica : replicas) {
-        replica.setChosenAsPrimary(false);
-      }
-    }
-    long mostRecentLastUpdate = 0;
-    ReplicaUnderConstruction primary = null;
-    primaryNodeIndex = -1;
-    for(int i = 0; i < replicas.length; i++) {
-      // Skip alive replicas which have been chosen for recovery.
-      if (!(replicas[i].isAlive() && !replicas[i].getChosenAsPrimary())) {
-        continue;
-      }
-      final ReplicaUnderConstruction ruc = replicas[i];
-      final long lastUpdate = ruc.getExpectedStorageLocation()
-          .getDatanodeDescriptor().getLastUpdateMonotonic();
-      if (lastUpdate > mostRecentLastUpdate) {
-        primaryNodeIndex = i;
-        primary = ruc;
-        mostRecentLastUpdate = lastUpdate;
-      }
-    }
-    if (primary != null) {
-      primary.getExpectedStorageLocation().getDatanodeDescriptor()
-          .addBlockToBeRecovered(this);
-      primary.setChosenAsPrimary(true);
-      NameNode.blockStateChangeLog.info(
-          "BLOCK* {} recovery started, primary={}", this, primary);
-    }
-  }
-
-  @Override
-  public void addReplicaIfNotPresent(DatanodeStorageInfo storage,
-      Block reportedBlock, ReplicaState rState) {
-    if (replicas == null) {
-      replicas = new ReplicaUnderConstruction[1];
-      replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage, rState);
-    } else {
-      for (int i = 0; i < replicas.length; i++) {
-        DatanodeStorageInfo expected = replicas[i].getExpectedStorageLocation();
-        if (expected == storage) {
-          replicas[i].setBlockId(reportedBlock.getBlockId());
-          replicas[i].setGenerationStamp(reportedBlock.getGenerationStamp());
-          return;
-        } else if (expected != null && expected.getDatanodeDescriptor() ==
-            storage.getDatanodeDescriptor()) {
-          // The Datanode reported that the block is on a different storage
-          // than the one chosen by BlockPlacementPolicy. This can occur as
-          // we allow Datanodes to choose the target storage. Update our
-          // state by removing the stale entry and adding a new one.
-          replicas[i] = new ReplicaUnderConstruction(reportedBlock, storage,
-              rState);
-          return;
-        }
-      }
-      ReplicaUnderConstruction[] newReplicas =
-          new ReplicaUnderConstruction[replicas.length + 1];
-      System.arraycopy(replicas, 0, newReplicas, 0, replicas.length);
-      newReplicas[newReplicas.length - 1] = new ReplicaUnderConstruction(
-          reportedBlock, storage, rState);
-      replicas = newReplicas;
-    }
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder b = new StringBuilder(100);
-    appendStringTo(b);
-    return b.toString();
-  }
-
-  @Override
-  public void appendStringTo(StringBuilder sb) {
-    super.appendStringTo(sb);
-    appendUCParts(sb);
-  }
-
-  private void appendUCParts(StringBuilder sb) {
-    sb.append("{UCState=").append(blockUCState).
-        append(", primaryNodeIndex=").append(primaryNodeIndex).
-        append(", replicas=[");
-    if (replicas != null) {
-      int i = 0;
-      for (ReplicaUnderConstruction r : replicas) {
-        r.appendStringTo(sb);
-        if (++i < replicas.length) {
-          sb.append(", ");
-        }
-      }
-    }
-    sb.append("]}");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
deleted file mode 100644
index 10a8cae..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-
-public interface BlockInfoUnderConstruction {
-  /**
-   * Create array of expected replica locations
-   * (as has been assigned by chooseTargets()).
-   */
-  public DatanodeStorageInfo[] getExpectedStorageLocations();
-
-  /** Get recover block */
-  public Block getTruncateBlock();
-
-  /** Convert to a Block object */
-  public Block toBlock();
-
-  /** Get block recovery ID */
-  public long getBlockRecoveryId();
-
-  /** Get the number of expected locations */
-  public int getNumExpectedLocations();
-
-  /** Set expected locations */
-  public void setExpectedLocations(DatanodeStorageInfo[] targets);
-
-  /**
-   * Process the recorded replicas. When about to commit or finish the
-   * pipeline recovery sort out bad replicas.
-   * @param genStamp  The final generation stamp for the block.
-   */
-  public void setGenerationStampAndVerifyReplicas(long genStamp);
-
-  /**
-   * Initialize lease recovery for this block.
-   * Find the first alive data-node starting from the previous primary and
-   * make it primary.
-   */
-  public void initializeBlockRecovery(long recoveryId);
-  
-  /** Add the reported replica if it is not already in the replica list. */
-  public void addReplicaIfNotPresent(DatanodeStorageInfo storage,
-      Block reportedBlock, ReplicaState rState);
-
-  /**
-   * Commit block's length and generation stamp as reported by the client.
-   * Set block state to {@link BlockUCState#COMMITTED}.
-   * @param block - contains client reported block length and generation 
-   * @throws IOException if block ids are inconsistent.
-   */
-  public void commitBlock(Block block) throws IOException;
-
-  /**
-   * Convert an under construction block to a complete block.
-   * 
-   * @return a complete block.
-   * @throws IOException
-   *           if the state of the block (the generation stamp and the length)
-   *           has not been committed by the client or it does not have at least
-   *           a minimal number of replicas reported from data-nodes.
-   */
-  public BlockInfo convertToCompleteBlock() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index dfea5f3..ae08825 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -644,19 +644,13 @@ public class BlockManager implements BlockStatsMXBean {
    */
   private static boolean commitBlock(final BlockInfo block,
       final Block commitBlock) throws IOException {
-    if (block instanceof BlockInfoUnderConstruction
-        && block.getBlockUCState() != BlockUCState.COMMITTED) {
-      final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block;
-
-      assert block.getNumBytes() <= commitBlock.getNumBytes() :
+    if (block.getBlockUCState() == BlockUCState.COMMITTED)
+      return false;
+    assert block.getNumBytes() <= commitBlock.getNumBytes() :
         "commitBlock length is less than the stored one "
-        + commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
-
-      uc.commitBlock(commitBlock);
-      return true;
-    }
-
-    return false;
+            + commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
+    block.commitBlock(commitBlock);
+    return true;
   }
   
   /**
@@ -713,9 +707,7 @@ public class BlockManager implements BlockStatsMXBean {
           "Cannot complete block: block has not been COMMITTED by the client");
     }
 
-    final BlockInfo completeBlock
-        = !(curBlock instanceof BlockInfoUnderConstruction)? curBlock
-            : ((BlockInfoUnderConstruction)curBlock).convertToCompleteBlock();
+    final BlockInfo completeBlock = curBlock.convertToCompleteBlock();
 
     // replace penultimate block in file
     bc.setBlock(blkIndex, completeBlock);
@@ -754,9 +746,7 @@ public class BlockManager implements BlockStatsMXBean {
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
       final BlockInfo block) throws IOException {
-    if (block instanceof BlockInfoUnderConstruction) {
-      ((BlockInfoUnderConstruction)block).commitBlock(block);
-    }
+    block.commitBlock(block);
     return completeBlock(bc, block, true);
   }
 
@@ -777,30 +767,28 @@ public class BlockManager implements BlockStatsMXBean {
    */
   public LocatedBlock convertLastBlockToUnderConstruction(
       BlockCollection bc, long bytesToRemove) throws IOException {
-    BlockInfo oldBlock = bc.getLastBlock();
-    if(oldBlock == null ||
-       bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
+    BlockInfo lastBlock = bc.getLastBlock();
+    if(lastBlock == null ||
+       bc.getPreferredBlockSize() == lastBlock.getNumBytes() - bytesToRemove)
       return null;
-    assert oldBlock == getStoredBlock(oldBlock) :
+    assert lastBlock == getStoredBlock(lastBlock) :
       "last block of the file is not in blocksMap";
 
-    DatanodeStorageInfo[] targets = getStorages(oldBlock);
+    DatanodeStorageInfo[] targets = getStorages(lastBlock);
 
-    // convert the last block to UC
-    bc.convertLastBlockToUC(oldBlock, targets);
-    // get the new created uc block
-    BlockInfo ucBlock = bc.getLastBlock();
-    blocksMap.replaceBlock(ucBlock);
+    // convert the last block to under construction. note no block replacement
+    // is happening
+    bc.convertLastBlockToUC(lastBlock, targets);
 
     // Remove block from replication queue.
-    NumberReplicas replicas = countNodes(ucBlock);
-    neededReplications.remove(ucBlock, replicas.liveReplicas(),
-        replicas.decommissionedAndDecommissioning(), getReplication(ucBlock));
-    pendingReplications.remove(ucBlock);
+    NumberReplicas replicas = countNodes(lastBlock);
+    neededReplications.remove(lastBlock, replicas.liveReplicas(),
+        replicas.decommissionedAndDecommissioning(), getReplication(lastBlock));
+    pendingReplications.remove(lastBlock);
 
     // remove this block from the list of pending blocks to be deleted. 
     for (DatanodeStorageInfo storage : targets) {
-      final Block b = getBlockOnStorage(oldBlock, storage);
+      final Block b = getBlockOnStorage(lastBlock, storage);
       if (b != null) {
         invalidateBlocks.remove(storage.getDatanodeDescriptor(), b);
       }
@@ -810,13 +798,15 @@ public class BlockManager implements BlockStatsMXBean {
     // count in safe-mode.
     namesystem.adjustSafeModeBlockTotals(
         // decrement safe if we had enough
-        hasMinStorage(oldBlock, targets.length) ? -1 : 0,
+        hasMinStorage(lastBlock, targets.length) ? -1 : 0,
         // always decrement total blocks
         -1);
 
-    final long fileLength = bc.computeContentSummary(getStoragePolicySuite()).getLength();
-    final long pos = fileLength - ucBlock.getNumBytes();
-    return createLocatedBlock(ucBlock, pos, BlockTokenIdentifier.AccessMode.WRITE);
+    final long fileLength = bc.computeContentSummary(
+        getStoragePolicySuite()).getLength();
+    final long pos = fileLength - lastBlock.getNumBytes();
+    return createLocatedBlock(lastBlock, pos,
+        BlockTokenIdentifier.AccessMode.WRITE);
   }
 
   /**
@@ -895,18 +885,14 @@ public class BlockManager implements BlockStatsMXBean {
   private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos)
       throws IOException {
     if (!blk.isComplete()) {
+      final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature();
       if (blk.isStriped()) {
-        final BlockInfoStripedUnderConstruction uc =
-            (BlockInfoStripedUnderConstruction) blk;
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
         return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
             false);
       } else {
-        assert blk instanceof BlockInfoContiguousUnderConstruction;
-        final BlockInfoContiguousUnderConstruction uc =
-            (BlockInfoContiguousUnderConstruction) blk;
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
@@ -1923,9 +1909,7 @@ public class BlockManager implements BlockStatsMXBean {
     
     StatefulBlockInfo(BlockInfo storedBlock,
         Block reportedBlock, ReplicaState reportedState) {
-      Preconditions.checkArgument(
-          storedBlock instanceof BlockInfoContiguousUnderConstruction ||
-          storedBlock instanceof BlockInfoStripedUnderConstruction);
+      Preconditions.checkArgument(!storedBlock.isComplete());
       this.storedBlock = storedBlock;
       this.reportedBlock = reportedBlock;
       this.reportedState = reportedState;
@@ -2335,13 +2319,14 @@ public class BlockManager implements BlockStatsMXBean {
       
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-        final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)storedBlock;
-        uc.addReplicaIfNotPresent(storageInfo, iblk, reportedState);
+        storedBlock.getUnderConstructionFeature()
+            .addReplicaIfNotPresent(storageInfo, iblk, reportedState);
         // OpenFileBlocks only inside snapshots also will be added to safemode
         // threshold. So we need to update such blocks to safemode
         // refer HDFS-5283
         if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) {
-          int numOfReplicas = uc.getNumExpectedLocations();
+          int numOfReplicas = storedBlock.getUnderConstructionFeature()
+              .getNumExpectedLocations();
           namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock);
         }
         //and fall through to next clause
@@ -2469,11 +2454,6 @@ public class BlockManager implements BlockStatsMXBean {
 
     // Ignore replicas already scheduled to be removed from the DN
     if(invalidateBlocks.contains(dn, block)) {
-      /*
-       * TODO: following assertion is incorrect, see HDFS-2668 assert
-       * storedBlock.findDatanode(dn) < 0 : "Block " + block +
-       * " in recentInvalidatesSet should not appear in DN " + dn;
-       */
       return storedBlock;
     }
 
@@ -2704,9 +2684,8 @@ public class BlockManager implements BlockStatsMXBean {
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
       DatanodeStorageInfo storageInfo) throws IOException {
     BlockInfo block = ucBlock.storedBlock;
-    final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block;
-    uc.addReplicaIfNotPresent(storageInfo, ucBlock.reportedBlock,
-        ucBlock.reportedState);
+    block.getUnderConstructionFeature().addReplicaIfNotPresent(storageInfo,
+        ucBlock.reportedBlock, ucBlock.reportedState);
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
         (block.findStorageInfo(storageInfo) < 0)) {
@@ -2766,8 +2745,7 @@ public class BlockManager implements BlockStatsMXBean {
     assert block != null && namesystem.hasWriteLock();
     BlockInfo storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
-    if (block instanceof BlockInfoContiguousUnderConstruction ||
-        block instanceof BlockInfoStripedUnderConstruction) {
+    if (!block.isComplete()) {
       //refresh our copy in case the block got completed in another thread
       storedBlock = getStoredBlock(block);
     } else {
@@ -4221,7 +4199,7 @@ public class BlockManager implements BlockStatsMXBean {
     final LocatedBlock lb;
     if (info.isStriped()) {
       lb = newLocatedStripedBlock(eb, locs,
-          ((BlockInfoStripedUnderConstruction)info).getBlockIndices(),
+          info.getUnderConstructionFeature().getBlockIndices(),
           offset, false);
     } else {
       lb = newLocatedBlock(eb, locs, offset, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
new file mode 100644
index 0000000..58b455e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
@@ -0,0 +1,269 @@
+/**
+ * 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.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE;
+
+/**
+ * Represents the under construction feature of a Block.
+ * This is usually the last block of a file opened for write or append.
+ */
+public class BlockUnderConstructionFeature {
+  private BlockUCState blockUCState;
+
+  /**
+   * Block replicas as assigned when the block was allocated.
+   */
+  private ReplicaUnderConstruction[] replicas;
+
+  /**
+   * Index of the primary data node doing the recovery. Useful for log
+   * messages.
+   */
+  private int primaryNodeIndex = -1;
+
+  /**
+   * The new generation stamp, which this block will have
+   * after the recovery succeeds. Also used as a recovery id to identify
+   * the right recovery if any of the abandoned recoveries re-appear.
+   */
+  private long blockRecoveryId = 0;
+
+  /**
+   * The block source to use in the event of copy-on-write truncate.
+   */
+  private Block truncateBlock;
+
+  public BlockUnderConstructionFeature(Block blk,
+      BlockUCState state, DatanodeStorageInfo[] targets, boolean isStriped) {
+    assert getBlockUCState() != COMPLETE :
+      "BlockUnderConstructionFeature cannot be in COMPLETE state";
+    this.blockUCState = state;
+    setExpectedLocations(blk, targets, isStriped);
+  }
+
+  /** Set expected locations */
+  public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets,
+      boolean isStriped) {
+    int numLocations = targets == null ? 0 : targets.length;
+    this.replicas = new ReplicaUnderConstruction[numLocations];
+    for(int i = 0; i < numLocations; i++) {
+      // when creating a new striped block we simply sequentially assign block
+      // index to each storage
+      Block replicaBlock = isStriped ?
+          new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
+          block;
+      replicas[i] = new ReplicaUnderConstruction(replicaBlock, targets[i],
+          ReplicaState.RBW);
+    }
+  }
+
+  /**
+   * Create array of expected replica locations
+   * (as has been assigned by chooseTargets()).
+   */
+  public DatanodeStorageInfo[] getExpectedStorageLocations() {
+    int numLocations = getNumExpectedLocations();
+    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
+    for (int i = 0; i < numLocations; i++) {
+      storages[i] = replicas[i].getExpectedStorageLocation();
+    }
+    return storages;
+  }
+
+  /**
+   * @return the index array indicating the block index in each storage. Used
+   * only by striped blocks.
+   */
+  public int[] getBlockIndices() {
+    int numLocations = getNumExpectedLocations();
+    int[] indices = new int[numLocations];
+    for (int i = 0; i < numLocations; i++) {
+      indices[i] = BlockIdManager.getBlockIndex(replicas[i]);
+    }
+    return indices;
+  }
+
+  public int getNumExpectedLocations() {
+    return replicas == null ? 0 : replicas.length;
+  }
+
+  /**
+   * Return the state of the block under construction.
+   * @see BlockUCState
+   */
+  public BlockUCState getBlockUCState() {
+    return blockUCState;
+  }
+
+  void setBlockUCState(BlockUCState s) {
+    blockUCState = s;
+  }
+
+  public long getBlockRecoveryId() {
+    return blockRecoveryId;
+  }
+
+  /** Get recover block */
+  public Block getTruncateBlock() {
+    return truncateBlock;
+  }
+
+  public void setTruncateBlock(Block recoveryBlock) {
+    this.truncateBlock = recoveryBlock;
+  }
+
+  /**
+   * Set {@link #blockUCState} to {@link BlockUCState#COMMITTED}.
+   */
+  void commit() {
+    blockUCState = BlockUCState.COMMITTED;
+  }
+
+  void removeStaleReplicas(BlockInfo block) {
+    final long genStamp = block.getGenerationStamp();
+    if (replicas != null) {
+      // Remove replicas with wrong gen stamp. The replica list is unchanged.
+      for (ReplicaUnderConstruction r : replicas) {
+        if (genStamp != r.getGenerationStamp()) {
+          r.getExpectedStorageLocation().removeBlock(block);
+          NameNode.blockStateChangeLog.debug("BLOCK* Removing stale replica "
+              + "from location: {}", r.getExpectedStorageLocation());
+        }
+      }
+    }
+  }
+
+  /**
+   * Initialize lease recovery for this block.
+   * Find the first alive data-node starting from the previous primary and
+   * make it primary.
+   */
+  public void initializeBlockRecovery(BlockInfo blockInfo, long recoveryId) {
+    setBlockUCState(BlockUCState.UNDER_RECOVERY);
+    blockRecoveryId = recoveryId;
+    if (replicas == null || replicas.length == 0) {
+      NameNode.blockStateChangeLog.warn("BLOCK*" +
+          " BlockUnderConstructionFeature.initLeaseRecovery:" +
+          " No blocks found, lease removed.");
+      // sets primary node index and return.
+      primaryNodeIndex = -1;
+      return;
+    }
+    boolean allLiveReplicasTriedAsPrimary = true;
+    for (ReplicaUnderConstruction replica : replicas) {
+      // Check if all replicas have been tried or not.
+      if (replica.isAlive()) {
+        allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary &&
+            replica.getChosenAsPrimary());
+      }
+    }
+    if (allLiveReplicasTriedAsPrimary) {
+      // Just set all the replicas to be chosen whether they are alive or not.
+      for (ReplicaUnderConstruction replica : replicas) {
+        replica.setChosenAsPrimary(false);
+      }
+    }
+    long mostRecentLastUpdate = 0;
+    ReplicaUnderConstruction primary = null;
+    primaryNodeIndex = -1;
+    for(int i = 0; i < replicas.length; i++) {
+      // Skip alive replicas which have been chosen for recovery.
+      if (!(replicas[i].isAlive() && !replicas[i].getChosenAsPrimary())) {
+        continue;
+      }
+      final ReplicaUnderConstruction ruc = replicas[i];
+      final long lastUpdate = ruc.getExpectedStorageLocation()
+          .getDatanodeDescriptor().getLastUpdateMonotonic();
+      if (lastUpdate > mostRecentLastUpdate) {
+        primaryNodeIndex = i;
+        primary = ruc;
+        mostRecentLastUpdate = lastUpdate;
+      }
+    }
+    if (primary != null) {
+      primary.getExpectedStorageLocation().getDatanodeDescriptor()
+          .addBlockToBeRecovered(blockInfo);
+      primary.setChosenAsPrimary(true);
+      NameNode.blockStateChangeLog.info(
+          "BLOCK* {} recovery started, primary={}", this, primary);
+    }
+  }
+
+  /** Add the reported replica if it is not already in the replica list. */
+  void addReplicaIfNotPresent(DatanodeStorageInfo storage,
+      Block reportedBlock, ReplicaState rState) {
+    if (replicas == null) {
+      replicas = new ReplicaUnderConstruction[1];
+      replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage, rState);
+    } else {
+      for (int i = 0; i < replicas.length; i++) {
+        DatanodeStorageInfo expected = replicas[i].getExpectedStorageLocation();
+        if (expected == storage) {
+          replicas[i].setBlockId(reportedBlock.getBlockId());
+          replicas[i].setGenerationStamp(reportedBlock.getGenerationStamp());
+          return;
+        } else if (expected != null && expected.getDatanodeDescriptor() ==
+            storage.getDatanodeDescriptor()) {
+          // The Datanode reported that the block is on a different storage
+          // than the one chosen by BlockPlacementPolicy. This can occur as
+          // we allow Datanodes to choose the target storage. Update our
+          // state by removing the stale entry and adding a new one.
+          replicas[i] = new ReplicaUnderConstruction(reportedBlock, storage,
+              rState);
+          return;
+        }
+      }
+      ReplicaUnderConstruction[] newReplicas =
+          new ReplicaUnderConstruction[replicas.length + 1];
+      System.arraycopy(replicas, 0, newReplicas, 0, replicas.length);
+      newReplicas[newReplicas.length - 1] = new ReplicaUnderConstruction(
+          reportedBlock, storage, rState);
+      replicas = newReplicas;
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder b = new StringBuilder(100);
+    appendUCParts(b);
+    return b.toString();
+  }
+
+  private void appendUCParts(StringBuilder sb) {
+    sb.append("{UCState=").append(blockUCState)
+      .append(", truncateBlock=").append(truncateBlock)
+      .append(", primaryNodeIndex=").append(primaryNodeIndex)
+      .append(", replicas=[");
+    if (replicas != null) {
+      int i = 0;
+      for (ReplicaUnderConstruction r : replicas) {
+        r.appendStringTo(sb);
+        if (++i < replicas.length) {
+          sb.append(", ");
+        }
+      }
+    }
+    sb.append("]}");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 87394f6..a4d5442 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -227,7 +227,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final BlockQueue<BlockECRecoveryInfo> erasurecodeBlocks =
       new BlockQueue<>();
   /** A queue of blocks to be recovered by this datanode */
-  private final BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
+  private final BlockQueue<BlockInfo> recoverBlocks =
       new BlockQueue<>();
   /** A set of blocks to be invalidated by this datanode */
   private final LightWeightHashSet<Block> invalidateBlocks =
@@ -624,7 +624,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Store block recovery work.
    */
-  void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
+  void addBlockToBeRecovered(BlockInfo block) {
     if(recoverBlocks.contains(block)) {
       // this prevents adding the same block twice to the recovery queue
       BlockManager.LOG.info(block + " is already in the recovery queue");
@@ -678,11 +678,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return erasurecodeBlocks.poll(maxTransfers);
   }
 
-  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
-    List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
+  public BlockInfo[] getLeaseRecoveryCommand(int maxTransfers) {
+    List<BlockInfo> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
       return null;
-    return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]);
+    return blocks.toArray(new BlockInfo[blocks.size()]);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
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 dbd07d4..8d2e750 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
@@ -1381,13 +1381,15 @@ public class DatanodeManager {
         }
 
         //check lease recovery
-        BlockInfoUnderConstruction[] blocks = nodeinfo
-            .getLeaseRecoveryCommand(Integer.MAX_VALUE);
+        BlockInfo[] blocks = nodeinfo.getLeaseRecoveryCommand(Integer.MAX_VALUE);
         if (blocks != null) {
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
               blocks.length);
-          for (BlockInfoUnderConstruction b : blocks) {
-            final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
+          for (BlockInfo b : blocks) {
+            final BlockUnderConstructionFeature uc =
+                b.getUnderConstructionFeature();
+            assert uc != null;
+            final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =
                 new ArrayList<>(storages.length);
@@ -1398,12 +1400,12 @@ public class DatanodeManager {
             }
             // If we are performing a truncate recovery than set recovery fields
             // to old block.
-            boolean truncateRecovery = b.getTruncateBlock() != null;
+            boolean truncateRecovery = uc.getTruncateBlock() != null;
             boolean copyOnTruncateRecovery = truncateRecovery &&
-                b.getTruncateBlock().getBlockId() != b.toBlock().getBlockId();
+                uc.getTruncateBlock().getBlockId() != b.getBlockId();
             ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ?
-                new ExtendedBlock(blockPoolId, b.getTruncateBlock()) :
-                new ExtendedBlock(blockPoolId, b.toBlock());
+                new ExtendedBlock(blockPoolId, uc.getTruncateBlock()) :
+                new ExtendedBlock(blockPoolId, b);
             // If we only get 1 replica after eliminating stale nodes, then choose all
             // replicas for recovery and let the primary data node handle failures.
             DatanodeInfo[] recoveryInfos;
@@ -1420,13 +1422,13 @@ public class DatanodeManager {
               recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages);
             }
             if(truncateRecovery) {
-              Block recoveryBlock = (copyOnTruncateRecovery) ? b.toBlock() :
-                  b.getTruncateBlock();
+              Block recoveryBlock = (copyOnTruncateRecovery) ? b :
+                  uc.getTruncateBlock();
               brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos,
                                                 recoveryBlock));
             } else {
               brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos,
-                                                b.getBlockRecoveryId()));
+                                                uc.getBlockRecoveryId()));
             }
           }
           return new DatanodeCommand[] { brCommand };

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
index 215a761..3d5da8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
@@ -28,8 +28,9 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -102,7 +103,7 @@ final class FSDirTruncateOp {
       final BlockInfo last = file.getLastBlock();
       if (last != null && last.getBlockUCState()
           == BlockUCState.UNDER_RECOVERY) {
-        final Block truncatedBlock = ((BlockInfoContiguousUnderConstruction) last)
+        final Block truncatedBlock = last.getUnderConstructionFeature()
             .getTruncateBlock();
         if (truncatedBlock != null) {
           final long truncateLength = file.computeFileSize(false, false)
@@ -231,43 +232,42 @@ final class FSDirTruncateOp {
               oldBlock)));
     }
 
-    BlockInfoContiguousUnderConstruction truncatedBlockUC;
+    final BlockInfo truncatedBlockUC;
     BlockManager blockManager = fsn.getFSDirectory().getBlockManager();
     if (shouldCopyOnTruncate) {
       // Add new truncateBlock into blocksMap and
       // use oldBlock as a source for copy-on-truncate recovery
-      truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock,
+      truncatedBlockUC = new BlockInfoContiguous(newBlock,
           file.getPreferredBlockReplication());
+      truncatedBlockUC.convertToBlockUnderConstruction(
+          BlockUCState.UNDER_CONSTRUCTION, blockManager.getStorages(oldBlock));
       truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
-      truncatedBlockUC.setTruncateBlock(oldBlock);
-      file.convertLastBlockToUC(truncatedBlockUC,
-          blockManager.getStorages(oldBlock));
+      truncatedBlockUC.getUnderConstructionFeature().setTruncateBlock(oldBlock);
+      file.setLastBlock(truncatedBlockUC);
       blockManager.addBlockCollection(truncatedBlockUC, file);
 
       NameNode.stateChangeLog.debug(
           "BLOCK* prepareFileForTruncate: Scheduling copy-on-truncate to new"
               + " size {}  new block {} old block {}",
-          truncatedBlockUC.getNumBytes(), newBlock,
-          truncatedBlockUC.getTruncateBlock());
+          truncatedBlockUC.getNumBytes(), newBlock, oldBlock);
     } else {
       // Use new generation stamp for in-place truncate recovery
       blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
       oldBlock = file.getLastBlock();
       assert !oldBlock.isComplete() : "oldBlock should be under construction";
-      truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock;
-      truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
-      truncatedBlockUC.getTruncateBlock().setNumBytes(
-          oldBlock.getNumBytes() - lastBlockDelta);
-      truncatedBlockUC.getTruncateBlock().setGenerationStamp(
-          newBlock.getGenerationStamp());
-
-      NameNode.stateChangeLog.debug(
-          "BLOCK* prepareFileForTruncate: {} Scheduling in-place block "
-              + "truncate to new size {}", truncatedBlockUC.getTruncateBlock()
-              .getNumBytes(), truncatedBlockUC);
+      BlockUnderConstructionFeature uc = oldBlock.getUnderConstructionFeature();
+      uc.setTruncateBlock(new Block(oldBlock));
+      uc.getTruncateBlock().setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
+      uc.getTruncateBlock().setGenerationStamp(newBlock.getGenerationStamp());
+      truncatedBlockUC = oldBlock;
+
+      NameNode.stateChangeLog.debug("BLOCK* prepareFileForTruncate: " +
+              "{} Scheduling in-place block truncate to new size {}",
+          uc, uc.getTruncateBlock().getNumBytes());
     }
     if (shouldRecoverNow) {
-      truncatedBlockUC.initializeBlockRecovery(newBlock.getGenerationStamp());
+      truncatedBlockUC.getUnderConstructionFeature().initializeBlockRecovery(
+          truncatedBlockUC, newBlock.getGenerationStamp());
     }
 
     return newBlock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index ffd8fbc..68aef76 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -45,10 +45,10 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -77,7 +77,7 @@ class FSDirWriteFileOp {
       Block block) throws IOException {
     // modify file-> block and blocksMap
     // fileNode should be under construction
-    BlockInfoUnderConstruction uc = fileNode.removeLastBlock(block);
+    BlockInfo uc = fileNode.removeLastBlock(block);
     if (uc == null) {
       return false;
     }
@@ -214,8 +214,8 @@ class FSDirWriteFileOp {
 
   static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
       DatanodeStorageInfo[] locs, long offset) throws IOException {
-    LocatedBlock lBlk = BlockManager.newLocatedBlock(fsn.getExtendedBlock(blk),
-        blk, locs, offset);
+    LocatedBlock lBlk = BlockManager.newLocatedBlock(
+        fsn.getExtendedBlock(new Block(blk)), blk, locs, offset);
     fsn.getBlockManager().setBlockToken(lBlk,
         BlockTokenIdentifier.AccessMode.WRITE);
     return lBlk;
@@ -247,8 +247,8 @@ class FSDirWriteFileOp {
       } else {
         // add new chosen targets to already allocated block and return
         BlockInfo lastBlockInFile = pendingFile.getLastBlock();
-        ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
-            .setExpectedLocations(targets);
+        lastBlockInFile.getUnderConstructionFeature().setExpectedLocations(
+            lastBlockInFile, targets, pendingFile.isStriped());
         offset = pendingFile.computeFileSize();
         return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
       }
@@ -542,7 +542,8 @@ class FSDirWriteFileOp {
         // check quota limits and updated space consumed
         fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
             numLocations, true);
-        blockInfo = new BlockInfoStripedUnderConstruction(block, ecPolicy,
+        blockInfo = new BlockInfoStriped(block, ecPolicy);
+        blockInfo.convertToBlockUnderConstruction(
             HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
       } else {
         // check quota limits and updated space consumed
@@ -550,9 +551,9 @@ class FSDirWriteFileOp {
             fileINode.getPreferredBlockReplication(), true);
 
         short numLocations = fileINode.getFileReplication();
-        blockInfo = new BlockInfoContiguousUnderConstruction(block,
-            numLocations, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
-            targets);
+        blockInfo = new BlockInfoContiguous(block, numLocations);
+        blockInfo.convertToBlockUnderConstruction(
+            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
       }
       fsd.getBlockManager().addBlockCollection(blockInfo, fileINode);
       fileINode.addBlock(blockInfo);
@@ -692,10 +693,10 @@ class FSDirWriteFileOp {
             "allocation of a new block in " + src + ". Returning previously" +
             " allocated block " + lastBlockInFile);
         long offset = file.computeFileSize();
-        BlockInfoUnderConstruction lastBlockUC =
-            (BlockInfoUnderConstruction) lastBlockInFile;
+        BlockUnderConstructionFeature uc =
+            lastBlockInFile.getUnderConstructionFeature();
         onRetryBlock[0] = makeLocatedBlock(fsn, lastBlockInFile,
-            lastBlockUC.getExpectedStorageLocations(), offset);
+            uc.getExpectedStorageLocations(), offset);
         return new FileState(file, src, iip);
       } else {
         // Case 3

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 008a327..a61161f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -42,15 +42,14 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -991,12 +990,14 @@ public class FSEditLogLoader {
     final BlockInfo newBlockInfo;
     boolean isStriped = ecZone != null;
     if (isStriped) {
-      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
+      newBlockInfo = new BlockInfoStriped(newBlock,
           ecZone.getErasureCodingPolicy());
     } else {
-      newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
+      newBlockInfo = new BlockInfoContiguous(newBlock,
           file.getPreferredBlockReplication());
     }
+    newBlockInfo.convertToBlockUnderConstruction(
+        BlockUCState.UNDER_CONSTRUCTION, null);
     fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file);
     file.addBlock(newBlockInfo);
     fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
@@ -1077,12 +1078,14 @@ public class FSEditLogLoader {
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
           if (isStriped) {
-            newBI = new BlockInfoStripedUnderConstruction(newBlock,
+            newBI = new BlockInfoStriped(newBlock,
                 ecZone.getErasureCodingPolicy());
           } else {
-            newBI = new BlockInfoContiguousUnderConstruction(newBlock,
+            newBI = new BlockInfoContiguous(newBlock,
                 file.getPreferredBlockReplication());
           }
+          newBI.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
+              null);
         } else {
           // OP_CLOSE should add finalized blocks. This code path
           // is only executed when loading edits written by prior

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index e7c87d6..0b1902f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -756,7 +755,7 @@ public class FSImageFormat {
       // file
       
       // read blocks
-      Block[] blocks = new BlockInfoContiguous[numBlocks];
+      BlockInfo[] blocks = new BlockInfoContiguous[numBlocks];
       for (int j = 0; j < numBlocks; j++) {
         blocks[j] = new BlockInfoContiguous(replication);
         blocks[j].readFields(in);
@@ -778,9 +777,9 @@ public class FSImageFormat {
             clientMachine = FSImageSerialization.readString(in);
             // convert the last block to BlockUC
             if (blocks.length > 0) {
-              Block lastBlk = blocks[blocks.length - 1];
-              blocks[blocks.length - 1] =
-                  new BlockInfoContiguousUnderConstruction(lastBlk, replication);
+              BlockInfo lastBlk = blocks[blocks.length - 1];
+              lastBlk.convertToBlockUnderConstruction(
+                  HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index 51b04d0..ffaf86b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -45,10 +45,9 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
@@ -378,11 +377,13 @@ public final class FSImageFormatPBINode {
           final BlockInfo ucBlk;
           if (isStriped) {
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
-            ucBlk = new BlockInfoStripedUnderConstruction(striped, ecPolicy);
+            ucBlk = new BlockInfoStriped(striped, ecPolicy);
           } else {
-            ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
+            ucBlk = new BlockInfoContiguous(lastBlk,
                 replication);
           }
+          ucBlk.convertToBlockUnderConstruction(
+              HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
           file.setBlock(file.numBlocks() - 1, ucBlk);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index af3f813..0567efd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -138,8 +137,9 @@ public class FSImageSerialization {
     // last block is UNDER_CONSTRUCTION
     if(numBlocks > 0) {
       blk.readFields(in);
-      blocksContiguous[i] = new BlockInfoContiguousUnderConstruction(
-          blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
+      blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication);
+      blocksContiguous[i].convertToBlockUnderConstruction(
+          BlockUCState.UNDER_CONSTRUCTION, null);
     }
 
     PermissionStatus perm = PermissionStatus.read(in);


[30/50] [abbrv] hadoop git commit: HDFS-9091. Erasure Coding: Provide DistributedFilesystem API to getAllErasureCodingPolicies. Contributed by Rakesh R.

Posted by wa...@apache.org.
HDFS-9091. Erasure Coding: Provide DistributedFilesystem API to getAllErasureCodingPolicies. Contributed by Rakesh R.

Change-Id: Id30a1ce9e2ce676d00a9220a2d3f14b8d9f00527


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

Branch: refs/heads/trunk
Commit: a9e6681edf17895fdb3530a8d75449b2d3207a1b
Parents: b762199
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Mon Sep 21 13:24:02 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Sep 21 13:24:02 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt             |  3 +++
 .../apache/hadoop/hdfs/DistributedFileSystem.java    | 11 +++++++++++
 .../hadoop/hdfs/TestErasureCodingPolicies.java       | 15 +++++++++++++++
 3 files changed, 29 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9e6681e/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index db63d53..0e21d22 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -441,3 +441,6 @@
 
     HDFS-9113. ErasureCodingWorker#processErasureCodingTasks should not fail to process
     remaining tasks due to one invalid ECTask (umamahesh)
+
+    HDFS-9091. Erasure Coding: Provide DistributedFilesystem API to 
+    getAllErasureCodingPolicies. (Rakesh R via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9e6681e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 903f763..ac927ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2324,4 +2324,15 @@ public class DistributedFileSystem extends FileSystem {
       }
     }.resolve(this, absF);
   }
+
+  /**
+   * Retrieve all the erasure coding policies supported by this file system.
+   *
+   * @return all erasure coding policies supported by this file system.
+   * @throws IOException
+   */
+  public Collection<ErasureCodingPolicy> getAllErasureCodingPolicies()
+      throws IOException {
+    return Arrays.asList(dfs.getErasureCodingPolicies());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9e6681e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index ed41f7a..0ababed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -32,6 +32,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.Collection;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.*;
@@ -231,4 +232,18 @@ public class TestErasureCodingPolicies {
     }
   }
 
+  @Test
+  public void testGetAllErasureCodingPolicies() throws Exception {
+    ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager
+        .getSystemPolices();
+    assertTrue("System ecPolicies should be of only 1 for now",
+        sysECPolicies.length == 1);
+
+    Collection<ErasureCodingPolicy> allECPolicies = fs
+        .getAllErasureCodingPolicies();
+    assertTrue("All ecPolicies should be of only 1 for now",
+        allECPolicies.size() == 1);
+    assertEquals("Erasure coding policy mismatches",
+        sysECPolicies[0], allECPolicies.iterator().next());
+  }
 }


[42/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
index 0000000,0000000..a313ecb
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@@ -1,0 -1,0 +1,187 @@@
++/**
++ * 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.hdfs;
++
++import java.io.IOException;
++import java.util.concurrent.atomic.AtomicReference;
++
++import org.apache.hadoop.fs.StorageType;
++import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator;
++import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
++import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
++import org.apache.hadoop.hdfs.protocol.LocatedBlock;
++import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
++import org.apache.hadoop.hdfs.util.ByteArrayManager;
++import org.apache.hadoop.io.erasurecode.ECSchema;
++import org.apache.hadoop.util.DataChecksum;
++import org.apache.hadoop.util.Progressable;
++
++import com.google.common.annotations.VisibleForTesting;
++
++/**
++ * This class extends {@link DataStreamer} to support writing striped blocks
++ * to datanodes.
++ * A {@link DFSStripedOutputStream} has multiple {@link StripedDataStreamer}s.
++ * Whenever the streamers need to talk the namenode, only the fastest streamer
++ * sends an rpc call to the namenode and then populates the result for the
++ * other streamers.
++ */
++public class StripedDataStreamer extends DataStreamer {
++  private final Coordinator coordinator;
++  private final int index;
++
++  StripedDataStreamer(HdfsFileStatus stat,
++                      DFSClient dfsClient, String src,
++                      Progressable progress, DataChecksum checksum,
++                      AtomicReference<CachingStrategy> cachingStrategy,
++                      ByteArrayManager byteArrayManage, String[] favoredNodes,
++                      short index, Coordinator coordinator) {
++    super(stat, null, dfsClient, src, progress, checksum, cachingStrategy,
++        byteArrayManage, favoredNodes);
++    this.index = index;
++    this.coordinator = coordinator;
++  }
++
++  int getIndex() {
++    return index;
++  }
++
++  boolean isHealthy() {
++    return !streamerClosed() && !getErrorState().hasInternalError();
++  }
++
++  @Override
++  protected void endBlock() {
++    coordinator.offerEndBlock(index, block);
++    super.endBlock();
++  }
++
++  /**
++   * The upper level DFSStripedOutputStream will allocate the new block group.
++   * All the striped data streamer only needs to fetch from the queue, which
++   * should be already be ready.
++   */
++  private LocatedBlock getFollowingBlock() throws IOException {
++    if (!this.isHealthy()) {
++      // No internal block for this streamer, maybe no enough healthy DN.
++      // Throw the exception which has been set by the StripedOutputStream.
++      this.getLastException().check(false);
++    }
++    return coordinator.getFollowingBlocks().poll(index);
++  }
++
++  @Override
++  protected LocatedBlock nextBlockOutputStream() throws IOException {
++    boolean success;
++    LocatedBlock lb = getFollowingBlock();
++    block = lb.getBlock();
++    block.setNumBytes(0);
++    bytesSent = 0;
++    accessToken = lb.getBlockToken();
++
++    DatanodeInfo[] nodes = lb.getLocations();
++    StorageType[] storageTypes = lb.getStorageTypes();
++
++    // Connect to the DataNode. If fail the internal error state will be set.
++    success = createBlockOutputStream(nodes, storageTypes, 0L, false);
++
++    if (!success) {
++      block = null;
++      final DatanodeInfo badNode = nodes[getErrorState().getBadNodeIndex()];
++      LOG.info("Excluding datanode " + badNode);
++      excludedNodes.put(badNode, badNode);
++      throw new IOException("Unable to create new block.");
++    }
++    return lb;
++  }
++
++  @VisibleForTesting
++  LocatedBlock peekFollowingBlock() {
++    return coordinator.getFollowingBlocks().peek(index);
++  }
++
++  @Override
++  protected void setupPipelineInternal(DatanodeInfo[] nodes,
++      StorageType[] nodeStorageTypes) throws IOException {
++    boolean success = false;
++    while (!success && !streamerClosed() && dfsClient.clientRunning) {
++      if (!handleRestartingDatanode()) {
++        return;
++      }
++      if (!handleBadDatanode()) {
++        // for striped streamer if it is datanode error then close the stream
++        // and return. no need to replace datanode
++        return;
++      }
++
++      // get a new generation stamp and an access token
++      final LocatedBlock lb = coordinator.getNewBlocks().take(index);
++      long newGS = lb.getBlock().getGenerationStamp();
++      setAccessToken(lb.getBlockToken());
++
++      // set up the pipeline again with the remaining nodes. when a striped
++      // data streamer comes here, it must be in external error state.
++      assert getErrorState().hasExternalError();
++      success = createBlockOutputStream(nodes, nodeStorageTypes, newGS, true);
++
++      failPacket4Testing();
++      getErrorState().checkRestartingNodeDeadline(nodes);
++
++      // notify coordinator the result of createBlockOutputStream
++      synchronized (coordinator) {
++        if (!streamerClosed()) {
++          coordinator.updateStreamer(this, success);
++          coordinator.notify();
++        } else {
++          success = false;
++        }
++      }
++
++      if (success) {
++        // wait for results of other streamers
++        success = coordinator.takeStreamerUpdateResult(index);
++        if (success) {
++          // if all succeeded, update its block using the new GS
++          block = newBlock(block, newGS);
++        } else {
++          // otherwise close the block stream and restart the recovery process
++          closeStream();
++        }
++      } else {
++        // if fail, close the stream. The internal error state and last
++        // exception have already been set in createBlockOutputStream
++        // TODO: wait for restarting DataNodes during RollingUpgrade
++        closeStream();
++        setStreamerAsClosed();
++      }
++    } // while
++  }
++
++  void setExternalError() {
++    getErrorState().setExternalError();
++    synchronized (dataQueue) {
++      dataQueue.notifyAll();
++    }
++  }
++
++  @Override
++  public String toString() {
++    return "#" + index + ": " + (!isHealthy() ? "failed, ": "") + super.toString();
++  }
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 0000000,f4ce46d..7b02691
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@@ -1,0 -1,1531 +1,1593 @@@
+ /**
+  * 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.hdfs.protocolPB;
+ 
+ import java.io.Closeable;
+ import java.io.FileNotFoundException;
+ import java.io.IOException;
+ import java.util.Arrays;
+ import java.util.EnumSet;
+ import java.util.List;
+ 
+ import com.google.common.collect.Lists;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.crypto.CryptoProtocolVersion;
+ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+ import org.apache.hadoop.fs.CacheFlag;
+ import org.apache.hadoop.fs.ContentSummary;
+ import org.apache.hadoop.fs.CreateFlag;
+ import org.apache.hadoop.fs.FileAlreadyExistsException;
+ import org.apache.hadoop.fs.FsServerDefaults;
+ import org.apache.hadoop.fs.Options.Rename;
+ import org.apache.hadoop.fs.ParentNotDirectoryException;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.fs.UnresolvedLinkException;
+ import org.apache.hadoop.fs.XAttr;
+ import org.apache.hadoop.fs.XAttrSetFlag;
+ import org.apache.hadoop.fs.permission.AclEntry;
+ import org.apache.hadoop.fs.permission.AclStatus;
+ import org.apache.hadoop.fs.permission.FsAction;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hdfs.inotify.EventBatchList;
+ import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+ import org.apache.hadoop.hdfs.protocol.DatanodeID;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+ import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.ModifyAclEntriesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclEntriesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveDefaultAclRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetCurrentEditLogTxidRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatusRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
 -import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
++import org.apache.hadoop.hdfs.protocol.proto.*;
+ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto;
++import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
+ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+ import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+ import org.apache.hadoop.io.EnumSetWritable;
+ import org.apache.hadoop.io.Text;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+ import org.apache.hadoop.ipc.ProtobufHelper;
+ import org.apache.hadoop.ipc.ProtocolMetaInterface;
+ import org.apache.hadoop.ipc.ProtocolTranslator;
+ import org.apache.hadoop.ipc.RPC;
+ import org.apache.hadoop.ipc.RpcClientUtil;
+ import org.apache.hadoop.security.AccessControlException;
+ import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
+ import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
+ import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
+ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
+ import org.apache.hadoop.security.token.Token;
+ 
+ import com.google.protobuf.ByteString;
+ import com.google.protobuf.ServiceException;
+ 
+ import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+ import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
+     .EncryptionZoneProto;
+ 
+ /**
+  * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
+  * while translating from the parameter types used in ClientProtocol to the
+  * new PB types.
+  */
+ @InterfaceAudience.Private
+ @InterfaceStability.Stable
+ public class ClientNamenodeProtocolTranslatorPB implements
+     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
+   final private ClientNamenodeProtocolPB rpcProxy;
+ 
+   static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST = 
+   GetServerDefaultsRequestProto.newBuilder().build();
+ 
+   private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
+   GetFsStatusRequestProto.newBuilder().build();
+ 
+   private final static SaveNamespaceRequestProto VOID_SAVE_NAMESPACE_REQUEST =
+   SaveNamespaceRequestProto.newBuilder().build();
+ 
+   private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST = 
+   RollEditsRequestProto.getDefaultInstance();
+ 
+   private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
+   RefreshNodesRequestProto.newBuilder().build();
+ 
+   private final static FinalizeUpgradeRequestProto
+   VOID_FINALIZE_UPGRADE_REQUEST =
+       FinalizeUpgradeRequestProto.newBuilder().build();
+ 
+   private final static GetDataEncryptionKeyRequestProto
+   VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
+       GetDataEncryptionKeyRequestProto.newBuilder().build();
+ 
+   private final static GetStoragePoliciesRequestProto
+   VOID_GET_STORAGE_POLICIES_REQUEST =
+       GetStoragePoliciesRequestProto.newBuilder().build();
+ 
++  private final static GetErasureCodingPoliciesRequestProto
++  VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto
++      .newBuilder().build();
++
+   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
+     rpcProxy = proxy;
+   }
+   
+   @Override
+   public void close() {
+     RPC.stopProxy(rpcProxy);
+   }
+ 
+   @Override
+   public LocatedBlocks getBlockLocations(String src, long offset, long length)
+       throws AccessControlException, FileNotFoundException,
+       UnresolvedLinkException, IOException {
+     GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
+         .newBuilder()
+         .setSrc(src)
+         .setOffset(offset)
+         .setLength(length)
+         .build();
+     try {
+       GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
+           req);
+       return resp.hasLocations() ?
+         PBHelperClient.convert(resp.getLocations()) : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public FsServerDefaults getServerDefaults() throws IOException {
+     GetServerDefaultsRequestProto req = VOID_GET_SERVER_DEFAULT_REQUEST;
+     try {
+       return PBHelperClient
+           .convert(rpcProxy.getServerDefaults(null, req).getServerDefaults());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public HdfsFileStatus create(String src, FsPermission masked,
+       String clientName, EnumSetWritable<CreateFlag> flag,
+       boolean createParent, short replication, long blockSize, 
+       CryptoProtocolVersion[] supportedVersions)
+       throws AccessControlException, AlreadyBeingCreatedException,
+       DSQuotaExceededException, FileAlreadyExistsException,
+       FileNotFoundException, NSQuotaExceededException,
+       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
+         .setSrc(src)
+         .setMasked(PBHelperClient.convert(masked))
+         .setClientName(clientName)
+         .setCreateFlag(PBHelperClient.convertCreateFlag(flag))
+         .setCreateParent(createParent)
+         .setReplication(replication)
+         .setBlockSize(blockSize);
+     builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
+     CreateRequestProto req = builder.build();
+     try {
+       CreateResponseProto res = rpcProxy.create(null, req);
+       return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+ 
+   }
+ 
+   @Override
+   public boolean truncate(String src, long newLength, String clientName)
+       throws IOException, UnresolvedLinkException {
+     TruncateRequestProto req = TruncateRequestProto.newBuilder()
+         .setSrc(src)
+         .setNewLength(newLength)
+         .setClientName(clientName)
+         .build();
+     try {
+       return rpcProxy.truncate(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public LastBlockWithStatus append(String src, String clientName,
+       EnumSetWritable<CreateFlag> flag) throws AccessControlException,
+       DSQuotaExceededException, FileNotFoundException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
+         .setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
+         .build();
+     try {
+       AppendResponseProto res = rpcProxy.append(null, req);
+       LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
 -          .convert(res.getBlock()) : null;
++          .convertLocatedBlockProto(res.getBlock()) : null;
+       HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
+           : null;
+       return new LastBlockWithStatus(lastBlock, stat);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean setReplication(String src, short replication)
+       throws AccessControlException, DSQuotaExceededException,
+       FileNotFoundException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
+         .setSrc(src)
+         .setReplication(replication)
+         .build();
+     try {
+       return rpcProxy.setReplication(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setPermission(String src, FsPermission permission)
+       throws AccessControlException, FileNotFoundException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
+         .setSrc(src)
+         .setPermission(PBHelperClient.convert(permission))
+         .build();
+     try {
+       rpcProxy.setPermission(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setOwner(String src, String username, String groupname)
+       throws AccessControlException, FileNotFoundException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
+         .setSrc(src);
+     if (username != null)
+         req.setUsername(username);
+     if (groupname != null)
+         req.setGroupname(groupname);
+     try {
+       rpcProxy.setOwner(null, req.build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void abandonBlock(ExtendedBlock b, long fileId, String src,
+       String holder) throws AccessControlException, FileNotFoundException,
+         UnresolvedLinkException, IOException {
+     AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
+         .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
+             .setFileId(fileId).build();
+     try {
+       rpcProxy.abandonBlock(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public LocatedBlock addBlock(String src, String clientName,
+       ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
+       String[] favoredNodes)
+       throws AccessControlException, FileNotFoundException,
+       NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
+         .setSrc(src).setClientName(clientName).setFileId(fileId);
+     if (previous != null) 
+       req.setPrevious(PBHelperClient.convert(previous));
+     if (excludeNodes != null)
+       req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes));
+     if (favoredNodes != null) {
+       req.addAllFavoredNodes(Arrays.asList(favoredNodes));
+     }
+     try {
 -      return PBHelperClient.convert(rpcProxy.addBlock(null, req.build()).getBlock());
++      return PBHelperClient.convertLocatedBlockProto(
++          rpcProxy.addBlock(null, req.build()).getBlock());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public LocatedBlock getAdditionalDatanode(String src, long fileId,
+       ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
+       DatanodeInfo[] excludes,
+       int numAdditionalNodes, String clientName) throws AccessControlException,
+       FileNotFoundException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
+         .newBuilder()
+         .setSrc(src)
+         .setFileId(fileId)
+         .setBlk(PBHelperClient.convert(blk))
+         .addAllExistings(PBHelperClient.convert(existings))
+         .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
+         .addAllExcludes(PBHelperClient.convert(excludes))
+         .setNumAdditionalNodes(numAdditionalNodes)
+         .setClientName(clientName)
+         .build();
+     try {
 -      return PBHelperClient.convert(rpcProxy.getAdditionalDatanode(null, req)
 -          .getBlock());
++      return PBHelperClient.convertLocatedBlockProto(
++          rpcProxy.getAdditionalDatanode(null, req).getBlock());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean complete(String src, String clientName,
+                           ExtendedBlock last, long fileId)
+       throws AccessControlException, FileNotFoundException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
+         .setSrc(src)
+         .setClientName(clientName)
+         .setFileId(fileId);
+     if (last != null)
+       req.setLast(PBHelperClient.convert(last));
+     try {
+       return rpcProxy.complete(null, req.build()).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+     ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
 -        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
++        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlocks(blocks)))
+         .build();
+     try {
+       rpcProxy.reportBadBlocks(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean rename(String src, String dst) throws UnresolvedLinkException,
+       IOException {
+     RenameRequestProto req = RenameRequestProto.newBuilder()
+         .setSrc(src)
+         .setDst(dst).build();
+     try {
+       return rpcProxy.rename(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+ 
+   @Override
+   public void rename2(String src, String dst, Rename... options)
+       throws AccessControlException, DSQuotaExceededException,
+       FileAlreadyExistsException, FileNotFoundException,
+       NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     boolean overwrite = false;
+     if (options != null) {
+       for (Rename option : options) {
+         if (option == Rename.OVERWRITE) {
+           overwrite = true;
+         }
+       }
+     }
+     Rename2RequestProto req = Rename2RequestProto.newBuilder().
+         setSrc(src).
+         setDst(dst).setOverwriteDest(overwrite).
+         build();
+     try {
+       rpcProxy.rename2(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+ 
+   }
+ 
+   @Override
+   public void concat(String trg, String[] srcs) throws IOException,
+       UnresolvedLinkException {
+     ConcatRequestProto req = ConcatRequestProto.newBuilder().
+         setTrg(trg).
+         addAllSrcs(Arrays.asList(srcs)).build();
+     try {
+       rpcProxy.concat(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+ 
+   @Override
+   public boolean delete(String src, boolean recursive)
+       throws AccessControlException, FileNotFoundException, SafeModeException,
+       UnresolvedLinkException, IOException {
+     DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
+     try {
+       return rpcProxy.delete(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+       throws AccessControlException, FileAlreadyExistsException,
+       FileNotFoundException, NSQuotaExceededException,
+       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
+         .setSrc(src)
+         .setMasked(PBHelperClient.convert(masked))
+         .setCreateParent(createParent).build();
+ 
+     try {
+       return rpcProxy.mkdirs(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public DirectoryListing getListing(String src, byte[] startAfter,
+       boolean needLocation) throws AccessControlException,
+       FileNotFoundException, UnresolvedLinkException, IOException {
+     GetListingRequestProto req = GetListingRequestProto.newBuilder()
+         .setSrc(src)
+         .setStartAfter(ByteString.copyFrom(startAfter))
+         .setNeedLocation(needLocation).build();
+     try {
+       GetListingResponseProto result = rpcProxy.getListing(null, req);
+       
+       if (result.hasDirList()) {
+         return PBHelperClient.convert(result.getDirList());
+       }
+       return null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void renewLease(String clientName) throws AccessControlException,
+       IOException {
+     RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
+         .setClientName(clientName).build();
+     try {
+       rpcProxy.renewLease(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean recoverLease(String src, String clientName)
+       throws IOException {
+     RecoverLeaseRequestProto req = RecoverLeaseRequestProto.newBuilder()
+         .setSrc(src)
+         .setClientName(clientName).build();
+     try {
+       return rpcProxy.recoverLease(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }  
+   }
+ 
+   @Override
+   public long[] getStats() throws IOException {
+     try {
+       return PBHelperClient.convert(rpcProxy.getFsStats(null,
+           VOID_GET_FSSTATUS_REQUEST));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
+       throws IOException {
+     GetDatanodeReportRequestProto req = GetDatanodeReportRequestProto
+         .newBuilder()
+         .setType(PBHelperClient.convert(type)).build();
+     try {
+       return PBHelperClient.convert(
+           rpcProxy.getDatanodeReport(null, req).getDiList());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
+       throws IOException {
+     final GetDatanodeStorageReportRequestProto req
+         = GetDatanodeStorageReportRequestProto.newBuilder()
+             .setType(PBHelperClient.convert(type)).build();
+     try {
+       return PBHelperClient.convertDatanodeStorageReports(
+           rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public long getPreferredBlockSize(String filename) throws IOException,
+       UnresolvedLinkException {
+     GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
+         .newBuilder()
+         .setFilename(filename)
+         .build();
+     try {
+       return rpcProxy.getPreferredBlockSize(null, req).getBsize();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
+     SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
+         .setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
+     try {
+       return rpcProxy.setSafeMode(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+     try {
+       SaveNamespaceRequestProto req = SaveNamespaceRequestProto.newBuilder()
+           .setTimeWindow(timeWindow).setTxGap(txGap).build();
+       return rpcProxy.saveNamespace(null, req).getSaved();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public long rollEdits() throws AccessControlException, IOException {
+     try {
+       RollEditsResponseProto resp = rpcProxy.rollEdits(null,
+           VOID_ROLLEDITS_REQUEST);
+       return resp.getNewSegmentTxId();
+     } catch (ServiceException se) {
+       throw ProtobufHelper.getRemoteException(se);
+     }
+   }
+ 
+   @Override
+   public boolean restoreFailedStorage(String arg) 
+       throws AccessControlException, IOException{
+     RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
+         .newBuilder()
+         .setArg(arg).build();
+     try {
+       return rpcProxy.restoreFailedStorage(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void refreshNodes() throws IOException {
+     try {
+       rpcProxy.refreshNodes(null, VOID_REFRESH_NODES_REQUEST);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void finalizeUpgrade() throws IOException {
+     try {
+       rpcProxy.finalizeUpgrade(null, VOID_FINALIZE_UPGRADE_REQUEST);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
+     final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
+         .setAction(PBHelperClient.convert(action)).build();
+     try {
+       final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
+       if (proto.hasRollingUpgradeInfo()) {
+         return PBHelperClient.convert(proto.getRollingUpgradeInfo());
+       }
+       return null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+       throws IOException {
+     ListCorruptFileBlocksRequestProto.Builder req = 
+         ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);   
+     if (cookie != null) 
+       req.setCookie(cookie);
+     try {
+       return PBHelperClient.convert(
+           rpcProxy.listCorruptFileBlocks(null, req.build()).getCorrupt());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void metaSave(String filename) throws IOException {
+     MetaSaveRequestProto req = MetaSaveRequestProto.newBuilder()
+         .setFilename(filename).build();
+     try {
+       rpcProxy.metaSave(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+ 
+   }
+ 
+   @Override
+   public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
+       FileNotFoundException, UnresolvedLinkException, IOException {
+     GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
+         .setSrc(src).build();
+     try {
+       GetFileInfoResponseProto res = rpcProxy.getFileInfo(null, req);
+       return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public HdfsFileStatus getFileLinkInfo(String src)
+       throws AccessControlException, UnresolvedLinkException, IOException {
+     GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
+         .setSrc(src).build();
+     try {
+       GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
+       return result.hasFs() ?  
+           PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public ContentSummary getContentSummary(String path)
+       throws AccessControlException, FileNotFoundException,
+       UnresolvedLinkException, IOException {
+     GetContentSummaryRequestProto req = GetContentSummaryRequestProto
+         .newBuilder()
+         .setPath(path)
+         .build();
+     try {
+       return PBHelperClient.convert(rpcProxy.getContentSummary(null, req)
+           .getSummary());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+                        StorageType type)
+       throws AccessControlException, FileNotFoundException,
+       UnresolvedLinkException, IOException {
+     final SetQuotaRequestProto.Builder builder
+         = SetQuotaRequestProto.newBuilder()
+         .setPath(path)
+         .setNamespaceQuota(namespaceQuota)
+         .setStoragespaceQuota(storagespaceQuota);
+     if (type != null) {
+       builder.setStorageType(PBHelperClient.convertStorageType(type));
+     }
+     final SetQuotaRequestProto req = builder.build();
+     try {
+       rpcProxy.setQuota(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void fsync(String src, long fileId, String client,
+                     long lastBlockLength)
+       throws AccessControlException, FileNotFoundException,
+       UnresolvedLinkException, IOException {
+     FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
+         .setClient(client).setLastBlockLength(lastBlockLength)
+             .setFileId(fileId).build();
+     try {
+       rpcProxy.fsync(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setTimes(String src, long mtime, long atime)
+       throws AccessControlException, FileNotFoundException,
+       UnresolvedLinkException, IOException {
+     SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
+         .setSrc(src)
+         .setMtime(mtime)
+         .setAtime(atime)
+         .build();
+     try {
+       rpcProxy.setTimes(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void createSymlink(String target, String link, FsPermission dirPerm,
+       boolean createParent) throws AccessControlException,
+       FileAlreadyExistsException, FileNotFoundException,
+       ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+       IOException {
+     CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
+         .setTarget(target)
+         .setLink(link)
+         .setDirPerm(PBHelperClient.convert(dirPerm))
+         .setCreateParent(createParent)
+         .build();
+     try {
+       rpcProxy.createSymlink(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public String getLinkTarget(String path) throws AccessControlException,
+       FileNotFoundException, IOException {
+     GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
+         .setPath(path).build();
+     try {
+       GetLinkTargetResponseProto rsp = rpcProxy.getLinkTarget(null, req);
+       return rsp.hasTargetPath() ? rsp.getTargetPath() : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
+       String clientName) throws IOException {
+     UpdateBlockForPipelineRequestProto req = UpdateBlockForPipelineRequestProto
+         .newBuilder()
+         .setBlock(PBHelperClient.convert(block))
+         .setClientName(clientName)
+         .build();
+     try {
 -      return PBHelperClient.convert(
++      return PBHelperClient.convertLocatedBlockProto(
+           rpcProxy.updateBlockForPipeline(null, req).getBlock());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
+       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
+     UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
+         .setClientName(clientName)
+         .setOldBlock(PBHelperClient.convert(oldBlock))
+         .setNewBlock(PBHelperClient.convert(newBlock))
+         .addAllNewNodes(Arrays.asList(PBHelperClient.convert(newNodes)))
+         .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs))
+         .build();
+     try {
+       rpcProxy.updatePipeline(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+       throws IOException {
+     GetDelegationTokenRequestProto req = GetDelegationTokenRequestProto
+         .newBuilder()
+         .setRenewer(renewer == null ? "" : renewer.toString())
+         .build();
+     try {
+       GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
+       return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
+           : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+       throws IOException {
+     RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
+         setToken(PBHelperClient.convert(token)).
+         build();
+     try {
+       return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+       throws IOException {
+     CancelDelegationTokenRequestProto req = CancelDelegationTokenRequestProto
+         .newBuilder()
+         .setToken(PBHelperClient.convert(token))
+         .build();
+     try {
+       rpcProxy.cancelDelegationToken(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setBalancerBandwidth(long bandwidth) throws IOException {
+     SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
+         .setBandwidth(bandwidth)
+         .build();
+     try {
+       rpcProxy.setBalancerBandwidth(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public boolean isMethodSupported(String methodName) throws IOException {
+     return RpcClientUtil.isMethodSupported(rpcProxy,
+         ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+         RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
+   }
+   
+   @Override
+   public DataEncryptionKey getDataEncryptionKey() throws IOException {
+     try {
+       GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
+           null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
+      return rsp.hasDataEncryptionKey() ? 
+           PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+ 
+   @Override
+   public boolean isFileClosed(String src) throws AccessControlException,
+       FileNotFoundException, UnresolvedLinkException, IOException {
+     IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
+         .setSrc(src).build();
+     try {
+       return rpcProxy.isFileClosed(null, req).getResult();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public Object getUnderlyingProxyObject() {
+     return rpcProxy;
+   }
+ 
+   @Override
+   public String createSnapshot(String snapshotRoot, String snapshotName)
+       throws IOException {
+     final CreateSnapshotRequestProto.Builder builder
+         = CreateSnapshotRequestProto.newBuilder().setSnapshotRoot(snapshotRoot);
+     if (snapshotName != null) {
+       builder.setSnapshotName(snapshotName);
+     }
+     final CreateSnapshotRequestProto req = builder.build();
+     try {
+       return rpcProxy.createSnapshot(null, req).getSnapshotPath();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public void deleteSnapshot(String snapshotRoot, String snapshotName)
+       throws IOException {
+     DeleteSnapshotRequestProto req = DeleteSnapshotRequestProto.newBuilder()
+         .setSnapshotRoot(snapshotRoot).setSnapshotName(snapshotName).build();
+     try {
+       rpcProxy.deleteSnapshot(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public void allowSnapshot(String snapshotRoot) throws IOException {
+     AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
+         .setSnapshotRoot(snapshotRoot).build();
+     try {
+       rpcProxy.allowSnapshot(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void disallowSnapshot(String snapshotRoot) throws IOException {
+     DisallowSnapshotRequestProto req = DisallowSnapshotRequestProto
+         .newBuilder().setSnapshotRoot(snapshotRoot).build();
+     try {
+       rpcProxy.disallowSnapshot(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+       String snapshotNewName) throws IOException {
+     RenameSnapshotRequestProto req = RenameSnapshotRequestProto.newBuilder()
+         .setSnapshotRoot(snapshotRoot).setSnapshotOldName(snapshotOldName)
+         .setSnapshotNewName(snapshotNewName).build();
+     try {
+       rpcProxy.renameSnapshot(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+       throws IOException {
+     GetSnapshottableDirListingRequestProto req = 
+         GetSnapshottableDirListingRequestProto.newBuilder().build();
+     try {
+       GetSnapshottableDirListingResponseProto result = rpcProxy
+           .getSnapshottableDirListing(null, req);
+       
+       if (result.hasSnapshottableDirList()) {
+         return PBHelperClient.convert(result.getSnapshottableDirList());
+       }
+       return null;
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+       String fromSnapshot, String toSnapshot) throws IOException {
+     GetSnapshotDiffReportRequestProto req = GetSnapshotDiffReportRequestProto
+         .newBuilder().setSnapshotRoot(snapshotRoot)
+         .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
+     try {
+       GetSnapshotDiffReportResponseProto result = 
+           rpcProxy.getSnapshotDiffReport(null, req);
+     
+       return PBHelperClient.convert(result.getDiffReport());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public long addCacheDirective(CacheDirectiveInfo directive,
+       EnumSet<CacheFlag> flags) throws IOException {
+     try {
+       AddCacheDirectiveRequestProto.Builder builder =
+           AddCacheDirectiveRequestProto.newBuilder().
+               setInfo(PBHelperClient.convert(directive));
+       if (!flags.isEmpty()) {
+         builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
+       }
+       return rpcProxy.addCacheDirective(null, builder.build()).getId();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void modifyCacheDirective(CacheDirectiveInfo directive,
+       EnumSet<CacheFlag> flags) throws IOException {
+     try {
+       ModifyCacheDirectiveRequestProto.Builder builder =
+           ModifyCacheDirectiveRequestProto.newBuilder().
+               setInfo(PBHelperClient.convert(directive));
+       if (!flags.isEmpty()) {
+         builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
+       }
+       rpcProxy.modifyCacheDirective(null, builder.build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeCacheDirective(long id)
+       throws IOException {
+     try {
+       rpcProxy.removeCacheDirective(null,
+           RemoveCacheDirectiveRequestProto.newBuilder().
+               setId(id).build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   private static class BatchedCacheEntries
+       implements BatchedEntries<CacheDirectiveEntry> {
+     private final ListCacheDirectivesResponseProto response;
+ 
+     BatchedCacheEntries(
+         ListCacheDirectivesResponseProto response) {
+       this.response = response;
+     }
+ 
+     @Override
+     public CacheDirectiveEntry get(int i) {
+       return PBHelperClient.convert(response.getElements(i));
+     }
+ 
+     @Override
+     public int size() {
+       return response.getElementsCount();
+     }
+     
+     @Override
+     public boolean hasMore() {
+       return response.getHasMore();
+     }
+   }
+ 
+   @Override
+   public BatchedEntries<CacheDirectiveEntry>
+       listCacheDirectives(long prevId,
+           CacheDirectiveInfo filter) throws IOException {
+     if (filter == null) {
+       filter = new CacheDirectiveInfo.Builder().build();
+     }
+     try {
+       return new BatchedCacheEntries(
+         rpcProxy.listCacheDirectives(null,
+           ListCacheDirectivesRequestProto.newBuilder().
+             setPrevId(prevId).
+             setFilter(PBHelperClient.convert(filter)).
+             build()));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void addCachePool(CachePoolInfo info) throws IOException {
+     AddCachePoolRequestProto.Builder builder = 
+         AddCachePoolRequestProto.newBuilder();
+     builder.setInfo(PBHelperClient.convert(info));
+     try {
+       rpcProxy.addCachePool(null, builder.build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void modifyCachePool(CachePoolInfo req) throws IOException {
+     ModifyCachePoolRequestProto.Builder builder = 
+         ModifyCachePoolRequestProto.newBuilder();
+     builder.setInfo(PBHelperClient.convert(req));
+     try {
+       rpcProxy.modifyCachePool(null, builder.build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeCachePool(String cachePoolName) throws IOException {
+     try {
+       rpcProxy.removeCachePool(null, 
+           RemoveCachePoolRequestProto.newBuilder().
+             setPoolName(cachePoolName).build());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   private static class BatchedCachePoolEntries
+     implements BatchedEntries<CachePoolEntry> {
+       private final ListCachePoolsResponseProto proto;
+     
+     public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
+       this.proto = proto;
+     }
+       
+     @Override
+     public CachePoolEntry get(int i) {
+       CachePoolEntryProto elem = proto.getEntries(i);
+       return PBHelperClient.convert(elem);
+     }
+ 
+     @Override
+     public int size() {
+       return proto.getEntriesCount();
+     }
+     
+     @Override
+     public boolean hasMore() {
+       return proto.getHasMore();
+     }
+   }
+ 
+   @Override
+   public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
+       throws IOException {
+     try {
+       return new BatchedCachePoolEntries(
+         rpcProxy.listCachePools(null,
+           ListCachePoolsRequestProto.newBuilder().
+             setPrevPoolName(prevKey).build()));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void modifyAclEntries(String src, List<AclEntry> aclSpec)
+       throws IOException {
+     ModifyAclEntriesRequestProto req = ModifyAclEntriesRequestProto
+         .newBuilder().setSrc(src)
+         .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
+     try {
+       rpcProxy.modifyAclEntries(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeAclEntries(String src, List<AclEntry> aclSpec)
+       throws IOException {
+     RemoveAclEntriesRequestProto req = RemoveAclEntriesRequestProto
+         .newBuilder().setSrc(src)
+         .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
+     try {
+       rpcProxy.removeAclEntries(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeDefaultAcl(String src) throws IOException {
+     RemoveDefaultAclRequestProto req = RemoveDefaultAclRequestProto
+         .newBuilder().setSrc(src).build();
+     try {
+       rpcProxy.removeDefaultAcl(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeAcl(String src) throws IOException {
+     RemoveAclRequestProto req = RemoveAclRequestProto.newBuilder()
+         .setSrc(src).build();
+     try {
+       rpcProxy.removeAcl(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
+     SetAclRequestProto req = SetAclRequestProto.newBuilder()
+         .setSrc(src)
+         .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec))
+         .build();
+     try {
+       rpcProxy.setAcl(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public AclStatus getAclStatus(String src) throws IOException {
+     GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder()
+         .setSrc(src).build();
+     try {
+       return PBHelperClient.convert(rpcProxy.getAclStatus(null, req));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void createEncryptionZone(String src, String keyName)
+     throws IOException {
+     final CreateEncryptionZoneRequestProto.Builder builder =
+       CreateEncryptionZoneRequestProto.newBuilder();
+     builder.setSrc(src);
+     if (keyName != null && !keyName.isEmpty()) {
+       builder.setKeyName(keyName);
+     }
+     CreateEncryptionZoneRequestProto req = builder.build();
+     try {
+       rpcProxy.createEncryptionZone(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public EncryptionZone getEZForPath(String src)
+       throws IOException {
+     final GetEZForPathRequestProto.Builder builder =
+         GetEZForPathRequestProto.newBuilder();
+     builder.setSrc(src);
+     final GetEZForPathRequestProto req = builder.build();
+     try {
+       final EncryptionZonesProtos.GetEZForPathResponseProto response =
+           rpcProxy.getEZForPath(null, req);
+       if (response.hasZone()) {
+         return PBHelperClient.convert(response.getZone());
+       } else {
+         return null;
+       }
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
+       throws IOException {
+     final ListEncryptionZonesRequestProto req =
+       ListEncryptionZonesRequestProto.newBuilder()
+           .setId(id)
+           .build();
+     try {
+       EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
+           rpcProxy.listEncryptionZones(null, req);
+       List<EncryptionZone> elements =
+           Lists.newArrayListWithCapacity(response.getZonesCount());
+       for (EncryptionZoneProto p : response.getZonesList()) {
+         elements.add(PBHelperClient.convert(p));
+       }
+       return new BatchedListEntries<EncryptionZone>(elements,
+           response.getHasMore());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
++  public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
++      throws IOException {
++    final SetErasureCodingPolicyRequestProto.Builder builder =
++        SetErasureCodingPolicyRequestProto.newBuilder();
++    builder.setSrc(src);
++    if (ecPolicy != null) {
++      builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
++    }
++    SetErasureCodingPolicyRequestProto req = builder.build();
++    try {
++      rpcProxy.setErasureCodingPolicy(null, req);
++    } catch (ServiceException e) {
++      throw ProtobufHelper.getRemoteException(e);
++    }
++  }
++
++  @Override
+   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
+       throws IOException {
+     SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()
+         .setSrc(src)
+         .setXAttr(PBHelperClient.convertXAttrProto(xAttr))
+         .setFlag(PBHelperClient.convert(flag))
+         .build();
+     try {
+       rpcProxy.setXAttr(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
+       throws IOException {
+     GetXAttrsRequestProto.Builder builder = GetXAttrsRequestProto.newBuilder();
+     builder.setSrc(src);
+     if (xAttrs != null) {
+       builder.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs));
+     }
+     GetXAttrsRequestProto req = builder.build();
+     try {
+       return PBHelperClient.convert(rpcProxy.getXAttrs(null, req));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+   
+   @Override
+   public List<XAttr> listXAttrs(String src)
+       throws IOException {
+     ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
+     builder.setSrc(src);
+     ListXAttrsRequestProto req = builder.build();
+     try {
+       return PBHelperClient.convert(rpcProxy.listXAttrs(null, req));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void removeXAttr(String src, XAttr xAttr) throws IOException {
+     RemoveXAttrRequestProto req = RemoveXAttrRequestProto
+         .newBuilder().setSrc(src)
+         .setXAttr(PBHelperClient.convertXAttrProto(xAttr)).build();
+     try {
+       rpcProxy.removeXAttr(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void checkAccess(String path, FsAction mode) throws IOException {
+     CheckAccessRequestProto req = CheckAccessRequestProto.newBuilder()
+         .setPath(path).setMode(PBHelperClient.convert(mode)).build();
+     try {
+       rpcProxy.checkAccess(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public void setStoragePolicy(String src, String policyName)
+       throws IOException {
+     SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto
+         .newBuilder().setSrc(src).setPolicyName(policyName).build();
+     try {
+       rpcProxy.setStoragePolicy(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+     GetStoragePolicyRequestProto request = GetStoragePolicyRequestProto
+         .newBuilder().setPath(path).build();
+     try {
+       return PBHelperClient.convert(rpcProxy.getStoragePolicy(null, request)
+           .getStoragePolicy());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+     try {
+       GetStoragePoliciesResponseProto response = rpcProxy
+           .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST);
+       return PBHelperClient.convertStoragePolicies(response.getPoliciesList());
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   public long getCurrentEditLogTxid() throws IOException {
+     GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto
+         .getDefaultInstance();
+     try {
+       return rpcProxy.getCurrentEditLogTxid(null, req).getTxid();
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
+ 
+   @Override
+   public EventBatchList getEditsFromTxid(long txid) throws IOException {
+     GetEditsFromTxidRequestProto req = GetEditsFromTxidRequestProto.newBuilder()
+         .setTxid(txid).build();
+     try {
+       return PBHelperClient.convert(rpcProxy.getEditsFromTxid(null, req));
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
++
++  @Override
++  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
++    try {
++      GetErasureCodingPoliciesResponseProto response = rpcProxy
++          .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST);
++      ErasureCodingPolicy[] ecPolicies =
++          new ErasureCodingPolicy[response.getEcPoliciesCount()];
++      int i = 0;
++      for (ErasureCodingPolicyProto ecPolicyProto : response.getEcPoliciesList()) {
++        ecPolicies[i++] = PBHelperClient.convertErasureCodingPolicy(ecPolicyProto);
++      }
++      return ecPolicies;
++    } catch (ServiceException e) {
++      throw ProtobufHelper.getRemoteException(e);
++    }
++  }
++
++  @Override
++  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
++    GetErasureCodingPolicyRequestProto req = GetErasureCodingPolicyRequestProto.newBuilder()
++        .setSrc(src).build();
++    try {
++      GetErasureCodingPolicyResponseProto response = rpcProxy.getErasureCodingPolicy(
++          null, req);
++      if (response.hasEcPolicy()) {
++        return PBHelperClient.convertErasureCodingPolicy(response.getEcPolicy());
++      }
++      return null;
++    } catch (ServiceException e) {
++      throw ProtobufHelper.getRemoteException(e);
++    }
++  }
+ }


[21/50] [abbrv] hadoop git commit: HDFS-8853. Erasure Coding: Provide ECSchema validation when setting EC policy. Contributed by J.Andreina.

Posted by wa...@apache.org.
HDFS-8853. Erasure Coding: Provide ECSchema validation when setting EC policy. Contributed by J.Andreina.

Change-Id: I9211d9728480225a407d82e6c0bea1a928adfa11


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

Branch: refs/heads/trunk
Commit: 96d6b516b2468fce346490e0b95931d1759b3d33
Parents: f62237b
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Thu Sep 10 16:31:37 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Thu Sep 10 16:31:37 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +++
 .../server/namenode/FSDirErasureCodingOp.java   | 22 +++++++++++++++++++
 .../hadoop/hdfs/TestErasureCodingPolicies.java  | 23 ++++++++++++++++++++
 3 files changed, 48 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96d6b516/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 2f13310..f49a974 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -415,3 +415,6 @@
 
     HDFS-8833. Erasure coding: store EC schema and cell size in INodeFile and 
     eliminate notion of EC zones. (zhz)
+
+    HDFS-8853. Erasure Coding: Provide ECSchema validation when setting EC
+    policy. (andreina via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96d6b516/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index 4162760..d39da28 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -28,6 +28,8 @@ import java.util.List;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -105,6 +107,26 @@ final class FSDirErasureCodingOp {
     // System default erasure coding policy will be used since no specified.
     if (ecPolicy == null) {
       ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    } else {
+      // If ecPolicy is specified check if it is one among active policies.
+      boolean validPolicy = false;
+      ErasureCodingPolicy[] activePolicies =
+          FSDirErasureCodingOp.getErasureCodingPolicies(fsd.getFSNamesystem());
+      for (ErasureCodingPolicy activePolicy : activePolicies) {
+        if (activePolicy.equals(ecPolicy)) {
+          validPolicy = true;
+          break;
+        }
+      }
+      if (!validPolicy) {
+        List<String> ecPolicyNames = new ArrayList<String>();
+        for (ErasureCodingPolicy activePolicy : activePolicies) {
+          ecPolicyNames.add(activePolicy.getName());
+        }
+        throw new HadoopIllegalArgumentException("Policy [ " +
+            ecPolicy.getName()+ " ] does not match any of the " +
+            "supported policies. Please select any one of " + ecPolicyNames);
+      }
     }
 
     final XAttr ecXAttr;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96d6b516/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index f60d77d..ed41f7a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -208,4 +209,26 @@ public class TestErasureCodingPolicies {
     assertEquals("Actually used ecPolicy should be equal with target ecPolicy",
         usingECPolicy, ecPolicy);
   }
+
+  @Test
+  public void testCreationErasureCodingZoneWithInvalidPolicy()
+      throws IOException {
+    ECSchema rsSchema = new ECSchema("rs", 4, 2);
+    String policyName = "RS-4-2-128k";
+    int cellSize = 128 * 1024;
+    ErasureCodingPolicy ecPolicy=
+        new ErasureCodingPolicy(policyName,rsSchema,cellSize);
+    String src = "/ecZone4-2";
+    final Path ecDir = new Path(src);
+    try {
+      fs.mkdir(ecDir, FsPermission.getDirDefault());
+      fs.getClient().setErasureCodingPolicy(src, ecPolicy);
+      fail("HadoopIllegalArgumentException should be thrown for"
+          + "setting an invalid erasure coding policy");
+    } catch (Exception e) {
+      assertExceptionContains("Policy [ RS-4-2-128k ] does not match " +
+          "any of the supported policies",e);
+    }
+  }
+
 }


[31/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 21e60c8,a655d66..25012e7
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@@ -74,10 -74,9 +74,10 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
  import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 +import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
  import org.apache.hadoop.hdfs.server.balancer.Balancer.Cli;
- import org.apache.hadoop.hdfs.server.balancer.Balancer.Parameters;
  import org.apache.hadoop.hdfs.server.balancer.Balancer.Result;
+ import org.apache.hadoop.hdfs.server.balancer.BalancerParameters;
  import org.apache.hadoop.hdfs.server.datanode.DataNode;
  import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
  import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase;
@@@ -1667,74 -1680,7 +1698,74 @@@ public class TestBalancer 
        cluster.shutdown();
      }
    }
 -  
 +  public void integrationTestWithStripedFile(Configuration conf) throws Exception {
 +    initConfWithStripe(conf);
 +    doTestBalancerWithStripedFile(conf);
 +  }
 +
 +  @Test(timeout = 100000)
 +  public void testBalancerWithStripedFile() throws Exception {
 +    Configuration conf = new Configuration();
 +    initConfWithStripe(conf);
 +    doTestBalancerWithStripedFile(conf);
 +  }
 +
 +  private void doTestBalancerWithStripedFile(Configuration conf) throws Exception {
 +    int numOfDatanodes = dataBlocks + parityBlocks + 2;
 +    int numOfRacks = dataBlocks;
 +    long capacity = 20 * DEFAULT_STRIPE_BLOCK_SIZE;
 +    long[] capacities = new long[numOfDatanodes];
 +    for (int i = 0; i < capacities.length; i++) {
 +      capacities[i] = capacity;
 +    }
 +    String[] racks = new String[numOfDatanodes];
 +    for (int i = 0; i < numOfDatanodes; i++) {
 +      racks[i] = "/rack" + (i % numOfRacks);
 +    }
 +    cluster = new MiniDFSCluster.Builder(conf)
 +        .numDataNodes(numOfDatanodes)
 +        .racks(racks)
 +        .simulatedCapacities(capacities)
 +        .build();
 +
 +    try {
 +      cluster.waitActive();
 +      client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(),
 +          ClientProtocol.class).getProxy();
 +      client.setErasureCodingPolicy("/", null);
 +
 +      long totalCapacity = sum(capacities);
 +
 +      // fill up the cluster with 30% data. It'll be 45% full plus parity.
 +      long fileLen = totalCapacity * 3 / 10;
 +      long totalUsedSpace = fileLen * (dataBlocks + parityBlocks) / dataBlocks;
 +      FileSystem fs = cluster.getFileSystem(0);
 +      DFSTestUtil.createFile(fs, filePath, fileLen, (short) 3, r.nextLong());
 +
 +      // verify locations of striped blocks
 +      LocatedBlocks locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
 +      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
 +
 +      // add one datanode
 +      String newRack = "/rack" + (++numOfRacks);
 +      cluster.startDataNodes(conf, 1, true, null,
 +          new String[]{newRack}, null, new long[]{capacity});
 +      totalCapacity += capacity;
 +      cluster.triggerHeartbeats();
 +
 +      // run balancer and validate results
-       Balancer.Parameters p = Balancer.Parameters.DEFAULT;
++      BalancerParameters p = BalancerParameters.DEFAULT;
 +      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
 +      runBalancer(conf, totalUsedSpace, totalCapacity, p, 0);
 +
 +      // verify locations of striped blocks
 +      locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
 +      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
 +    } finally {
 +      cluster.shutdown();
 +    }
 +  }
 +
    /**
     * @param args
     */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
index 2a593d5,0000000..c827582
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@@ -1,284 -1,0 +1,284 @@@
 +/**
 + * 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.hdfs.server.namenode;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.assertFalse;
 +
 +import java.io.IOException;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.fs.permission.FsPermission;
 +import org.apache.hadoop.fs.permission.PermissionStatus;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.DFSTestUtil;
 +import org.apache.hadoop.hdfs.DistributedFileSystem;
 +import org.apache.hadoop.hdfs.MiniDFSCluster;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 +
 +import org.junit.Test;
 +
 +/**
 + * This class tests INodeFile with striped feature.
 + */
 +public class TestStripedINodeFile {
 +  public static final Log LOG = LogFactory.getLog(TestINodeFile.class);
 +
 +  private static final PermissionStatus perm = new PermissionStatus(
 +      "userName", null, FsPermission.getDefault());
 +
 +  private final BlockStoragePolicySuite defaultSuite =
 +      BlockStoragePolicySuite.createDefaultSuite();
 +  private final BlockStoragePolicy defaultPolicy =
 +      defaultSuite.getDefaultPolicy();
 +
 +  private static final ErasureCodingPolicy testECPolicy
 +      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 +
 +  private static INodeFile createStripedINodeFile() {
 +    return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-         null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID, true);
++        null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID, true);
 +  }
 +
 +  @Test
 +  public void testBlockStripedFeature()
 +      throws IOException, InterruptedException{
 +    INodeFile inf = createStripedINodeFile();
 +    assertTrue(inf.isStriped());
 +  }
 +
 +  @Test
 +  public void testBlockStripedTotalBlockCount() {
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    assertEquals(9, blockInfoStriped.getTotalBlockNum());
 +  }
 +
 +  @Test
 +  public void testBlockStripedLength()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    inf.addBlock(blockInfoStriped);
 +    assertEquals(1, inf.getBlocks().length);
 +  }
 +
 +  @Test
 +  public void testBlockStripedConsumedSpace()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(1);
 +    inf.addBlock(blockInfoStriped);
 +    //   0. Calculate the total bytes per stripes <Num Bytes per Stripes>
 +    //   1. Calculate the number of stripes in this block group. <Num Stripes>
 +    //   2. Calculate the last remaining length which does not make a stripe. <Last Stripe Length>
 +    //   3. Total consumed space is the total of
 +    //     a. The total of the full cells of data blocks and parity blocks.
 +    //     b. The remaining of data block which does not make a stripe.
 +    //     c. The last parity block cells. These size should be same
 +    //        to the first cell in this stripe.
 +    // So the total consumed space is the sum of
 +    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
 +    //  b. <Num Bytes> % <Num Bytes per Stripes> = 1
 +    //  c. <Last Stripe Length> * <Parity Block Num> = 1 * 3
 +    assertEquals(4, inf.storagespaceConsumedStriped().getStorageSpace());
 +    assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
 +  }
 +
 +  @Test
 +  public void testMultipleBlockStripedConsumedSpace()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk1 = new Block(1);
 +    BlockInfoStriped blockInfoStriped1
 +        = new BlockInfoStriped(blk1, testECPolicy);
 +    blockInfoStriped1.setNumBytes(1);
 +    Block blk2 = new Block(2);
 +    BlockInfoStriped blockInfoStriped2
 +        = new BlockInfoStriped(blk2, testECPolicy);
 +    blockInfoStriped2.setNumBytes(1);
 +    inf.addBlock(blockInfoStriped1);
 +    inf.addBlock(blockInfoStriped2);
 +    // This is the double size of one block in above case.
 +    assertEquals(4 * 2, inf.storagespaceConsumedStriped().getStorageSpace());
 +    assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
 +  }
 +
 +  @Test
 +  public void testBlockStripedFileSize()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(100);
 +    inf.addBlock(blockInfoStriped);
 +    // Compute file size should return actual data
 +    // size which is retained by this file.
 +    assertEquals(100, inf.computeFileSize());
 +    assertEquals(100, inf.computeFileSize(false, false));
 +  }
 +
 +  @Test
 +  public void testBlockStripedUCFileSize()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy);
 +    bInfoUCStriped.convertToBlockUnderConstruction(
 +        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
 +    bInfoUCStriped.setNumBytes(100);
 +    inf.addBlock(bInfoUCStriped);
 +    assertEquals(100, inf.computeFileSize());
 +    assertEquals(0, inf.computeFileSize(false, false));
 +  }
 +
 +  @Test
 +  public void testBlockStripedComputeQuotaUsage()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(100);
 +    inf.addBlock(blockInfoStriped);
 +
 +    QuotaCounts counts =
 +        inf.computeQuotaUsageWithStriped(defaultPolicy,
 +            new QuotaCounts.Builder().build());
 +    assertEquals(1, counts.getNameSpace());
 +    // The total consumed space is the sum of
 +    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
 +    //  b. <Num Bytes> % <Num Bytes per Stripes> = 100
 +    //  c. <Last Stripe Length> * <Parity Block Num> = 100 * 3
 +    assertEquals(400, counts.getStorageSpace());
 +  }
 +
 +  @Test
 +  public void testBlockStripedUCComputeQuotaUsage()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy);
 +    bInfoUCStriped.convertToBlockUnderConstruction(
 +        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
 +    bInfoUCStriped.setNumBytes(100);
 +    inf.addBlock(bInfoUCStriped);
 +
 +    QuotaCounts counts
 +        = inf.computeQuotaUsageWithStriped(defaultPolicy,
 +              new QuotaCounts.Builder().build());
 +    assertEquals(1024, inf.getPreferredBlockSize());
 +    assertEquals(1, counts.getNameSpace());
 +    // Consumed space in the case of BlockInfoStripedUC can be calculated
 +    // by using preferred block size. This is 1024 and total block num
 +    // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216.
 +    assertEquals(9216, counts.getStorageSpace());
 +  }
 +
 +  /**
 +   * Test the behavior of striped and contiguous block deletions.
 +   */
 +  @Test(timeout = 60000)
 +  public void testDeleteOp() throws Exception {
 +    MiniDFSCluster cluster = null;
 +    try {
 +      final int len = 1024;
 +      final Path parentDir = new Path("/parentDir");
 +      final Path ecDir = new Path(parentDir, "ecDir");
 +      final Path ecFile = new Path(ecDir, "ecFile");
 +      final Path contiguousFile = new Path(parentDir, "someFile");
 +      final DistributedFileSystem dfs;
 +      final Configuration conf = new Configuration();
 +      final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS
 +          + HdfsConstants.NUM_PARITY_BLOCKS;
 +      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 2);
 +
 +      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE)
 +          .build();
 +      cluster.waitActive();
 +
 +      FSNamesystem fsn = cluster.getNamesystem();
 +      dfs = cluster.getFileSystem();
 +      dfs.mkdirs(ecDir);
 +
 +      // set erasure coding policy
 +      dfs.setErasureCodingPolicy(ecDir, null);
 +      DFSTestUtil.createFile(dfs, ecFile, len, (short) 1, 0xFEED);
 +      DFSTestUtil.createFile(dfs, contiguousFile, len, (short) 1, 0xFEED);
 +      final FSDirectory fsd = fsn.getFSDirectory();
 +
 +      // Case-1: Verify the behavior of striped blocks
 +      // Get blocks of striped file
 +      INode inodeStriped = fsd.getINode("/parentDir/ecDir/ecFile");
 +      assertTrue("Failed to get INodeFile for /parentDir/ecDir/ecFile",
 +          inodeStriped instanceof INodeFile);
 +      INodeFile inodeStripedFile = (INodeFile) inodeStriped;
 +      BlockInfo[] stripedBlks = inodeStripedFile.getBlocks();
 +      for (BlockInfo blockInfo : stripedBlks) {
 +        assertFalse("Mistakenly marked the block as deleted!",
 +            blockInfo.isDeleted());
 +      }
 +
 +      // delete directory with erasure coding policy
 +      dfs.delete(ecDir, true);
 +      for (BlockInfo blockInfo : stripedBlks) {
 +        assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
 +      }
 +
 +      // Case-2: Verify the behavior of contiguous blocks
 +      // Get blocks of contiguous file
 +      INode inode = fsd.getINode("/parentDir/someFile");
 +      assertTrue("Failed to get INodeFile for /parentDir/someFile",
 +          inode instanceof INodeFile);
 +      INodeFile inodeFile = (INodeFile) inode;
 +      BlockInfo[] contiguousBlks = inodeFile.getBlocks();
 +      for (BlockInfo blockInfo : contiguousBlks) {
 +        assertFalse("Mistakenly marked the block as deleted!",
 +            blockInfo.isDeleted());
 +      }
 +
 +      // delete parent directory
 +      dfs.delete(parentDir, true);
 +      for (BlockInfo blockInfo : contiguousBlks) {
 +        assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
 +      }
 +    } finally {
 +      if (cluster != null) {
 +        cluster.shutdown();
 +      }
 +    }
 +  }
 +}


[19/50] [abbrv] hadoop git commit: HDFS-8833. Erasure coding: store EC schema and cell size in INodeFile and eliminate notion of EC zones.

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
index f3260da..b857d3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.shell.Command;
 import org.apache.hadoop.fs.shell.CommandFactory;
 import org.apache.hadoop.fs.shell.PathData;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.util.StringUtils;
@@ -45,9 +44,9 @@ public abstract class ECCommand extends Command {
   public static void registerCommands(CommandFactory factory) {
     // Register all commands of Erasure CLI, with a '-' at the beginning in name
     // of the command.
-    factory.addClass(CreateECZoneCommand.class, "-" + CreateECZoneCommand.NAME);
-    factory.addClass(GetECZoneCommand.class, "-"
-        + GetECZoneCommand.NAME);
+    factory.addClass(SetECPolicyCommand.class, "-" + SetECPolicyCommand.NAME);
+    factory.addClass(GetECPolicyCommand.class, "-"
+        + GetECPolicyCommand.NAME);
     factory.addClass(ListPolicies.class, "-" + ListPolicies.NAME);
   }
 
@@ -76,17 +75,18 @@ public abstract class ECCommand extends Command {
   }
 
   /**
-   * A command to create an EC zone for a path, with a erasure coding policy name.
+   * A command to set the erasure coding policy for a directory, with the name
+   * of the policy.
    */
-  static class CreateECZoneCommand extends ECCommand {
-    public static final String NAME = "createZone";
+  static class SetECPolicyCommand extends ECCommand {
+    public static final String NAME = "setPolicy";
     public static final String USAGE = "[-s <policyName>] <path>";
     public static final String DESCRIPTION = 
-        "Create a zone to encode files using a specified policy\n"
+        "Set a specified erasure coding policy to a directory\n"
         + "Options :\n"
         + "  -s <policyName> : erasure coding policy name to encode files. "
         + "If not passed the default policy will be used\n"
-        + "  <path>  : Path to an empty directory. Under this directory "
+        + "  <path>  : Path to a directory. Under this directory "
         + "files will be encoded using specified erasure coding policy";
     private String ecPolicyName;
     private ErasureCodingPolicy ecPolicy = null;
@@ -129,23 +129,23 @@ public abstract class ECCommand extends Command {
             throw new HadoopIllegalArgumentException(sb.toString());
           }
         }
-        dfs.createErasureCodingZone(item.path, ecPolicy);
-        out.println("EC Zone created successfully at " + item.path);
+        dfs.setErasureCodingPolicy(item.path, ecPolicy);
+        out.println("EC policy set successfully at " + item.path);
       } catch (IOException e) {
-        throw new IOException("Unable to create EC zone for the path "
+        throw new IOException("Unable to set EC policy for the path "
             + item.path + ". " + e.getMessage());
       }
     }
   }
 
   /**
-   * Get the information about the zone
+   * Get the erasure coding policy of a file or directory
    */
-  static class GetECZoneCommand extends ECCommand {
-    public static final String NAME = "getZone";
+  static class GetECPolicyCommand extends ECCommand {
+    public static final String NAME = "getPolicy";
     public static final String USAGE = "<path>";
     public static final String DESCRIPTION =
-        "Get information about the EC zone at specified path\n";
+        "Get erasure coding policy information about at specified path\n";
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
@@ -162,14 +162,14 @@ public abstract class ECCommand extends Command {
       super.processPath(item);
       DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
       try {
-        ErasureCodingZone ecZone = dfs.getErasureCodingZone(item.path);
-        if (ecZone != null) {
-          out.println(ecZone.toString());
+        ErasureCodingPolicy ecPolicy = dfs.getErasureCodingPolicy(item.path);
+        if (ecPolicy != null) {
+          out.println(ecPolicy.toString());
         } else {
-          out.println("Path " + item.path + " is not in EC zone");
+          out.println("Path " + item.path + " is not erasure coded.");
         }
       } catch (IOException e) {
-        throw new IOException("Unable to get EC zone for the path "
+        throw new IOException("Unable to get EC policy for the path "
             + item.path + ". " + e.getMessage());
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 6500b96..3551055 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -1892,12 +1892,12 @@ public class DFSTestUtil {
   public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir,
       int numBlocks, int numStripesPerBlk, boolean toMkdir) throws Exception {
     DistributedFileSystem dfs = cluster.getFileSystem();
-    // If outer test already created EC zone, dir should be left as null
+    // If outer test already set EC policy, dir should be left as null
     if (toMkdir) {
       assert dir != null;
       dfs.mkdirs(dir);
       try {
-        dfs.getClient().createErasureCodingZone(dir.toString(), null);
+        dfs.getClient().setErasureCodingPolicy(dir.toString(), null);
       } catch (IOException e) {
         if (!e.getMessage().contains("non-empty directory")) {
           throw e;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
index a09f0f0..6fcf644 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
@@ -80,7 +80,7 @@ public class TestDFSStripedInputStream {
     }
     fs = cluster.getFileSystem();
     fs.mkdirs(dirPath);
-    fs.getClient().createErasureCodingZone(dirPath.toString(), null);
+    fs.getClient().setErasureCodingPolicy(dirPath.toString(), null);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index a467f40..4263ffa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -68,7 +68,7 @@ public class TestDFSStripedOutputStream {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
     fs = cluster.getFileSystem();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index 2aa8484..afb7f95 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -118,7 +118,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     cluster.waitActive();
     dfs = cluster.getFileSystem();
     dfs.mkdirs(dir);
-    dfs.createErasureCodingZone(dir, null);
+    dfs.setErasureCodingPolicy(dir, null);
   }
 
   private void tearDown() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
new file mode 100644
index 0000000..f60d77d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -0,0 +1,211 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.junit.Assert.*;
+
+public class TestErasureCodingPolicies {
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private static final int BLOCK_SIZE = 1024;
+  private FSNamesystem namesystem;
+
+  @Before
+  public void setupCluster() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    cluster = new MiniDFSCluster.Builder(conf).
+        numDataNodes(1).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    namesystem = cluster.getNamesystem();
+  }
+
+  @After
+  public void shutdownCluster() throws IOException {
+    cluster.shutdown();
+  }
+
+  @Test
+  public void testBasicSetECPolicy()
+      throws IOException, InterruptedException {
+    final Path testDir = new Path("/ec");
+    fs.mkdir(testDir, FsPermission.getDirDefault());
+
+    /* Normal creation of an erasure coding directory */
+    fs.getClient().setErasureCodingPolicy(testDir.toString(), null);
+
+    /* Verify files under the directory are striped */
+    final Path ECFilePath = new Path(testDir, "foo");
+    fs.create(ECFilePath);
+    INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString());
+    assertTrue(inode.asFile().isStriped());
+
+    /**
+     * Verify that setting EC policy on non-empty directory only affects
+     * newly created files under the directory.
+     */
+    final Path notEmpty = new Path("/nonEmpty");
+    fs.mkdir(notEmpty, FsPermission.getDirDefault());
+    final Path oldFile = new Path(notEmpty, "old");
+    fs.create(oldFile);
+    fs.getClient().setErasureCodingPolicy(notEmpty.toString(), null);
+    final Path newFile = new Path(notEmpty, "new");
+    fs.create(newFile);
+    INode oldInode = namesystem.getFSDirectory().getINode(oldFile.toString());
+    assertFalse(oldInode.asFile().isStriped());
+    INode newInode = namesystem.getFSDirectory().getINode(newFile.toString());
+    assertTrue(newInode.asFile().isStriped());
+
+    /* Verify that nested EC policies not supported */
+    final Path dir1 = new Path("/dir1");
+    final Path dir2 = new Path(dir1, "dir2");
+    fs.mkdir(dir1, FsPermission.getDirDefault());
+    fs.getClient().setErasureCodingPolicy(dir1.toString(), null);
+    fs.mkdir(dir2, FsPermission.getDirDefault());
+    try {
+      fs.getClient().setErasureCodingPolicy(dir2.toString(), null);
+      fail("Nested erasure coding policies");
+    } catch (IOException e) {
+      assertExceptionContains("already has an erasure coding policy", e);
+    }
+
+    /* Verify that EC policy cannot be set on a file */
+    final Path fPath = new Path("/file");
+    fs.create(fPath);
+    try {
+      fs.getClient().setErasureCodingPolicy(fPath.toString(), null);
+      fail("Erasure coding policy on file");
+    } catch (IOException e) {
+      assertExceptionContains("erasure coding policy for a file", e);
+    }
+  }
+
+  @Test
+  public void testMoveValidity() throws IOException, InterruptedException {
+    final Path srcECDir = new Path("/srcEC");
+    final Path dstECDir = new Path("/dstEC");
+    fs.mkdir(srcECDir, FsPermission.getDirDefault());
+    fs.mkdir(dstECDir, FsPermission.getDirDefault());
+    fs.getClient().setErasureCodingPolicy(srcECDir.toString(), null);
+    fs.getClient().setErasureCodingPolicy(dstECDir.toString(), null);
+    final Path srcFile = new Path(srcECDir, "foo");
+    fs.create(srcFile);
+
+    // Test move dir
+    // Move EC dir under non-EC dir
+    final Path newDir = new Path("/srcEC_new");
+    fs.rename(srcECDir, newDir);
+    fs.rename(newDir, srcECDir); // move back
+
+    // Move EC dir under another EC dir
+    fs.rename(srcECDir, dstECDir);
+    fs.rename(new Path("/dstEC/srcEC"), srcECDir); // move back
+
+    // Test move file
+    /* Verify that a file can be moved between 2 EC dirs */
+    fs.rename(srcFile, dstECDir);
+    fs.rename(new Path(dstECDir, "foo"), srcECDir); // move back
+
+    /* Verify that a file can be moved from a non-EC dir to an EC dir */
+    final Path nonECDir = new Path("/nonEC");
+    fs.mkdir(nonECDir, FsPermission.getDirDefault());
+    fs.rename(srcFile, nonECDir);
+
+    /* Verify that a file can be moved from an EC dir to a non-EC dir */
+    final Path nonECFile = new Path(nonECDir, "nonECFile");
+    fs.create(nonECFile);
+    fs.rename(nonECFile, dstECDir);
+  }
+
+  @Test
+  public void testReplication() throws IOException {
+    final Path testDir = new Path("/ec");
+    fs.mkdir(testDir, FsPermission.getDirDefault());
+    fs.setErasureCodingPolicy(testDir, null);
+    final Path fooFile = new Path(testDir, "foo");
+    // create ec file with replication=0
+    fs.create(fooFile, FsPermission.getFileDefault(), true,
+        conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short)0, fs.getDefaultBlockSize(fooFile), null);
+    // set replication should be a no-op
+    fs.setReplication(fooFile, (short) 3);
+  }
+
+  @Test
+  public void testGetErasureCodingPolicyWithSystemDefaultECPolicy() throws Exception {
+    String src = "/ec";
+    final Path ecDir = new Path(src);
+    fs.mkdir(ecDir, FsPermission.getDirDefault());
+    // dir EC policy should be null
+    assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy());
+    // dir EC policy after setting
+    fs.getClient().setErasureCodingPolicy(src, null); //Default one will be used.
+    ErasureCodingPolicy sysDefaultECPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    verifyErasureCodingInfo(src, sysDefaultECPolicy);
+    fs.create(new Path(ecDir, "child1")).close();
+    // verify for the files in ec dir
+    verifyErasureCodingInfo(src + "/child1", sysDefaultECPolicy);
+  }
+
+  @Test
+  public void testGetErasureCodingPolicy() throws Exception {
+    ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager.getSystemPolices();
+    assertTrue("System ecPolicies should be of only 1 for now",
+        sysECPolicies.length == 1);
+
+    ErasureCodingPolicy usingECPolicy = sysECPolicies[0];
+    String src = "/ec2";
+    final Path ecDir = new Path(src);
+    fs.mkdir(ecDir, FsPermission.getDirDefault());
+    // dir ECInfo before being set
+    assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy());
+    // dir ECInfo after set
+    fs.getClient().setErasureCodingPolicy(src, usingECPolicy);
+    verifyErasureCodingInfo(src, usingECPolicy);
+    fs.create(new Path(ecDir, "child1")).close();
+    // verify for the files in ec dir
+    verifyErasureCodingInfo(src + "/child1", usingECPolicy);
+  }
+
+  private void verifyErasureCodingInfo(
+      String src, ErasureCodingPolicy usingECPolicy) throws IOException {
+    HdfsFileStatus hdfsFileStatus = fs.getClient().getFileInfo(src);
+    ErasureCodingPolicy ecPolicy = hdfsFileStatus.getErasureCodingPolicy();
+    assertNotNull(ecPolicy);
+    assertEquals("Actually used ecPolicy should be equal with target ecPolicy",
+        usingECPolicy, ecPolicy);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
deleted file mode 100644
index b68aab9..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
+++ /dev/null
@@ -1,217 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
-import static org.junit.Assert.*;
-
-public class TestErasureCodingZones {
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private DistributedFileSystem fs;
-  private static final int BLOCK_SIZE = 1024;
-  private FSNamesystem namesystem;
-
-  @Before
-  public void setupCluster() throws IOException {
-    conf = new HdfsConfiguration();
-    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    cluster = new MiniDFSCluster.Builder(conf).
-        numDataNodes(1).build();
-    cluster.waitActive();
-    fs = cluster.getFileSystem();
-    namesystem = cluster.getNamesystem();
-  }
-
-  @After
-  public void shutdownCluster() throws IOException {
-    cluster.shutdown();
-  }
-
-  @Test
-  public void testCreateECZone()
-      throws IOException, InterruptedException {
-    final Path testDir = new Path("/ec");
-    fs.mkdir(testDir, FsPermission.getDirDefault());
-
-    /* Normal creation of an erasure coding zone */
-    fs.getClient().createErasureCodingZone(testDir.toString(), null);
-
-    /* Verify files under the zone are striped */
-    final Path ECFilePath = new Path(testDir, "foo");
-    fs.create(ECFilePath);
-    INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString());
-    assertTrue(inode.asFile().isStriped());
-
-    /* Verify that EC zone cannot be created on non-empty dir */
-    final Path notEmpty = new Path("/nonEmpty");
-    fs.mkdir(notEmpty, FsPermission.getDirDefault());
-    fs.create(new Path(notEmpty, "foo"));
-    try {
-      fs.getClient().createErasureCodingZone(notEmpty.toString(), null);
-      fail("Erasure coding zone on non-empty dir");
-    } catch (IOException e) {
-      assertExceptionContains("erasure coding zone for a non-empty directory", e);
-    }
-
-    /* Verify that nested EC zones cannot be created */
-    final Path zone1 = new Path("/zone1");
-    final Path zone2 = new Path(zone1, "zone2");
-    fs.mkdir(zone1, FsPermission.getDirDefault());
-    fs.getClient().createErasureCodingZone(zone1.toString(), null);
-    fs.mkdir(zone2, FsPermission.getDirDefault());
-    try {
-      fs.getClient().createErasureCodingZone(zone2.toString(), null);
-      fail("Nested erasure coding zones");
-    } catch (IOException e) {
-      assertExceptionContains("already in an erasure coding zone", e);
-    }
-
-    /* Verify that EC zone cannot be created on a file */
-    final Path fPath = new Path("/file");
-    fs.create(fPath);
-    try {
-      fs.getClient().createErasureCodingZone(fPath.toString(), null);
-      fail("Erasure coding zone on file");
-    } catch (IOException e) {
-      assertExceptionContains("erasure coding zone for a file", e);
-    }
-  }
-
-  @Test
-  public void testMoveValidity() throws IOException, InterruptedException {
-    final Path srcECDir = new Path("/srcEC");
-    final Path dstECDir = new Path("/dstEC");
-    fs.mkdir(srcECDir, FsPermission.getDirDefault());
-    fs.mkdir(dstECDir, FsPermission.getDirDefault());
-    fs.getClient().createErasureCodingZone(srcECDir.toString(), null);
-    fs.getClient().createErasureCodingZone(dstECDir.toString(), null);
-    final Path srcFile = new Path(srcECDir, "foo");
-    fs.create(srcFile);
-
-    // Test move dir
-    // Move EC dir under non-EC dir
-    final Path newDir = new Path("/srcEC_new");
-    fs.rename(srcECDir, newDir);
-    fs.rename(newDir, srcECDir); // move back
-
-    // Move EC dir under another EC dir
-    fs.rename(srcECDir, dstECDir);
-    fs.rename(new Path("/dstEC/srcEC"), srcECDir); // move back
-
-    // Test move file
-    /* Verify that a file can be moved between 2 EC zones */
-    fs.rename(srcFile, dstECDir);
-    fs.rename(new Path(dstECDir, "foo"), srcECDir); // move back
-
-    /* Verify that a file cannot be moved from a non-EC dir to an EC zone */
-    final Path nonECDir = new Path("/nonEC");
-    fs.mkdir(nonECDir, FsPermission.getDirDefault());
-    try {
-      fs.rename(srcFile, nonECDir);
-      fail("A file shouldn't be able to move from a non-EC dir to an EC zone");
-    } catch (IOException e) {
-      assertExceptionContains("can't be moved because the source and " +
-          "destination have different erasure coding policies", e);
-    }
-
-    /* Verify that a file cannot be moved from an EC zone to a non-EC dir */
-    final Path nonECFile = new Path(nonECDir, "nonECFile");
-    fs.create(nonECFile);
-    try {
-      fs.rename(nonECFile, dstECDir);
-    } catch (IOException e) {
-      assertExceptionContains("can't be moved because the source and " +
-          "destination have different erasure coding policies", e);
-    }
-  }
-
-  @Test
-  public void testReplication() throws IOException {
-    final Path testDir = new Path("/ec");
-    fs.mkdir(testDir, FsPermission.getDirDefault());
-    fs.createErasureCodingZone(testDir, null);
-    final Path fooFile = new Path(testDir, "foo");
-    // create ec file with replication=0
-    fs.create(fooFile, FsPermission.getFileDefault(), true,
-        conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short)0, fs.getDefaultBlockSize(fooFile), null);
-    // set replication should be a no-op
-    fs.setReplication(fooFile, (short) 3);
-  }
-
-  @Test
-  public void testGetErasureCodingInfoWithSystemDefaultECPolicy() throws Exception {
-    String src = "/ec";
-    final Path ecDir = new Path(src);
-    fs.mkdir(ecDir, FsPermission.getDirDefault());
-    // dir ECInfo before creating ec zone
-    assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy());
-    // dir ECInfo after creating ec zone
-    fs.getClient().createErasureCodingZone(src, null); //Default one will be used.
-    ErasureCodingPolicy sysDefaultECPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
-    verifyErasureCodingInfo(src, sysDefaultECPolicy);
-    fs.create(new Path(ecDir, "child1")).close();
-    // verify for the files in ec zone
-    verifyErasureCodingInfo(src + "/child1", sysDefaultECPolicy);
-  }
-
-  @Test
-  public void testGetErasureCodingInfo() throws Exception {
-    ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager.getSystemPolices();
-    assertTrue("System ecPolicies should be of only 1 for now",
-        sysECPolicies.length == 1);
-
-    ErasureCodingPolicy usingECPolicy = sysECPolicies[0];
-    String src = "/ec2";
-    final Path ecDir = new Path(src);
-    fs.mkdir(ecDir, FsPermission.getDirDefault());
-    // dir ECInfo before creating ec zone
-    assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy());
-    // dir ECInfo after creating ec zone
-    fs.getClient().createErasureCodingZone(src, usingECPolicy);
-    verifyErasureCodingInfo(src, usingECPolicy);
-    fs.create(new Path(ecDir, "child1")).close();
-    // verify for the files in ec zone
-    verifyErasureCodingInfo(src + "/child1", usingECPolicy);
-  }
-
-  private void verifyErasureCodingInfo(
-      String src, ErasureCodingPolicy usingECPolicy) throws IOException {
-    HdfsFileStatus hdfsFileStatus = fs.getClient().getFileInfo(src);
-    ErasureCodingPolicy ecPolicy = hdfsFileStatus.getErasureCodingPolicy();
-    assertNotNull(ecPolicy);
-    assertEquals("Actually used ecPolicy should be equal with target ecPolicy",
-        usingECPolicy, ecPolicy);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java
index 4610ced..64063d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java
@@ -38,24 +38,24 @@ public class TestFileStatusWithECPolicy {
 
   @Test
   public void testFileStatusWithECPolicy() throws Exception {
-    // test directory not in EC zone
+    // test directory doesn't have an EC policy
     final Path dir = new Path("/foo");
     assertTrue(fs.mkdir(dir, FsPermission.getDirDefault()));
     assertNull(client.getFileInfo(dir.toString()).getErasureCodingPolicy());
-    // test file not in EC zone
+    // test file doesn't have an EC policy
     final Path file = new Path(dir, "foo");
     fs.create(file).close();
     assertNull(client.getFileInfo(file.toString()).getErasureCodingPolicy());
     fs.delete(file, true);
 
     final ErasureCodingPolicy ecPolicy1 = ErasureCodingPolicyManager.getSystemDefaultPolicy();
-    // create EC zone on dir
-    fs.createErasureCodingZone(dir, ecPolicy1);
+    // set EC policy on dir
+    fs.setErasureCodingPolicy(dir, ecPolicy1);
     final ErasureCodingPolicy ecPolicy2 = client.getFileInfo(dir.toUri().getPath()).getErasureCodingPolicy();
     assertNotNull(ecPolicy2);
     assertTrue(ecPolicy1.equals(ecPolicy2));
 
-    // test file in EC zone
+    // test file doesn't have an EC policy
     fs.create(file).close();
     final ErasureCodingPolicy ecPolicy3 =
         fs.getClient().getFileInfo(file.toUri().getPath()).getErasureCodingPolicy();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
index 9048fbd..68cd25e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
@@ -68,7 +68,7 @@ public class TestReadStripedFileWithDecoding {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
         .numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
     fs = cluster.getFileSystem();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
index 08a5f58..ca376af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
@@ -53,7 +53,7 @@ public class TestReadStripedFileWithMissingBlocks {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
     fs = cluster.getFileSystem();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
index 7a0851f..b581845 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
@@ -78,7 +78,7 @@ public class TestRecoverStripedFile {
     cluster.waitActive();
     
     fs = cluster.getFileSystem();
-    fs.getClient().createErasureCodingZone("/", null);
+    fs.getClient().setErasureCodingPolicy("/", null);
 
     List<DataNode> datanodes = cluster.getDataNodes();
     for (int i = 0; i < dnNum; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
index f577ddb..318eb9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
@@ -54,7 +54,7 @@ public class TestSafeModeWithStripedFile {
     conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
     cluster.waitActive();
 
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
index b383c85..5381fca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
@@ -57,7 +57,7 @@ public class TestWriteReadStripedFile {
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
     fs = cluster.getFileSystem();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
index deffbcc..50f98a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
@@ -30,7 +30,6 @@ import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize;
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks;
@@ -48,7 +47,7 @@ public class TestWriteStripedFileWithFailure {
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
     fs = cluster.getFileSystem();
   }
 
@@ -158,4 +157,4 @@ public class TestWriteStripedFileWithFailure {
       throw new IOException("Failed at i=" + i, e);
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index eb24fb0..21e60c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -82,7 +82,6 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
@@ -1702,7 +1701,7 @@ public class TestBalancer {
       cluster.waitActive();
       client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(),
           ClientProtocol.class).getProxy();
-      client.createErasureCodingZone("/", null);
+      client.setErasureCodingPolicy("/", null);
 
       long totalCapacity = sum(capacities);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
index 2202b34..9dc537c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
@@ -56,7 +56,7 @@ public class TestBlockTokenWithDFSStriped extends TestBlockTokenWithDFS {
     conf = getConf();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.getFileSystem().getClient()
-        .createErasureCodingZone("/", null);
+        .setErasureCodingPolicy("/", null);
     try {
       cluster.waitActive();
       doTestRead(conf, cluster, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java
index 2e084fc..9f4c47d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java
@@ -71,7 +71,7 @@ public class TestSequentialBlockGroupId {
   private MiniDFSCluster cluster;
   private FileSystem fs;
   private SequentialBlockGroupIdGenerator blockGrpIdGenerator;
-  private Path eczone = new Path("/eczone");
+  private Path ecDir = new Path("/ecDir");
 
   @Before
   public void setup() throws Exception {
@@ -84,9 +84,9 @@ public class TestSequentialBlockGroupId {
     fs = cluster.getFileSystem();
     blockGrpIdGenerator = cluster.getNamesystem().getBlockIdManager()
         .getBlockGroupIdGenerator();
-    fs.mkdirs(eczone);
+    fs.mkdirs(ecDir);
     cluster.getFileSystem().getClient()
-        .createErasureCodingZone("/eczone", null);
+        .setErasureCodingPolicy("/ecDir", null);
   }
 
   @After
@@ -104,7 +104,7 @@ public class TestSequentialBlockGroupId {
     long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue();
 
     // Create a file that is 4 blocks long.
-    Path path = new Path(eczone, "testBlockGrpIdGeneration.dat");
+    Path path = new Path(ecDir, "testBlockGrpIdGeneration.dat");
     DFSTestUtil.createFile(fs, path, cellSize, fileLen, blockSize, REPLICATION,
         SEED);
     List<LocatedBlock> blocks = DFSTestUtil.getAllBlocks(fs, path);
@@ -134,7 +134,7 @@ public class TestSequentialBlockGroupId {
 
     // Create a file with a few blocks to rev up the global block ID
     // counter.
-    Path path1 = new Path(eczone, "testBlockGrpIdCollisionDetection_file1.dat");
+    Path path1 = new Path(ecDir, "testBlockGrpIdCollisionDetection_file1.dat");
     DFSTestUtil.createFile(fs, path1, cellSize, fileLen, blockSize,
         REPLICATION, SEED);
     List<LocatedBlock> blocks1 = DFSTestUtil.getAllBlocks(fs, path1);
@@ -145,7 +145,7 @@ public class TestSequentialBlockGroupId {
     blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue);
 
     // Trigger collisions by creating a new file.
-    Path path2 = new Path(eczone, "testBlockGrpIdCollisionDetection_file2.dat");
+    Path path2 = new Path(ecDir, "testBlockGrpIdCollisionDetection_file2.dat");
     DFSTestUtil.createFile(fs, path2, cellSize, fileLen, blockSize,
         REPLICATION, SEED);
     List<LocatedBlock> blocks2 = DFSTestUtil.getAllBlocks(fs, path2);
@@ -204,7 +204,7 @@ public class TestSequentialBlockGroupId {
     // Reset back to the initial value to trigger collision
     blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue);
     // Trigger collisions by creating a new file.
-    Path path2 = new Path(eczone, "testCollisionWithLegacyBlock_file2.dat");
+    Path path2 = new Path(ecDir, "testCollisionWithLegacyBlock_file2.dat");
     DFSTestUtil.createFile(fs, path2, cellSize, fileLen, blockSize,
         REPLICATION, SEED);
     List<LocatedBlock> blocks2 = DFSTestUtil.getAllBlocks(fs, path2);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index 2598fa8..7794f04 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@ -470,8 +470,8 @@ public class TestMover {
       client.mkdirs(barDir, new FsPermission((short) 777), true);
       client.setStoragePolicy(barDir,
           HdfsConstants.HOT_STORAGE_POLICY_NAME);
-      // set "/bar" directory with EC zone.
-      client.createErasureCodingZone(barDir, null);
+      // set an EC policy on "/bar" directory
+      client.setErasureCodingPolicy(barDir, null);
 
       // write file to barDir
       final String fooFile = "/bar/foo";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
index 7d06a9b..ae2cbab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.junit.After;
@@ -39,7 +38,6 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.HashSet;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
@@ -70,7 +68,7 @@ public class TestAddOverReplicatedStripedBlocks {
     cluster.waitActive();
     fs = cluster.getFileSystem();
     fs.mkdirs(dirPath);
-    fs.getClient().createErasureCodingZone(dirPath.toString(), null);
+    fs.getClient().setErasureCodingPolicy(dirPath.toString(), null);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index c472cd8..4351fb9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -74,7 +74,7 @@ public class TestAddStripedBlocks {
         .numDataNodes(GROUP_SIZE).build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
-    dfs.getClient().createErasureCodingZone("/", null);
+    dfs.getClient().setErasureCodingPolicy("/", null);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index 694411f..40572f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -452,7 +452,7 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.getClient().getNamenode().createErasureCodingZone(testDir, null);
+      fs.getClient().getNamenode().setErasureCodingPolicy(testDir, null);
 
       // Create a file with striped block
       Path p = new Path(testFilePath);
@@ -524,7 +524,7 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.getClient().getNamenode().createErasureCodingZone(testDir, null);
+      fs.getClient().getNamenode().setErasureCodingPolicy(testDir, null);
 
       //create a file with striped blocks
       Path p = new Path(testFilePath);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index d3689f3..7565e86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -141,7 +141,7 @@ public class TestFSImage {
   private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf,
                                                boolean isUC) throws IOException{
     // contruct a INode with StripedBlock for saving and loading
-    fsn.createErasureCodingZone("/", null, false);
+    fsn.setErasureCodingPolicy("/", null, false);
     long id = 123456789;
     byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
     PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
@@ -425,7 +425,7 @@ public class TestFSImage {
           .build();
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
-      fs.getClient().getNamenode().createErasureCodingZone("/", null);
+      fs.getClient().getNamenode().setErasureCodingPolicy("/", null);
       Path file = new Path("/striped");
       FSDataOutputStream out = fs.create(file);
       byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
index de29997..efa5027 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -66,7 +65,7 @@ public class TestQuotaWithStripedBlocks {
     dfs = cluster.getFileSystem();
 
     dfs.mkdirs(ecDir);
-    dfs.getClient().createErasureCodingZone(ecDir.toString(), ecPolicy);
+    dfs.getClient().setErasureCodingPolicy(ecDir.toString(), ecPolicy);
     dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA);
     dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA);
     dfs.setStoragePolicy(ecDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
index 48b22c0..2a593d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@ -217,8 +217,8 @@ public class TestStripedINodeFile {
     try {
       final int len = 1024;
       final Path parentDir = new Path("/parentDir");
-      final Path zone = new Path(parentDir, "zone");
-      final Path zoneFile = new Path(zone, "zoneFile");
+      final Path ecDir = new Path(parentDir, "ecDir");
+      final Path ecFile = new Path(ecDir, "ecFile");
       final Path contiguousFile = new Path(parentDir, "someFile");
       final DistributedFileSystem dfs;
       final Configuration conf = new Configuration();
@@ -232,18 +232,18 @@ public class TestStripedINodeFile {
 
       FSNamesystem fsn = cluster.getNamesystem();
       dfs = cluster.getFileSystem();
-      dfs.mkdirs(zone);
+      dfs.mkdirs(ecDir);
 
-      // create erasure zone
-      dfs.createErasureCodingZone(zone, null);
-      DFSTestUtil.createFile(dfs, zoneFile, len, (short) 1, 0xFEED);
+      // set erasure coding policy
+      dfs.setErasureCodingPolicy(ecDir, null);
+      DFSTestUtil.createFile(dfs, ecFile, len, (short) 1, 0xFEED);
       DFSTestUtil.createFile(dfs, contiguousFile, len, (short) 1, 0xFEED);
       final FSDirectory fsd = fsn.getFSDirectory();
 
       // Case-1: Verify the behavior of striped blocks
       // Get blocks of striped file
-      INode inodeStriped = fsd.getINode("/parentDir/zone/zoneFile");
-      assertTrue("Failed to get INodeFile for /parentDir/zone/zoneFile",
+      INode inodeStriped = fsd.getINode("/parentDir/ecDir/ecFile");
+      assertTrue("Failed to get INodeFile for /parentDir/ecDir/ecFile",
           inodeStriped instanceof INodeFile);
       INodeFile inodeStripedFile = (INodeFile) inodeStriped;
       BlockInfo[] stripedBlks = inodeStripedFile.getBlocks();
@@ -252,8 +252,8 @@ public class TestStripedINodeFile {
             blockInfo.isDeleted());
       }
 
-      // delete erasure zone directory
-      dfs.delete(zone, true);
+      // delete directory with erasure coding policy
+      dfs.delete(ecDir, true);
       for (BlockInfo blockInfo : stripedBlks) {
         assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
index 7bfaab6..1e2326a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
@@ -60,7 +60,7 @@ public class TestOfflineImageViewerWithStripedBlocks {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.waitActive();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
     fs = cluster.getFileSystem();
     Path eczone = new Path("/eczone");
     fs.mkdirs(eczone);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index 5e60658..3a10a50 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -48,39 +48,39 @@
     </test>
 
     <test>
-      <description>help: createZone command</description>
+      <description>help: setPolicy command</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -help createZone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -help setPolicy</ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^[ \t]*Create a zone to encode files using a specified policy( )*</expected-output>
+          <expected-output>^[ \t]*Set a specified erasure coding policy to a directory( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-createZone \[-s &lt;policyName&gt;\] &lt;path&gt;(.)*</expected-output>
+          <expected-output>^-setPolicy \[-s &lt;policyName&gt;\] &lt;path&gt;(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>help: getZone command</description>
+      <description>help: getPolicy command</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -help getZone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -help getPolicy</ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Get information about the EC zone at specified path</expected-output>
+          <expected-output>Get erasure coding policy information about at specified path</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-getZone &lt;path&gt;(.)*</expected-output>
+          <expected-output>^-getPolicy &lt;path&gt;(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -106,63 +106,63 @@
 
   <!-- Test erasure code commands -->
     <test>
-      <description>createZone : create a zone to encode files</description>
+      <description>setPolicy : set erasure coding policy on a directory to encode files</description>
       <test-commands>
-        <command>-fs NAMENODE -mkdir /eczone</command>
-        <ec-admin-command>-fs NAMENODE -createZone -s RS-6-3-64k /eczone</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
-        <command>-fs NAMENODE -rmdir /eczone</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>EC Zone created successfully at NAMENODE/eczone</expected-output>
+          <expected-output>EC policy set successfully at NAMENODE/ecdir</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>createZone : create a zone twice</description>
+      <description>setPolicy : set a policy twice</description>
       <test-commands>
-        <command>-fs NAMENODE -mkdir /eczone</command>
-        <ec-admin-command>-fs NAMENODE -createZone /eczone</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -createZone /eczone</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
-        <command>-fs NAMENODE -rmdir /eczone</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Directory /eczone is already in an erasure coding zone</expected-output>
+          <expected-output>Directory /ecdir already has an erasure coding policy</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>createZone : default policy</description>
+      <description>setPolicy : default policy</description>
       <test-commands>
-        <command>-fs NAMENODE -mkdir /eczone</command>
-        <ec-admin-command>-fs NAMENODE -createZone /eczone</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -getZone /eczone</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
-        <command>-fs NAMENODE -rmdir /eczone</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Dir: /eczone, Policy: ErasureCodingPolicy=[Name=RS-6-3-64k</expected-output>
+          <expected-output>ErasureCodingPolicy=[Name=RS-6-3-64k</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>getZone : get information about the EC zone at specified path not in zone</description>
+      <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /noec</command>
-        <ec-admin-command>-fs NAMENODE -getZone /noec</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy /noec</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /noec</command>
@@ -170,45 +170,45 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Path NAMENODE/noec is not in EC zone</expected-output>
+          <expected-output>Path NAMENODE/noec is not erasure coded</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>getZone : get information about the EC zone at specified path</description>
+      <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
       <test-commands>
-        <command>-fs NAMENODE -mkdir /eczone</command>
-        <ec-admin-command>-fs NAMENODE -createZone -s RS-6-3-64k /eczone</ec-admin-command>
-        <ec-admin-command>-fs NAMENODE -getZone /eczone</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
-        <command>-fs NAMENODE -rmdir /eczone</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Dir: /eczone, Policy: ErasureCodingPolicy=[Name=RS-6-3-64k</expected-output>
+          <expected-output>ErasureCodingPolicy=[Name=RS-6-3-64k</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>getZone : get EC zone at specified file path</description>
+      <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
       <test-commands>
-        <command>-fs NAMENODE -mkdir /eczone</command>
-        <ec-admin-command>-fs NAMENODE -createZone -s RS-6-3-64k /eczone</ec-admin-command>
-        <command>-fs NAMENODE -touchz /eczone/ecfile</command>
-        <ec-admin-command>-fs NAMENODE -getZone /eczone/ecfile</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir</ec-admin-command>
+        <command>-fs NAMENODE -touchz /ecdir/ecfile</command>
+        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir/ecfile</ec-admin-command>
       </test-commands>
       <cleanup-commands>
-        <command>-fs NAMENODE -rm /eczone/ecfile</command>
-        <command>-fs NAMENODE -rmdir /eczone</command>
+        <command>-fs NAMENODE -rm /ecdir/ecfile</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Dir: /eczone, Policy: ErasureCodingPolicy=[Name=RS-6-3-64k</expected-output>
+          <expected-output>ErasureCodingPolicy=[Name=RS-6-3-64k</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -230,64 +230,64 @@
 
 <!-- Test illegal parameters -->
     <test>
-      <description>createZone : illegal parameters - path is missing</description>
+      <description>setPolicy : illegal parameters - path is missing</description>
       <test-commands>
-        <command>-fs NAMENODE -mkdir /eczone</command>
-        <ec-admin-command>-fs NAMENODE -createZone</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy</ec-admin-command>
       </test-commands>
       <cleanup-commands>
-        <command>-fs NAMENODE -rmdir /eczone</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-createZone: &lt;path&gt; is missing(.)*</expected-output>
+          <expected-output>^-setPolicy: &lt;path&gt; is missing(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>createZone : illegal parameters - policy name is missing</description>
+      <description>setPolicy : illegal parameters - policy name is missing</description>
       <test-commands>
-        <command>-fs NAMENODE -mkdir /eczone</command>
-        <ec-admin-command>-fs NAMENODE -createZone -s</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -s</ec-admin-command>
       </test-commands>
       <cleanup-commands>
-        <command>-fs NAMENODE -rmdir /eczone</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-createZone: option -s requires 1 argument(.)*</expected-output>
+          <expected-output>^-setPolicy: option -s requires 1 argument(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>createZone : illegal parameters - too many arguments</description>
+      <description>setPolicy : illegal parameters - too many arguments</description>
       <test-commands>
-        <command>-fs NAMENODE -mkdir /eczone</command>
-        <ec-admin-command>-fs NAMENODE -createZone /eczone1 /eczone2</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir1 /ecdir2</ec-admin-command>
       </test-commands>
       <cleanup-commands>
-        <command>-fs NAMENODE -rmdir /eczone</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>-createZone: Too many arguments</expected-output>
+          <expected-output>-setPolicy: Too many arguments</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>createZone : illegal parameters - invalidpolicy</description>
+      <description>setPolicy : illegal parameters - invalidpolicy</description>
       <test-commands>
-        <command>-fs NAMENODE -mkdir /eczone</command>
-        <ec-admin-command>-fs NAMENODE -createZone -s invalidpolicy /eczone</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -s invalidpolicy /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
-        <command>-fs NAMENODE -rmdir /eczone</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
@@ -298,62 +298,62 @@
     </test>
 
     <test>
-      <description>createZone : illegal parameters - no such file</description>
+      <description>setPolicy : illegal parameters - no such file</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -createZone /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^createZone: `/eczone': No such file or directory(.)*</expected-output>
+          <expected-output>^setPolicy: `/ecdir': No such file or directory(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>getZone : illegal parameters - path is missing</description>
+      <description>getPolicy : illegal parameters - path is missing</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getZone </ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy </ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-getZone: &lt;path&gt; is missing(.)*</expected-output>
+          <expected-output>^-getPolicy: &lt;path&gt; is missing(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>getZone : illegal parameters - too many arguments</description>
+      <description>getPolicy : illegal parameters - too many arguments</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getZone /eczone /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
-        <command>-fs NAMENODE -rm /eczone</command>
+        <command>-fs NAMENODE -rm /ecdir</command>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>-getZone: Too many arguments</expected-output>
+          <expected-output>-getPolicy: Too many arguments</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>getZone : illegal parameters - no such file</description>
+      <description>getPolicy : illegal parameters - no such file</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getZone /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^getZone: `/eczone': No such file or directory(.)*</expected-output>
+          <expected-output>^getPolicy: `/ecdir': No such file or directory(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -361,7 +361,7 @@
     <test>
       <description>listPolicies : illegal parameters - too many parameters</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -listPolicies /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -listPolicies /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>


[43/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 0000000,ea86d2d..18cc124
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@@ -1,0 -1,2259 +1,2335 @@@
+ /**
+  * 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.hdfs;
+ 
+ import java.io.FileNotFoundException;
+ import java.io.IOException;
+ import java.net.InetSocketAddress;
+ import java.net.URI;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.EnumSet;
+ import java.util.List;
+ import java.util.Map;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.fs.BlockLocation;
+ import org.apache.hadoop.fs.BlockStoragePolicySpi;
+ import org.apache.hadoop.fs.CacheFlag;
+ import org.apache.hadoop.fs.ContentSummary;
+ import org.apache.hadoop.fs.CreateFlag;
+ import org.apache.hadoop.fs.FSDataInputStream;
+ import org.apache.hadoop.fs.FSDataOutputStream;
+ import org.apache.hadoop.fs.FSLinkResolver;
+ import org.apache.hadoop.fs.FileAlreadyExistsException;
+ import org.apache.hadoop.fs.FileChecksum;
+ import org.apache.hadoop.fs.FileStatus;
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.FileSystemLinkResolver;
+ import org.apache.hadoop.fs.FsServerDefaults;
+ import org.apache.hadoop.fs.FsStatus;
+ import org.apache.hadoop.fs.LocatedFileStatus;
+ import org.apache.hadoop.fs.Options;
+ import org.apache.hadoop.fs.XAttrSetFlag;
+ import org.apache.hadoop.fs.Options.ChecksumOpt;
+ import org.apache.hadoop.fs.ParentNotDirectoryException;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.fs.PathFilter;
+ import org.apache.hadoop.fs.RemoteIterator;
+ import org.apache.hadoop.fs.UnresolvedLinkException;
+ import org.apache.hadoop.fs.UnsupportedFileSystemException;
+ import org.apache.hadoop.fs.permission.AclEntry;
+ import org.apache.hadoop.fs.permission.AclStatus;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.fs.permission.FsAction;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+ import org.apache.hadoop.hdfs.client.impl.CorruptFileBlockIterator;
+ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+ import org.apache.hadoop.io.Text;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+ import org.apache.hadoop.net.NetUtils;
+ import org.apache.hadoop.security.AccessControlException;
+ import org.apache.hadoop.security.Credentials;
+ import org.apache.hadoop.security.token.Token;
+ import org.apache.hadoop.util.Progressable;
+ import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.base.Preconditions;
+ 
+ 
+ /****************************************************************
+  * Implementation of the abstract FileSystem for the DFS system.
+  * This object is the way end-user code interacts with a Hadoop
+  * DistributedFileSystem.
+  *
+  *****************************************************************/
+ @InterfaceAudience.LimitedPrivate({ "MapReduce", "HBase" })
+ @InterfaceStability.Unstable
+ public class DistributedFileSystem extends FileSystem {
+   private Path workingDir;
+   private URI uri;
+   private String homeDirPrefix =
+       HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT;
+ 
+   DFSClient dfs;
+   private boolean verifyChecksum = true;
+   
+   static{
+     HdfsConfigurationLoader.init();
+   }
+ 
+   public DistributedFileSystem() {
+   }
+ 
+   /**
+    * Return the protocol scheme for the FileSystem.
+    * <p/>
+    *
+    * @return <code>hdfs</code>
+    */
+   @Override
+   public String getScheme() {
+     return HdfsConstants.HDFS_URI_SCHEME;
+   }
+ 
+   @Override
+   public URI getUri() { return uri; }
+ 
+   @Override
+   public void initialize(URI uri, Configuration conf) throws IOException {
+     super.initialize(uri, conf);
+     setConf(conf);
+ 
+     String host = uri.getHost();
+     if (host == null) {
+       throw new IOException("Incomplete HDFS URI, no host: "+ uri);
+     }
+     homeDirPrefix = conf.get(
+         HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_KEY,
+         HdfsClientConfigKeys.DFS_USER_HOME_DIR_PREFIX_DEFAULT);
+ 
+     this.dfs = new DFSClient(uri, conf, statistics);
+     this.uri = URI.create(uri.getScheme()+"://"+uri.getAuthority());
+     this.workingDir = getHomeDirectory();
+   }
+ 
+   @Override
+   public Path getWorkingDirectory() {
+     return workingDir;
+   }
+ 
+   @Override
+   public long getDefaultBlockSize() {
+     return dfs.getConf().getDefaultBlockSize();
+   }
+ 
+   @Override
+   public short getDefaultReplication() {
+     return dfs.getConf().getDefaultReplication();
+   }
+ 
+   @Override
+   public void setWorkingDirectory(Path dir) {
+     String result = fixRelativePart(dir).toUri().getPath();
+     if (!DFSUtilClient.isValidName(result)) {
+       throw new IllegalArgumentException("Invalid DFS directory name " + 
+                                          result);
+     }
+     workingDir = fixRelativePart(dir);
+   }
+ 
+   @Override
+   public Path getHomeDirectory() {
+     return makeQualified(new Path(homeDirPrefix + "/"
+         + dfs.ugi.getShortUserName()));
+   }
+ 
+   /**
+    * Checks that the passed URI belongs to this filesystem and returns
+    * just the path component. Expects a URI with an absolute path.
+    * 
+    * @param file URI with absolute path
+    * @return path component of {file}
+    * @throws IllegalArgumentException if URI does not belong to this DFS
+    */
+   private String getPathName(Path file) {
+     checkPath(file);
+     String result = file.toUri().getPath();
+     if (!DFSUtilClient.isValidName(result)) {
+       throw new IllegalArgumentException("Pathname " + result + " from " +
+                                          file+" is not a valid DFS filename.");
+     }
+     return result;
+   }
+   
+   @Override
+   public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
+       long len) throws IOException {
+     if (file == null) {
+       return null;
+     }
+     return getFileBlockLocations(file.getPath(), start, len);
+   }
+   
+   @Override
+   public BlockLocation[] getFileBlockLocations(Path p, 
+       final long start, final long len) throws IOException {
+     statistics.incrementReadOps(1);
+     final Path absF = fixRelativePart(p);
+     return new FileSystemLinkResolver<BlockLocation[]>() {
+       @Override
+       public BlockLocation[] doCall(final Path p) throws IOException {
+         return dfs.getBlockLocations(getPathName(p), start, len);
+       }
+       @Override
+       public BlockLocation[] next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.getFileBlockLocations(p, start, len);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void setVerifyChecksum(boolean verifyChecksum) {
+     this.verifyChecksum = verifyChecksum;
+   }
+ 
+   /** 
+    * Start the lease recovery of a file
+    *
+    * @param f a file
+    * @return true if the file is already closed
+    * @throws IOException if an error occurs
+    */
+   public boolean recoverLease(final Path f) throws IOException {
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.recoverLease(getPathName(p));
+       }
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           return myDfs.recoverLease(p);
+         }
+         throw new UnsupportedOperationException("Cannot recoverLease through" +
+             " a symlink to a non-DistributedFileSystem: " + f + " -> " + p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public FSDataInputStream open(Path f, final int bufferSize)
+       throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FSDataInputStream>() {
+       @Override
+       public FSDataInputStream doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         final DFSInputStream dfsis =
+           dfs.open(getPathName(p), bufferSize, verifyChecksum);
+         return dfs.createWrappedInputStream(dfsis);
+       }
+       @Override
+       public FSDataInputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.open(p, bufferSize);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public FSDataOutputStream append(Path f, final int bufferSize,
+       final Progressable progress) throws IOException {
+     return append(f, EnumSet.of(CreateFlag.APPEND), bufferSize, progress);
+   }
+ 
+   /**
+    * Append to an existing file (optional operation).
+    * 
+    * @param f the existing file to be appended.
+    * @param flag Flags for the Append operation. CreateFlag.APPEND is mandatory
+    *          to be present.
+    * @param bufferSize the size of the buffer to be used.
+    * @param progress for reporting progress if it is not null.
+    * @return Returns instance of {@link FSDataOutputStream}
+    * @throws IOException
+    */
+   public FSDataOutputStream append(Path f, final EnumSet<CreateFlag> flag,
+       final int bufferSize, final Progressable progress) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FSDataOutputStream>() {
+       @Override
+       public FSDataOutputStream doCall(final Path p)
+           throws IOException {
+         return dfs.append(getPathName(p), bufferSize, flag, progress,
+             statistics);
+       }
+       @Override
+       public FSDataOutputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.append(p, bufferSize);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * Append to an existing file (optional operation).
+    * 
+    * @param f the existing file to be appended.
+    * @param flag Flags for the Append operation. CreateFlag.APPEND is mandatory
+    *          to be present.
+    * @param bufferSize the size of the buffer to be used.
+    * @param progress for reporting progress if it is not null.
+    * @param favoredNodes Favored nodes for new blocks
+    * @return Returns instance of {@link FSDataOutputStream}
+    * @throws IOException
+    */
+   public FSDataOutputStream append(Path f, final EnumSet<CreateFlag> flag,
+       final int bufferSize, final Progressable progress,
+       final InetSocketAddress[] favoredNodes) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FSDataOutputStream>() {
+       @Override
+       public FSDataOutputStream doCall(final Path p)
+           throws IOException {
+         return dfs.append(getPathName(p), bufferSize, flag, progress,
+             statistics, favoredNodes);
+       }
+       @Override
+       public FSDataOutputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.append(p, bufferSize);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public FSDataOutputStream create(Path f, FsPermission permission,
+       boolean overwrite, int bufferSize, short replication, long blockSize,
+       Progressable progress) throws IOException {
+     return this.create(f, permission,
+         overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
+             : EnumSet.of(CreateFlag.CREATE), bufferSize, replication,
+         blockSize, progress, null);
+   }
+ 
+   /**
+    * Same as  
+    * {@link #create(Path, FsPermission, boolean, int, short, long, 
+    * Progressable)} with the addition of favoredNodes that is a hint to 
+    * where the namenode should place the file blocks.
+    * The favored nodes hint is not persisted in HDFS. Hence it may be honored
+    * at the creation time only. And with favored nodes, blocks will be pinned
+    * on the datanodes to prevent balancing move the block. HDFS could move the
+    * blocks during replication, to move the blocks from favored nodes. A value
+    * of null means no favored nodes for this create
+    */
+   public HdfsDataOutputStream create(final Path f,
+       final FsPermission permission, final boolean overwrite,
+       final int bufferSize, final short replication, final long blockSize,
+       final Progressable progress, final InetSocketAddress[] favoredNodes)
+           throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<HdfsDataOutputStream>() {
+       @Override
+       public HdfsDataOutputStream doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         final DFSOutputStream out = dfs.create(getPathName(f), permission,
+             overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
+                 : EnumSet.of(CreateFlag.CREATE),
+             true, replication, blockSize, progress, bufferSize, null,
+             favoredNodes);
+         return dfs.createWrappedOutputStream(out, statistics);
+       }
+       @Override
+       public HdfsDataOutputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           return myDfs.create(p, permission, overwrite, bufferSize, replication,
+               blockSize, progress, favoredNodes);
+         }
+         throw new UnsupportedOperationException("Cannot create with" +
+             " favoredNodes through a symlink to a non-DistributedFileSystem: "
+             + f + " -> " + p);
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public FSDataOutputStream create(final Path f, final FsPermission permission,
+     final EnumSet<CreateFlag> cflags, final int bufferSize,
+     final short replication, final long blockSize, final Progressable progress,
+     final ChecksumOpt checksumOpt) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FSDataOutputStream>() {
+       @Override
+       public FSDataOutputStream doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
+                 cflags, replication, blockSize, progress, bufferSize,
+                 checksumOpt);
+         return dfs.createWrappedOutputStream(dfsos, statistics);
+       }
+       @Override
+       public FSDataOutputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.create(p, permission, cflags, bufferSize,
+             replication, blockSize, progress, checksumOpt);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   protected HdfsDataOutputStream primitiveCreate(Path f,
+     FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
+     short replication, long blockSize, Progressable progress,
+     ChecksumOpt checksumOpt) throws IOException {
+     statistics.incrementWriteOps(1);
+     final DFSOutputStream dfsos = dfs.primitiveCreate(
+       getPathName(fixRelativePart(f)),
+       absolutePermission, flag, true, replication, blockSize,
+       progress, bufferSize, checksumOpt);
+     return dfs.createWrappedOutputStream(dfsos, statistics);
+   }
+ 
+   /**
+    * Same as create(), except fails if parent directory doesn't already exist.
+    */
+   @Override
+   public FSDataOutputStream createNonRecursive(final Path f,
+       final FsPermission permission, final EnumSet<CreateFlag> flag,
+       final int bufferSize, final short replication, final long blockSize,
+       final Progressable progress) throws IOException {
+     statistics.incrementWriteOps(1);
+     if (flag.contains(CreateFlag.OVERWRITE)) {
+       flag.add(CreateFlag.CREATE);
+     }
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FSDataOutputStream>() {
+       @Override
+       public FSDataOutputStream doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
+           flag, false, replication, blockSize, progress, bufferSize, null);
+         return dfs.createWrappedOutputStream(dfsos, statistics);
+       }
+ 
+       @Override
+       public FSDataOutputStream next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.createNonRecursive(p, permission, flag, bufferSize,
+             replication, blockSize, progress);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public boolean setReplication(Path src, 
+                                 final short replication
+                                ) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(src);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.setReplication(getPathName(p), replication);
+       }
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.setReplication(p, replication);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * Set the source path to the specified storage policy.
+    *
+    * @param src The source path referring to either a directory or a file.
+    * @param policyName The name of the storage policy.
+    */
+   @Override
+   public void setStoragePolicy(final Path src, final String policyName)
+       throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(src);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.setStoragePolicy(getPathName(p), policyName);
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         fs.setStoragePolicy(p, policyName);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public BlockStoragePolicySpi getStoragePolicy(Path path) throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(path);
+ 
+     return new FileSystemLinkResolver<BlockStoragePolicySpi>() {
+       @Override
+       public BlockStoragePolicySpi doCall(final Path p) throws IOException {
+         return getClient().getStoragePolicy(getPathName(p));
+       }
+ 
+       @Override
+       public BlockStoragePolicySpi next(final FileSystem fs, final Path p)
+           throws IOException, UnresolvedLinkException {
+         return fs.getStoragePolicy(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public Collection<BlockStoragePolicy> getAllStoragePolicies()
+       throws IOException {
+     return Arrays.asList(dfs.getStoragePolicies());
+   }
+ 
+   /**
+    * Deprecated. Prefer {@link FileSystem#getAllStoragePolicies()}
+    * @return
+    * @throws IOException
+    */
+   @Deprecated
+   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+     statistics.incrementReadOps(1);
+     return dfs.getStoragePolicies();
+   }
+ 
+   /**
+    * Move blocks from srcs to trg and delete srcs afterwards.
+    * The file block sizes must be the same.
+    * 
+    * @param trg existing file to append to
+    * @param psrcs list of files (same block size, same replication)
+    * @throws IOException
+    */
+   @Override
+   public void concat(Path trg, Path [] psrcs) throws IOException {
+     statistics.incrementWriteOps(1);
+     // Make target absolute
+     Path absF = fixRelativePart(trg);
+     // Make all srcs absolute
+     Path[] srcs = new Path[psrcs.length];
+     for (int i=0; i<psrcs.length; i++) {
+       srcs[i] = fixRelativePart(psrcs[i]);
+     }
+     // Try the concat without resolving any links
+     String[] srcsStr = new String[psrcs.length];
+     try {
+       for (int i=0; i<psrcs.length; i++) {
+         srcsStr[i] = getPathName(srcs[i]);
+       }
+       dfs.concat(getPathName(trg), srcsStr);
+     } catch (UnresolvedLinkException e) {
+       // Exception could be from trg or any src.
+       // Fully resolve trg and srcs. Fail if any of them are a symlink.
+       FileStatus stat = getFileLinkStatus(absF);
+       if (stat.isSymlink()) {
+         throw new IOException("Cannot concat with a symlink target: "
+             + trg + " -> " + stat.getPath());
+       }
+       absF = fixRelativePart(stat.getPath());
+       for (int i=0; i<psrcs.length; i++) {
+         stat = getFileLinkStatus(srcs[i]);
+         if (stat.isSymlink()) {
+           throw new IOException("Cannot concat with a symlink src: "
+               + psrcs[i] + " -> " + stat.getPath());
+         }
+         srcs[i] = fixRelativePart(stat.getPath());
+       }
+       // Try concat again. Can still race with another symlink.
+       for (int i=0; i<psrcs.length; i++) {
+         srcsStr[i] = getPathName(srcs[i]);
+       }
+       dfs.concat(getPathName(absF), srcsStr);
+     }
+   }
+ 
+   
+   @SuppressWarnings("deprecation")
+   @Override
+   public boolean rename(Path src, Path dst) throws IOException {
+     statistics.incrementWriteOps(1);
+ 
+     final Path absSrc = fixRelativePart(src);
+     final Path absDst = fixRelativePart(dst);
+ 
+     // Try the rename without resolving first
+     try {
+       return dfs.rename(getPathName(absSrc), getPathName(absDst));
+     } catch (UnresolvedLinkException e) {
+       // Fully resolve the source
+       final Path source = getFileLinkStatus(absSrc).getPath();
+       // Keep trying to resolve the destination
+       return new FileSystemLinkResolver<Boolean>() {
+         @Override
+         public Boolean doCall(final Path p)
+             throws IOException, UnresolvedLinkException {
+           return dfs.rename(getPathName(source), getPathName(p));
+         }
+         @Override
+         public Boolean next(final FileSystem fs, final Path p)
+             throws IOException {
+           // Should just throw an error in FileSystem#checkPath
+           return doCall(p);
+         }
+       }.resolve(this, absDst);
+     }
+   }
+ 
+   /** 
+    * This rename operation is guaranteed to be atomic.
+    */
+   @SuppressWarnings("deprecation")
+   @Override
+   public void rename(Path src, Path dst, final Options.Rename... options)
+       throws IOException {
+     statistics.incrementWriteOps(1);
+     final Path absSrc = fixRelativePart(src);
+     final Path absDst = fixRelativePart(dst);
+     // Try the rename without resolving first
+     try {
+       dfs.rename(getPathName(absSrc), getPathName(absDst), options);
+     } catch (UnresolvedLinkException e) {
+       // Fully resolve the source
+       final Path source = getFileLinkStatus(absSrc).getPath();
+       // Keep trying to resolve the destination
+       new FileSystemLinkResolver<Void>() {
+         @Override
+         public Void doCall(final Path p)
+             throws IOException, UnresolvedLinkException {
+           dfs.rename(getPathName(source), getPathName(p), options);
+           return null;
+         }
+         @Override
+         public Void next(final FileSystem fs, final Path p)
+             throws IOException {
+           // Should just throw an error in FileSystem#checkPath
+           return doCall(p);
+         }
+       }.resolve(this, absDst);
+     }
+   }
+ 
+   @Override
+   public boolean truncate(Path f, final long newLength) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.truncate(getPathName(p), newLength);
+       }
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.truncate(p, newLength);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public boolean delete(Path f, final boolean recursive) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.delete(getPathName(p), recursive);
+       }
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.delete(p, recursive);
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public ContentSummary getContentSummary(Path f) throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<ContentSummary>() {
+       @Override
+       public ContentSummary doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.getContentSummary(getPathName(p));
+       }
+       @Override
+       public ContentSummary next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.getContentSummary(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /** Set a directory's quotas
+    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long, StorageType)
+    */
+   public void setQuota(Path src, final long namespaceQuota,
+       final long storagespaceQuota) throws IOException {
+     Path absF = fixRelativePart(src);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.setQuota(getPathName(p), namespaceQuota, storagespaceQuota);
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         // setQuota is not defined in FileSystem, so we only can resolve
+         // within this DFS
+         return doCall(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * Set the per type storage quota of a directory.
+    *
+    * @param src target directory whose quota is to be modified.
+    * @param type storage type of the specific storage type quota to be modified.
+    * @param quota value of the specific storage type quota to be modified.
+    * Maybe {@link HdfsConstants#QUOTA_RESET} to clear quota by storage type.
+    */
+   public void setQuotaByStorageType(
+     Path src, final StorageType type, final long quota)
+     throws IOException {
+     Path absF = fixRelativePart(src);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+         throws IOException, UnresolvedLinkException {
+         dfs.setQuotaByStorageType(getPathName(p), type, quota);
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+         throws IOException {
+         // setQuotaByStorageType is not defined in FileSystem, so we only can resolve
+         // within this DFS
+         return doCall(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   private FileStatus[] listStatusInternal(Path p) throws IOException {
+     String src = getPathName(p);
+ 
+     // fetch the first batch of entries in the directory
+     DirectoryListing thisListing = dfs.listPaths(
+         src, HdfsFileStatus.EMPTY_NAME);
+ 
+     if (thisListing == null) { // the directory does not exist
+       throw new FileNotFoundException("File " + p + " does not exist.");
+     }
+     
+     HdfsFileStatus[] partialListing = thisListing.getPartialListing();
+     if (!thisListing.hasMore()) { // got all entries of the directory
+       FileStatus[] stats = new FileStatus[partialListing.length];
+       for (int i = 0; i < partialListing.length; i++) {
+         stats[i] = partialListing[i].makeQualified(getUri(), p);
+       }
+       statistics.incrementReadOps(1);
+       return stats;
+     }
+ 
+     // The directory size is too big that it needs to fetch more
+     // estimate the total number of entries in the directory
+     int totalNumEntries =
+       partialListing.length + thisListing.getRemainingEntries();
+     ArrayList<FileStatus> listing =
+       new ArrayList<FileStatus>(totalNumEntries);
+     // add the first batch of entries to the array list
+     for (HdfsFileStatus fileStatus : partialListing) {
+       listing.add(fileStatus.makeQualified(getUri(), p));
+     }
+     statistics.incrementLargeReadOps(1);
+  
+     // now fetch more entries
+     do {
+       thisListing = dfs.listPaths(src, thisListing.getLastName());
+  
+       if (thisListing == null) { // the directory is deleted
+         throw new FileNotFoundException("File " + p + " does not exist.");
+       }
+  
+       partialListing = thisListing.getPartialListing();
+       for (HdfsFileStatus fileStatus : partialListing) {
+         listing.add(fileStatus.makeQualified(getUri(), p));
+       }
+       statistics.incrementLargeReadOps(1);
+     } while (thisListing.hasMore());
+  
+     return listing.toArray(new FileStatus[listing.size()]);
+   }
+ 
+   /**
+    * List all the entries of a directory
+    *
+    * Note that this operation is not atomic for a large directory.
+    * The entries of a directory may be fetched from NameNode multiple times.
+    * It only guarantees that  each name occurs once if a directory
+    * undergoes changes between the calls.
+    */
+   @Override
+   public FileStatus[] listStatus(Path p) throws IOException {
+     Path absF = fixRelativePart(p);
+     return new FileSystemLinkResolver<FileStatus[]>() {
+       @Override
+       public FileStatus[] doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return listStatusInternal(p);
+       }
+       @Override
+       public FileStatus[] next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.listStatus(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path p,
+       final PathFilter filter)
+   throws IOException {
+     Path absF = fixRelativePart(p);
+     return new FileSystemLinkResolver<RemoteIterator<LocatedFileStatus>>() {
+       @Override
+       public RemoteIterator<LocatedFileStatus> doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return new DirListingIterator<LocatedFileStatus>(p, filter, true);
+       }
+ 
+       @Override
+       public RemoteIterator<LocatedFileStatus> next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           return ((DistributedFileSystem)fs).listLocatedStatus(p, filter);
+         }
+         // symlink resolution for this methos does not work cross file systems
+         // because it is a protected method.
+         throw new IOException("Link resolution does not work with multiple " +
+             "file systems for listLocatedStatus(): " + p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+ 
+   /**
+    * Returns a remote iterator so that followup calls are made on demand
+    * while consuming the entries. This reduces memory consumption during
+    * listing of a large directory.
+    *
+    * @param p target path
+    * @return remote iterator
+    */
+   @Override
+   public RemoteIterator<FileStatus> listStatusIterator(final Path p)
+   throws IOException {
+     Path absF = fixRelativePart(p);
+     return new FileSystemLinkResolver<RemoteIterator<FileStatus>>() {
+       @Override
+       public RemoteIterator<FileStatus> doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return new DirListingIterator<FileStatus>(p, false);
+       }
+ 
+       @Override
+       public RemoteIterator<FileStatus> next(final FileSystem fs, final Path p)
+           throws IOException {
+           return ((DistributedFileSystem)fs).listStatusIterator(p);
+       }
+     }.resolve(this, absF);
+ 
+   }
+ 
+   /**
+    * This class defines an iterator that returns
+    * the file status of each file/subdirectory of a directory
+    * 
+    * if needLocation, status contains block location if it is a file
+    * throws a RuntimeException with the error as its cause.
+    * 
+    * @param <T> the type of the file status
+    */
+   private class  DirListingIterator<T extends FileStatus>
+   implements RemoteIterator<T> {
+     private DirectoryListing thisListing;
+     private int i;
+     private Path p;
+     private String src;
+     private T curStat = null;
+     private PathFilter filter;
+     private boolean needLocation;
+ 
+     private DirListingIterator(Path p, PathFilter filter,
+         boolean needLocation) throws IOException {
+       this.p = p;
+       this.src = getPathName(p);
+       this.filter = filter;
+       this.needLocation = needLocation;
+       // fetch the first batch of entries in the directory
+       thisListing = dfs.listPaths(src, HdfsFileStatus.EMPTY_NAME,
+           needLocation);
+       statistics.incrementReadOps(1);
+       if (thisListing == null) { // the directory does not exist
+         throw new FileNotFoundException("File " + p + " does not exist.");
+       }
+       i = 0;
+     }
+ 
+     private DirListingIterator(Path p, boolean needLocation)
+         throws IOException {
+       this(p, null, needLocation);
+     }
+ 
+     @Override
+     @SuppressWarnings("unchecked")
+     public boolean hasNext() throws IOException {
+       while (curStat == null && hasNextNoFilter()) {
+         T next;
+         HdfsFileStatus fileStat = thisListing.getPartialListing()[i++];
+         if (needLocation) {
+           next = (T)((HdfsLocatedFileStatus)fileStat)
+               .makeQualifiedLocated(getUri(), p);
+         } else {
+           next = (T)fileStat.makeQualified(getUri(), p);
+         }
+           // apply filter if not null
+         if (filter == null || filter.accept(next.getPath())) {
+           curStat = next;
+         }
+       }
+       return curStat != null;
+     }
+       
+     /** Check if there is a next item before applying the given filter */
+     private boolean hasNextNoFilter() throws IOException {
+       if (thisListing == null) {
+         return false;
+       }
+       if (i >= thisListing.getPartialListing().length
+           && thisListing.hasMore()) { 
+         // current listing is exhausted & fetch a new listing
+         thisListing = dfs.listPaths(src, thisListing.getLastName(),
+             needLocation);
+         statistics.incrementReadOps(1);
+         if (thisListing == null) {
+           return false;
+         }
+         i = 0;
+       }
+       return (i < thisListing.getPartialListing().length);
+     }
+ 
+     @Override
+     public T next() throws IOException {
+       if (hasNext()) {
+         T tmp = curStat;
+         curStat = null;
+         return tmp;
+       } 
+       throw new java.util.NoSuchElementException("No more entry in " + p);
+     }
+   }
+   
+   /**
+    * Create a directory, only when the parent directories exist.
+    *
+    * See {@link FsPermission#applyUMask(FsPermission)} for details of how
+    * the permission is applied.
+    *
+    * @param f           The path to create
+    * @param permission  The permission.  See FsPermission#applyUMask for 
+    *                    details about how this is used to calculate the
+    *                    effective permission.
+    */
+   public boolean mkdir(Path f, FsPermission permission) throws IOException {
+     return mkdirsInternal(f, permission, false);
+   }
+ 
+   /**
+    * Create a directory and its parent directories.
+    *
+    * See {@link FsPermission#applyUMask(FsPermission)} for details of how
+    * the permission is applied.
+    *
+    * @param f           The path to create
+    * @param permission  The permission.  See FsPermission#applyUMask for 
+    *                    details about how this is used to calculate the
+    *                    effective permission.
+    */
+   @Override
+   public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+     return mkdirsInternal(f, permission, true);
+   }
+ 
+   private boolean mkdirsInternal(Path f, final FsPermission permission,
+       final boolean createParent) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.mkdirs(getPathName(p), permission, createParent);
+       }
+ 
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         // FileSystem doesn't have a non-recursive mkdir() method
+         // Best we can do is error out
+         if (!createParent) {
+           throw new IOException("FileSystem does not support non-recursive"
+               + "mkdir");
+         }
+         return fs.mkdirs(p, permission);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @SuppressWarnings("deprecation")
+   @Override
+   protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
+     throws IOException {
+     statistics.incrementWriteOps(1);
+     return dfs.primitiveMkdir(getPathName(f), absolutePermission);
+   }
+ 
+  
+   @Override
+   public void close() throws IOException {
+     try {
+       dfs.closeOutputStreams(false);
+       super.close();
+     } finally {
+       dfs.close();
+     }
+   }
+ 
+   @Override
+   public String toString() {
+     return "DFS[" + dfs + "]";
+   }
+ 
+   @InterfaceAudience.Private
+   @VisibleForTesting
+   public DFSClient getClient() {
+     return dfs;
+   }        
+   
+   @Override
+   public FsStatus getStatus(Path p) throws IOException {
+     statistics.incrementReadOps(1);
+     return dfs.getDiskStatus();
+   }
+ 
+   /**
+    * Returns count of blocks with no good replicas left. Normally should be
+    * zero.
+    * 
+    * @throws IOException
+    */
+   public long getMissingBlocksCount() throws IOException {
+     return dfs.getMissingBlocksCount();
+   }
+ 
+   /**
+    * Returns count of blocks with replication factor 1 and have
+    * lost the only replica.
+    *
+    * @throws IOException
+    */
+   public long getMissingReplOneBlocksCount() throws IOException {
+     return dfs.getMissingReplOneBlocksCount();
+   }
+ 
+   /**
+    * Returns count of blocks with one of more replica missing.
+    * 
+    * @throws IOException
+    */
+   public long getUnderReplicatedBlocksCount() throws IOException {
+     return dfs.getUnderReplicatedBlocksCount();
+   }
+ 
+   /**
+    * Returns count of blocks with at least one replica marked corrupt.
+    * 
+    * @throws IOException
+    */
+   public long getCorruptBlocksCount() throws IOException {
+     return dfs.getCorruptBlocksCount();
+   }
+ 
+   @Override
+   public RemoteIterator<Path> listCorruptFileBlocks(Path path)
+     throws IOException {
+     return new CorruptFileBlockIterator(dfs, path);
+   }
+ 
+   /** @return datanode statistics. */
+   public DatanodeInfo[] getDataNodeStats() throws IOException {
+     return getDataNodeStats(DatanodeReportType.ALL);
+   }
+ 
+   /** @return datanode statistics for the given type. */
+   public DatanodeInfo[] getDataNodeStats(final DatanodeReportType type
+       ) throws IOException {
+     return dfs.datanodeReport(type);
+   }
+ 
+   /**
+    * Enter, leave or get safe mode.
+    *  
+    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(
+    *    HdfsConstants.SafeModeAction,boolean)
+    */
+   public boolean setSafeMode(HdfsConstants.SafeModeAction action) 
+   throws IOException {
+     return setSafeMode(action, false);
+   }
+ 
+   /**
+    * Enter, leave or get safe mode.
+    * 
+    * @param action
+    *          One of SafeModeAction.ENTER, SafeModeAction.LEAVE and
+    *          SafeModeAction.GET
+    * @param isChecked
+    *          If true check only for Active NNs status, else check first NN's
+    *          status
+    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(SafeModeAction, boolean)
+    */
+   public boolean setSafeMode(HdfsConstants.SafeModeAction action,
+       boolean isChecked) throws IOException {
+     return dfs.setSafeMode(action, isChecked);
+   }
+ 
+   /**
+    * Save namespace image.
+    *
+    * @param timeWindow NameNode can ignore this command if the latest
+    *                   checkpoint was done within the given time period (in
+    *                   seconds).
+    * @return true if a new checkpoint has been made
+    * @see ClientProtocol#saveNamespace(long, long)
+    */
+   public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+     return dfs.saveNamespace(timeWindow, txGap);
+   }
+ 
+   /**
+    * Save namespace image. NameNode always does the checkpoint.
+    */
+   public void saveNamespace() throws IOException {
+     saveNamespace(0, 0);
+   }
+ 
+   /**
+    * Rolls the edit log on the active NameNode.
+    * Requires super-user privileges.
+    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#rollEdits()
+    * @return the transaction ID of the newly created segment
+    */
+   public long rollEdits() throws AccessControlException, IOException {
+     return dfs.rollEdits();
+   }
+ 
+   /**
+    * enable/disable/check restoreFaileStorage
+    * 
+    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
+    */
+   public boolean restoreFailedStorage(String arg)
+       throws AccessControlException, IOException {
+     return dfs.restoreFailedStorage(arg);
+   }
+   
+ 
+   /**
+    * Refreshes the list of hosts and excluded hosts from the configured 
+    * files.  
+    */
+   public void refreshNodes() throws IOException {
+     dfs.refreshNodes();
+   }
+ 
+   /**
+    * Finalize previously upgraded files system state.
+    * @throws IOException
+    */
+   public void finalizeUpgrade() throws IOException {
+     dfs.finalizeUpgrade();
+   }
+ 
+   /**
+    * Rolling upgrade: prepare/finalize/query.
+    */
+   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
+       throws IOException {
+     return dfs.rollingUpgrade(action);
+   }
+ 
+   /*
+    * Requests the namenode to dump data strcutures into specified 
+    * file.
+    */
+   public void metaSave(String pathname) throws IOException {
+     dfs.metaSave(pathname);
+   }
+ 
+   @Override
+   public FsServerDefaults getServerDefaults() throws IOException {
+     return dfs.getServerDefaults();
+   }
+ 
+   /**
+    * Returns the stat information about the file.
+    * @throws FileNotFoundException if the file does not exist.
+    */
+   @Override
+   public FileStatus getFileStatus(Path f) throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FileStatus>() {
+       @Override
+       public FileStatus doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         HdfsFileStatus fi = dfs.getFileInfo(getPathName(p));
+         if (fi != null) {
+           return fi.makeQualified(getUri(), p);
+         } else {
+           throw new FileNotFoundException("File does not exist: " + p);
+         }
+       }
+       @Override
+       public FileStatus next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.getFileStatus(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @SuppressWarnings("deprecation")
+   @Override
+   public void createSymlink(final Path target, final Path link,
+       final boolean createParent) throws AccessControlException,
+       FileAlreadyExistsException, FileNotFoundException,
+       ParentNotDirectoryException, UnsupportedFileSystemException, 
+       IOException {
+     if (!FileSystem.areSymlinksEnabled()) {
+       throw new UnsupportedOperationException("Symlinks not supported");
+     }
+     statistics.incrementWriteOps(1);
+     final Path absF = fixRelativePart(link);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         dfs.createSymlink(target.toString(), getPathName(p), createParent);
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException, UnresolvedLinkException {
+         fs.createSymlink(target, p, createParent);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public boolean supportsSymlinks() {
+     return true;
+   }
+ 
+   @Override
+   public FileStatus getFileLinkStatus(final Path f)
+       throws AccessControlException, FileNotFoundException,
+       UnsupportedFileSystemException, IOException {
+     statistics.incrementReadOps(1);
+     final Path absF = fixRelativePart(f);
+     FileStatus status = new FileSystemLinkResolver<FileStatus>() {
+       @Override
+       public FileStatus doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
+         if (fi != null) {
+           return fi.makeQualified(getUri(), p);
+         } else {
+           throw new FileNotFoundException("File does not exist: " + p);
+         }
+       }
+       @Override
+       public FileStatus next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getFileLinkStatus(p);
+       }
+     }.resolve(this, absF);
+     // Fully-qualify the symlink
+     if (status.isSymlink()) {
+       Path targetQual = FSLinkResolver.qualifySymlinkTarget(this.getUri(),
+           status.getPath(), status.getSymlink());
+       status.setSymlink(targetQual);
+     }
+     return status;
+   }
+ 
+   @Override
+   public Path getLinkTarget(final Path f) throws AccessControlException,
+       FileNotFoundException, UnsupportedFileSystemException, IOException {
+     statistics.incrementReadOps(1);
+     final Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<Path>() {
+       @Override
+       public Path doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
+         if (fi != null) {
+           return fi.makeQualified(getUri(), p).getSymlink();
+         } else {
+           throw new FileNotFoundException("File does not exist: " + p);
+         }
+       }
+       @Override
+       public Path next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getLinkTarget(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   protected Path resolveLink(Path f) throws IOException {
+     statistics.incrementReadOps(1);
+     String target = dfs.getLinkTarget(getPathName(fixRelativePart(f)));
+     if (target == null) {
+       throw new FileNotFoundException("File does not exist: " + f.toString());
+     }
+     return new Path(target);
+   }
+ 
+   @Override
+   public FileChecksum getFileChecksum(Path f) throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FileChecksum>() {
+       @Override
+       public FileChecksum doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.getFileChecksum(getPathName(p), Long.MAX_VALUE);
+       }
+ 
+       @Override
+       public FileChecksum next(final FileSystem fs, final Path p)
+           throws IOException {
+         return fs.getFileChecksum(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public FileChecksum getFileChecksum(Path f, final long length)
+       throws IOException {
+     statistics.incrementReadOps(1);
+     Path absF = fixRelativePart(f);
+     return new FileSystemLinkResolver<FileChecksum>() {
+       @Override
+       public FileChecksum doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.getFileChecksum(getPathName(p), length);
+       }
+ 
+       @Override
+       public FileChecksum next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           return ((DistributedFileSystem) fs).getFileChecksum(p, length);
+         } else {
+           throw new UnsupportedFileSystemException(
+               "getFileChecksum(Path, long) is not supported by "
+                   + fs.getClass().getSimpleName()); 
+         }
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void setPermission(Path p, final FsPermission permission
+       ) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(p);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.setPermission(getPathName(p), permission);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         fs.setPermission(p, permission);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void setOwner(Path p, final String username, final String groupname
+       ) throws IOException {
+     if (username == null && groupname == null) {
+       throw new IOException("username == null && groupname == null");
+     }
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(p);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.setOwner(getPathName(p), username, groupname);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         fs.setOwner(p, username, groupname);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void setTimes(Path p, final long mtime, final long atime
+       ) throws IOException {
+     statistics.incrementWriteOps(1);
+     Path absF = fixRelativePart(p);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.setTimes(getPathName(p), mtime, atime);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         fs.setTimes(p, mtime, atime);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+   
+ 
+   @Override
+   protected int getDefaultPort() {
+     return HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT;
+   }
+ 
+   @Override
+   public Token<DelegationTokenIdentifier> getDelegationToken(String renewer)
+       throws IOException {
+     Token<DelegationTokenIdentifier> result =
+       dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
+     return result;
+   }
+ 
+   /**
+    * Requests the namenode to tell all datanodes to use a new, non-persistent
+    * bandwidth value for dfs.balance.bandwidthPerSec.
+    * The bandwidth parameter is the max number of bytes per second of network
+    * bandwidth to be used by a datanode during balancing.
+    *
+    * @param bandwidth Balancer bandwidth in bytes per second for all datanodes.
+    * @throws IOException
+    */
+   public void setBalancerBandwidth(long bandwidth) throws IOException {
+     dfs.setBalancerBandwidth(bandwidth);
+   }
+ 
+   /**
+    * Get a canonical service name for this file system. If the URI is logical,
+    * the hostname part of the URI will be returned.
+    * @return a service string that uniquely identifies this file system.
+    */
+   @Override
+   public String getCanonicalServiceName() {
+     return dfs.getCanonicalServiceName();
+   }
+   
+   @Override
+   protected URI canonicalizeUri(URI uri) {
+     if (HAUtilClient.isLogicalUri(getConf(), uri)) {
+       // Don't try to DNS-resolve logical URIs, since the 'authority'
+       // portion isn't a proper hostname
+       return uri;
+     } else {
+       return NetUtils.getCanonicalUri(uri, getDefaultPort());
+     }
+   }
+ 
+   /**
+    * Utility function that returns if the NameNode is in safemode or not. In HA
+    * mode, this API will return only ActiveNN's safemode status.
+    * 
+    * @return true if NameNode is in safemode, false otherwise.
+    * @throws IOException
+    *           when there is an issue communicating with the NameNode
+    */
+   public boolean isInSafeMode() throws IOException {
+     return setSafeMode(SafeModeAction.SAFEMODE_GET, true);
+   }
+ 
+   /** @see HdfsAdmin#allowSnapshot(Path) */
+   public void allowSnapshot(final Path path) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.allowSnapshot(getPathName(p));
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           myDfs.allowSnapshot(p);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + path + " -> " + p);
+         }
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+   
+   /** @see HdfsAdmin#disallowSnapshot(Path) */
+   public void disallowSnapshot(final Path path) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.disallowSnapshot(getPathName(p));
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           myDfs.disallowSnapshot(p);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + path + " -> " + p);
+         }
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public Path createSnapshot(final Path path, final String snapshotName) 
+       throws IOException {
+     Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<Path>() {
+       @Override
+       public Path doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return new Path(dfs.createSnapshot(getPathName(p), snapshotName));
+       }
+ 
+       @Override
+       public Path next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           return myDfs.createSnapshot(p);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + path + " -> " + p);
+         }
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public void renameSnapshot(final Path path, final String snapshotOldName,
+       final String snapshotNewName) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.renameSnapshot(getPathName(p), snapshotOldName, snapshotNewName);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           myDfs.renameSnapshot(p, snapshotOldName, snapshotNewName);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + path + " -> " + p);
+         }
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+   
+   /**
+    * @return All the snapshottable directories
+    * @throws IOException
+    */
+   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+       throws IOException {
+     return dfs.getSnapshottableDirListing();
+   }
+   
+   @Override
+   public void deleteSnapshot(final Path snapshotDir, final String snapshotName)
+       throws IOException {
+     Path absF = fixRelativePart(snapshotDir);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         dfs.deleteSnapshot(getPathName(p), snapshotName);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           myDfs.deleteSnapshot(p, snapshotName);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + snapshotDir + " -> " + p);
+         }
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * Get the difference between two snapshots, or between a snapshot and the
+    * current tree of a directory.
+    * 
+    * @see DFSClient#getSnapshotDiffReport(String, String, String)
+    */
+   public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
+       final String fromSnapshot, final String toSnapshot) throws IOException {
+     Path absF = fixRelativePart(snapshotDir);
+     return new FileSystemLinkResolver<SnapshotDiffReport>() {
+       @Override
+       public SnapshotDiffReport doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.getSnapshotDiffReport(getPathName(p), fromSnapshot,
+             toSnapshot);
+       }
+ 
+       @Override
+       public SnapshotDiffReport next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           myDfs.getSnapshotDiffReport(p, fromSnapshot, toSnapshot);
+         } else {
+           throw new UnsupportedOperationException("Cannot perform snapshot"
+               + " operations on a symlink to a non-DistributedFileSystem: "
+               + snapshotDir + " -> " + p);
+         }
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+  
+   /**
+    * Get the close status of a file
+    * @param src The path to the file
+    *
+    * @return return true if file is closed
+    * @throws FileNotFoundException if the file does not exist.
+    * @throws IOException If an I/O error occurred     
+    */
+   public boolean isFileClosed(final Path src) throws IOException {
+     Path absF = fixRelativePart(src);
+     return new FileSystemLinkResolver<Boolean>() {
+       @Override
+       public Boolean doCall(final Path p)
+           throws IOException, UnresolvedLinkException {
+         return dfs.isFileClosed(getPathName(p));
+       }
+ 
+       @Override
+       public Boolean next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+           return myDfs.isFileClosed(p);
+         } else {
+           throw new UnsupportedOperationException("Cannot call isFileClosed"
+               + " on a symlink to a non-DistributedFileSystem: "
+               + src + " -> " + p);
+         }
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * @see {@link #addCacheDirective(CacheDirectiveInfo, EnumSet)}
+    */
+   public long addCacheDirective(CacheDirectiveInfo info) throws IOException {
+     return addCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
+   }
+ 
+   /**
+    * Add a new CacheDirective.
+    * 
+    * @param info Information about a directive to add.
+    * @param flags {@link CacheFlag}s to use for this operation.
+    * @return the ID of the directive that was created.
+    * @throws IOException if the directive could not be added
+    */
+   public long addCacheDirective(
+       CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+     Preconditions.checkNotNull(info.getPath());
+     Path path = new Path(getPathName(fixRelativePart(info.getPath()))).
+         makeQualified(getUri(), getWorkingDirectory());
+     return dfs.addCacheDirective(
+         new CacheDirectiveInfo.Builder(info).
+             setPath(path).
+             build(),
+         flags);
+   }
+ 
+   /**
+    * @see {@link #modifyCacheDirective(CacheDirectiveInfo, EnumSet)}
+    */
+   public void modifyCacheDirective(CacheDirectiveInfo info) throws IOException {
+     modifyCacheDirective(info, EnumSet.noneOf(CacheFlag.class));
+   }
+ 
+   /**
+    * Modify a CacheDirective.
+    * 
+    * @param info Information about the directive to modify. You must set the ID
+    *          to indicate which CacheDirective you want to modify.
+    * @param flags {@link CacheFlag}s to use for this operation.
+    * @throws IOException if the directive could not be modified
+    */
+   public void modifyCacheDirective(
+       CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
+     if (info.getPath() != null) {
+       info = new CacheDirectiveInfo.Builder(info).
+           setPath(new Path(getPathName(fixRelativePart(info.getPath()))).
+               makeQualified(getUri(), getWorkingDirectory())).build();
+     }
+     dfs.modifyCacheDirective(info, flags);
+   }
+ 
+   /**
+    * Remove a CacheDirectiveInfo.
+    * 
+    * @param id identifier of the CacheDirectiveInfo to remove
+    * @throws IOException if the directive could not be removed
+    */
+   public void removeCacheDirective(long id)
+       throws IOException {
+     dfs.removeCacheDirective(id);
+   }
+   
+   /**
+    * List cache directives.  Incrementally fetches results from the server.
+    * 
+    * @param filter Filter parameters to use when listing the directives, null to
+    *               list all directives visible to us.
+    * @return A RemoteIterator which returns CacheDirectiveInfo objects.
+    */
+   public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+       CacheDirectiveInfo filter) throws IOException {
+     if (filter == null) {
+       filter = new CacheDirectiveInfo.Builder().build();
+     }
+     if (filter.getPath() != null) {
+       filter = new CacheDirectiveInfo.Builder(filter).
+           setPath(new Path(getPathName(fixRelativePart(filter.getPath())))).
+           build();
+     }
+     final RemoteIterator<CacheDirectiveEntry> iter =
+         dfs.listCacheDirectives(filter);
+     return new RemoteIterator<CacheDirectiveEntry>() {
+       @Override
+       public boolean hasNext() throws IOException {
+         return iter.hasNext();
+       }
+ 
+       @Override
+       public CacheDirectiveEntry next() throws IOException {
+         // Although the paths we get back from the NameNode should always be
+         // absolute, we call makeQualified to add the scheme and authority of
+         // this DistributedFilesystem.
+         CacheDirectiveEntry desc = iter.next();
+         CacheDirectiveInfo info = desc.getInfo();
+         Path p = info.getPath().makeQualified(getUri(), getWorkingDirectory());
+         return new CacheDirectiveEntry(
+             new CacheDirectiveInfo.Builder(info).setPath(p).build(),
+             desc.getStats());
+       }
+     };
+   }
+ 
+   /**
+    * Add a cache pool.
+    *
+    * @param info
+    *          The request to add a cache pool.
+    * @throws IOException 
+    *          If the request could not be completed.
+    */
+   public void addCachePool(CachePoolInfo info) throws IOException {
+     CachePoolInfo.validate(info);
+     dfs.addCachePool(info);
+   }
+ 
+   /**
+    * Modify an existing cache pool.
+    *
+    * @param info
+    *          The request to modify a cache pool.
+    * @throws IOException 
+    *          If the request could not be completed.
+    */
+   public void modifyCachePool(CachePoolInfo info) throws IOException {
+     CachePoolInfo.validate(info);
+     dfs.modifyCachePool(info);
+   }
+     
+   /**
+    * Remove a cache pool.
+    *
+    * @param poolName
+    *          Name of the cache pool to remove.
+    * @throws IOException 
+    *          if the cache pool did not exist, or could not be removed.
+    */
+   public void removeCachePool(String poolName) throws IOException {
+     CachePoolInfo.validateName(poolName);
+     dfs.removeCachePool(poolName);
+   }
+ 
+   /**
+    * List all cache pools.
+    *
+    * @return A remote iterator from which you can get CachePoolEntry objects.
+    *          Requests will be made as needed.
+    * @throws IOException
+    *          If there was an error listing cache pools.
+    */
+   public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
+     return dfs.listCachePools();
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void modifyAclEntries(Path path, final List<AclEntry> aclSpec)
+       throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.modifyAclEntries(getPathName(p), aclSpec);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         fs.modifyAclEntries(p, aclSpec);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void removeAclEntries(Path path, final List<AclEntry> aclSpec)
+       throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.removeAclEntries(getPathName(p), aclSpec);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         fs.removeAclEntries(p, aclSpec);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void removeDefaultAcl(Path path) throws IOException {
+     final Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.removeDefaultAcl(getPathName(p));
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         fs.removeDefaultAcl(p);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void removeAcl(Path path) throws IOException {
+     final Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.removeAcl(getPathName(p));
+         return null;
+       }
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         fs.removeAcl(p);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void setAcl(Path path, final List<AclEntry> aclSpec) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.setAcl(getPathName(p), aclSpec);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         fs.setAcl(p, aclSpec);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public AclStatus getAclStatus(Path path) throws IOException {
+     final Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<AclStatus>() {
+       @Override
+       public AclStatus doCall(final Path p) throws IOException {
+         return dfs.getAclStatus(getPathName(p));
+       }
+       @Override
+       public AclStatus next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getAclStatus(p);
+       }
+     }.resolve(this, absF);
+   }
+   
+   /* HDFS only */
+   public void createEncryptionZone(final Path path, final String keyName)
+     throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         dfs.createEncryptionZone(getPathName(p), keyName);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+           myDfs.createEncryptionZone(p, keyName);
+           return null;
+         } else {
+           throw new UnsupportedOperationException(
+               "Cannot call createEncryptionZone"
+                   + " on a symlink to a non-DistributedFileSystem: " + path
+                   + " -> " + p);
+         }
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /* HDFS only */
+   public EncryptionZone getEZForPath(final Path path)
+           throws IOException {
+     Preconditions.checkNotNull(path);
+     Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<EncryptionZone>() {
+       @Override
+       public EncryptionZone doCall(final Path p) throws IOException,
+           UnresolvedLinkException {
+         return dfs.getEZForPath(getPathName(p));
+       }
+ 
+       @Override
+       public EncryptionZone next(final FileSystem fs, final Path p)
+           throws IOException {
+         if (fs instanceof DistributedFileSystem) {
+           DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+           return myDfs.getEZForPath(p);
+         } else {
+           throw new UnsupportedOperationException(
+               "Cannot call getEZForPath"
+                   + " on a symlink to a non-DistributedFileSystem: " + path
+                   + " -> " + p);
+         }
+       }
+     }.resolve(this, absF);
+   }
+ 
+   /* HDFS only */
+   public RemoteIterator<EncryptionZone> listEncryptionZones()
+       throws IOException {
+     return dfs.listEncryptionZones();
+   }
+ 
+   @Override
+   public void setXAttr(Path path, final String name, final byte[] value, 
+       final EnumSet<XAttrSetFlag> flag) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+ 
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.setXAttr(getPathName(p), name, value, flag);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         fs.setXAttr(p, name, value, flag);
+         return null;
+       }      
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public byte[] getXAttr(Path path, final String name) throws IOException {
+     final Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<byte[]>() {
+       @Override
+       public byte[] doCall(final Path p) throws IOException {
+         return dfs.getXAttr(getPathName(p), name);
+       }
+       @Override
+       public byte[] next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getXAttr(p, name);
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public Map<String, byte[]> getXAttrs(Path path) throws IOException {
+     final Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<Map<String, byte[]>>() {
+       @Override
+       public Map<String, byte[]> doCall(final Path p) throws IOException {
+         return dfs.getXAttrs(getPathName(p));
+       }
+       @Override
+       public Map<String, byte[]> next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getXAttrs(p);
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public Map<String, byte[]> getXAttrs(Path path, final List<String> names) 
+       throws IOException {
+     final Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<Map<String, byte[]>>() {
+       @Override
+       public Map<String, byte[]> doCall(final Path p) throws IOException {
+         return dfs.getXAttrs(getPathName(p), names);
+       }
+       @Override
+       public Map<String, byte[]> next(final FileSystem fs, final Path p)
+         throws IOException, UnresolvedLinkException {
+         return fs.getXAttrs(p, names);
+       }
+     }.resolve(this, absF);
+   }
+   
+   @Override
+   public List<String> listXAttrs(Path path)
+           throws IOException {
+     final Path absF = fixRelativePart(path);
+     return new FileSystemLinkResolver<List<String>>() {
+       @Override
+       public List<String> doCall(final Path p) throws IOException {
+         return dfs.listXAttrs(getPathName(p));
+       }
+       @Override
+       public List<String> next(final FileSystem fs, final Path p)
+               throws IOException, UnresolvedLinkException {
+         return fs.listXAttrs(p);
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void removeXAttr(Path path, final String name) throws IOException {
+     Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.removeXAttr(getPathName(p), name);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p) throws IOException {
+         fs.removeXAttr(p, name);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public void access(Path path, final FsAction mode) throws IOException {
+     final Path absF = fixRelativePart(path);
+     new FileSystemLinkResolver<Void>() {
+       @Override
+       public Void doCall(final Path p) throws IOException {
+         dfs.checkAccess(getPathName(p), mode);
+         return null;
+       }
+ 
+       @Override
+       public Void next(final FileSystem fs, final Path p)
+           throws IOException {
+         fs.access(p, mode);
+         return null;
+       }
+     }.resolve(this, absF);
+   }
+ 
+   @Override
+   public Token<?>[] addDelegationTokens(
+       final String renewer, Credentials credentials) throws IOException {
+     Token<?>[] tokens = super.addDelegationTokens(renewer, credentials);
+     if (dfs.isHDFSEncryptionEnabled()) {
+       KeyProviderDelegationTokenExtension keyProviderDelegationTokenExtension =
+           KeyProviderDelegationTokenExtension.
+               createKeyProviderDelegationTokenExtension(dfs.getKeyProvider());
+       Token<?>[] kpTokens = keyProviderDelegationTokenExtension.
+           addDelegationTokens(renewer, credentials);
+       if (tokens != null && kpTokens != null) {
+         Token<?>[] all = new Token<?>[tokens.length + kpTokens.length];
+         System.arraycopy(tokens, 0, all, 0, tokens.length);
+         System.arraycopy(kpTokens, 0, all, tokens.length, kpTokens.length);
+         tokens = all;
+       } else {
+         tokens = (tokens != null) ? tokens : kpTokens;
+       }
+     }
+     return tokens;
+   }
+ 
+   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
+     return dfs.getInotifyEventStream();
+   }
+ 
+   public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid)
+       throws IOException {
+     return dfs.getInotifyEventStream(lastReadTxid);
+   }
++
++  /**
++   * Set the source path to the specified erasure coding policy.
++   *
++   * @param path     The directory to set the policy
++   * @param ecPolicy The erasure coding policy. If not specified default will be used.
++   * @throws IOException
++   */
++  public void setErasureCodingPolicy(final Path path, final ErasureCodingPolicy ecPolicy)
++      throws IOException {
++    Path absF = fixRelativePart(path);
++    new FileSystemLinkResolver<Void>() {
++      @Override
++      public Void doCall(final Path p) throws IOException,
++          UnresolvedLinkException {
++        dfs.setErasureCodingPolicy(getPathName(p), ecPolicy);
++        return null;
++      }
++
++      @Override
++      public Void next(final FileSystem fs, final Path p) throws IOException {
++        if (fs instanceof DistributedFileSystem) {
++          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
++          myDfs.setErasureCodingPolicy(p, ecPolicy);
++          return null;
++        }
++        throw new UnsupportedOperationException(
++            "Cannot setErasureCodingPolicy through a symlink to a "
++                + "non-DistributedFileSystem: " + path + " -> " + p);
++      }
++    }.resolve(this, absF);
++  }
++
++  /**
++   * Get erasure coding policy information for the specified path
++   *
++   * @param path The path of the file or directory
++   * @return Returns the policy information if file or directory on the path
++   * is erasure coded, null otherwise
++   * @throws IOException
++   */
++  public ErasureCodingPolicy getErasureCodingPolicy(final Path path)
++      throws IOException {
++    Path absF = fixRelativePart(path);
++    return new FileSystemLinkResolver<ErasureCodingPolicy>() {
++      @Override
++      public ErasureCodingPolicy doCall(final Path p) throws IOException,
++          UnresolvedLinkException {
++        return dfs.getErasureCodingPolicy(getPathName(p));
++      }
++
++      @Override
++      public ErasureCodingPolicy next(final FileSystem fs, final Path p)
++          throws IOException {
++        if (fs instanceof DistributedFileSystem) {
++          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
++          return myDfs.getErasureCodingPolicy(p);
++        }
++        throw new UnsupportedOperationException(
++            "Cannot getErasureCodingPolicy through a symlink to a "
++                + "non-DistributedFileSystem: " + path + " -> " + p);
++      }
++    }.resolve(this, absF);
++  }
++
++  /**
++   * Retrieve all the erasure coding policies supported by this file system.
++   *
++   * @return all erasure coding policies supported by this file system.
++   * @throws IOException
++   */
++  public Collection<ErasureCodingPolicy> getAllErasureCodingPolicies()
++      throws IOException {
++    return Arrays.asList(dfs.getErasureCodingPolicies());
++  }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------


[49/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

Change-Id: Ic7946c4ea35bed587fe879ce58b959b25ecc0823


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

Branch: refs/heads/trunk
Commit: 8fd55202468b28422b0df888641c9b08906fe4a7
Parents: 6419900 151fca5
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Tue Sep 29 01:39:16 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue Sep 29 01:39:16 2015 -0700

----------------------------------------------------------------------
 dev-support/test-patch.sh                       |   29 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |   33 +-
 hadoop-common-project/hadoop-common/pom.xml     |   45 +-
 .../hadoop/fs/CommonConfigurationKeys.java      |    3 +
 .../fs/CommonConfigurationKeysPublic.java       |    6 +
 .../org/apache/hadoop/fs/FSOutputSummer.java    |   12 +-
 .../java/org/apache/hadoop/fs/FileContext.java  |    7 +
 .../java/org/apache/hadoop/fs/FileSystem.java   |   27 +-
 .../org/apache/hadoop/fs/FilterFileSystem.java  |    1 -
 .../main/java/org/apache/hadoop/fs/FsShell.java |   26 +-
 .../java/org/apache/hadoop/fs/FsTracer.java     |   64 +
 .../main/java/org/apache/hadoop/fs/Globber.java |   15 +-
 .../org/apache/hadoop/fs/HarFileSystem.java     |    1 -
 .../org/apache/hadoop/fs/LocalDirAllocator.java |    4 +-
 .../apache/hadoop/fs/RawLocalFileSystem.java    |    1 -
 .../hadoop/fs/shell/CommandWithDestination.java |    8 +-
 .../hadoop/fs/viewfs/ChRootedFileSystem.java    |    1 -
 .../main/java/org/apache/hadoop/ipc/Client.java |   13 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |   17 +-
 .../main/java/org/apache/hadoop/ipc/Server.java |   51 +-
 .../apache/hadoop/ipc/WritableRpcEngine.java    |   13 +-
 .../main/java/org/apache/hadoop/net/DNS.java    |  129 +-
 .../apache/hadoop/security/SecurityUtil.java    |   37 +-
 .../apache/hadoop/tracing/SpanReceiverHost.java |  208 --
 .../org/apache/hadoop/tracing/TraceUtils.java   |   24 +-
 .../tracing/TracerConfigurationManager.java     |  100 +
 .../java/org/apache/hadoop/util/ProtoUtil.java  |   13 +-
 .../src/main/proto/RpcHeader.proto              |    5 +-
 .../src/main/resources/core-default.xml         |   40 +
 .../hadoop-common/src/site/markdown/Metrics.md  |    1 -
 .../hadoop-common/src/site/markdown/Tracing.md  |   80 +-
 .../apache/hadoop/conf/TestConfiguration.java   |    2 +-
 .../java/org/apache/hadoop/fs/TestFsShell.java  |   44 +-
 .../org/apache/hadoop/fs/TestHarFileSystem.java |    1 -
 .../apache/hadoop/fs/TestLocalDirAllocator.java |   26 +-
 .../apache/hadoop/fs/TestLocalFileSystem.java   |    1 +
 .../AbstractContractRootDirectoryTest.java      |   14 +
 .../apache/hadoop/ha/ClientBaseWithFixes.java   |   40 +-
 .../http/TestAuthenticationSessionCookie.java   |    5 +-
 .../apache/hadoop/http/TestHttpCookieFlag.java  |    7 +-
 .../apache/hadoop/http/TestSSLHttpServer.java   |    4 +-
 .../java/org/apache/hadoop/ipc/TestRPC.java     |    4 +-
 .../org/apache/hadoop/net/ServerSocketUtil.java |    7 +-
 .../java/org/apache/hadoop/net/TestDNS.java     |  110 +-
 .../hadoop/security/TestSecurityUtil.java       |    2 +-
 .../hadoop/security/ssl/KeyStoreTestUtil.java   |   60 +-
 .../apache/hadoop/tracing/SetSpanReceiver.java  |   13 +-
 .../apache/hadoop/tracing/TestTraceUtils.java   |    2 +-
 .../dev-support/findbugsExcludeFile.xml         |   24 +
 .../org/apache/hadoop/fs/HdfsBlockLocation.java |   47 +
 .../hadoop/hdfs/BlockMissingException.java      |   65 +
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  906 +++++
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |   21 +-
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |   30 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 3240 +++++++++++++++++
 .../hadoop/hdfs/DFSClientFaultInjector.java     |   60 +
 .../hadoop/hdfs/DFSHedgedReadMetrics.java       |   58 +
 .../hadoop/hdfs/DFSInotifyEventInputStream.java |  234 ++
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 1889 ++++++++++
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  982 ++++++
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  364 ++
 .../hadoop/hdfs/DFSStripedInputStream.java      |  972 ++++++
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  953 +++++
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |   32 +
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 1944 +++++++++++
 .../hadoop/hdfs/DistributedFileSystem.java      | 2335 +++++++++++++
 .../apache/hadoop/hdfs/ExternalBlockReader.java |   20 +-
 .../hadoop/hdfs/HdfsConfigurationLoader.java    |   44 +
 .../hadoop/hdfs/NameNodeProxiesClient.java      |  366 ++
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |   20 +-
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |   23 +-
 .../apache/hadoop/hdfs/RemotePeerFactory.java   |   43 +
 .../org/apache/hadoop/hdfs/ReplicaAccessor.java |   10 +-
 .../hadoop/hdfs/ReplicaAccessorBuilder.java     |    3 +
 .../apache/hadoop/hdfs/StripedDataStreamer.java |  187 +
 .../hdfs/UnknownCipherSuiteException.java       |   35 +
 .../UnknownCryptoProtocolVersionException.java  |   38 +
 .../org/apache/hadoop/hdfs/XAttrHelper.java     |  174 +
 .../hdfs/client/HdfsClientConfigKeys.java       |   11 +
 .../hadoop/hdfs/client/HdfsDataInputStream.java |  113 +
 .../hdfs/client/HdfsDataOutputStream.java       |  112 +
 .../client/impl/CorruptFileBlockIterator.java   |  105 +
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  524 +++
 .../hdfs/inotify/MissingEventsException.java    |   54 +
 .../hadoop/hdfs/protocol/AclException.java      |   39 +
 .../hdfs/protocol/CacheDirectiveIterator.java   |  129 +
 .../hadoop/hdfs/protocol/CachePoolIterator.java |   62 +
 .../hdfs/protocol/EncryptionZoneIterator.java   |   62 +
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   23 +
 .../QuotaByStorageTypeExceededException.java    |   56 +
 .../hdfs/protocol/UnresolvedPathException.java  |   87 +
 .../datatransfer/DataTransferProtoUtil.java     |   43 +-
 .../datatransfer/ReplaceDatanodeOnFailure.java  |  200 ++
 .../hdfs/protocol/datatransfer/Sender.java      |   22 +-
 .../protocolPB/ClientNamenodeProtocolPB.java    |   46 +
 .../ClientNamenodeProtocolTranslatorPB.java     | 1593 +++++++++
 .../datanode/ReplicaNotFoundException.java      |   53 +
 .../namenode/RetryStartFileException.java       |   36 +
 .../ha/AbstractNNFailoverProxyProvider.java     |   55 +
 .../ha/WrappedFailoverProxyProvider.java        |   71 +
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  952 +++++
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |    1 -
 .../src/main/proto/hdfs.proto                   |  178 +-
 .../hadoop/hdfs/nfs/nfs3/OffsetRange.java       |    4 +
 .../hadoop/hdfs/nfs/nfs3/OpenFileCtx.java       |  141 +-
 .../apache/hadoop/hdfs/nfs/nfs3/WriteCtx.java   |   82 +-
 .../apache/hadoop/hdfs/nfs/nfs3/TestWrites.java |   92 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   80 +-
 .../dev-support/findbugsExcludeFile.xml         |   19 -
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |    3 +-
 .../hadoop-hdfs/src/contrib/bkjournal/pom.xml   |    1 +
 .../bkjournal/src/main/proto/bkjournal.proto    |    1 +
 .../org/apache/hadoop/fs/HdfsBlockLocation.java |   47 -
 .../hadoop/hdfs/BlockMissingException.java      |   65 -
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |  891 -----
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 3244 ------------------
 .../hadoop/hdfs/DFSClientFaultInjector.java     |   57 -
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   39 +-
 .../hadoop/hdfs/DFSHedgedReadMetrics.java       |   58 -
 .../hadoop/hdfs/DFSInotifyEventInputStream.java |  239 --
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 1887 ----------
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  976 ------
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  359 --
 .../hadoop/hdfs/DFSStripedInputStream.java      |  973 ------
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  954 -----
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   31 -
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 1900 ----------
 .../hadoop/hdfs/DistributedFileSystem.java      | 2338 -------------
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |    9 +-
 .../apache/hadoop/hdfs/HdfsConfiguration.java   |   11 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |  275 +-
 .../apache/hadoop/hdfs/RemotePeerFactory.java   |   43 -
 .../apache/hadoop/hdfs/StripedDataStreamer.java |  187 -
 .../hdfs/UnknownCipherSuiteException.java       |   35 -
 .../UnknownCryptoProtocolVersionException.java  |   38 -
 .../org/apache/hadoop/hdfs/XAttrHelper.java     |  174 -
 .../hadoop/hdfs/client/HdfsDataInputStream.java |  113 -
 .../hdfs/client/HdfsDataOutputStream.java       |  112 -
 .../client/impl/CorruptFileBlockIterator.java   |  105 -
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  524 ---
 .../hdfs/inotify/MissingEventsException.java    |   54 -
 .../hadoop/hdfs/protocol/AclException.java      |   39 -
 .../hdfs/protocol/CacheDirectiveIterator.java   |  130 -
 .../hadoop/hdfs/protocol/CachePoolIterator.java |   63 -
 .../hdfs/protocol/EncryptionZoneIterator.java   |   64 -
 .../QuotaByStorageTypeExceededException.java    |   56 -
 .../hdfs/protocol/UnresolvedPathException.java  |   87 -
 .../hdfs/protocol/datatransfer/Receiver.java    |   35 +-
 .../datatransfer/ReplaceDatanodeOnFailure.java  |  200 --
 .../protocolPB/ClientNamenodeProtocolPB.java    |   46 -
 .../ClientNamenodeProtocolTranslatorPB.java     | 1593 ---------
 .../DatanodeProtocolClientSideTranslatorPB.java |    2 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |    4 +-
 .../NamenodeProtocolServerSideTranslatorPB.java |    4 +-
 .../NamenodeProtocolTranslatorPB.java           |    5 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |   33 +-
 .../hdfs/qjournal/server/JournalNode.java       |   13 +-
 .../qjournal/server/JournalNodeRpcServer.java   |    1 +
 .../hadoop/hdfs/server/balancer/Dispatcher.java |    3 +-
 .../server/blockmanagement/DatanodeManager.java |   45 +-
 .../blockmanagement/HeartbeatManager.java       |   30 +-
 .../hdfs/server/common/HdfsServerConstants.java |   27 +-
 .../hdfs/server/datanode/BlockReceiver.java     |    4 +-
 .../hdfs/server/datanode/BlockSender.java       |   13 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   73 +-
 .../hdfs/server/datanode/DataXceiver.java       |    8 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |  337 +-
 .../datanode/ReplicaNotFoundException.java      |   53 -
 .../erasurecode/ErasureCodingWorker.java        |    6 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |    4 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |    8 +-
 .../impl/RamDiskAsyncLazyPersistService.java    |    4 +-
 .../hadoop/hdfs/server/namenode/BackupNode.java |    3 +-
 .../hdfs/server/namenode/FSDirConcatOp.java     |    2 +-
 .../server/namenode/FSDirEncryptionZoneOp.java  |  303 ++
 .../server/namenode/FSDirStatAndListingOp.java  |   27 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |    6 +-
 .../hdfs/server/namenode/FSDirectory.java       |  147 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  141 +-
 .../hdfs/server/namenode/LeaseManager.java      |    5 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |   25 +-
 .../hdfs/server/namenode/NameNodeMXBean.java    |    7 -
 .../hdfs/server/namenode/NameNodeRpcServer.java |   10 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |    5 +
 .../namenode/RetryStartFileException.java       |   36 -
 .../hdfs/server/namenode/TransferFsImage.java   |    4 +-
 .../ha/AbstractNNFailoverProxyProvider.java     |   51 -
 .../ha/ConfiguredFailoverProxyProvider.java     |    2 +-
 .../ha/WrappedFailoverProxyProvider.java        |   80 -
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |    2 +-
 .../hadoop/hdfs/tools/NNHAServiceTarget.java    |   16 +-
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  946 -----
 .../main/native/fuse-dfs/test/TestFuseDFS.java  |    4 +-
 .../src/main/proto/DatanodeProtocol.proto       |    1 +
 .../hadoop-hdfs/src/main/proto/HdfsServer.proto |  198 ++
 .../src/main/proto/InterDatanodeProtocol.proto  |    1 +
 .../src/main/proto/JournalProtocol.proto        |    1 +
 .../src/main/proto/NamenodeProtocol.proto       |    1 +
 .../src/main/proto/QJournalProtocol.proto       |    1 +
 .../src/main/resources/hdfs-default.xml         |   58 +-
 .../hadoop-hdfs/src/site/markdown/Federation.md |    2 +-
 .../org/apache/hadoop/hdfs/TestFiPipelines.java |   17 +-
 .../datanode/TestFiDataTransferProtocol.java    |    4 +-
 .../datanode/TestFiDataTransferProtocol2.java   |    8 +-
 .../apache/hadoop/fs/TestFcHdfsSetUMask.java    |    4 +-
 .../TestSWebHdfsFileContextMainOperations.java  |    5 +
 .../TestWebHdfsFileContextMainOperations.java   |    5 +
 .../apache/hadoop/hdfs/BenchmarkThroughput.java |    3 +-
 .../apache/hadoop/hdfs/BlockReaderTestUtil.java |    2 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |    2 +-
 .../hdfs/MiniDFSClusterWithNodeGroup.java       |    2 +-
 .../hadoop/hdfs/TestBlockReaderLocal.java       |    2 +
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |    1 -
 .../TestClientProtocolForPipelineRecovery.java  |    6 +-
 .../apache/hadoop/hdfs/TestCrcCorruption.java   |    2 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |    3 +-
 .../apache/hadoop/hdfs/TestDFSOutputStream.java |    5 +-
 .../org/apache/hadoop/hdfs/TestDFSPacket.java   |   25 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |   13 +
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     |    8 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |   43 +
 .../hadoop/hdfs/TestDistributedFileSystem.java  |    3 +-
 .../hadoop/hdfs/TestExternalBlockReader.java    |   43 +-
 .../org/apache/hadoop/hdfs/TestFileAppend4.java |    4 +-
 .../hadoop/hdfs/TestFileConcurrentReader.java   |    9 +-
 .../apache/hadoop/hdfs/TestFileCreation.java    |   12 +-
 .../hadoop/hdfs/TestFileCreationClient.java     |   11 +-
 .../org/apache/hadoop/hdfs/TestFileStatus.java  |    7 +-
 .../java/org/apache/hadoop/hdfs/TestHFlush.java |    6 +-
 .../org/apache/hadoop/hdfs/TestLargeBlock.java  |   10 +-
 .../java/org/apache/hadoop/hdfs/TestLease.java  |    8 +-
 .../apache/hadoop/hdfs/TestLeaseRecovery2.java  |   13 +-
 .../apache/hadoop/hdfs/TestListFilesInDFS.java  |    4 +-
 .../hadoop/hdfs/TestListFilesInFileContext.java |    4 +-
 .../org/apache/hadoop/hdfs/TestLocalDFS.java    |    7 +-
 .../apache/hadoop/hdfs/TestPersistBlocks.java   |    5 +-
 .../java/org/apache/hadoop/hdfs/TestPread.java  |   10 +-
 .../hadoop/hdfs/TestReadWhileWriting.java       |    7 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |   24 +-
 .../hdfs/qjournal/client/TestQJMWithFaults.java |    3 +-
 .../client/TestQuorumJournalManager.java        |    3 +-
 .../client/TestQuorumJournalManagerUnit.java    |    3 +-
 .../TestClientProtocolWithDelegationToken.java  |   12 +-
 .../hdfs/security/TestDelegationToken.java      |    4 +-
 .../security/token/block/TestBlockToken.java    |   12 +-
 .../hdfs/server/balancer/TestBalancer.java      |    3 +-
 .../TestBalancerWithMultipleNameNodes.java      |    4 +-
 .../blockmanagement/TestBlockTokenWithDFS.java  |    4 +-
 .../TestBlocksWithNotEnoughRacks.java           |    6 +-
 .../blockmanagement/TestHeartbeatHandling.java  |   27 +
 .../TestPendingInvalidateBlock.java             |    9 +-
 .../blockmanagement/TestReplicationPolicy.java  |    4 +-
 .../server/datanode/TestBPOfferService.java     |    3 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |    5 +-
 .../TestDataNodeVolumeFailureReporting.java     |    5 +-
 .../TestDatanodeProtocolRetryPolicy.java        |    3 +-
 .../server/datanode/TestDirectoryScanner.java   |  234 +-
 .../hdfs/server/datanode/TestTransferRbw.java   |    4 +-
 .../fsdataset/impl/TestSpaceReservation.java    |    5 +-
 .../fsdataset/impl/TestWriteToReplica.java      |    4 +-
 .../hdfs/server/mover/TestStorageMover.java     |   13 +-
 .../server/namenode/TestAuditLogAtDebug.java    |    4 +-
 .../hdfs/server/namenode/TestBackupNode.java    |    5 +-
 .../server/namenode/TestCacheDirectives.java    |    4 +-
 .../hdfs/server/namenode/TestCheckpoint.java    |    3 +-
 .../hdfs/server/namenode/TestEditLog.java       |    3 +-
 .../hdfs/server/namenode/TestEditLogRace.java   |    4 +-
 .../server/namenode/TestFSEditLogLoader.java    |    6 +-
 .../namenode/TestFSImageWithSnapshot.java       |    4 +-
 .../namenode/TestFavoredNodesEndToEnd.java      |    5 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |    5 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |    8 +-
 .../server/namenode/TestNameNodeMXBean.java     |    3 -
 .../namenode/TestNameNodeRetryCacheMetrics.java |    4 +-
 .../hdfs/server/namenode/TestSaveNamespace.java |    3 +-
 .../ha/TestDNFencingWithReplication.java        |   11 +-
 .../server/namenode/ha/TestEditLogTailer.java   |    5 +-
 .../hdfs/server/namenode/ha/TestHAFsck.java     |    5 +-
 .../namenode/ha/TestHAStateTransitions.java     |    3 +-
 .../ha/TestLossyRetryInvocationHandler.java     |    8 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |    4 +-
 .../namenode/metrics/TestNameNodeMetrics.java   |    6 +-
 ...tINodeFileUnderConstructionWithSnapshot.java |   12 +-
 .../server/namenode/snapshot/TestSnapshot.java  |    3 +-
 .../hdfs/tools/TestDFSHAAdminMiniCluster.java   |    4 +-
 .../hadoop/hdfs/util/TestByteArrayManager.java  |   11 +-
 .../hdfs/web/TestFSMainOperationsWebHdfs.java   |    4 +-
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |    4 +-
 .../web/TestWebHdfsWithMultipleNameNodes.java   |    3 +-
 .../hadoop/tools/TestHdfsConfigFields.java      |    6 +-
 .../apache/hadoop/tracing/TestTraceAdmin.java   |    9 +-
 .../org/apache/hadoop/tracing/TestTracing.java  |   86 +-
 .../TestTracingShortCircuitLocalRead.java       |   18 +-
 hadoop-mapreduce-project/CHANGES.txt            |   17 +-
 .../v2/app/job/impl/TaskAttemptImpl.java        |    4 +-
 .../v2/app/job/impl/TestTaskAttempt.java        |    8 +
 .../v2/app/launcher/TestContainerLauncher.java  |   11 +
 .../app/launcher/TestContainerLauncherImpl.java |    9 +
 .../app/local/TestLocalContainerAllocator.java  |    6 +-
 .../v2/app/rm/TestRMContainerAllocator.java     |   19 +-
 .../java/org/apache/hadoop/mapred/Master.java   |   27 +-
 .../org/apache/hadoop/mapred/TestMaster.java    |   13 +
 hadoop-project/pom.xml                          |    6 +-
 .../dev-support/findbugs-exclude.xml            |   32 +
 hadoop-tools/hadoop-archive-logs/pom.xml        |   18 +
 .../apache/hadoop/tools/HadoopArchiveLogs.java  |  243 +-
 .../hadoop/tools/TestHadoopArchiveLogs.java     |  231 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |    3 +
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |    3 -
 .../fs/azure/NativeAzureFileSystemBaseTest.java |    1 -
 .../src/site/markdown/DistCp.md.vm              |    5 +-
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |   14 +
 .../yarn/sls/scheduler/RMNodeWrapper.java       |   13 +
 .../sls/scheduler/ResourceSchedulerWrapper.java |   21 +-
 .../sls/scheduler/SLSCapacityScheduler.java     |   19 +-
 .../org/apache/hadoop/streaming/UtilTest.java   |    5 -
 hadoop-yarn-project/CHANGES.txt                 |   80 +-
 .../yarn/api/ContainerManagementProtocol.java   |   30 +-
 .../api/protocolrecords/AllocateRequest.java    |   57 +-
 .../api/protocolrecords/AllocateResponse.java   |   42 +-
 .../IncreaseContainersResourceRequest.java      |   75 +
 .../IncreaseContainersResourceResponse.java     |   93 +
 .../records/ContainerResourceChangeRequest.java |  117 +
 .../api/records/ContainerResourceDecrease.java  |   78 -
 .../api/records/ContainerResourceIncrease.java  |   84 -
 .../ContainerResourceIncreaseRequest.java       |   80 -
 .../yarn/api/records/ContainerStatus.java       |   13 +
 .../proto/containermanagement_protocol.proto    |    1 +
 .../src/main/proto/yarn_protos.proto            |   14 +-
 .../src/main/proto/yarn_service_protos.proto    |   16 +-
 .../yarn/client/api/impl/YarnClientImpl.java    |    7 +-
 .../yarn/client/TestResourceTrackerOnHA.java    |    2 +-
 .../api/impl/TestAMRMClientOnRMRestart.java     |    8 +-
 ...ContainerManagementProtocolPBClientImpl.java |   20 +
 ...ontainerManagementProtocolPBServiceImpl.java |   22 +
 .../impl/pb/AllocateRequestPBImpl.java          |  119 +-
 .../impl/pb/AllocateResponsePBImpl.java         |  175 +-
 ...IncreaseContainersResourceRequestPBImpl.java |  170 +
 ...ncreaseContainersResourceResponsePBImpl.java |  241 ++
 .../ContainerResourceChangeRequestPBImpl.java   |  141 +
 .../pb/ContainerResourceDecreasePBImpl.java     |  136 -
 .../pb/ContainerResourceIncreasePBImpl.java     |  171 -
 .../ContainerResourceIncreaseRequestPBImpl.java |  141 -
 .../records/impl/pb/ContainerStatusPBImpl.java  |   31 +-
 .../resource/DefaultResourceCalculator.java     |    5 +
 .../resource/DominantResourceCalculator.java    |    6 +
 .../yarn/util/resource/ResourceCalculator.java  |    5 +
 .../hadoop/yarn/util/resource/Resources.java    |    5 +
 .../hadoop/yarn/TestContainerLaunchRPC.java     |    8 +
 .../yarn/TestContainerResourceIncreaseRPC.java  |  162 +
 .../java/org/apache/hadoop/yarn/TestRPC.java    |    8 +
 .../hadoop/yarn/api/TestAllocateRequest.java    |   73 -
 .../hadoop/yarn/api/TestAllocateResponse.java   |  114 -
 .../yarn/api/TestContainerResourceDecrease.java |   66 -
 .../yarn/api/TestContainerResourceIncrease.java |   74 -
 .../TestContainerResourceIncreaseRequest.java   |   68 -
 .../hadoop/yarn/api/TestPBImplRecords.java      |   54 +-
 .../util/resource/TestResourceCalculator.java   |   30 +-
 ...pplicationHistoryManagerOnTimelineStore.java |   12 +
 .../ApplicationHistoryServer.java               |   13 +-
 ...pplicationHistoryManagerOnTimelineStore.java |   43 +-
 .../TestApplicationHistoryServer.java           |    4 +-
 .../protocolrecords/NodeHeartbeatResponse.java  |    5 +
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |   77 +-
 .../yarn/server/api/records/NodeStatus.java     |   15 +-
 .../api/records/impl/pb/NodeStatusPBImpl.java   |   75 +-
 .../metrics/ApplicationMetricsConstants.java    |    3 +
 .../hadoop/yarn/server/utils/BuilderUtils.java  |    4 +-
 .../main/proto/yarn_server_common_protos.proto  |    3 +-
 .../yarn_server_common_service_protos.proto     |    1 +
 .../hadoop/yarn/TestYarnServerApiClasses.java   |   39 +-
 .../CMgrDecreaseContainersResourceEvent.java    |   37 +
 .../nodemanager/ContainerManagerEventType.java  |    1 +
 .../hadoop/yarn/server/nodemanager/Context.java |    3 +
 .../nodemanager/LocalDirsHandlerService.java    |   33 +-
 .../yarn/server/nodemanager/NodeManager.java    |   10 +
 .../nodemanager/NodeStatusUpdaterImpl.java      |   57 +-
 .../containermanager/ContainerManagerImpl.java  |  302 +-
 .../containermanager/container/Container.java   |    2 +
 .../container/ContainerImpl.java                |   24 +-
 .../logaggregation/LogAggregationService.java   |   11 +-
 .../ChangeMonitoringContainerResourceEvent.java |   37 +
 .../monitor/ContainersMonitorEventType.java     |    3 +-
 .../monitor/ContainersMonitorImpl.java          |  207 +-
 .../nodemanager/metrics/NodeManagerMetrics.java |   11 +
 .../recovery/NMLeveldbStateStoreService.java    |   22 +
 .../recovery/NMNullStateStoreService.java       |    6 +
 .../recovery/NMStateStoreService.java           |   15 +
 .../nodemanager/DummyContainerManager.java      |    6 +-
 .../TestContainerManagerWithLCE.java            |   33 +
 .../TestLocalDirsHandlerService.java            |   18 +
 .../nodemanager/TestNodeManagerResync.java      |  258 ++
 .../nodemanager/TestNodeStatusUpdater.java      |    2 +-
 .../amrmproxy/BaseAMRMProxyTest.java            |    5 +
 .../amrmproxy/MockResourceManagerFacade.java    |    6 +-
 .../BaseContainerManagerTest.java               |   43 +-
 .../containermanager/TestContainerManager.java  |  288 +-
 .../TestContainerManagerRecovery.java           |  233 +-
 .../TestLogAggregationService.java              |   19 +
 .../monitor/MockResourceCalculatorPlugin.java   |   69 +
 .../MockResourceCalculatorProcessTree.java      |   57 +
 .../TestContainersMonitorResourceChange.java    |  248 ++
 .../metrics/TestNodeManagerMetrics.java         |   18 +-
 .../recovery/NMMemoryStateStoreService.java     |   11 +-
 .../TestNMLeveldbStateStoreService.java         |   11 +
 .../nodemanager/webapp/MockContainer.java       |    4 +
 .../ApplicationMasterService.java               |   22 +-
 .../server/resourcemanager/ClientRMService.java |   30 +-
 .../server/resourcemanager/RMAuditLogger.java   |    2 +
 .../server/resourcemanager/RMServerUtils.java   |  164 +
 .../resourcemanager/ResourceTrackerService.java |    7 +-
 .../resourcemanager/amlauncher/AMLauncher.java  |   27 +-
 .../metrics/ApplicationUpdatedEvent.java        |   54 +
 .../metrics/SystemMetricsEventType.java         |    1 +
 .../metrics/SystemMetricsPublisher.java         |   29 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |    5 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |    4 +-
 .../rmcontainer/RMContainer.java                |    4 +
 .../RMContainerChangeResourceEvent.java         |   44 +
 .../rmcontainer/RMContainerEventType.java       |   13 +-
 .../rmcontainer/RMContainerImpl.java            |  121 +-
 .../RMContainerUpdatesAcquiredEvent.java        |   35 +
 .../server/resourcemanager/rmnode/RMNode.java   |    9 +
 .../rmnode/RMNodeDecreaseContainerEvent.java    |   39 +
 .../resourcemanager/rmnode/RMNodeEventType.java |    1 +
 .../resourcemanager/rmnode/RMNodeImpl.java      |   93 +
 .../rmnode/RMNodeStatusEvent.java               |   33 +-
 .../scheduler/AbstractYarnScheduler.java        |  150 +-
 .../resourcemanager/scheduler/Allocation.java   |   22 +-
 .../scheduler/AppSchedulingInfo.java            |  249 +-
 .../resourcemanager/scheduler/QueueMetrics.java |   16 +-
 .../scheduler/SchedContainerChangeRequest.java  |  118 +
 .../scheduler/SchedulerApplication.java         |    2 +-
 .../scheduler/SchedulerApplicationAttempt.java  |  250 +-
 .../scheduler/SchedulerNode.java                |   31 +
 .../scheduler/SchedulerUtils.java               |   11 +-
 .../scheduler/YarnScheduler.java                |   14 +-
 .../scheduler/capacity/AbstractCSQueue.java     |   23 +-
 .../scheduler/capacity/CSAssignment.java        |    9 +
 .../scheduler/capacity/CSQueue.java             |   16 +
 .../scheduler/capacity/CapacityScheduler.java   |   87 +-
 .../scheduler/capacity/LeafQueue.java           |  127 +-
 .../scheduler/capacity/ParentQueue.java         |  115 +-
 .../allocator/AbstractContainerAllocator.java   |  131 +
 .../capacity/allocator/ContainerAllocator.java  |  149 +-
 .../allocator/IncreaseContainerAllocator.java   |  365 ++
 .../allocator/RegularContainerAllocator.java    |   30 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |   68 +-
 .../scheduler/fair/FSLeafQueue.java             |    5 +-
 .../scheduler/fair/FSParentQueue.java           |    3 +-
 .../scheduler/fair/FairScheduler.java           |   35 +-
 .../scheduler/fair/QueueManager.java            |   24 +-
 .../scheduler/fifo/FifoScheduler.java           |   25 +-
 .../server/resourcemanager/Application.java     |    2 +-
 .../yarn/server/resourcemanager/MockAM.java     |    9 +
 .../yarn/server/resourcemanager/MockNM.java     |    2 +-
 .../yarn/server/resourcemanager/MockNodes.java  |   13 +
 .../yarn/server/resourcemanager/MockRM.java     |   25 +-
 .../server/resourcemanager/NodeManager.java     |   13 +-
 .../resourcemanager/TestAMAuthorization.java    |    8 +
 .../resourcemanager/TestApplicationCleanup.java |    6 +-
 .../TestApplicationMasterLauncher.java          |   75 +
 .../TestApplicationMasterService.java           |  144 +-
 .../resourcemanager/TestClientRMService.java    |   36 +-
 .../applicationsmanager/TestAMRestart.java      |   15 +-
 .../TestRMAppLogAggregationStatus.java          |   10 +-
 .../metrics/TestSystemMetricsPublisher.java     |   50 +-
 .../attempt/TestRMAppAttemptTransitions.java    |   53 +-
 .../rmcontainer/TestRMContainerImpl.java        |  117 +-
 .../capacity/TestCapacityScheduler.java         |  130 +-
 .../scheduler/capacity/TestChildQueueOrder.java |    4 +-
 .../capacity/TestContainerAllocation.java       |   50 +-
 .../capacity/TestContainerResizing.java         |  963 ++++++
 .../scheduler/capacity/TestLeafQueue.java       |    4 +-
 .../scheduler/capacity/TestParentQueue.java     |    4 +-
 .../scheduler/capacity/TestReservations.java    |    9 +-
 .../scheduler/fair/FairSchedulerTestBase.java   |    6 +-
 .../fair/TestContinuousScheduling.java          |    2 +-
 .../scheduler/fair/TestFSParentQueue.java       |   79 +
 .../scheduler/fair/TestFairScheduler.java       |   30 +-
 .../scheduler/fifo/TestFifoScheduler.java       |   32 +-
 .../security/TestAMRMTokens.java                |    3 +-
 .../yarn/server/webproxy/AppReportFetcher.java  |   49 +-
 .../server/webproxy/WebAppProxyServlet.java     |   46 +-
 .../server/webproxy/TestWebAppProxyServlet.java |  113 +-
 485 files changed, 31104 insertions(+), 23102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
----------------------------------------------------------------------
diff --cc hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
index a8a7494,648043e..616ff3a
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
@@@ -196,12 -198,8 +198,12 @@@ abstract public class FSOutputSummer ex
      return sum.getChecksumSize();
    }
  
 +  protected DataChecksum getDataChecksum() {
 +    return sum;
 +  }
 +
    protected TraceScope createWriteTraceScope() {
-     return NullScope.INSTANCE;
+     return null;
    }
  
    /** Generate checksums for the given data chunks and output chunks & checksums

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------


[13/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
Merge remote-tracking branch 'apache/trunk' into HDFS-7285


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

Branch: refs/heads/trunk
Commit: ab56fcdb1219d03713b408dd3a95d7405635254d
Parents: 164cbe6 cbb2495
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Thu Aug 27 16:23:41 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue Sep 1 14:30:25 2015 -0700

----------------------------------------------------------------------
 .../server/AuthenticationFilter.java            |   63 +-
 .../server/AuthenticationToken.java             |   12 +
 .../security/authentication/util/AuthToken.java |   35 +-
 .../server/TestAuthenticationFilter.java        |  163 ++-
 hadoop-common-project/hadoop-common/CHANGES.txt |   34 +
 .../src/main/conf/log4j.properties              |   13 +
 .../fs/CommonConfigurationKeysPublic.java       |    5 +
 .../java/org/apache/hadoop/fs/CreateFlag.java   |    2 +-
 .../apache/hadoop/fs/TrashPolicyDefault.java    |   11 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |    5 +-
 .../main/java/org/apache/hadoop/ipc/Server.java |   60 +
 .../apache/hadoop/ipc/WritableRpcEngine.java    |    3 +
 .../apache/hadoop/ipc/metrics/RpcMetrics.java   |   48 +
 .../apache/hadoop/metrics2/lib/MutableStat.java |    7 +-
 .../org/apache/hadoop/metrics2/util/MBeans.java |   37 +-
 .../org/apache/hadoop/util/HostsFileReader.java |    7 +-
 .../main/java/org/apache/hadoop/util/Shell.java |   11 +-
 .../org/apache/hadoop/util/StringUtils.java     |   29 +-
 .../src/main/resources/core-default.xml         |    9 +
 .../src/site/markdown/HttpAuthentication.md     |    8 +-
 .../hadoop-common/src/site/markdown/Metrics.md  |    2 +
 .../src/site/markdown/SingleCluster.md.vm       |    2 +-
 .../org/apache/hadoop/ipc/TestProtoBufRpc.java  |   77 +-
 .../org/apache/hadoop/test/MetricsAsserts.java  |    2 +-
 .../java/org/apache/hadoop/util/TestShell.java  |   39 +
 .../hadoop-common/src/test/proto/test.proto     |    7 +
 .../src/test/proto/test_rpc_service.proto       |    1 +
 .../dev-support/findbugsExcludeFile.xml         |   10 +
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |   26 +
 .../org/apache/hadoop/hdfs/ExtendedBlockId.java |   82 ++
 .../org/apache/hadoop/hdfs/ReplicaAccessor.java |   88 ++
 .../hadoop/hdfs/ReplicaAccessorBuilder.java     |  101 ++
 .../hdfs/client/HdfsClientConfigKeys.java       |   76 +-
 .../hadoop/hdfs/client/impl/DfsClientConf.java  |  794 +++++++++++++
 .../hadoop/hdfs/client/impl/package-info.java   |   18 +
 .../org/apache/hadoop/hdfs/net/DomainPeer.java  |  132 +++
 .../java/org/apache/hadoop/hdfs/net/Peer.java   |  123 ++
 .../hadoop/hdfs/protocol/HdfsConstants.java     |    7 +
 .../datatransfer/BlockConstructionStage.java    |   62 +
 .../datatransfer/DataTransferProtoUtil.java     |  146 +++
 .../datatransfer/DataTransferProtocol.java      |  202 ++++
 .../hadoop/hdfs/protocol/datatransfer/Op.java   |   66 ++
 .../hdfs/protocol/datatransfer/Sender.java      |  261 +++++
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  254 +++++
 .../token/block/InvalidBlockTokenException.java |   41 +
 .../server/datanode/BlockMetadataHeader.java    |  209 ++++
 .../hdfs/server/datanode/CachingStrategy.java   |   76 ++
 .../hadoop/hdfs/shortcircuit/ClientMmap.java    |   75 ++
 .../hadoop/hdfs/shortcircuit/DfsClientShm.java  |  119 ++
 .../hdfs/shortcircuit/DfsClientShmManager.java  |  522 +++++++++
 .../hdfs/shortcircuit/DomainSocketFactory.java  |  196 ++++
 .../hdfs/shortcircuit/ShortCircuitCache.java    | 1066 +++++++++++++++++
 .../hdfs/shortcircuit/ShortCircuitReplica.java  |  352 ++++++
 .../shortcircuit/ShortCircuitReplicaInfo.java   |   64 ++
 .../hdfs/shortcircuit/ShortCircuitShm.java      |  647 +++++++++++
 .../hadoop/hdfs/util/ByteArrayManager.java      |  422 +++++++
 .../hadoop/hdfs/util/ExactSizeInputStream.java  |  125 ++
 .../apache/hadoop/hdfs/util/IOUtilsClient.java  |   46 +
 .../apache/hadoop/hdfs/util/package-info.java   |   18 +
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |   20 +
 .../hdfs/web/resources/CreateFlagParam.java     |   48 +
 .../hdfs/web/resources/CreateParentParam.java   |    2 +-
 .../src/main/proto/ClientDatanodeProtocol.proto |   33 -
 .../src/main/proto/datatransfer.proto           |    4 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  125 +-
 .../dev-support/findbugsExcludeFile.xml         |   10 -
 .../hadoop-hdfs/src/CMakeLists.txt              |    1 +
 .../apache/hadoop/fs/BlockStorageLocation.java  |   52 -
 .../java/org/apache/hadoop/fs/HdfsVolumeId.java |   73 --
 .../java/org/apache/hadoop/fs/VolumeId.java     |   40 -
 .../apache/hadoop/hdfs/BlockReaderFactory.java  |   65 +-
 .../hadoop/hdfs/BlockStorageLocationUtil.java   |  368 ------
 .../org/apache/hadoop/hdfs/ClientContext.java   |    5 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  134 +--
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  186 ++-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |    3 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |    6 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |   39 -
 .../org/apache/hadoop/hdfs/ExtendedBlockId.java |   82 --
 .../apache/hadoop/hdfs/ExternalBlockReader.java |  126 +++
 .../apache/hadoop/hdfs/HdfsConfiguration.java   |    8 +-
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |    6 +-
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |    6 +-
 .../org/apache/hadoop/hdfs/XAttrHelper.java     |   13 +-
 .../hadoop/hdfs/client/impl/DfsClientConf.java  |  765 -------------
 .../org/apache/hadoop/hdfs/net/DomainPeer.java  |  132 ---
 .../java/org/apache/hadoop/hdfs/net/Peer.java   |  123 --
 .../hdfs/protocol/ClientDatanodeProtocol.java   |   19 -
 .../hdfs/protocol/HdfsBlocksMetadata.java       |  111 --
 .../datatransfer/BlockConstructionStage.java    |   62 -
 .../datatransfer/DataTransferProtoUtil.java     |  148 ---
 .../datatransfer/DataTransferProtocol.java      |  201 ----
 .../hadoop/hdfs/protocol/datatransfer/Op.java   |   65 --
 .../hdfs/protocol/datatransfer/PipelineAck.java |    2 +-
 .../hdfs/protocol/datatransfer/Receiver.java    |    8 +-
 .../hdfs/protocol/datatransfer/Sender.java      |  261 -----
 .../datatransfer/sasl/DataTransferSaslUtil.java |    2 +-
 ...tDatanodeProtocolServerSideTranslatorPB.java |   43 +-
 .../ClientDatanodeProtocolTranslatorPB.java     |   49 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |    6 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |   28 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |    4 +-
 .../InterDatanodeProtocolTranslatorPB.java      |    2 +-
 .../NamenodeProtocolTranslatorPB.java           |    2 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  237 +---
 .../token/block/InvalidBlockTokenException.java |   41 -
 .../hadoop/hdfs/server/balancer/Balancer.java   |  133 ++-
 .../hadoop/hdfs/server/balancer/Dispatcher.java |   46 +-
 .../server/blockmanagement/BlockCollection.java |    9 +-
 .../hdfs/server/blockmanagement/BlockInfo.java  |   59 +-
 .../blockmanagement/BlockInfoContiguous.java    |   11 -
 .../blockmanagement/BlockInfoStriped.java       |    5 -
 .../server/blockmanagement/BlockManager.java    |  115 +-
 .../BlockPlacementPolicyDefault.java            |  177 +--
 .../BlockPlacementPolicyWithNodeGroup.java      |   35 +-
 .../BlockStoragePolicySuite.java                |    5 +-
 .../BlockUnderConstructionFeature.java          |   31 +-
 .../hdfs/server/blockmanagement/BlocksMap.java  |   14 +-
 .../blockmanagement/DatanodeDescriptor.java     |   26 +-
 .../server/blockmanagement/DatanodeManager.java |    3 +-
 .../blockmanagement/DecommissionManager.java    |   21 +-
 .../SequentialBlockGroupIdGenerator.java        |    5 +-
 .../SequentialBlockIdGenerator.java             |    5 +-
 .../hdfs/server/common/HdfsServerConstants.java |    6 -
 .../server/datanode/BlockMetadataHeader.java    |  211 ----
 .../hdfs/server/datanode/CachingStrategy.java   |   76 --
 .../hadoop/hdfs/server/datanode/DNConf.java     |   17 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   44 +-
 .../hdfs/server/datanode/DataXceiver.java       |   23 +-
 .../server/datanode/SecureDataNodeStarter.java  |    4 +-
 .../server/datanode/fsdataset/FsDatasetSpi.java |   13 -
 .../datanode/fsdataset/impl/BlockPoolSlice.java |    2 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   64 +-
 .../impl/RamDiskAsyncLazyPersistService.java    |    8 +-
 .../datanode/web/webhdfs/ParameterParser.java   |   14 +
 .../datanode/web/webhdfs/WebHdfsHandler.java    |   23 +-
 .../hdfs/server/namenode/BackupImage.java       |    8 +-
 .../namenode/ErasureCodingZoneManager.java      |    2 +-
 .../hdfs/server/namenode/FSDirAppendOp.java     |    2 +-
 .../hdfs/server/namenode/FSDirAttrOp.java       |   49 +-
 .../hdfs/server/namenode/FSDirConcatOp.java     |    4 +-
 .../hdfs/server/namenode/FSDirDeleteOp.java     |    5 +-
 .../hdfs/server/namenode/FSDirRenameOp.java     |    7 +-
 .../hdfs/server/namenode/FSDirSnapshotOp.java   |    2 +
 .../hdfs/server/namenode/FSDirTruncateOp.java   |    2 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |    3 +-
 .../hdfs/server/namenode/FSDirXAttrOp.java      |   29 +-
 .../hdfs/server/namenode/FSDirectory.java       |   77 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   11 +
 .../hdfs/server/namenode/FSEditLogLoader.java   |    8 +-
 .../hadoop/hdfs/server/namenode/FSImage.java    |  153 ++-
 .../server/namenode/FSImageFormatPBINode.java   |   11 +-
 .../server/namenode/FSImageSerialization.java   |    4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   70 +-
 .../namenode/FileUnderConstructionFeature.java  |    2 +-
 .../hadoop/hdfs/server/namenode/INode.java      |   52 +-
 .../hdfs/server/namenode/INodeDirectory.java    |   11 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   34 +-
 .../hadoop/hdfs/server/namenode/INodeId.java    |    1 +
 .../hadoop/hdfs/server/namenode/NameNode.java   |  176 ++-
 .../hdfs/server/namenode/NamenodeFsck.java      |   20 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |    7 +-
 .../hdfs/server/namenode/QuotaCounts.java       |   10 +-
 .../server/namenode/SerialNumberManager.java    |   44 -
 .../hdfs/server/namenode/SerialNumberMap.java   |   79 ++
 .../hdfs/server/namenode/XAttrFeature.java      |   78 +-
 .../hdfs/server/namenode/XAttrFormat.java       |  161 +++
 .../server/namenode/XAttrPermissionFilter.java  |    6 +-
 .../hdfs/server/namenode/XAttrStorage.java      |   62 +-
 .../namenode/metrics/FSNamesystemMBean.java     |   10 +
 .../snapshot/FSImageFormatPBSnapshot.java       |   10 +-
 .../snapshot/FileWithSnapshotFeature.java       |   44 +-
 .../web/resources/NamenodeWebHdfsMethods.java   |   20 +-
 .../hdfs/server/protocol/NamespaceInfo.java     |   17 +-
 .../hadoop/hdfs/shortcircuit/ClientMmap.java    |   75 --
 .../hadoop/hdfs/shortcircuit/DfsClientShm.java  |  119 --
 .../hdfs/shortcircuit/DfsClientShmManager.java  |  514 ---------
 .../hdfs/shortcircuit/DomainSocketFactory.java  |  194 ----
 .../hdfs/shortcircuit/ShortCircuitCache.java    | 1068 ------------------
 .../hdfs/shortcircuit/ShortCircuitReplica.java  |  349 ------
 .../shortcircuit/ShortCircuitReplicaInfo.java   |   64 --
 .../hdfs/shortcircuit/ShortCircuitShm.java      |  646 -----------
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |    5 +
 .../hadoop/hdfs/util/ByteArrayManager.java      |  418 -------
 .../hadoop/hdfs/util/ExactSizeInputStream.java  |  125 --
 .../hadoop/hdfs/util/LightWeightHashSet.java    |   21 +-
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |    6 +-
 .../src/main/resources/hdfs-default.xml         |   50 +-
 .../hadoop-hdfs/src/site/markdown/HdfsDesign.md |    5 +-
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    |   21 +-
 .../org/apache/hadoop/hdfs/TestFiPipelines.java |    9 +-
 .../datanode/TestFiDataTransferProtocol.java    |    3 +-
 .../datanode/TestFiDataTransferProtocol2.java   |    5 +-
 .../hadoop/fs/TestEnhancedByteBufferAccess.java |   10 +-
 .../java/org/apache/hadoop/fs/TestUnbuffer.java |    7 +-
 .../java/org/apache/hadoop/fs/TestVolumeId.java |  146 ---
 .../fs/viewfs/TestViewFsDefaultValue.java       |    8 +-
 .../apache/hadoop/hdfs/BlockReaderTestUtil.java |    6 +-
 .../org/apache/hadoop/hdfs/FileAppendTest4.java |    5 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |    1 +
 .../hadoop/hdfs/TestAppendSnapshotTruncate.java |   13 +-
 .../hadoop/hdfs/TestBlockReaderFactory.java     |    4 +-
 .../hadoop/hdfs/TestBlockReaderLocal.java       |    4 +-
 .../hadoop/hdfs/TestBlockReaderLocalLegacy.java |    6 +-
 .../TestClientProtocolForPipelineRecovery.java  |    4 +-
 .../org/apache/hadoop/hdfs/TestConnCache.java   |    5 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |    8 +-
 .../hdfs/TestDFSInotifyEventInputStream.java    |    2 +-
 .../apache/hadoop/hdfs/TestDFSInputStream.java  |    2 +-
 .../org/apache/hadoop/hdfs/TestDFSUpgrade.java  |   78 +-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |  107 +-
 .../hadoop/hdfs/TestDataTransferKeepalive.java  |    8 +-
 .../hadoop/hdfs/TestDataTransferProtocol.java   |    8 +-
 .../apache/hadoop/hdfs/TestDatanodeDeath.java   |    5 +-
 .../hadoop/hdfs/TestDisableConnCache.java       |    3 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |  291 +----
 .../hadoop/hdfs/TestExternalBlockReader.java    |  298 +++++
 .../org/apache/hadoop/hdfs/TestFileAppend.java  |    4 +-
 .../org/apache/hadoop/hdfs/TestFileAppend2.java |    6 +-
 .../org/apache/hadoop/hdfs/TestFileAppend4.java |    5 +-
 .../apache/hadoop/hdfs/TestFileCreation.java    |  165 +--
 .../java/org/apache/hadoop/hdfs/TestHFlush.java |    3 +-
 .../apache/hadoop/hdfs/TestParallelRead.java    |    2 +-
 .../TestParallelShortCircuitLegacyRead.java     |    4 +-
 .../TestParallelShortCircuitReadUnCached.java   |    6 +-
 .../hadoop/hdfs/TestParallelUnixDomainRead.java |    2 +-
 .../org/apache/hadoop/hdfs/TestPipelines.java   |    9 +-
 .../java/org/apache/hadoop/hdfs/TestPread.java  |    6 +-
 .../java/org/apache/hadoop/hdfs/TestRead.java   |    5 +-
 .../hadoop/hdfs/TestRemoteBlockReader.java      |    4 +-
 .../hdfs/TestReplaceDatanodeOnFailure.java      |    4 +-
 .../hdfs/protocol/TestBlockListAsLongs.java     |    9 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |   20 +-
 .../hdfs/server/balancer/TestBalancer.java      |  166 ++-
 .../server/blockmanagement/TestBlockInfo.java   |   17 +-
 .../blockmanagement/TestBlockInfoStriped.java   |    4 +-
 .../TestBlockInfoUnderConstruction.java         |   80 --
 .../blockmanagement/TestBlockManager.java       |   18 +-
 .../TestBlockReportRateLimiting.java            |    2 -
 .../blockmanagement/TestBlockTokenWithDFS.java  |    6 +-
 .../TestBlockUnderConstructionFeature.java      |   80 ++
 .../blockmanagement/TestPendingReplication.java |    1 -
 .../blockmanagement/TestReplicationPolicy.java  |   94 +-
 .../server/datanode/SimulatedFSDataset.java     |    7 -
 .../server/datanode/TestBlockReplacement.java   |    7 +-
 .../server/datanode/TestCachingStrategy.java    |   18 +-
 .../datanode/TestDataNodeVolumeFailure.java     |    6 +-
 .../extdataset/ExternalDatasetImpl.java         |    6 -
 .../fsdataset/impl/LazyPersistTestCase.java     |    5 +-
 .../fsdataset/impl/TestDatanodeRestart.java     |    7 +-
 .../TestCommitBlockSynchronization.java         |    9 +-
 .../TestDefaultBlockPlacementPolicy.java        |   49 +-
 .../namenode/TestDiskspaceQuotaUpdate.java      |   64 ++
 .../namenode/TestFSImageWithSnapshot.java       |    4 +-
 .../server/namenode/TestFSNamesystemMBean.java  |   34 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   56 +-
 .../hdfs/server/namenode/TestINodeFile.java     |    7 +-
 .../namenode/TestNameNodeMetricsLogger.java     |  193 ++++
 .../hdfs/server/namenode/TestStartup.java       |   27 +-
 .../hdfs/server/namenode/TestXAttrFeature.java  |  119 ++
 .../namenode/snapshot/SnapshotTestHelper.java   |    4 +-
 .../snapshot/TestFileWithSnapshotFeature.java   |    7 +-
 .../snapshot/TestSnapshotBlocksMap.java         |   30 +-
 .../namenode/snapshot/TestSnapshotDeletion.java |   25 +-
 .../snapshot/TestSnapshotReplication.java       |   31 +-
 .../shortcircuit/TestShortCircuitCache.java     |   16 +-
 .../shortcircuit/TestShortCircuitLocalRead.java |  106 +-
 .../hadoop/hdfs/tools/TestDFSAdminWithHA.java   |    7 +
 .../hdfs/util/TestLightWeightHashSet.java       |   29 +-
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |   25 +
 .../src/test/resources/hadoop-252-dfs-dir.tgz   |  Bin 0 -> 14112 bytes
 .../src/test/resources/log4j.properties         |   13 +
 hadoop-mapreduce-project/CHANGES.txt            |   23 +
 .../apache/hadoop/mapred/LocalJobRunner.java    |   27 +
 .../mapreduce/lib/output/MultipleOutputs.java   |   14 +-
 .../hadoop/mapred/ResourceMgrDelegate.java      |    7 +
 .../hadoop/mapred/TestClientRedirect.java       |    9 +
 .../hadoop/mapred/TestLocalJobSubmission.java   |   25 +
 hadoop-project/pom.xml                          |    4 +-
 .../org/apache/hadoop/tools/CopyListing.java    |   15 +-
 .../java/org/apache/hadoop/tools/DiffInfo.java  |   32 +-
 .../java/org/apache/hadoop/tools/DistCp.java    |   27 +-
 .../org/apache/hadoop/tools/DistCpOptions.java  |    4 +-
 .../org/apache/hadoop/tools/DistCpSync.java     |  308 ++++-
 .../apache/hadoop/tools/SimpleCopyListing.java  |  151 ++-
 .../org/apache/hadoop/tools/TestDistCpSync.java |  345 +++++-
 .../apache/hadoop/tools/TestOptionsParser.java  |   22 +-
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |    3 +
 .../yarn/sls/scheduler/RMNodeWrapper.java       |    5 +
 hadoop-yarn-project/CHANGES.txt                 |   48 +-
 .../yarn/api/ApplicationClientProtocol.java     |   18 +
 .../UpdateApplicationPriorityRequest.java       |   80 ++
 .../UpdateApplicationPriorityResponse.java      |   47 +
 .../yarn/api/records/LogAggregationContext.java |   95 ++
 .../hadoop/yarn/conf/YarnConfiguration.java     |   36 +
 .../api/ContainerLogAggregationPolicy.java      |   54 +
 .../yarn/server/api/ContainerLogContext.java    |   71 ++
 .../ResourceManagerAdministrationProtocol.java  |    8 +
 .../RefreshClusterMaxPriorityRequest.java       |   35 +
 .../RefreshClusterMaxPriorityResponse.java      |   36 +
 .../main/proto/applicationclient_protocol.proto |    1 +
 ...esourcemanager_administration_protocol.proto |    1 +
 ..._server_resourcemanager_service_protos.proto |    5 +
 .../src/main/proto/yarn_protos.proto            |    2 +
 .../src/main/proto/yarn_service_protos.proto    |    8 +
 .../hadoop/yarn/client/api/YarnClient.java      |   17 +
 .../yarn/client/api/impl/YarnClientImpl.java    |   11 +
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   29 +
 .../hadoop/yarn/client/cli/RMAdminCLI.java      |   15 +
 .../hadoop/yarn/client/cli/TestRMAdminCLI.java  |    9 +
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |   29 +
 .../ApplicationClientProtocolPBClientImpl.java  |   20 +
 .../ApplicationClientProtocolPBServiceImpl.java |   22 +
 .../UpdateApplicationPriorityRequestPBImpl.java |  171 +++
 ...UpdateApplicationPriorityResponsePBImpl.java |   69 ++
 .../impl/pb/LogAggregationContextPBImpl.java    |   40 +
 .../ContainerLogsRetentionPolicy.java           |   29 -
 .../nodelabels/CommonNodeLabelsManager.java     |    2 +-
 ...nagerAdministrationProtocolPBClientImpl.java |   20 +
 ...agerAdministrationProtocolPBServiceImpl.java |   23 +
 .../RefreshClusterMaxPriorityRequestPBImpl.java |   74 ++
 ...RefreshClusterMaxPriorityResponsePBImpl.java |   73 ++
 .../src/main/resources/yarn-default.xml         |   71 ++
 .../hadoop-yarn/hadoop-yarn-registry/pom.xml    |    4 +-
 .../hadoop/yarn/server/webapp/AppBlock.java     |    6 +-
 .../yarn/server/nodemanager/NodeManager.java    |   39 +-
 .../nodemanager/NodeStatusUpdaterImpl.java      |  259 +++--
 .../application/ApplicationImpl.java            |    5 +-
 .../AMOnlyLogAggregationPolicy.java             |   31 +
 ...AMOrFailedContainerLogAggregationPolicy.java |   35 +
 .../AbstractContainerLogAggregationPolicy.java  |   31 +
 .../AllContainerLogAggregationPolicy.java       |   30 +
 .../logaggregation/AppLogAggregator.java        |    5 +-
 .../logaggregation/AppLogAggregatorImpl.java    |  131 ++-
 .../FailedContainerLogAggregationPolicy.java    |   33 +
 ...edOrKilledContainerLogAggregationPolicy.java |   30 +
 .../logaggregation/LogAggregationService.java   |   19 +-
 .../NoneContainerLogAggregationPolicy.java      |   30 +
 .../SampleContainerLogAggregationPolicy.java    |  124 ++
 .../event/LogHandlerAppStartedEvent.java        |   15 +-
 .../monitor/ContainersMonitorImpl.java          |   10 +-
 .../nodelabels/AbstractNodeLabelsProvider.java  |  146 +++
 .../ConfigurationNodeLabelsProvider.java        |   81 ++
 .../server/nodemanager/TestNodeManager.java     |   50 +-
 .../TestNodeStatusUpdaterForLabels.java         |   76 +-
 .../containermanager/TestAuxServices.java       |    1 +
 .../TestLogAggregationService.java              |  677 +++++++++--
 .../TestNonAggregatingLogHandler.java           |   12 +-
 .../TestConfigurationNodeLabelsProvider.java    |  146 +++
 .../server/resourcemanager/AdminService.java    |   28 +
 .../ApplicationMasterService.java               |    3 +-
 .../server/resourcemanager/ClientRMService.java |   76 ++
 .../server/resourcemanager/RMAuditLogger.java   |    2 +
 .../resourcemanager/ResourceTrackerService.java |    2 +
 .../resourcemanager/recovery/RMStateStore.java  |   12 +-
 .../recovery/RMStateUpdateAppEvent.java         |   13 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |    3 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |    8 +-
 .../server/resourcemanager/rmnode/RMNode.java   |    7 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |   15 +-
 .../scheduler/AbstractYarnScheduler.java        |   25 +
 .../scheduler/YarnScheduler.java                |   20 +
 .../scheduler/capacity/CapacityScheduler.java   |   25 +-
 .../ClientToAMTokenSecretManagerInRM.java       |    7 +
 .../yarn/server/resourcemanager/MockNodes.java  |    4 +
 .../resourcemanager/TestClientRMService.java    |   75 ++
 .../resourcemanager/TestRMAdminService.java     |   34 +
 .../TestWorkPreservingRMRestart.java            |    2 +-
 .../resourcetracker/TestNMReconnect.java        |   39 +
 .../attempt/TestRMAppAttemptTransitions.java    |   32 +
 .../capacity/TestContainerAllocation.java       |   12 +-
 371 files changed, 15142 insertions(+), 9357 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
index 0000000,d46ab47..b99e3ba
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
@@@ -1,0 -1,777 +1,794 @@@
+ /**
+  * 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.hdfs.client.impl;
+ 
+ import com.google.common.annotations.VisibleForTesting;
++import com.google.common.base.Preconditions;
+ import org.apache.hadoop.HadoopIllegalArgumentException;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+ import org.apache.hadoop.fs.Options.ChecksumOpt;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hdfs.ReplicaAccessorBuilder;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+ import org.apache.hadoop.hdfs.util.ByteArrayManager;
+ import org.apache.hadoop.ipc.Client;
+ import org.apache.hadoop.util.DataChecksum;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BLOCK_SIZE_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CHECKSUM_TYPE_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_MS;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DOMAIN_SOCKET_PATH_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_REPLICATION_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_REPLICATION_KEY;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Failover;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.HedgedRead;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Mmap;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Read;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Retry;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.ShortCircuit;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write;
+ 
+ import java.lang.Class;
+ import java.util.ArrayList;
+ import java.util.Collections;
+ import java.util.List;
+ 
+ /**
+  * DFSClient configuration.
+  */
+ public class DfsClientConf {
+   private static final Logger LOG = LoggerFactory.getLogger(DfsClientConf
+                                                                 .class);
+ 
+   private final int hdfsTimeout;    // timeout value for a DFS operation.
+ 
+   private final int maxFailoverAttempts;
+   private final int maxRetryAttempts;
+   private final int failoverSleepBaseMillis;
+   private final int failoverSleepMaxMillis;
+   private final int maxBlockAcquireFailures;
+   private final int datanodeSocketWriteTimeout;
+   private final int ioBufferSize;
+   private final ChecksumOpt defaultChecksumOpt;
+   private final int writePacketSize;
+   private final int writeMaxPackets;
+   private final ByteArrayManager.Conf writeByteArrayManagerConf;
+   private final int socketTimeout;
+   private final long excludedNodesCacheExpiry;
+   /** Wait time window (in msec) if BlockMissingException is caught. */
+   private final int timeWindow;
+   private final int numCachedConnRetry;
+   private final int numBlockWriteRetry;
+   private final int numBlockWriteLocateFollowingRetry;
+   private final int blockWriteLocateFollowingInitialDelayMs;
+   private final long defaultBlockSize;
+   private final long prefetchSize;
+   private final short defaultReplication;
+   private final String taskId;
+   private final FsPermission uMask;
+   private final boolean connectToDnViaHostname;
+   private final int retryTimesForGetLastBlockLength;
+   private final int retryIntervalForGetLastBlockLength;
+   private final long datanodeRestartTimeout;
+   private final long slowIoWarningThresholdMs;
+ 
+   private final ShortCircuitConf shortCircuitConf;
+ 
+   private final long hedgedReadThresholdMillis;
+   private final int hedgedReadThreadpoolSize;
+   private final List<Class<? extends ReplicaAccessorBuilder>>
+       replicaAccessorBuilderClasses;
+ 
++  private final int stripedReadThreadpoolSize;
++
++
+   public DfsClientConf(Configuration conf) {
+     // The hdfsTimeout is currently the same as the ipc timeout
+     hdfsTimeout = Client.getTimeout(conf);
+ 
+     maxRetryAttempts = conf.getInt(
+         Retry.MAX_ATTEMPTS_KEY,
+         Retry.MAX_ATTEMPTS_DEFAULT);
+     timeWindow = conf.getInt(
+         Retry.WINDOW_BASE_KEY,
+         Retry.WINDOW_BASE_DEFAULT);
+     retryTimesForGetLastBlockLength = conf.getInt(
+         Retry.TIMES_GET_LAST_BLOCK_LENGTH_KEY,
+         Retry.TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT);
+     retryIntervalForGetLastBlockLength = conf.getInt(
+         Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_KEY,
+         Retry.INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT);
+ 
+     maxFailoverAttempts = conf.getInt(
+         Failover.MAX_ATTEMPTS_KEY,
+         Failover.MAX_ATTEMPTS_DEFAULT);
+     failoverSleepBaseMillis = conf.getInt(
+         Failover.SLEEPTIME_BASE_KEY,
+         Failover.SLEEPTIME_BASE_DEFAULT);
+     failoverSleepMaxMillis = conf.getInt(
+         Failover.SLEEPTIME_MAX_KEY,
+         Failover.SLEEPTIME_MAX_DEFAULT);
+ 
+     maxBlockAcquireFailures = conf.getInt(
+         DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
+         DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
+     datanodeSocketWriteTimeout = conf.getInt(
+         DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
+         HdfsConstants.WRITE_TIMEOUT);
+     ioBufferSize = conf.getInt(
+         CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
+         CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
+     defaultChecksumOpt = getChecksumOptFromConf(conf);
+     socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
+         HdfsConstants.READ_TIMEOUT);
+     /** dfs.write.packet.size is an internal config variable */
+     writePacketSize = conf.getInt(
+         DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
+         DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
+     writeMaxPackets = conf.getInt(
+         Write.MAX_PACKETS_IN_FLIGHT_KEY,
+         Write.MAX_PACKETS_IN_FLIGHT_DEFAULT);
+ 
+     final boolean byteArrayManagerEnabled = conf.getBoolean(
+         Write.ByteArrayManager.ENABLED_KEY,
+         Write.ByteArrayManager.ENABLED_DEFAULT);
+     if (!byteArrayManagerEnabled) {
+       writeByteArrayManagerConf = null;
+     } else {
+       final int countThreshold = conf.getInt(
+           Write.ByteArrayManager.COUNT_THRESHOLD_KEY,
+           Write.ByteArrayManager.COUNT_THRESHOLD_DEFAULT);
+       final int countLimit = conf.getInt(
+           Write.ByteArrayManager.COUNT_LIMIT_KEY,
+           Write.ByteArrayManager.COUNT_LIMIT_DEFAULT);
+       final long countResetTimePeriodMs = conf.getLong(
+           Write.ByteArrayManager.COUNT_RESET_TIME_PERIOD_MS_KEY,
+           Write.ByteArrayManager.COUNT_RESET_TIME_PERIOD_MS_DEFAULT);
+       writeByteArrayManagerConf = new ByteArrayManager.Conf(
+           countThreshold, countLimit, countResetTimePeriodMs);
+     }
+ 
+     defaultBlockSize = conf.getLongBytes(DFS_BLOCK_SIZE_KEY,
+         DFS_BLOCK_SIZE_DEFAULT);
+     defaultReplication = (short) conf.getInt(
+         DFS_REPLICATION_KEY, DFS_REPLICATION_DEFAULT);
+     taskId = conf.get("mapreduce.task.attempt.id", "NONMAPREDUCE");
+     excludedNodesCacheExpiry = conf.getLong(
+         Write.EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY,
+         Write.EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT);
+     prefetchSize = conf.getLong(Read.PREFETCH_SIZE_KEY,
+         10 * defaultBlockSize);
+     numCachedConnRetry = conf.getInt(DFS_CLIENT_CACHED_CONN_RETRY_KEY,
+         DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
+     numBlockWriteRetry = conf.getInt(
+         BlockWrite.RETRIES_KEY,
+         BlockWrite.RETRIES_DEFAULT);
+     numBlockWriteLocateFollowingRetry = conf.getInt(
+         BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
+         BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
+     blockWriteLocateFollowingInitialDelayMs = conf.getInt(
+         BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY,
+         BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT);
+     uMask = FsPermission.getUMask(conf);
+     connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
+         DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
+ 
+     datanodeRestartTimeout = conf.getLong(
+         DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,
+         DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT) * 1000;
+     slowIoWarningThresholdMs = conf.getLong(
+         DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY,
+         DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
+ 
+     shortCircuitConf = new ShortCircuitConf(conf);
+ 
+     hedgedReadThresholdMillis = conf.getLong(
+         HedgedRead.THRESHOLD_MILLIS_KEY,
+         HedgedRead.THRESHOLD_MILLIS_DEFAULT);
+     hedgedReadThreadpoolSize = conf.getInt(
+         HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY,
+         HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_DEFAULT);
+ 
++    stripedReadThreadpoolSize = conf.getInt(
++        HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY,
++        HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_DEFAULT);
++    Preconditions.checkArgument(stripedReadThreadpoolSize > 0, "The value of " +
++        HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY +
++        " must be greater than 0.");
+     replicaAccessorBuilderClasses = loadReplicaAccessorBuilderClasses(conf);
+   }
+ 
+   @SuppressWarnings("unchecked")
+   private List<Class<? extends ReplicaAccessorBuilder>>
+       loadReplicaAccessorBuilderClasses(Configuration conf)
+   {
+     String classNames[] = conf.getTrimmedStrings(
+         HdfsClientConfigKeys.REPLICA_ACCESSOR_BUILDER_CLASSES_KEY);
+     if (classNames.length == 0) {
+       return Collections.emptyList();
+     }
+     ArrayList<Class<? extends ReplicaAccessorBuilder>> classes =
+         new ArrayList<>();
+     ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+     for (String className: classNames) {
+       try {
+         Class<? extends ReplicaAccessorBuilder> cls =
+           (Class<? extends ReplicaAccessorBuilder>)
+             classLoader.loadClass(className);
+         classes.add(cls);
+       } catch (Throwable t) {
+         LOG.warn("Unable to load " + className, t);
+       }
+     }
+     return classes;
+   }
+ 
+   private DataChecksum.Type getChecksumType(Configuration conf) {
+     final String checksum = conf.get(
+         DFS_CHECKSUM_TYPE_KEY,
+         DFS_CHECKSUM_TYPE_DEFAULT);
+     try {
+       return DataChecksum.Type.valueOf(checksum);
+     } catch(IllegalArgumentException iae) {
+       LOG.warn("Bad checksum type: {}. Using default {}", checksum,
+                DFS_CHECKSUM_TYPE_DEFAULT);
+       return DataChecksum.Type.valueOf(
+           DFS_CHECKSUM_TYPE_DEFAULT);
+     }
+   }
+ 
+   // Construct a checksum option from conf
+   private ChecksumOpt getChecksumOptFromConf(Configuration conf) {
+     DataChecksum.Type type = getChecksumType(conf);
+     int bytesPerChecksum = conf.getInt(DFS_BYTES_PER_CHECKSUM_KEY,
+         DFS_BYTES_PER_CHECKSUM_DEFAULT);
+     return new ChecksumOpt(type, bytesPerChecksum);
+   }
+ 
+   /** create a DataChecksum with the given option. */
+   public DataChecksum createChecksum(ChecksumOpt userOpt) {
+     // Fill in any missing field with the default.
+     ChecksumOpt opt = ChecksumOpt.processChecksumOpt(
+         defaultChecksumOpt, userOpt);
+     DataChecksum dataChecksum = DataChecksum.newDataChecksum(
+         opt.getChecksumType(),
+         opt.getBytesPerChecksum());
+     if (dataChecksum == null) {
+       throw new HadoopIllegalArgumentException("Invalid checksum type: userOpt="
+           + userOpt + ", default=" + defaultChecksumOpt
+           + ", effective=null");
+     }
+     return dataChecksum;
+   }
+ 
+   @VisibleForTesting
+   public int getBlockWriteLocateFollowingInitialDelayMs() {
+     return blockWriteLocateFollowingInitialDelayMs;
+   }
+ 
+   /**
+    * @return the hdfsTimeout
+    */
+   public int getHdfsTimeout() {
+     return hdfsTimeout;
+   }
+ 
+   /**
+    * @return the maxFailoverAttempts
+    */
+   public int getMaxFailoverAttempts() {
+     return maxFailoverAttempts;
+   }
+ 
+   /**
+    * @return the maxRetryAttempts
+    */
+   public int getMaxRetryAttempts() {
+     return maxRetryAttempts;
+   }
+ 
+   /**
+    * @return the failoverSleepBaseMillis
+    */
+   public int getFailoverSleepBaseMillis() {
+     return failoverSleepBaseMillis;
+   }
+ 
+   /**
+    * @return the failoverSleepMaxMillis
+    */
+   public int getFailoverSleepMaxMillis() {
+     return failoverSleepMaxMillis;
+   }
+ 
+   /**
+    * @return the maxBlockAcquireFailures
+    */
+   public int getMaxBlockAcquireFailures() {
+     return maxBlockAcquireFailures;
+   }
+ 
+   /**
+    * @return the datanodeSocketWriteTimeout
+    */
+   public int getDatanodeSocketWriteTimeout() {
+     return datanodeSocketWriteTimeout;
+   }
+ 
+   /**
+    * @return the ioBufferSize
+    */
+   public int getIoBufferSize() {
+     return ioBufferSize;
+   }
+ 
+   /**
+    * @return the defaultChecksumOpt
+    */
+   public ChecksumOpt getDefaultChecksumOpt() {
+     return defaultChecksumOpt;
+   }
+ 
+   /**
+    * @return the writePacketSize
+    */
+   public int getWritePacketSize() {
+     return writePacketSize;
+   }
+ 
+   /**
+    * @return the writeMaxPackets
+    */
+   public int getWriteMaxPackets() {
+     return writeMaxPackets;
+   }
+ 
+   /**
+    * @return the writeByteArrayManagerConf
+    */
+   public ByteArrayManager.Conf getWriteByteArrayManagerConf() {
+     return writeByteArrayManagerConf;
+   }
+ 
+   /**
+    * @return the socketTimeout
+    */
+   public int getSocketTimeout() {
+     return socketTimeout;
+   }
+ 
+   /**
+    * @return the excludedNodesCacheExpiry
+    */
+   public long getExcludedNodesCacheExpiry() {
+     return excludedNodesCacheExpiry;
+   }
+ 
+   /**
+    * @return the timeWindow
+    */
+   public int getTimeWindow() {
+     return timeWindow;
+   }
+ 
+   /**
+    * @return the numCachedConnRetry
+    */
+   public int getNumCachedConnRetry() {
+     return numCachedConnRetry;
+   }
+ 
+   /**
+    * @return the numBlockWriteRetry
+    */
+   public int getNumBlockWriteRetry() {
+     return numBlockWriteRetry;
+   }
+ 
+   /**
+    * @return the numBlockWriteLocateFollowingRetry
+    */
+   public int getNumBlockWriteLocateFollowingRetry() {
+     return numBlockWriteLocateFollowingRetry;
+   }
+ 
+   /**
+    * @return the defaultBlockSize
+    */
+   public long getDefaultBlockSize() {
+     return defaultBlockSize;
+   }
+ 
+   /**
+    * @return the prefetchSize
+    */
+   public long getPrefetchSize() {
+     return prefetchSize;
+   }
+ 
+   /**
+    * @return the defaultReplication
+    */
+   public short getDefaultReplication() {
+     return defaultReplication;
+   }
+ 
+   /**
+    * @return the taskId
+    */
+   public String getTaskId() {
+     return taskId;
+   }
+ 
+   /**
+    * @return the uMask
+    */
+   public FsPermission getUMask() {
+     return uMask;
+   }
+ 
+   /**
+    * @return the connectToDnViaHostname
+    */
+   public boolean isConnectToDnViaHostname() {
+     return connectToDnViaHostname;
+   }
+ 
+   /**
+    * @return the retryTimesForGetLastBlockLength
+    */
+   public int getRetryTimesForGetLastBlockLength() {
+     return retryTimesForGetLastBlockLength;
+   }
+ 
+   /**
+    * @return the retryIntervalForGetLastBlockLength
+    */
+   public int getRetryIntervalForGetLastBlockLength() {
+     return retryIntervalForGetLastBlockLength;
+   }
+ 
+   /**
+    * @return the datanodeRestartTimeout
+    */
+   public long getDatanodeRestartTimeout() {
+     return datanodeRestartTimeout;
+   }
+ 
+   /**
+    * @return the slowIoWarningThresholdMs
+    */
+   public long getSlowIoWarningThresholdMs() {
+     return slowIoWarningThresholdMs;
+   }
+ 
+   /**
+    * @return the hedgedReadThresholdMillis
+    */
+   public long getHedgedReadThresholdMillis() {
+     return hedgedReadThresholdMillis;
+   }
+ 
+   /**
+    * @return the hedgedReadThreadpoolSize
+    */
+   public int getHedgedReadThreadpoolSize() {
+     return hedgedReadThreadpoolSize;
+   }
+ 
+   /**
++   * @return the stripedReadThreadpoolSize
++   */
++  public int getStripedReadThreadpoolSize() {
++    return stripedReadThreadpoolSize;
++  }
++
++  /**
+    * @return the replicaAccessorBuilderClasses
+    */
+   public List<Class<? extends ReplicaAccessorBuilder>>
+         getReplicaAccessorBuilderClasses() {
+     return replicaAccessorBuilderClasses;
+   }
+ 
+   /**
+    * @return the shortCircuitConf
+    */
+   public ShortCircuitConf getShortCircuitConf() {
+     return shortCircuitConf;
+   }
+ 
+   /**
+    * Configuration for short-circuit reads.
+    */
+   public static class ShortCircuitConf {
+     private static final Logger LOG = DfsClientConf.LOG;
+ 
+     private final int socketCacheCapacity;
+     private final long socketCacheExpiry;
+ 
+     private final boolean useLegacyBlockReader;
+     private final boolean useLegacyBlockReaderLocal;
+     private final String domainSocketPath;
+     private final boolean skipShortCircuitChecksums;
+ 
+     private final int shortCircuitBufferSize;
+     private final boolean shortCircuitLocalReads;
+     private final boolean domainSocketDataTraffic;
+     private final int shortCircuitStreamsCacheSize;
+     private final long shortCircuitStreamsCacheExpiryMs;
+     private final int shortCircuitSharedMemoryWatcherInterruptCheckMs;
+ 
+     private final boolean shortCircuitMmapEnabled;
+     private final int shortCircuitMmapCacheSize;
+     private final long shortCircuitMmapCacheExpiryMs;
+     private final long shortCircuitMmapCacheRetryTimeout;
+     private final long shortCircuitCacheStaleThresholdMs;
+ 
+     private final long keyProviderCacheExpiryMs;
+ 
+     public ShortCircuitConf(Configuration conf) {
+       socketCacheCapacity = conf.getInt(
+           DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY,
+           DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT);
+       socketCacheExpiry = conf.getLong(
+           DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY,
+           DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT);
+ 
+       useLegacyBlockReader = conf.getBoolean(
+           DFS_CLIENT_USE_LEGACY_BLOCKREADER,
+           DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT);
+       useLegacyBlockReaderLocal = conf.getBoolean(
+           DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
+           DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT);
+       shortCircuitLocalReads = conf.getBoolean(
+           Read.ShortCircuit.KEY,
+           Read.ShortCircuit.DEFAULT);
+       domainSocketDataTraffic = conf.getBoolean(
+           DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
+           DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT);
+       domainSocketPath = conf.getTrimmed(
+           DFS_DOMAIN_SOCKET_PATH_KEY,
+           DFS_DOMAIN_SOCKET_PATH_DEFAULT);
+ 
+       LOG.debug(DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL
+                     + " = {}", useLegacyBlockReaderLocal);
+       LOG.debug(Read.ShortCircuit.KEY
+                     + " = {}", shortCircuitLocalReads);
+       LOG.debug(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC
+                     + " = {}", domainSocketDataTraffic);
+       LOG.debug(DFS_DOMAIN_SOCKET_PATH_KEY
+                     + " = {}", domainSocketPath);
+ 
+       skipShortCircuitChecksums = conf.getBoolean(
+           Read.ShortCircuit.SKIP_CHECKSUM_KEY,
+           Read.ShortCircuit.SKIP_CHECKSUM_DEFAULT);
+       shortCircuitBufferSize = conf.getInt(
+           Read.ShortCircuit.BUFFER_SIZE_KEY,
+           Read.ShortCircuit.BUFFER_SIZE_DEFAULT);
+       shortCircuitStreamsCacheSize = conf.getInt(
+           Read.ShortCircuit.STREAMS_CACHE_SIZE_KEY,
+           Read.ShortCircuit.STREAMS_CACHE_SIZE_DEFAULT);
+       shortCircuitStreamsCacheExpiryMs = conf.getLong(
+           Read.ShortCircuit.STREAMS_CACHE_EXPIRY_MS_KEY,
+           Read.ShortCircuit.STREAMS_CACHE_EXPIRY_MS_DEFAULT);
+       shortCircuitMmapEnabled = conf.getBoolean(
+           Mmap.ENABLED_KEY,
+           Mmap.ENABLED_DEFAULT);
+       shortCircuitMmapCacheSize = conf.getInt(
+           Mmap.CACHE_SIZE_KEY,
+           Mmap.CACHE_SIZE_DEFAULT);
+       shortCircuitMmapCacheExpiryMs = conf.getLong(
+           Mmap.CACHE_TIMEOUT_MS_KEY,
+           Mmap.CACHE_TIMEOUT_MS_DEFAULT);
+       shortCircuitMmapCacheRetryTimeout = conf.getLong(
+           Mmap.RETRY_TIMEOUT_MS_KEY,
+           Mmap.RETRY_TIMEOUT_MS_DEFAULT);
+       shortCircuitCacheStaleThresholdMs = conf.getLong(
+           ShortCircuit.REPLICA_STALE_THRESHOLD_MS_KEY,
+           ShortCircuit.REPLICA_STALE_THRESHOLD_MS_DEFAULT);
+       shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt(
+           DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
+           DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT);
+ 
+       keyProviderCacheExpiryMs = conf.getLong(
+           DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_MS,
+           DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT);
+     }
+ 
+     /**
+      * @return the socketCacheCapacity
+      */
+     public int getSocketCacheCapacity() {
+       return socketCacheCapacity;
+     }
+ 
+     /**
+      * @return the socketCacheExpiry
+      */
+     public long getSocketCacheExpiry() {
+       return socketCacheExpiry;
+     }
+ 
+     public boolean isUseLegacyBlockReaderLocal() {
+       return useLegacyBlockReaderLocal;
+     }
+ 
+     public String getDomainSocketPath() {
+       return domainSocketPath;
+     }
+ 
+     public boolean isShortCircuitLocalReads() {
+       return shortCircuitLocalReads;
+     }
+ 
+     public boolean isDomainSocketDataTraffic() {
+       return domainSocketDataTraffic;
+     }
+     /**
+      * @return the useLegacyBlockReader
+      */
+     public boolean isUseLegacyBlockReader() {
+       return useLegacyBlockReader;
+     }
+ 
+     /**
+      * @return the skipShortCircuitChecksums
+      */
+     public boolean isSkipShortCircuitChecksums() {
+       return skipShortCircuitChecksums;
+     }
+ 
+     /**
+      * @return the shortCircuitBufferSize
+      */
+     public int getShortCircuitBufferSize() {
+       return shortCircuitBufferSize;
+     }
+ 
+     /**
+      * @return the shortCircuitStreamsCacheSize
+      */
+     public int getShortCircuitStreamsCacheSize() {
+       return shortCircuitStreamsCacheSize;
+     }
+ 
+     /**
+      * @return the shortCircuitStreamsCacheExpiryMs
+      */
+     public long getShortCircuitStreamsCacheExpiryMs() {
+       return shortCircuitStreamsCacheExpiryMs;
+     }
+ 
+     /**
+      * @return the shortCircuitSharedMemoryWatcherInterruptCheckMs
+      */
+     public int getShortCircuitSharedMemoryWatcherInterruptCheckMs() {
+       return shortCircuitSharedMemoryWatcherInterruptCheckMs;
+     }
+ 
+     /**
+      * @return the shortCircuitMmapEnabled
+      */
+     public boolean isShortCircuitMmapEnabled() {
+       return shortCircuitMmapEnabled;
+     }
+ 
+     /**
+      * @return the shortCircuitMmapCacheSize
+      */
+     public int getShortCircuitMmapCacheSize() {
+       return shortCircuitMmapCacheSize;
+     }
+ 
+     /**
+      * @return the shortCircuitMmapCacheExpiryMs
+      */
+     public long getShortCircuitMmapCacheExpiryMs() {
+       return shortCircuitMmapCacheExpiryMs;
+     }
+ 
+     /**
+      * @return the shortCircuitMmapCacheRetryTimeout
+      */
+     public long getShortCircuitMmapCacheRetryTimeout() {
+       return shortCircuitMmapCacheRetryTimeout;
+     }
+ 
+     /**
+      * @return the shortCircuitCacheStaleThresholdMs
+      */
+     public long getShortCircuitCacheStaleThresholdMs() {
+       return shortCircuitCacheStaleThresholdMs;
+     }
+ 
+     /**
+      * @return the keyProviderCacheExpiryMs
+      */
+     public long getKeyProviderCacheExpiryMs() {
+       return keyProviderCacheExpiryMs;
+     }
+ 
+     public String confAsString() {
+ 
+       return "shortCircuitStreamsCacheSize = "
+           + shortCircuitStreamsCacheSize
+           + ", shortCircuitStreamsCacheExpiryMs = "
+           + shortCircuitStreamsCacheExpiryMs
+           + ", shortCircuitMmapCacheSize = "
+           + shortCircuitMmapCacheSize
+           + ", shortCircuitMmapCacheExpiryMs = "
+           + shortCircuitMmapCacheExpiryMs
+           + ", shortCircuitMmapCacheRetryTimeout = "
+           + shortCircuitMmapCacheRetryTimeout
+           + ", shortCircuitCacheStaleThresholdMs = "
+           + shortCircuitCacheStaleThresholdMs
+           + ", socketCacheCapacity = "
+           + socketCacheCapacity
+           + ", socketCacheExpiry = "
+           + socketCacheExpiry
+           + ", shortCircuitLocalReads = "
+           + shortCircuitLocalReads
+           + ", useLegacyBlockReaderLocal = "
+           + useLegacyBlockReaderLocal
+           + ", domainSocketDataTraffic = "
+           + domainSocketDataTraffic
+           + ", shortCircuitSharedMemoryWatcherInterruptCheckMs = "
+           + shortCircuitSharedMemoryWatcherInterruptCheckMs
+           + ", keyProviderCacheExpiryMs = "
+           + keyProviderCacheExpiryMs;
+     }
+   }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index 3d19ab9,d5f4d53..7f45132
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@@ -78,17 -78,13 +78,24 @@@ public final class HdfsConstants 
    public static final String CLIENT_NAMENODE_PROTOCOL_NAME =
        "org.apache.hadoop.hdfs.protocol.ClientProtocol";
  
 +  /*
 +   * These values correspond to the values used by the system default erasure
 +   * coding policy.
 +   * TODO: get these values from ec policy of the associated INodeFile
 +   */
 +
 +  public static final byte NUM_DATA_BLOCKS = 6;
 +  public static final byte NUM_PARITY_BLOCKS = 3;
 +  // The chunk size for striped block which is used by erasure coding
 +  public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024;
 +
+   // Timeouts for communicating with DataNode for streaming writes/reads
+   public static final int READ_TIMEOUT = 60 * 1000;
+   public static final int READ_TIMEOUT_EXTENSION = 5 * 1000;
+   public static final int WRITE_TIMEOUT = 8 * 60 * 1000;
+   //for write pipeline
+   public static final int WRITE_TIMEOUT_EXTENSION = 5 * 1000;
+ 
    // SafeMode actions
    public enum SafeModeAction {
      SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
index 0000000,e135d8e..f908dd3
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
@@@ -1,0 -1,120 +1,126 @@@
+ /**
+  * 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.hdfs;
+ 
+ import java.io.IOException;
+ import java.nio.ByteBuffer;
+ import java.util.EnumSet;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.fs.ReadOption;
+ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
++import org.apache.hadoop.util.DataChecksum;
+ 
+ /**
+  * An ExternalBlockReader uses pluggable ReplicaAccessor objects to read from
+  * replicas.
+  */
+ @InterfaceAudience.Private
+ public final class ExternalBlockReader implements BlockReader {
+   private final ReplicaAccessor accessor;
+   private final long visibleLength;
+   private long pos;
+ 
+   ExternalBlockReader(ReplicaAccessor accessor, long visibleLength,
+                       long startOffset) {
+     this.accessor = accessor;
+     this.visibleLength = visibleLength;
+     this.pos = startOffset;
+   }
+ 
+   @Override
+   public int read(byte[] buf, int off, int len) throws IOException {
+     int nread = accessor.read(pos, buf, off, len);
+     pos += nread;
+     return nread;
+   }
+ 
+   @Override
+   public int read(ByteBuffer buf) throws IOException {
+     int nread = accessor.read(pos, buf);
+     pos += nread;
+     return nread;
+   }
+ 
+   @Override
+   public long skip(long n) throws IOException {
+     // You cannot skip backwards
+     if (n <= 0) {
+       return 0;
+     }
+     // You can't skip past the end of the replica.
+     long oldPos = pos;
+     pos += n;
+     if (pos > visibleLength) {
+       pos = visibleLength;
+     }
+     return pos - oldPos;
+   }
+ 
+   @Override
+   public int available() throws IOException {
+     // We return the amount of bytes that we haven't read yet from the
+     // replica, based on our current position.  Some of the other block
+     // readers return a shorter length than that.  The only advantage to
+     // returning a shorter length is that the DFSInputStream will
+     // trash your block reader and create a new one if someone tries to
+     // seek() beyond the available() region.
+     long diff = visibleLength - pos;
+     if (diff > Integer.MAX_VALUE) {
+       return Integer.MAX_VALUE;
+     } else {
+       return (int)diff;
+     }
+   }
+ 
+   @Override
+   public void close() throws IOException {
+     accessor.close();
+   }
+ 
+   @Override
+   public void readFully(byte[] buf, int offset, int len) throws IOException {
+     BlockReaderUtil.readFully(this, buf, offset, len);
+   }
+ 
+   @Override
+   public int readAll(byte[] buf, int offset, int len) throws IOException {
+     return BlockReaderUtil.readAll(this, buf, offset, len);
+   }
+ 
+   @Override
+   public boolean isLocal() {
+     return accessor.isLocal();
+   }
+ 
+   @Override
+   public boolean isShortCircuit() {
+     return accessor.isShortCircuit();
+   }
+ 
+   @Override
+   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
+     // For now, pluggable ReplicaAccessors do not support zero-copy.
+     return null;
+   }
++
++  @Override
++  public DataChecksum getDataChecksum() {
++    return null;
++  }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index c083b5e,887accf..f292ee8
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@@ -445,17 -415,12 +431,17 @@@ public class PBHelper 
    }
  
    public static BlockWithLocationsProto convert(BlockWithLocations blk) {
 -    return BlockWithLocationsProto.newBuilder()
 -        .setBlock(convert(blk.getBlock()))
 +    BlockWithLocationsProto.Builder builder = BlockWithLocationsProto
 +        .newBuilder().setBlock(convert(blk.getBlock()))
          .addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids()))
          .addAllStorageUuids(Arrays.asList(blk.getStorageIDs()))
-         .addAllStorageTypes(convertStorageTypes(blk.getStorageTypes()));
 -        .addAllStorageTypes(PBHelperClient.convertStorageTypes(blk.getStorageTypes()))
 -        .build();
++        .addAllStorageTypes(PBHelperClient.convertStorageTypes(blk.getStorageTypes()));
 +    if (blk instanceof StripedBlockWithLocations) {
 +      StripedBlockWithLocations sblk = (StripedBlockWithLocations) blk;
 +      builder.setIndices(getByteString(sblk.getIndices()));
 +      builder.setDataBlockNum(sblk.getDataBlockNum());
 +    }
 +    return builder.build();
    }
  
    public static BlockWithLocations convert(BlockWithLocationsProto b) {
@@@ -806,26 -697,17 +723,26 @@@
  
      StorageType[] storageTypes = b.getStorageTypes();
      if (storageTypes != null) {
 -      for (int i = 0; i < storageTypes.length; ++i) {
 -        builder.addStorageTypes(PBHelperClient.convertStorageType(storageTypes[i]));
 +      for (StorageType storageType : storageTypes) {
-         builder.addStorageTypes(PBHelper.convertStorageType(storageType));
++        builder.addStorageTypes(PBHelperClient.convertStorageType(storageType));
        }
      }
      final String[] storageIDs = b.getStorageIDs();
      if (storageIDs != null) {
        builder.addAllStorageIDs(Arrays.asList(storageIDs));
      }
 +    if (b instanceof LocatedStripedBlock) {
 +      LocatedStripedBlock sb = (LocatedStripedBlock) b;
 +      int[] indices = sb.getBlockIndices();
 +      Token<BlockTokenIdentifier>[] blockTokens = sb.getBlockTokens();
 +      for (int i = 0; i < indices.length; i++) {
 +        builder.addBlockIndex(indices[i]);
-         builder.addBlockTokens(PBHelper.convert(blockTokens[i]));
++        builder.addBlockTokens(PBHelperClient.convert(blockTokens[i]));
 +      }
 +    }
  
-     return builder.setB(PBHelper.convert(b.getBlock()))
-         .setBlockToken(PBHelper.convert(b.getBlockToken()))
+     return builder.setB(PBHelperClient.convert(b.getBlock()))
+         .setBlockToken(PBHelperClient.convert(b.getBlockToken()))
          .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
    }
    
@@@ -3144,191 -2880,4 +2954,192 @@@
          setLeaseId(context.getLeaseId()).
          build();
    }
 +
 +  public static ECSchema convertECSchema(ECSchemaProto schema) {
 +    List<ECSchemaOptionEntryProto> optionsList = schema.getOptionsList();
 +    Map<String, String> options = new HashMap<>(optionsList.size());
 +    for (ECSchemaOptionEntryProto option : optionsList) {
 +      options.put(option.getKey(), option.getValue());
 +    }
 +    return new ECSchema(schema.getCodecName(), schema.getDataUnits(),
 +        schema.getParityUnits(), options);
 +  }
 +
 +  public static ECSchemaProto convertECSchema(ECSchema schema) {
 +    ECSchemaProto.Builder builder = ECSchemaProto.newBuilder()
 +        .setCodecName(schema.getCodecName())
 +        .setDataUnits(schema.getNumDataUnits())
 +        .setParityUnits(schema.getNumParityUnits());
 +    Set<Entry<String, String>> entrySet = schema.getExtraOptions().entrySet();
 +    for (Entry<String, String> entry : entrySet) {
 +      builder.addOptions(ECSchemaOptionEntryProto.newBuilder()
 +          .setKey(entry.getKey()).setValue(entry.getValue()).build());
 +    }
 +    return builder.build();
 +  }
 +
 +  public static ErasureCodingPolicy convertErasureCodingPolicy(
 +      ErasureCodingPolicyProto policy) {
 +    return new ErasureCodingPolicy(policy.getName(),
 +        convertECSchema(policy.getSchema()),
 +        policy.getCellSize());
 +  }
 +
 +  public static ErasureCodingPolicyProto convertErasureCodingPolicy(
 +      ErasureCodingPolicy policy) {
 +    ErasureCodingPolicyProto.Builder builder = ErasureCodingPolicyProto
 +        .newBuilder()
 +        .setName(policy.getName())
 +        .setSchema(convertECSchema(policy.getSchema()))
 +        .setCellSize(policy.getCellSize());
 +    return builder.build();
 +  }
 +
 +  public static ErasureCodingZoneProto convertErasureCodingZone(
 +      ErasureCodingZone ecZone) {
 +    return ErasureCodingZoneProto.newBuilder().setDir(ecZone.getDir())
 +        .setEcPolicy(convertErasureCodingPolicy(ecZone.getErasureCodingPolicy()))
 +        .build();
 +  }
 +
 +  public static ErasureCodingZone convertErasureCodingZone(
 +      ErasureCodingZoneProto ecZoneProto) {
 +    return new ErasureCodingZone(ecZoneProto.getDir(),
 +        convertErasureCodingPolicy(ecZoneProto.getEcPolicy()));
 +  }
 +  
 +  public static BlockECRecoveryInfo convertBlockECRecoveryInfo(
 +      BlockECRecoveryInfoProto blockEcRecoveryInfoProto) {
 +    ExtendedBlockProto blockProto = blockEcRecoveryInfoProto.getBlock();
 +    ExtendedBlock block = convert(blockProto);
 +
 +    DatanodeInfosProto sourceDnInfosProto = blockEcRecoveryInfoProto
 +        .getSourceDnInfos();
 +    DatanodeInfo[] sourceDnInfos = convert(sourceDnInfosProto);
 +
 +    DatanodeInfosProto targetDnInfosProto = blockEcRecoveryInfoProto
 +        .getTargetDnInfos();
 +    DatanodeInfo[] targetDnInfos = convert(targetDnInfosProto);
 +
 +    StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto
 +        .getTargetStorageUuids();
 +    String[] targetStorageUuids = convert(targetStorageUuidsProto);
 +
 +    StorageTypesProto targetStorageTypesProto = blockEcRecoveryInfoProto
 +        .getTargetStorageTypes();
 +    StorageType[] convertStorageTypes = convertStorageTypes(
 +        targetStorageTypesProto.getStorageTypesList(), targetStorageTypesProto
 +            .getStorageTypesList().size());
 +
 +    List<Integer> liveBlockIndicesList = blockEcRecoveryInfoProto
 +        .getLiveBlockIndicesList();
 +    short[] liveBlkIndices = new short[liveBlockIndicesList.size()];
 +    for (int i = 0; i < liveBlockIndicesList.size(); i++) {
 +      liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue();
 +    }
 +
 +    ErasureCodingPolicy ecPolicy =
 +        convertErasureCodingPolicy(blockEcRecoveryInfoProto.getEcPolicy());
 +
 +    return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos,
 +        targetStorageUuids, convertStorageTypes, liveBlkIndices, ecPolicy);
 +  }
 +
 +  public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo(
 +      BlockECRecoveryInfo blockEcRecoveryInfo) {
 +    BlockECRecoveryInfoProto.Builder builder = BlockECRecoveryInfoProto
 +        .newBuilder();
-     builder.setBlock(convert(blockEcRecoveryInfo.getExtendedBlock()));
++    builder.setBlock(PBHelperClient.convert(
++        blockEcRecoveryInfo.getExtendedBlock()));
 +
 +    DatanodeInfo[] sourceDnInfos = blockEcRecoveryInfo.getSourceDnInfos();
 +    builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos));
 +
 +    DatanodeInfo[] targetDnInfos = blockEcRecoveryInfo.getTargetDnInfos();
 +    builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos));
 +
 +    String[] targetStorageIDs = blockEcRecoveryInfo.getTargetStorageIDs();
 +    builder.setTargetStorageUuids(convertStorageIDs(targetStorageIDs));
 +
 +    StorageType[] targetStorageTypes = blockEcRecoveryInfo
 +        .getTargetStorageTypes();
 +    builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes));
 +
 +    short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices();
 +    builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices));
 +
 +    builder.setEcPolicy(convertErasureCodingPolicy(blockEcRecoveryInfo
 +        .getErasureCodingPolicy()));
 +
 +    return builder.build();
 +  }
 +
 +  private static List<Integer> convertIntArray(short[] liveBlockIndices) {
 +    List<Integer> liveBlockIndicesList = new ArrayList<Integer>();
 +    for (short s : liveBlockIndices) {
 +      liveBlockIndicesList.add((int) s);
 +    }
 +    return liveBlockIndicesList;
 +  }
 +
 +  private static StorageTypesProto convertStorageTypesProto(
 +      StorageType[] targetStorageTypes) {
 +    StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
 +    for (StorageType storageType : targetStorageTypes) {
-       builder.addStorageTypes(convertStorageType(storageType));
++      builder.addStorageTypes(PBHelperClient.convertStorageType(storageType));
 +    }
 +    return builder.build();
 +  }
 +
 +  private static StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) {
 +    StorageUuidsProto.Builder builder = StorageUuidsProto.newBuilder();
 +    for (String storageUuid : targetStorageIDs) {
 +      builder.addStorageUuids(storageUuid);
 +    }
 +    return builder.build();
 +  }
 +
 +  private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) {
 +    DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder();
 +    for (DatanodeInfo datanodeInfo : dnInfos) {
-       builder.addDatanodes(convert(datanodeInfo));
++      builder.addDatanodes(PBHelperClient.convert(datanodeInfo));
 +    }
 +    return builder.build();
 +  }
 +
 +  private static String[] convert(StorageUuidsProto targetStorageUuidsProto) {
 +    List<String> storageUuidsList = targetStorageUuidsProto
 +        .getStorageUuidsList();
 +    String[] storageUuids = new String[storageUuidsList.size()];
 +    for (int i = 0; i < storageUuidsList.size(); i++) {
 +      storageUuids[i] = storageUuidsList.get(i);
 +    }
 +    return storageUuids;
 +  }
 +  
 +  public static BlockECRecoveryCommandProto convert(
 +      BlockECRecoveryCommand blkECRecoveryCmd) {
 +    BlockECRecoveryCommandProto.Builder builder = BlockECRecoveryCommandProto
 +        .newBuilder();
 +    Collection<BlockECRecoveryInfo> blockECRecoveryInfos = blkECRecoveryCmd
 +        .getECTasks();
 +    for (BlockECRecoveryInfo blkECRecoveryInfo : blockECRecoveryInfos) {
 +      builder
 +          .addBlockECRecoveryinfo(convertBlockECRecoveryInfo(blkECRecoveryInfo));
 +    }
 +    return builder.build();
 +  }
 +  
 +  public static BlockECRecoveryCommand convert(
 +      BlockECRecoveryCommandProto blkECRecoveryCmdProto) {
 +    Collection<BlockECRecoveryInfo> blkECRecoveryInfos = new ArrayList<BlockECRecoveryInfo>();
 +    List<BlockECRecoveryInfoProto> blockECRecoveryinfoList = blkECRecoveryCmdProto
 +        .getBlockECRecoveryinfoList();
 +    for (BlockECRecoveryInfoProto blockECRecoveryInfoProto : blockECRecoveryinfoList) {
 +      blkECRecoveryInfos
 +          .add(convertBlockECRecoveryInfo(blockECRecoveryInfoProto));
 +    }
 +    return new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY,
 +        blkECRecoveryInfos);
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index 555f506,be1a9ef..07c3c01
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@@ -17,8 -17,7 +17,8 @@@
   */
  package org.apache.hadoop.hdfs.server.balancer;
  
- import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
 +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
+ import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
  
  import java.io.BufferedInputStream;
  import java.io.BufferedOutputStream;
@@@ -64,10 -62,8 +63,9 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
  import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
  import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
- import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
  import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
  import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
  import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
  import org.apache.hadoop.io.IOUtils;
  import org.apache.hadoop.net.NetUtils;
@@@ -857,10 -794,14 +854,14 @@@ public class Dispatcher 
          if (shouldFetchMoreBlocks()) {
            // fetch new blocks
            try {
-             blocksToReceive -= getBlockList();
+             final long received = getBlockList();
+             if (received == 0) {
+               return;
+             }
+             blocksToReceive -= received;
              continue;
            } catch (IOException e) {
 -            LOG.warn("Exception while getting block list", e);
 +            LOG.warn("Exception while getting reportedBlock list", e);
              return;
            }
          } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
index 928424b,95d9983..2f214be
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
@@@ -55,12 -55,6 +55,12 @@@ public interface BlockCollection 
    public long getPreferredBlockSize();
  
    /**
 +   * Get block replication for the collection.
 +   * @return block replication value. Return 0 if the file is erasure coded.
 +   */
 +  public short getPreferredBlockReplication();
 +
-   /** 
++  /**
     * @return the storage policy ID.
     */
    public byte getStoragePolicyID();
@@@ -88,7 -81,7 +88,12 @@@
    public boolean isUnderConstruction();
  
    /**
 +   * @return whether the block collection is in striping format
 +   */
-   public boolean isStriped();
++  boolean isStriped();
++
++  /**
+    * @return the id for the block collection
+    */
+   long getId();
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index f440e14,706cbcd..dc296ac
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@@ -19,25 -19,39 +19,38 @@@ package org.apache.hadoop.hdfs.server.b
  
  import java.io.IOException;
  import java.util.LinkedList;
+ import java.util.List;
  
  import com.google.common.base.Preconditions;
 -import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.hdfs.protocol.Block;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  import org.apache.hadoop.util.LightWeightGSet;
  
+ import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID;
+ 
  /**
 - * BlockInfo class maintains for a given block
 - * the {@link BlockCollection} it is part of and datanodes where the replicas of
 - * the block are stored.
 + * For a given block (or an erasure coding block group), BlockInfo class
 + * maintains 1) the {@link BlockCollection} it is part of, and 2) datanodes
 + * where the replicas of the block, or blocks belonging to the erasure coding
 + * block group, are stored.
   */
 -@InterfaceAudience.Private
 -public abstract class  BlockInfo extends Block
 +public abstract class BlockInfo extends Block
      implements LightWeightGSet.LinkedElement {
+ 
    public static final BlockInfo[] EMPTY_ARRAY = {};
  
-   private BlockCollection bc;
+   /**
+    * Replication factor.
+    */
+   private short replication;
+ 
+   /**
+    * Block collection ID.
+    */
+   private long bcId;
  
-   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
+   /** For implementing {@link LightWeightGSet.LinkedElement} interface. */
    private LightWeightGSet.LinkedElement nextLinkedElement;
  
    /**
@@@ -58,26 -72,48 +71,40 @@@
  
    /**
     * Construct an entry for blocksmap
 -   * @param replication the block's replication factor
 +   * @param size the block's replication factor, or the total number of blocks
 +   *             in the block group
     */
 -  public BlockInfo(short replication) {
 -    this.triplets = new Object[3*replication];
 +  public BlockInfo(short size) {
 +    this.triplets = new Object[3 * size];
-     this.bc = null;
+     this.bcId = INVALID_INODE_ID;
 -    this.replication = replication;
++    this.replication = isStriped() ? 0 : size;
    }
  
 -  public BlockInfo(Block blk, short replication) {
 +  public BlockInfo(Block blk, short size) {
      super(blk);
-     this.triplets = new Object[3 * size];
-     this.bc = null;
 -    this.triplets = new Object[3*replication];
++    this.triplets = new Object[3*size];
+     this.bcId = INVALID_INODE_ID;
 -    this.replication = replication;
 -  }
 -
 -  /**
 -   * Copy construction.
 -   * @param from BlockInfo to copy from.
 -   */
 -  protected BlockInfo(BlockInfo from) {
 -    this(from, from.getReplication());
 -    this.bcId = from.bcId;
++    this.replication = isStriped() ? 0 : size;
+   }
+ 
+   public short getReplication() {
+     return replication;
    }
  
-   public BlockCollection getBlockCollection() {
-     return bc;
+   public void setReplication(short repl) {
+     this.replication = repl;
    }
  
-   public void setBlockCollection(BlockCollection bc) {
-     this.bc = bc;
+   public long getBlockCollectionId() {
+     return bcId;
+   }
+ 
+   public void setBlockCollectionId(long id) {
+     this.bcId = id;
+   }
+ 
+   public boolean isDeleted() {
+     return bcId == INVALID_INODE_ID;
    }
  
    public DatanodeDescriptor getDatanode(int index) {
@@@ -342,7 -363,7 +365,8 @@@
    public void convertToBlockUnderConstruction(BlockUCState s,
        DatanodeStorageInfo[] targets) {
      if (isComplete()) {
-       uc = new BlockUnderConstructionFeature(this, s, targets, this.isStriped());
 -      uc = new BlockUnderConstructionFeature(this, s, targets);
++      uc = new BlockUnderConstructionFeature(this, s, targets,
++          this.isStriped());
      } else {
        // the block is already under construction
        uc.setBlockUCState(s);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index 12b4fd3,42934c3..b9d8486
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@@ -35,17 -36,14 +35,6 @@@ public class BlockInfoContiguous extend
    }
  
    /**
-    * Copy construction. This is used to convert
-    * BlockReplicationInfoUnderConstruction
-    *
-    * @param from BlockReplicationInfo to copy from.
 -   * Copy construction.
 -   * @param from BlockInfoContiguous to copy from.
--   */
--  protected BlockInfoContiguous(BlockInfoContiguous from) {
-     this(from, (short) (from.triplets.length / 3));
-     this.setBlockCollection(from.getBlockCollection());
 -    super(from);
--  }
--
--  /**
     * Ensure that there is enough  space to include num more triplets.
     * @return first free triplet index.
     */


[14/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 6c6d758,1346ab3..8232ab9
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@@ -674,8 -648,8 +674,8 @@@ public class BlockManager implements Bl
        return false; // already completed (e.g. by syncBlock)
      
      final boolean b = commitBlock(lastBlock, commitBlock);
-     if (hasMinStorage(lastBlock)) {
-       completeBlock(bc, bc.numBlocks() - 1, false);
 -    if (countNodes(lastBlock).liveReplicas() >= minReplication) {
++      if (hasMinStorage(lastBlock)) {
+       completeBlock(lastBlock, false);
      }
      return b;
    }
@@@ -698,9 -666,9 +692,9 @@@
      }
  
      int numNodes = curBlock.numNodes();
 -    if (!force && numNodes < minReplication) {
 +    if (!force && !hasMinStorage(curBlock, numNodes)) {
-       throw new IOException("Cannot complete block: " +
-           "block does not satisfy minimal replication requirement.");
+       throw new IOException("Cannot complete block: "
+           + "block does not satisfy minimal replication requirement.");
      }
      if (!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
        throw new IOException(
@@@ -718,26 -683,10 +709,12 @@@
      // a "forced" completion when a file is getting closed by an
      // OP_CLOSE edit on the standby).
      namesystem.adjustSafeModeBlockTotals(0, 1);
 +    final int minStorage = curBlock.isStriped() ?
 +        ((BlockInfoStriped) curBlock).getRealDataBlockNum() : minReplication;
      namesystem.incrementSafeBlockCount(
 -        Math.min(numNodes, minReplication));
 +        Math.min(numNodes, minStorage), curBlock);
-     
-     // replace block in the blocksMap
-     return blocksMap.replaceBlock(completeBlock);
    }
  
-   private BlockInfo completeBlock(final BlockCollection bc,
-       final BlockInfo block, boolean force) throws IOException {
-     BlockInfo[] fileBlocks = bc.getBlocks();
-     for (int idx = 0; idx < fileBlocks.length; idx++) {
-       if (fileBlocks[idx] == block) {
-         return completeBlock(bc, idx, force);
-       }
-     }
-     return block;
-   }
-   
    /**
     * Force the given block in the given file to be marked as complete,
     * regardless of whether enough replicas are present. This is necessary
@@@ -1270,37 -1162,29 +1245,36 @@@
    private void markBlockAsCorrupt(BlockToMarkCorrupt b,
        DatanodeStorageInfo storageInfo,
        DatanodeDescriptor node) throws IOException {
--
-     if (b.stored.isDeleted()) {
 -    if (b.getCorrupted().isDeleted()) {
++    if (b.getStored().isDeleted()) {
        blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
            " corrupt as it does not belong to any file", b);
-       addToInvalidates(b.corrupted, node);
+       addToInvalidates(b.getCorrupted(), node);
        return;
 -    } 
 -    short expectedReplicas = b.getCorrupted().getReplication();
 +    }
 +    short expectedReplicas =
-         getExpectedReplicaNum(b.stored);
++        getExpectedReplicaNum(b.getStored());
  
      // Add replica to the data-node if it is not already there
      if (storageInfo != null) {
-       storageInfo.addBlock(b.stored, b.corrupted);
 -      storageInfo.addBlock(b.getStored());
++      storageInfo.addBlock(b.getStored(), b.getCorrupted());
      }
  
 -    // Add this replica to corruptReplicas Map
 -    corruptReplicas.addToCorruptReplicasMap(b.getCorrupted(), node,
 -        b.getReason(), b.getReasonCode());
 +    // Add this replica to corruptReplicas Map. For striped blocks, we always
 +    // use the id of whole striped block group when adding to corruptReplicas
-     Block corrupted = new Block(b.corrupted);
-     if (b.stored.isStriped()) {
-       corrupted.setBlockId(b.stored.getBlockId());
++    Block corrupted = new Block(b.getCorrupted());
++    if (b.getStored().isStriped()) {
++      corrupted.setBlockId(b.getStored().getBlockId());
 +    }
-     corruptReplicas.addToCorruptReplicasMap(corrupted, node, b.reason,
-         b.reasonCode);
++    corruptReplicas.addToCorruptReplicasMap(corrupted, node, b.getReason(),
++        b.getReasonCode());
  
-     NumberReplicas numberOfReplicas = countNodes(b.stored);
+     NumberReplicas numberOfReplicas = countNodes(b.getStored());
      boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >=
          expectedReplicas;
 -    boolean minReplicationSatisfied =
 -        numberOfReplicas.liveReplicas() >= minReplication;
 +
-     boolean minReplicationSatisfied = hasMinStorage(b.stored,
++    boolean minReplicationSatisfied = hasMinStorage(b.getStored(),
 +        numberOfReplicas.liveReplicas());
 +
      boolean hasMoreCorruptReplicas = minReplicationSatisfied &&
          (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) >
          expectedReplicas;
@@@ -1315,10 -1199,10 +1289,10 @@@
      if (hasEnoughLiveReplicas || hasMoreCorruptReplicas
          || corruptedDuringWrite) {
        // the block is over-replicated so invalidate the replicas immediately
 -      invalidateBlock(b, node);
 +      invalidateBlock(b, node, numberOfReplicas);
      } else if (namesystem.isPopulatingReplQueues()) {
        // add the block to neededReplication
-       updateNeededReplications(b.stored, -1, 0);
+       updateNeededReplications(b.getStored(), -1, 0);
      }
    }
  
@@@ -1342,13 -1227,12 +1316,13 @@@
            "invalidation of {} on {} because {} replica(s) are located on " +
            "nodes with potentially out-of-date block reports", b, dn,
            nr.replicasOnStaleNodes());
-       postponeBlock(b.corrupted);
+       postponeBlock(b.getCorrupted());
        return false;
 -    } else if (nr.liveReplicas() >= 1) {
 -      // If we have at least one copy on a live node, then we can delete it.
 +    } else {
 +      // we already checked the number of replicas in the caller of this
 +      // function and know there are enough live replicas, so we can delete it.
-       addToInvalidates(b.corrupted, dn);
-       removeStoredBlock(b.stored, node);
+       addToInvalidates(b.getCorrupted(), dn);
+       removeStoredBlock(b.getStored(), node);
        blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.",
            b, dn);
        return true;
@@@ -1446,72 -1326,11 +1420,11 @@@
      namesystem.writeLock();
      try {
        synchronized (neededReplications) {
 -        for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
 -          for (BlockInfo block : blocksToReplicate.get(priority)) {
 -            ReplicationWork rw = scheduleReplication(block, priority);
 +        for (int priority = 0; priority < blocksToRecover.size(); priority++) {
 +          for (BlockInfo block : blocksToRecover.get(priority)) {
-             // block should belong to a file
-             bc = getBlockCollection(block);
-             // abandoned block or block reopened for append
-             if (bc == null
-                 || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
-               // remove from neededReplications
-               neededReplications.remove(block, priority);
-               continue;
-             }
- 
-             requiredReplication = getExpectedReplicaNum(block);
- 
-             // get a source data-node
-             containingNodes = new ArrayList<>();
-             List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
-             NumberReplicas numReplicas = new NumberReplicas();
-             List<Short> liveBlockIndices = new ArrayList<>();
-             final DatanodeDescriptor[] srcNodes = chooseSourceDatanodes(block,
-                 containingNodes, liveReplicaNodes, numReplicas,
-                 liveBlockIndices, priority);
-             if(srcNodes == null || srcNodes.length == 0) {
-               // block can not be replicated from any node
-               LOG.debug("Block " + block + " cannot be recovered " +
-                   "from any node");
-               continue;
-             }
- 
-             // liveReplicaNodes can include READ_ONLY_SHARED replicas which are
-             // not included in the numReplicas.liveReplicas() count
-             assert liveReplicaNodes.size() >= numReplicas.liveReplicas();
- 
-             // do not schedule more if enough replicas is already pending
-             numEffectiveReplicas = numReplicas.liveReplicas() +
-                                     pendingReplications.getNumReplicas(block);
- 
-             if (numEffectiveReplicas >= requiredReplication) {
-               if ( (pendingReplications.getNumReplicas(block) > 0) ||
-                    (blockHasEnoughRacks(block, requiredReplication)) ) {
-                 neededReplications.remove(block, priority); // remove from neededReplications
-                 blockLog.debug("BLOCK* Removing {} from neededReplications as" +
-                         " it has enough replicas", block);
-                 continue;
-               }
-             }
- 
-             if (numReplicas.liveReplicas() < requiredReplication) {
-               additionalReplRequired = requiredReplication
-                   - numEffectiveReplicas;
-             } else {
-               additionalReplRequired = 1; // Needed on a new rack
-             }
-             if (block.isStriped()) {
-               short[] indices = new short[liveBlockIndices.size()];
-               for (int i = 0 ; i < liveBlockIndices.size(); i++) {
-                 indices[i] = liveBlockIndices.get(i);
-               }
-               ErasureCodingWork ecw = new ErasureCodingWork(block, bc, srcNodes,
-                   containingNodes, liveReplicaNodes, additionalReplRequired,
-                   priority, indices);
-               recovWork.add(ecw);
-             } else {
-               recovWork.add(new ReplicationWork(block, bc, srcNodes,
-                   containingNodes, liveReplicaNodes, additionalReplRequired,
-                   priority));
++            BlockRecoveryWork rw = scheduleRecovery(block, priority);
+             if (rw != null) {
 -              work.add(rw);
++              recovWork.add(rw);
              }
            }
          }
@@@ -1520,9 -1339,8 +1433,9 @@@
        namesystem.writeUnlock();
      }
  
 +    // Step 2: choose target nodes for each recovery task
-     final Set<Node> excludedNodes = new HashSet<Node>();
+     final Set<Node> excludedNodes = new HashSet<>();
 -    for(ReplicationWork rw : work){
 +    for(BlockRecoveryWork rw : recovWork){
        // Exclude all of the containing nodes from being targets.
        // This list includes decommissioning or corrupt nodes.
        excludedNodes.clear();
@@@ -1533,101 -1351,21 +1446,24 @@@
        // choose replication targets: NOT HOLDING THE GLOBAL LOCK
        // It is costly to extract the filename for which chooseTargets is called,
        // so for now we pass in the block collection itself.
 -      rw.chooseTargets(blockplacement, storagePolicySuite, excludedNodes);
 +      final BlockPlacementPolicy placementPolicy =
-           placementPolicies.getPolicy(rw.block.isStriped());
++          placementPolicies.getPolicy(rw.getBlock().isStriped());
 +      rw.chooseTargets(placementPolicy, storagePolicySuite, excludedNodes);
      }
  
 +    // Step 3: add tasks to the DN
      namesystem.writeLock();
      try {
 -      for(ReplicationWork rw : work){
 +      for(BlockRecoveryWork rw : recovWork){
-         final DatanodeStorageInfo[] targets = rw.targets;
+         final DatanodeStorageInfo[] targets = rw.getTargets();
          if(targets == null || targets.length == 0){
-           rw.targets = null;
+           rw.resetTargets();
            continue;
          }
  
          synchronized (neededReplications) {
-           BlockInfo block = rw.block;
-           int priority = rw.priority;
-           // Recheck since global lock was released
-           // block should belong to a file
-           bc = getBlockCollection(block);
-           // abandoned block or block reopened for append
-           if(bc == null || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
-             neededReplications.remove(block, priority); // remove from neededReplications
-             rw.targets = null;
-             continue;
-           }
-           requiredReplication = getExpectedReplicaNum(block);
- 
-           // do not schedule more if enough replicas is already pending
-           NumberReplicas numReplicas = countNodes(block);
-           numEffectiveReplicas = numReplicas.liveReplicas() +
-             pendingReplications.getNumReplicas(block);
- 
-           if (numEffectiveReplicas >= requiredReplication) {
-             if ( (pendingReplications.getNumReplicas(block) > 0) ||
-                  (blockHasEnoughRacks(block, requiredReplication)) ) {
-               neededReplications.remove(block, priority); // remove from neededReplications
-               rw.targets = null;
-               blockLog.debug("BLOCK* Removing {} from neededReplications as" +
-                       " it has enough replicas", block);
-               continue;
-             }
-           }
- 
-           if ( (numReplicas.liveReplicas() >= requiredReplication) &&
-                (!blockHasEnoughRacks(block, requiredReplication)) ) {
-             if (rw.srcNodes[0].getNetworkLocation().equals(
-                 targets[0].getDatanodeDescriptor().getNetworkLocation())) {
-               //No use continuing, unless a new rack in this case
-               continue;
-             }
-           }
- 
-           // Add block to the to be replicated list
-           if (block.isStriped()) {
-             assert rw instanceof ErasureCodingWork;
-             assert rw.targets.length > 0;
-             String src = getBlockCollection(block).getName();
-             ErasureCodingZone ecZone = null;
-             try {
-               ecZone = namesystem.getErasureCodingZoneForPath(src);
-             } catch (IOException e) {
-               blockLog
-                   .warn("Failed to get the EC zone for the file {} ", src);
-             }
-             if (ecZone == null) {
-               blockLog.warn("No erasure coding policy found for the file {}. "
-                   + "So cannot proceed for recovery", src);
-               // TODO: we may have to revisit later for what we can do better to
-               // handle this case.
-               continue;
-             }
-             rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
-                 new ExtendedBlock(namesystem.getBlockPoolId(), block),
-                 rw.srcNodes, rw.targets,
-                 ((ErasureCodingWork) rw).liveBlockIndicies,
-                 ecZone.getErasureCodingPolicy());
-           } else {
-             rw.srcNodes[0].addBlockToBeReplicated(block, targets);
-           }
-           scheduledWork++;
-           DatanodeStorageInfo.incrementBlocksScheduled(targets);
- 
-           // Move the block-replication into a "pending" state.
-           // The reason we use 'pending' is so we can retry
-           // replications that fail after an appropriate amount of time.
-           pendingReplications.increment(block,
-               DatanodeStorageInfo.toDatanodeDescriptors(targets));
-           blockLog.debug("BLOCK* block {} is moved from neededReplications to "
-                   + "pendingReplications", block);
- 
-           // remove from neededReplications
-           if(numEffectiveReplicas + targets.length >= requiredReplication) {
-             neededReplications.remove(block, priority); // remove from neededReplications
 -          if (validateReplicationWork(rw)) {
++          if (validateRecoveryWork(rw)) {
+             scheduledWork++;
            }
          }
        }
@@@ -1637,16 -1375,16 +1473,16 @@@
  
      if (blockLog.isInfoEnabled()) {
        // log which blocks have been scheduled for replication
 -      for(ReplicationWork rw : work){
 +      for(BlockRecoveryWork rw : recovWork){
-         DatanodeStorageInfo[] targets = rw.targets;
+         DatanodeStorageInfo[] targets = rw.getTargets();
          if (targets != null && targets.length != 0) {
            StringBuilder targetList = new StringBuilder("datanode(s)");
            for (DatanodeStorageInfo target : targets) {
              targetList.append(' ');
              targetList.append(target.getDatanodeDescriptor());
            }
-           blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.srcNodes,
-               rw.block, targetList);
 -          blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.getSrcNode(),
++          blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.getSrcNodes(),
+               rw.getBlock(), targetList);
          }
        }
      }
@@@ -1658,6 -1396,118 +1494,160 @@@
      return scheduledWork;
    }
  
+   boolean hasEnoughEffectiveReplicas(BlockInfo block,
+       NumberReplicas numReplicas, int pendingReplicaNum, int required) {
+     int numEffectiveReplicas = numReplicas.liveReplicas() + pendingReplicaNum;
+     return (numEffectiveReplicas >= required) &&
 -        (pendingReplicaNum > 0 || blockHasEnoughRacks(block));
++        (pendingReplicaNum > 0 || blockHasEnoughRacks(block, required));
+   }
+ 
 -  private ReplicationWork scheduleReplication(BlockInfo block, int priority) {
++  private BlockRecoveryWork scheduleRecovery(BlockInfo block, int priority) {
+     // block should belong to a file
+     BlockCollection bc = getBlockCollection(block);
+     // abandoned block or block reopened for append
+     if (bc == null
+         || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
+       // remove from neededReplications
+       neededReplications.remove(block, priority);
+       return null;
+     }
+ 
+     short requiredReplication = getExpectedReplicaNum(block);
+ 
+     // get a source data-node
+     List<DatanodeDescriptor> containingNodes = new ArrayList<>();
+     List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
+     NumberReplicas numReplicas = new NumberReplicas();
 -    DatanodeDescriptor srcNode = chooseSourceDatanode(block, containingNodes,
 -        liveReplicaNodes, numReplicas, priority);
 -    if (srcNode == null) { // block can not be replicated from any node
 -      LOG.debug("Block " + block + " cannot be repl from any node");
++    List<Short> liveBlockIndices = new ArrayList<>();
++    final DatanodeDescriptor[] srcNodes = chooseSourceDatanodes(block,
++        containingNodes, liveReplicaNodes, numReplicas,
++        liveBlockIndices, priority);
++    if(srcNodes == null || srcNodes.length == 0) {
++      // block can not be recovered from any node
++      LOG.debug("Block " + block + " cannot be recovered " +
++          "from any node");
+       return null;
+     }
+ 
+     // liveReplicaNodes can include READ_ONLY_SHARED replicas which are
+     // not included in the numReplicas.liveReplicas() count
+     assert liveReplicaNodes.size() >= numReplicas.liveReplicas();
+ 
+     int pendingNum = pendingReplications.getNumReplicas(block);
+     if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum,
+         requiredReplication)) {
+       neededReplications.remove(block, priority);
+       blockLog.debug("BLOCK* Removing {} from neededReplications as" +
+           " it has enough replicas", block);
+       return null;
+     }
+ 
+     final int additionalReplRequired;
+     if (numReplicas.liveReplicas() < requiredReplication) {
+       additionalReplRequired = requiredReplication - numReplicas.liveReplicas()
+           - pendingNum;
+     } else {
+       additionalReplRequired = 1; // Needed on a new rack
+     }
 -    return new ReplicationWork(block, bc, srcNode, containingNodes,
 -        liveReplicaNodes, additionalReplRequired, priority);
++
++    if (block.isStriped()) {
++      short[] indices = new short[liveBlockIndices.size()];
++      for (int i = 0 ; i < liveBlockIndices.size(); i++) {
++        indices[i] = liveBlockIndices.get(i);
++      }
++      return new ErasureCodingWork(block, bc, srcNodes,
++          containingNodes, liveReplicaNodes, additionalReplRequired,
++          priority, indices);
++    } else {
++      return new ReplicationWork(block, bc, srcNodes,
++          containingNodes, liveReplicaNodes, additionalReplRequired,
++          priority);
++    }
+   }
+ 
 -  private boolean validateReplicationWork(ReplicationWork rw) {
++  private boolean validateRecoveryWork(BlockRecoveryWork rw) {
+     BlockInfo block = rw.getBlock();
+     int priority = rw.getPriority();
+     // Recheck since global lock was released
+     // block should belong to a file
+     BlockCollection bc = getBlockCollection(block);
+     // abandoned block or block reopened for append
+     if (bc == null
+         || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
+       neededReplications.remove(block, priority);
+       rw.resetTargets();
+       return false;
+     }
+ 
+     // do not schedule more if enough replicas is already pending
+     final short requiredReplication = getExpectedReplicaNum(block);
+     NumberReplicas numReplicas = countNodes(block);
+     final int pendingNum = pendingReplications.getNumReplicas(block);
+     if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum,
+         requiredReplication)) {
+       neededReplications.remove(block, priority);
+       rw.resetTargets();
+       blockLog.debug("BLOCK* Removing {} from neededReplications as" +
+           " it has enough replicas", block);
+       return false;
+     }
+ 
+     DatanodeStorageInfo[] targets = rw.getTargets();
+     if ( (numReplicas.liveReplicas() >= requiredReplication) &&
 -        (!blockHasEnoughRacks(block)) ) {
 -      if (rw.getSrcNode().getNetworkLocation().equals(
++        (!blockHasEnoughRacks(block, requiredReplication)) ) {
++      if (rw.getSrcNodes()[0].getNetworkLocation().equals(
+           targets[0].getDatanodeDescriptor().getNetworkLocation())) {
+         //No use continuing, unless a new rack in this case
+         return false;
+       }
+     }
+ 
 -    // Add block to the to be replicated list
 -    rw.getSrcNode().addBlockToBeReplicated(block, targets);
++    // Add block to the to be recovered list
++    if (block.isStriped()) {
++      assert rw instanceof ErasureCodingWork;
++      assert rw.getTargets().length > 0;
++      String src = getBlockCollection(block).getName();
++      ErasureCodingZone ecZone = null;
++      try {
++        ecZone = namesystem.getErasureCodingZoneForPath(src);
++      } catch (IOException e) {
++        blockLog
++            .warn("Failed to get the EC zone for the file {} ", src);
++      }
++      if (ecZone == null) {
++        blockLog.warn("No erasure coding policy found for the file {}. "
++            + "So cannot proceed for recovery", src);
++        // TODO: we may have to revisit later for what we can do better to
++        // handle this case.
++        return false;
++      }
++      rw.getTargets()[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
++          new ExtendedBlock(namesystem.getBlockPoolId(), block),
++          rw.getSrcNodes(), rw.getTargets(),
++          ((ErasureCodingWork) rw).getLiveBlockIndicies(),
++          ecZone.getErasureCodingPolicy());
++    } else {
++      rw.getSrcNodes()[0].addBlockToBeReplicated(block, targets);
++    }
++
+     DatanodeStorageInfo.incrementBlocksScheduled(targets);
+ 
+     // Move the block-replication into a "pending" state.
+     // The reason we use 'pending' is so we can retry
+     // replications that fail after an appropriate amount of time.
+     pendingReplications.increment(block,
+         DatanodeStorageInfo.toDatanodeDescriptors(targets));
+     blockLog.debug("BLOCK* block {} is moved from neededReplications to "
+         + "pendingReplications", block);
+ 
+     int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum;
+     // remove from neededReplications
+     if(numEffectiveReplicas + targets.length >= requiredReplication) {
+       neededReplications.remove(block, priority);
+     }
+     return true;
+   }
+ 
    /** Choose target for WebHDFS redirection. */
    public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src,
        DatanodeDescriptor clientnode, Set<Node> excludes, long blocksize) {
@@@ -1916,59 -1755,7 +1906,17 @@@
      }
    }
  
 +  private static class BlockInfoToAdd {
 +    final BlockInfo stored;
 +    final Block reported;
 +
 +    BlockInfoToAdd(BlockInfo stored, Block reported) {
 +      this.stored = stored;
 +      this.reported = reported;
 +    }
 +  }
 +
    /**
-    * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a
-    * list of blocks that should be considered corrupt due to a block report.
-    */
-   private static class BlockToMarkCorrupt {
-     /**
-      * The corrupted block in a datanode. This is the one reported by the
-      * datanode.
-      */
-     final Block corrupted;
-     /** The corresponding block stored in the BlockManager. */
-     final BlockInfo stored;
-     /** The reason to mark corrupt. */
-     final String reason;
-     /** The reason code to be stored */
-     final Reason reasonCode;
- 
-     BlockToMarkCorrupt(Block corrupted, BlockInfo stored, String reason,
-         Reason reasonCode) {
-       Preconditions.checkNotNull(corrupted, "corrupted is null");
-       Preconditions.checkNotNull(stored, "stored is null");
- 
-       this.corrupted = corrupted;
-       this.stored = stored;
-       this.reason = reason;
-       this.reasonCode = reasonCode;
-     }
- 
-     BlockToMarkCorrupt(Block corrupted, BlockInfo stored, long gs,
-         String reason, Reason reasonCode) {
-       this(corrupted, stored, reason, reasonCode);
-       //the corrupted block in datanode has a different generation stamp
-       corrupted.setGenerationStamp(gs);
-     }
- 
-     @Override
-     public String toString() {
-       return corrupted + "("
-           + (corrupted == stored? "same as stored": "stored=" + stored) + ")";
-     }
-   }
- 
-   /**
     * The given storage is reporting all its blocks.
     * Update the (storage-->block list) and (block-->storage list) maps.
     *
@@@ -2721,8 -2484,8 +2669,8 @@@
      // Now check for completion of blocks and safe block count
      int numCurrentReplica = countLiveNodes(storedBlock);
      if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
 -        && numCurrentReplica >= minReplication) {
 +        && hasMinStorage(storedBlock, numCurrentReplica)) {
-       completeBlock(getBlockCollection(storedBlock), storedBlock, false);
+       completeBlock(storedBlock, false);
      } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
        // check whether safe replication is reached for the block
        // only complete blocks are counted towards that.
@@@ -2796,8 -2558,8 +2744,8 @@@
        + pendingReplications.getNumReplicas(storedBlock);
  
      if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
 -        numLiveReplicas >= minReplication) {
 +        hasMinStorage(storedBlock, numLiveReplicas)) {
-       storedBlock = completeBlock(bc, storedBlock, false);
+       completeBlock(storedBlock, false);
      } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
        // check whether safe replication is reached for the block
        // only complete blocks are counted towards that
@@@ -4171,138 -3740,7 +4119,32 @@@
          null);
    }
  
 +  public static LocatedStripedBlock newLocatedStripedBlock(
 +      ExtendedBlock b, DatanodeStorageInfo[] storages,
 +      int[] indices, long startOffset, boolean corrupt) {
 +    // startOffset is unknown
 +    return new LocatedStripedBlock(
 +        b, DatanodeStorageInfo.toDatanodeInfos(storages),
 +        DatanodeStorageInfo.toStorageIDs(storages),
 +        DatanodeStorageInfo.toStorageTypes(storages),
 +        indices, startOffset, corrupt,
 +        null);
 +  }
 +
 +  public static LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info,
 +      DatanodeStorageInfo[] locs, long offset) throws IOException {
 +    final LocatedBlock lb;
 +    if (info.isStriped()) {
 +      lb = newLocatedStripedBlock(eb, locs,
 +          info.getUnderConstructionFeature().getBlockIndices(),
 +          offset, false);
 +    } else {
 +      lb = newLocatedBlock(eb, locs, offset, false);
 +    }
 +    return lb;
 +  }
 +
    /**
-    * This class is used internally by {@link this#computeRecoveryWorkForBlocks}
-    * to represent a task to recover a block through replication or erasure
-    * coding. Recovery is done by transferring data from srcNodes to targets
-    */
-   private abstract static class BlockRecoveryWork {
-     final BlockInfo block;
-     final BlockCollection bc;
- 
-     /**
-      * An erasure coding recovery task has multiple source nodes.
-      * A replication task only has 1 source node, stored on top of the array
-      */
-     final DatanodeDescriptor[] srcNodes;
-     /** Nodes containing the block; avoid them in choosing new targets */
-     final List<DatanodeDescriptor> containingNodes;
-     /** Required by {@link BlockPlacementPolicy#chooseTarget} */
-     final List<DatanodeStorageInfo> liveReplicaStorages;
-     final int additionalReplRequired;
- 
-     DatanodeStorageInfo[] targets;
-     final int priority;
- 
-     BlockRecoveryWork(BlockInfo block,
-         BlockCollection bc,
-         DatanodeDescriptor[] srcNodes,
-         List<DatanodeDescriptor> containingNodes,
-         List<DatanodeStorageInfo> liveReplicaStorages,
-         int additionalReplRequired,
-         int priority) {
-       this.block = block;
-       this.bc = bc;
-       this.srcNodes = srcNodes;
-       this.containingNodes = containingNodes;
-       this.liveReplicaStorages = liveReplicaStorages;
-       this.additionalReplRequired = additionalReplRequired;
-       this.priority = priority;
-       this.targets = null;
-     }
- 
-     abstract void chooseTargets(BlockPlacementPolicy blockplacement,
-         BlockStoragePolicySuite storagePolicySuite,
-         Set<Node> excludedNodes);
-   }
- 
-   private static class ReplicationWork extends BlockRecoveryWork {
-     ReplicationWork(BlockInfo block,
-         BlockCollection bc,
-         DatanodeDescriptor[] srcNodes,
-         List<DatanodeDescriptor> containingNodes,
-         List<DatanodeStorageInfo> liveReplicaStorages,
-         int additionalReplRequired,
-         int priority) {
-       super(block, bc, srcNodes, containingNodes,
-           liveReplicaStorages, additionalReplRequired, priority);
-       LOG.debug("Creating a ReplicationWork to recover " + block);
-     }
- 
-     @Override
-     void chooseTargets(BlockPlacementPolicy blockplacement,
-         BlockStoragePolicySuite storagePolicySuite,
-         Set<Node> excludedNodes) {
-       assert srcNodes.length > 0
-           : "At least 1 source node should have been selected";
-       try {
-         targets = blockplacement.chooseTarget(bc.getName(),
-             additionalReplRequired, srcNodes[0], liveReplicaStorages, false,
-             excludedNodes, block.getNumBytes(),
-             storagePolicySuite.getPolicy(bc.getStoragePolicyID()));
-       } finally {
-         srcNodes[0].decrementPendingReplicationWithoutTargets();
-       }
-     }
-   }
- 
-   private static class ErasureCodingWork extends BlockRecoveryWork {
-     final short[] liveBlockIndicies;
- 
-     ErasureCodingWork(BlockInfo block,
-         BlockCollection bc,
-         DatanodeDescriptor[] srcNodes,
-         List<DatanodeDescriptor> containingNodes,
-         List<DatanodeStorageInfo> liveReplicaStorages,
-         int additionalReplRequired,
-         int priority, short[] liveBlockIndicies) {
-       super(block, bc, srcNodes, containingNodes,
-           liveReplicaStorages, additionalReplRequired, priority);
-       this.liveBlockIndicies = liveBlockIndicies;
-       LOG.debug("Creating an ErasureCodingWork to recover " + block);
-     }
- 
-     @Override
-     void chooseTargets(BlockPlacementPolicy blockplacement,
-         BlockStoragePolicySuite storagePolicySuite,
-         Set<Node> excludedNodes) {
-       try {
-         // TODO: new placement policy for EC considering multiple writers
-         targets = blockplacement.chooseTarget(bc.getName(),
-             additionalReplRequired, srcNodes[0], liveReplicaStorages, false,
-             excludedNodes, block.getNumBytes(),
-             storagePolicySuite.getPolicy(bc.getStoragePolicyID()));
-       } finally {
-       }
-     }
-   }
- 
-   /**
     * A simple result enum for the result of
     * {@link BlockManager#processMisReplicatedBlock(BlockInfo)}.
     */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockRecoveryWork.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockRecoveryWork.java
index 0000000,0000000..ed546df
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockRecoveryWork.java
@@@ -1,0 -1,0 +1,111 @@@
++/**
++ * 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.hdfs.server.blockmanagement;
++
++import org.apache.hadoop.net.Node;
++
++import java.util.Collections;
++import java.util.List;
++import java.util.Set;
++
++/**
++ * This class is used internally by
++ * {@link BlockManager#computeRecoveryWorkForBlocks} to represent a task to
++ * recover a block through replication or erasure coding. Recovery is done by
++ * transferring data from srcNodes to targets
++ */
++abstract class BlockRecoveryWork {
++  private final BlockInfo block;
++
++  private final BlockCollection bc;
++
++  /**
++   * An erasure coding recovery task has multiple source nodes.
++   * A replication task only has 1 source node, stored on top of the array
++   */
++  private final DatanodeDescriptor[] srcNodes;
++  /** Nodes containing the block; avoid them in choosing new targets */
++  private final List<DatanodeDescriptor> containingNodes;
++  /** Required by {@link BlockPlacementPolicy#chooseTarget} */
++  private  final List<DatanodeStorageInfo> liveReplicaStorages;
++  private final int additionalReplRequired;
++
++  private DatanodeStorageInfo[] targets;
++  private final int priority;
++
++  public BlockRecoveryWork(BlockInfo block,
++      BlockCollection bc,
++      DatanodeDescriptor[] srcNodes,
++      List<DatanodeDescriptor> containingNodes,
++      List<DatanodeStorageInfo> liveReplicaStorages,
++      int additionalReplRequired,
++      int priority) {
++    this.block = block;
++    this.bc = bc;
++    this.srcNodes = srcNodes;
++    this.containingNodes = containingNodes;
++    this.liveReplicaStorages = liveReplicaStorages;
++    this.additionalReplRequired = additionalReplRequired;
++    this.priority = priority;
++    this.targets = null;
++  }
++
++  DatanodeStorageInfo[] getTargets() {
++    return targets;
++  }
++
++  void resetTargets() {
++    this.targets = null;
++  }
++
++  void setTargets(DatanodeStorageInfo[] targets) {
++    this.targets = targets;
++  }
++
++  List<DatanodeDescriptor> getContainingNodes() {
++    return Collections.unmodifiableList(containingNodes);
++  }
++
++  public int getPriority() {
++    return priority;
++  }
++
++  public BlockInfo getBlock() {
++    return block;
++  }
++
++  public DatanodeDescriptor[] getSrcNodes() {
++    return srcNodes;
++  }
++
++  BlockCollection getBc() {
++    return bc;
++  }
++
++  List<DatanodeStorageInfo> getLiveReplicaStorages() {
++    return liveReplicaStorages;
++  }
++
++  public int getAdditionalReplRequired() {
++    return additionalReplRequired;
++  }
++
++  abstract void chooseTargets(BlockPlacementPolicy blockplacement,
++      BlockStoragePolicySuite storagePolicySuite,
++      Set<Node> excludedNodes);
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java
index 0000000,3842e56..a871390
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java
@@@ -1,0 -1,87 +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.hdfs.server.blockmanagement;
+ 
+ import static org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
+ 
+ import com.google.common.base.Preconditions;
++import org.apache.hadoop.hdfs.protocol.Block;
+ 
+ /**
+  * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a
+  * list of blocks that should be considered corrupt due to a block report.
+  */
+ class BlockToMarkCorrupt {
+   /** The corrupted block in a datanode. */
 -  private final BlockInfo corrupted;
++  private final Block corrupted;
+   /** The corresponding block stored in the BlockManager. */
+   private final BlockInfo stored;
+   /** The reason to mark corrupt. */
+   private final String reason;
+   /** The reason code to be stored */
+   private final CorruptReplicasMap.Reason reasonCode;
+ 
 -  BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason,
++  BlockToMarkCorrupt(Block corrupted, BlockInfo stored, String reason,
+       CorruptReplicasMap.Reason reasonCode) {
+     Preconditions.checkNotNull(corrupted, "corrupted is null");
+     Preconditions.checkNotNull(stored, "stored is null");
+ 
+     this.corrupted = corrupted;
+     this.stored = stored;
+     this.reason = reason;
+     this.reasonCode = reasonCode;
+   }
+ 
 -  BlockToMarkCorrupt(BlockInfo stored, String reason,
++  BlockToMarkCorrupt(Block corrupted, BlockInfo stored, long gs, String reason,
+       CorruptReplicasMap.Reason reasonCode) {
 -    this(stored, stored, reason, reasonCode);
 -  }
 -
 -  BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
 -      CorruptReplicasMap.Reason reasonCode) {
 -    this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored,
 -        reason, reasonCode);
++    this(corrupted, stored, reason, reasonCode);
+     //the corrupted block in datanode has a different generation stamp
 -    corrupted.setGenerationStamp(gs);
++    this.corrupted.setGenerationStamp(gs);
+   }
+ 
+   public boolean isCorruptedDuringWrite() {
+     return stored.getGenerationStamp() > corrupted.getGenerationStamp();
+   }
+ 
 -  public BlockInfo getCorrupted() {
++  public Block getCorrupted() {
+     return corrupted;
+   }
+ 
+   public BlockInfo getStored() {
+     return stored;
+   }
+ 
+   public String getReason() {
+     return reason;
+   }
+ 
+   public Reason getReasonCode() {
+     return reasonCode;
+   }
+ 
+   @Override
+   public String toString() {
+     return corrupted + "("
+         + (corrupted == stored ? "same as stored": "stored=" + stored) + ")";
+   }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 29e541c,0b398c5..b258f06
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@@ -38,9 -39,8 +39,10 @@@ import org.apache.hadoop.fs.StorageType
  import org.apache.hadoop.hdfs.protocol.Block;
  import org.apache.hadoop.hdfs.protocol.DatanodeID;
  import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 +import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
  import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
  import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
  import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
  import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
@@@ -696,29 -663,32 +698,39 @@@ public class DatanodeDescriptor extend
      }
    }
  
 +  @VisibleForTesting
 +  public boolean containsInvalidateBlock(Block block) {
 +    synchronized (invalidateBlocks) {
 +      return invalidateBlocks.contains(block);
 +    }
 +  }
 +
    /**
-    * Return the sum of remaining spaces of the specified type. If the remaining
-    * space of a storage is less than minSize, it won't be counted toward the
-    * sum.
+    * Find whether the datanode contains good storage of given type to
+    * place block of size <code>blockSize</code>.
+    *
+    * <p>Currently datanode only cares about the storage type, in this
+    * method, the first storage of given type we see is returned.
     *
-    * @param t The storage type. If null, the type is ignored.
-    * @param minSize The minimum free space required.
-    * @return the sum of remaining spaces that are bigger than minSize.
+    * @param t requested storage type
+    * @param blockSize requested block size
+    * @return
     */
-   public long getRemaining(StorageType t, long minSize) {
+   public DatanodeStorageInfo chooseStorage4Block(StorageType t,
+       long blockSize) {
+     final long requiredSize =
+         blockSize * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE;
+     final long scheduledSize = blockSize * getBlocksScheduled(t);
      long remaining = 0;
+     DatanodeStorageInfo storage = null;
      for (DatanodeStorageInfo s : getStorageInfos()) {
        if (s.getState() == State.NORMAL &&
-           (t == null || s.getStorageType() == t)) {
+           s.getStorageType() == t) {
+         if (storage == null) {
+           storage = s;
+         }
          long r = s.getRemaining();
-         if (r >= minSize) {
+         if (r >= requiredSize) {
            remaining += r;
          }
        }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java
index 0000000,0000000..761d6d0
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java
@@@ -1,0 -1,0 +1,60 @@@
++/**
++ * 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.hdfs.server.blockmanagement;
++
++import org.apache.hadoop.net.Node;
++
++import java.util.List;
++import java.util.Set;
++
++class ErasureCodingWork extends BlockRecoveryWork {
++  private final short[] liveBlockIndicies;
++
++  public ErasureCodingWork(BlockInfo block,
++      BlockCollection bc,
++      DatanodeDescriptor[] srcNodes,
++      List<DatanodeDescriptor> containingNodes,
++      List<DatanodeStorageInfo> liveReplicaStorages,
++      int additionalReplRequired,
++      int priority, short[] liveBlockIndicies) {
++    super(block, bc, srcNodes, containingNodes,
++        liveReplicaStorages, additionalReplRequired, priority);
++    this.liveBlockIndicies = liveBlockIndicies;
++    BlockManager.LOG.debug("Creating an ErasureCodingWork to recover " + block);
++  }
++
++  short[] getLiveBlockIndicies() {
++    return liveBlockIndicies;
++  }
++
++  @Override
++  void chooseTargets(BlockPlacementPolicy blockplacement,
++      BlockStoragePolicySuite storagePolicySuite,
++      Set<Node> excludedNodes) {
++    try {
++      // TODO: new placement policy for EC considering multiple writers
++      DatanodeStorageInfo[] chosenTargets = blockplacement.chooseTarget(
++          getBc().getName(), getAdditionalReplRequired(), getSrcNodes()[0],
++          getLiveReplicaStorages(), false, excludedNodes,
++          getBlock().getNumBytes(),
++          storagePolicySuite.getPolicy(getBc().getStoragePolicyID()));
++      setTargets(chosenTargets);
++    } finally {
++    }
++  }
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java
index 0000000,f8a6dad..8266f45
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java
@@@ -1,0 -1,87 +1,53 @@@
+ /**
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hdfs.server.blockmanagement;
+ 
+ import org.apache.hadoop.net.Node;
+ 
+ import java.util.Collections;
+ import java.util.List;
+ import java.util.Set;
+ 
 -class ReplicationWork {
 -  private final BlockInfo block;
 -  private final BlockCollection bc;
 -  private final DatanodeDescriptor srcNode;
 -  private final int additionalReplRequired;
 -  private final int priority;
 -  private final List<DatanodeDescriptor> containingNodes;
 -  private final List<DatanodeStorageInfo> liveReplicaStorages;
 -  private DatanodeStorageInfo[] targets;
 -
++class ReplicationWork extends BlockRecoveryWork {
+   public ReplicationWork(BlockInfo block, BlockCollection bc,
 -      DatanodeDescriptor srcNode, List<DatanodeDescriptor> containingNodes,
++      DatanodeDescriptor[] srcNodes, List<DatanodeDescriptor> containingNodes,
+       List<DatanodeStorageInfo> liveReplicaStorages, int additionalReplRequired,
+       int priority) {
 -    this.block = block;
 -    this.bc = bc;
 -    this.srcNode = srcNode;
 -    this.srcNode.incrementPendingReplicationWithoutTargets();
 -    this.containingNodes = containingNodes;
 -    this.liveReplicaStorages = liveReplicaStorages;
 -    this.additionalReplRequired = additionalReplRequired;
 -    this.priority = priority;
 -    this.targets = null;
++    super(block, bc, srcNodes, containingNodes,
++        liveReplicaStorages, additionalReplRequired, priority);
++    BlockManager.LOG.debug("Creating a ReplicationWork to recover " + block);
+   }
+ 
++  @Override
+   void chooseTargets(BlockPlacementPolicy blockplacement,
+       BlockStoragePolicySuite storagePolicySuite,
+       Set<Node> excludedNodes) {
++    assert getSrcNodes().length > 0
++        : "At least 1 source node should have been selected";
+     try {
 -      targets = blockplacement.chooseTarget(bc.getName(),
 -          additionalReplRequired, srcNode, liveReplicaStorages, false,
 -          excludedNodes, block.getNumBytes(),
 -          storagePolicySuite.getPolicy(bc.getStoragePolicyID()));
++      DatanodeStorageInfo[] chosenTargets = blockplacement.chooseTarget(
++          getBc().getName(), getAdditionalReplRequired(), getSrcNodes()[0],
++          getLiveReplicaStorages(), false, excludedNodes,
++          getBlock().getNumBytes(),
++          storagePolicySuite.getPolicy(getBc().getStoragePolicyID()));
++      setTargets(chosenTargets);
+     } finally {
 -      srcNode.decrementPendingReplicationWithoutTargets();
++      getSrcNodes()[0].decrementPendingReplicationWithoutTargets();
+     }
+   }
 -
 -  DatanodeStorageInfo[] getTargets() {
 -    return targets;
 -  }
 -
 -  void resetTargets() {
 -    this.targets = null;
 -  }
 -
 -  List<DatanodeDescriptor> getContainingNodes() {
 -    return Collections.unmodifiableList(containingNodes);
 -  }
 -
 -  public int getPriority() {
 -    return priority;
 -  }
 -
 -  public BlockInfo getBlock() {
 -    return block;
 -  }
 -
 -  public DatanodeDescriptor getSrcNode() {
 -    return srcNode;
 -  }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
index 3c77120,0000000..6cc1dcd
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
@@@ -1,250 -1,0 +1,220 @@@
 +/**
 + * 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.hdfs.server.blockmanagement;
 +
 +import org.apache.hadoop.hdfs.DFSTestUtil;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
 +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.junit.Assert;
 +import org.junit.Test;
 +import org.mockito.internal.util.reflection.Whitebox;
 +
 +import java.io.DataOutput;
 +import java.io.DataOutputStream;
 +import java.io.ByteArrayOutputStream;
 +import java.nio.ByteBuffer;
 +
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
 +import static org.junit.Assert.assertArrayEquals;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.fail;
 +
 +/**
 + * Test {@link BlockInfoStriped}
 + */
 +public class TestBlockInfoStriped {
 +  private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
 +  private static final long BASE_ID = -1600;
 +  private static final Block baseBlock = new Block(BASE_ID);
 +  private static final ErasureCodingPolicy testECPolicy
 +      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 +  private final BlockInfoStriped info = new BlockInfoStriped(baseBlock,
 +      testECPolicy);
 +
 +  private Block[] createReportedBlocks(int num) {
 +    Block[] blocks = new Block[num];
 +    for (int i = 0; i < num; i++) {
 +      blocks[i] = new Block(BASE_ID + i);
 +    }
 +    return blocks;
 +  }
 +
 +  /**
 +   * Test adding storage and reported block
 +   */
 +  @Test
 +  public void testAddStorage() {
 +    // first add NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS storages, i.e., a complete
 +    // group of blocks/storages
 +    DatanodeStorageInfo[] storageInfos = DFSTestUtil.createDatanodeStorageInfos(
 +        TOTAL_NUM_BLOCKS);
 +    Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS);
 +    int i = 0;
 +    for (; i < storageInfos.length; i += 2) {
 +      info.addStorage(storageInfos[i], blocks[i]);
 +      Assert.assertEquals(i/2 + 1, info.numNodes());
 +    }
 +    i /= 2;
 +    for (int j = 1; j < storageInfos.length; j += 2) {
 +      Assert.assertTrue(info.addStorage(storageInfos[j], blocks[j]));
 +      Assert.assertEquals(i + (j+1)/2, info.numNodes());
 +    }
 +
 +    // check
 +    byte[] indices = (byte[]) Whitebox.getInternalState(info, "indices");
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, indices.length);
 +    i = 0;
 +    for (DatanodeStorageInfo storage : storageInfos) {
 +      int index = info.findStorageInfo(storage);
 +      Assert.assertEquals(i++, index);
 +      Assert.assertEquals(index, indices[index]);
 +    }
 +
 +    // the same block is reported from the same storage twice
 +    i = 0;
 +    for (DatanodeStorageInfo storage : storageInfos) {
 +      Assert.assertTrue(info.addStorage(storage, blocks[i++]));
 +    }
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.numNodes());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, indices.length);
 +    i = 0;
 +    for (DatanodeStorageInfo storage : storageInfos) {
 +      int index = info.findStorageInfo(storage);
 +      Assert.assertEquals(i++, index);
 +      Assert.assertEquals(index, indices[index]);
 +    }
 +
 +    // the same block is reported from another storage
 +    DatanodeStorageInfo[] storageInfos2 = DFSTestUtil.createDatanodeStorageInfos(
 +        TOTAL_NUM_BLOCKS * 2);
 +    // only add the second half of info2
 +    for (i = TOTAL_NUM_BLOCKS; i < storageInfos2.length; i++) {
 +      info.addStorage(storageInfos2[i], blocks[i % TOTAL_NUM_BLOCKS]);
 +      Assert.assertEquals(i + 1, info.getCapacity());
 +      Assert.assertEquals(i + 1, info.numNodes());
 +      indices = (byte[]) Whitebox.getInternalState(info, "indices");
 +      Assert.assertEquals(i + 1, indices.length);
 +    }
 +    for (i = TOTAL_NUM_BLOCKS; i < storageInfos2.length; i++) {
 +      int index = info.findStorageInfo(storageInfos2[i]);
 +      Assert.assertEquals(i++, index);
 +      Assert.assertEquals(index - TOTAL_NUM_BLOCKS, indices[index]);
 +    }
 +  }
 +
 +  @Test
 +  public void testRemoveStorage() {
 +    // first add TOTAL_NUM_BLOCKS into the BlockInfoStriped
 +    DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(
 +        TOTAL_NUM_BLOCKS);
 +    Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS);
 +    for (int i = 0; i < storages.length; i++) {
 +      info.addStorage(storages[i], blocks[i]);
 +    }
 +
 +    // remove two storages
 +    info.removeStorage(storages[0]);
 +    info.removeStorage(storages[2]);
 +
 +    // check
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS - 2, info.numNodes());
 +    byte[] indices = (byte[]) Whitebox.getInternalState(info, "indices");
 +    for (int i = 0; i < storages.length; i++) {
 +      int index = info.findStorageInfo(storages[i]);
 +      if (i != 0 && i != 2) {
 +        Assert.assertEquals(i, index);
 +        Assert.assertEquals(index, indices[index]);
 +      } else {
 +        Assert.assertEquals(-1, index);
 +        Assert.assertEquals(-1, indices[i]);
 +      }
 +    }
 +
 +    // the same block is reported from another storage
 +    DatanodeStorageInfo[] storages2 = DFSTestUtil.createDatanodeStorageInfos(
 +        TOTAL_NUM_BLOCKS * 2);
 +    for (int i = TOTAL_NUM_BLOCKS; i < storages2.length; i++) {
 +      info.addStorage(storages2[i], blocks[i % TOTAL_NUM_BLOCKS]);
 +    }
 +    // now we should have 8 storages
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.numNodes());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.getCapacity());
 +    indices = (byte[]) Whitebox.getInternalState(info, "indices");
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, indices.length);
 +    int j = TOTAL_NUM_BLOCKS;
 +    for (int i = TOTAL_NUM_BLOCKS; i < storages2.length; i++) {
 +      int index = info.findStorageInfo(storages2[i]);
 +      if (i == TOTAL_NUM_BLOCKS || i == TOTAL_NUM_BLOCKS + 2) {
 +        Assert.assertEquals(i - TOTAL_NUM_BLOCKS, index);
 +      } else {
 +        Assert.assertEquals(j++, index);
 +      }
 +    }
 +
 +    // remove the storages from storages2
 +    for (int i = 0; i < TOTAL_NUM_BLOCKS; i++) {
 +      info.removeStorage(storages2[i + TOTAL_NUM_BLOCKS]);
 +    }
 +    // now we should have 3 storages
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS - 2, info.numNodes());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.getCapacity());
 +    indices = (byte[]) Whitebox.getInternalState(info, "indices");
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, indices.length);
 +    for (int i = 0; i < TOTAL_NUM_BLOCKS; i++) {
 +      if (i == 0 || i == 2) {
 +        int index = info.findStorageInfo(storages2[i + TOTAL_NUM_BLOCKS]);
 +        Assert.assertEquals(-1, index);
 +      } else {
 +        int index = info.findStorageInfo(storages[i]);
 +        Assert.assertEquals(i, index);
 +      }
 +    }
 +    for (int i = TOTAL_NUM_BLOCKS; i < TOTAL_NUM_BLOCKS * 2 - 2; i++) {
 +      Assert.assertEquals(-1, indices[i]);
 +      Assert.assertNull(info.getDatanode(i));
 +    }
 +  }
 +
 +  @Test
-   public void testReplaceBlock() {
-     DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(
-         TOTAL_NUM_BLOCKS);
-     Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS);
-     // add block/storage 0, 2, 4 into the BlockInfoStriped
-     for (int i = 0; i < storages.length; i += 2) {
-       Assert.assertEquals(AddBlockResult.ADDED,
-           storages[i].addBlock(info, blocks[i]));
-     }
- 
-     BlockInfoStriped newBlockInfo = new BlockInfoStriped(info,
-         info.getErasureCodingPolicy());
-     newBlockInfo.setBlockCollectionId(info.getBlockCollectionId());
-     info.replaceBlock(newBlockInfo);
- 
-     // make sure the newBlockInfo is correct
-     byte[] indices = (byte[]) Whitebox.getInternalState(newBlockInfo, "indices");
-     for (int i = 0; i < storages.length; i += 2) {
-       int index = newBlockInfo.findStorageInfo(storages[i]);
-       Assert.assertEquals(i, index);
-       Assert.assertEquals(index, indices[i]);
- 
-       // make sure the newBlockInfo is added to the linked list of the storage
-       Assert.assertSame(newBlockInfo, storages[i].getBlockListHeadForTesting());
-       Assert.assertEquals(1, storages[i].numBlocks());
-       Assert.assertNull(newBlockInfo.getNext());
-     }
-   }
- 
-   @Test
 +  public void testWrite() {
 +    long blkID = 1;
 +    long numBytes = 1;
 +    long generationStamp = 1;
 +    ByteBuffer byteBuffer = ByteBuffer.allocate(Long.SIZE / Byte.SIZE * 3);
 +    byteBuffer.putLong(blkID).putLong(numBytes).putLong(generationStamp);
 +
 +    ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
 +    DataOutput out = new DataOutputStream(byteStream);
 +    BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes,
 +        generationStamp), testECPolicy);
 +
 +    try {
 +      blk.write(out);
 +    } catch(Exception ex) {
 +      fail("testWrite error:" + ex.getMessage());
 +    }
 +    assertEquals(byteBuffer.array().length, byteStream.toByteArray().length);
 +    assertArrayEquals(byteBuffer.array(), byteStream.toByteArray());
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------


[11/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
index 1b23600,0000000..3c77120
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
@@@ -1,248 -1,0 +1,250 @@@
 +/**
 + * 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.hdfs.server.blockmanagement;
 +
 +import org.apache.hadoop.hdfs.DFSTestUtil;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
 +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.junit.Assert;
 +import org.junit.Test;
 +import org.mockito.internal.util.reflection.Whitebox;
 +
 +import java.io.DataOutput;
 +import java.io.DataOutputStream;
 +import java.io.ByteArrayOutputStream;
 +import java.nio.ByteBuffer;
 +
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
 +import static org.junit.Assert.assertArrayEquals;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.fail;
 +
 +/**
 + * Test {@link BlockInfoStriped}
 + */
 +public class TestBlockInfoStriped {
 +  private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
 +  private static final long BASE_ID = -1600;
 +  private static final Block baseBlock = new Block(BASE_ID);
 +  private static final ErasureCodingPolicy testECPolicy
 +      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 +  private final BlockInfoStriped info = new BlockInfoStriped(baseBlock,
 +      testECPolicy);
 +
 +  private Block[] createReportedBlocks(int num) {
 +    Block[] blocks = new Block[num];
 +    for (int i = 0; i < num; i++) {
 +      blocks[i] = new Block(BASE_ID + i);
 +    }
 +    return blocks;
 +  }
 +
 +  /**
 +   * Test adding storage and reported block
 +   */
 +  @Test
 +  public void testAddStorage() {
 +    // first add NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS storages, i.e., a complete
 +    // group of blocks/storages
 +    DatanodeStorageInfo[] storageInfos = DFSTestUtil.createDatanodeStorageInfos(
 +        TOTAL_NUM_BLOCKS);
 +    Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS);
 +    int i = 0;
 +    for (; i < storageInfos.length; i += 2) {
 +      info.addStorage(storageInfos[i], blocks[i]);
 +      Assert.assertEquals(i/2 + 1, info.numNodes());
 +    }
 +    i /= 2;
 +    for (int j = 1; j < storageInfos.length; j += 2) {
 +      Assert.assertTrue(info.addStorage(storageInfos[j], blocks[j]));
 +      Assert.assertEquals(i + (j+1)/2, info.numNodes());
 +    }
 +
 +    // check
 +    byte[] indices = (byte[]) Whitebox.getInternalState(info, "indices");
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, indices.length);
 +    i = 0;
 +    for (DatanodeStorageInfo storage : storageInfos) {
 +      int index = info.findStorageInfo(storage);
 +      Assert.assertEquals(i++, index);
 +      Assert.assertEquals(index, indices[index]);
 +    }
 +
 +    // the same block is reported from the same storage twice
 +    i = 0;
 +    for (DatanodeStorageInfo storage : storageInfos) {
 +      Assert.assertTrue(info.addStorage(storage, blocks[i++]));
 +    }
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.numNodes());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, indices.length);
 +    i = 0;
 +    for (DatanodeStorageInfo storage : storageInfos) {
 +      int index = info.findStorageInfo(storage);
 +      Assert.assertEquals(i++, index);
 +      Assert.assertEquals(index, indices[index]);
 +    }
 +
 +    // the same block is reported from another storage
 +    DatanodeStorageInfo[] storageInfos2 = DFSTestUtil.createDatanodeStorageInfos(
 +        TOTAL_NUM_BLOCKS * 2);
 +    // only add the second half of info2
 +    for (i = TOTAL_NUM_BLOCKS; i < storageInfos2.length; i++) {
 +      info.addStorage(storageInfos2[i], blocks[i % TOTAL_NUM_BLOCKS]);
 +      Assert.assertEquals(i + 1, info.getCapacity());
 +      Assert.assertEquals(i + 1, info.numNodes());
 +      indices = (byte[]) Whitebox.getInternalState(info, "indices");
 +      Assert.assertEquals(i + 1, indices.length);
 +    }
 +    for (i = TOTAL_NUM_BLOCKS; i < storageInfos2.length; i++) {
 +      int index = info.findStorageInfo(storageInfos2[i]);
 +      Assert.assertEquals(i++, index);
 +      Assert.assertEquals(index - TOTAL_NUM_BLOCKS, indices[index]);
 +    }
 +  }
 +
 +  @Test
 +  public void testRemoveStorage() {
 +    // first add TOTAL_NUM_BLOCKS into the BlockInfoStriped
 +    DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(
 +        TOTAL_NUM_BLOCKS);
 +    Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS);
 +    for (int i = 0; i < storages.length; i++) {
 +      info.addStorage(storages[i], blocks[i]);
 +    }
 +
 +    // remove two storages
 +    info.removeStorage(storages[0]);
 +    info.removeStorage(storages[2]);
 +
 +    // check
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS - 2, info.numNodes());
 +    byte[] indices = (byte[]) Whitebox.getInternalState(info, "indices");
 +    for (int i = 0; i < storages.length; i++) {
 +      int index = info.findStorageInfo(storages[i]);
 +      if (i != 0 && i != 2) {
 +        Assert.assertEquals(i, index);
 +        Assert.assertEquals(index, indices[index]);
 +      } else {
 +        Assert.assertEquals(-1, index);
 +        Assert.assertEquals(-1, indices[i]);
 +      }
 +    }
 +
 +    // the same block is reported from another storage
 +    DatanodeStorageInfo[] storages2 = DFSTestUtil.createDatanodeStorageInfos(
 +        TOTAL_NUM_BLOCKS * 2);
 +    for (int i = TOTAL_NUM_BLOCKS; i < storages2.length; i++) {
 +      info.addStorage(storages2[i], blocks[i % TOTAL_NUM_BLOCKS]);
 +    }
 +    // now we should have 8 storages
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.numNodes());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.getCapacity());
 +    indices = (byte[]) Whitebox.getInternalState(info, "indices");
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, indices.length);
 +    int j = TOTAL_NUM_BLOCKS;
 +    for (int i = TOTAL_NUM_BLOCKS; i < storages2.length; i++) {
 +      int index = info.findStorageInfo(storages2[i]);
 +      if (i == TOTAL_NUM_BLOCKS || i == TOTAL_NUM_BLOCKS + 2) {
 +        Assert.assertEquals(i - TOTAL_NUM_BLOCKS, index);
 +      } else {
 +        Assert.assertEquals(j++, index);
 +      }
 +    }
 +
 +    // remove the storages from storages2
 +    for (int i = 0; i < TOTAL_NUM_BLOCKS; i++) {
 +      info.removeStorage(storages2[i + TOTAL_NUM_BLOCKS]);
 +    }
 +    // now we should have 3 storages
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS - 2, info.numNodes());
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.getCapacity());
 +    indices = (byte[]) Whitebox.getInternalState(info, "indices");
 +    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, indices.length);
 +    for (int i = 0; i < TOTAL_NUM_BLOCKS; i++) {
 +      if (i == 0 || i == 2) {
 +        int index = info.findStorageInfo(storages2[i + TOTAL_NUM_BLOCKS]);
 +        Assert.assertEquals(-1, index);
 +      } else {
 +        int index = info.findStorageInfo(storages[i]);
 +        Assert.assertEquals(i, index);
 +      }
 +    }
 +    for (int i = TOTAL_NUM_BLOCKS; i < TOTAL_NUM_BLOCKS * 2 - 2; i++) {
 +      Assert.assertEquals(-1, indices[i]);
 +      Assert.assertNull(info.getDatanode(i));
 +    }
 +  }
 +
 +  @Test
 +  public void testReplaceBlock() {
 +    DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(
 +        TOTAL_NUM_BLOCKS);
 +    Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS);
 +    // add block/storage 0, 2, 4 into the BlockInfoStriped
 +    for (int i = 0; i < storages.length; i += 2) {
 +      Assert.assertEquals(AddBlockResult.ADDED,
 +          storages[i].addBlock(info, blocks[i]));
 +    }
 +
-     BlockInfoStriped newBlockInfo = new BlockInfoStriped(info);
++    BlockInfoStriped newBlockInfo = new BlockInfoStriped(info,
++        info.getErasureCodingPolicy());
++    newBlockInfo.setBlockCollectionId(info.getBlockCollectionId());
 +    info.replaceBlock(newBlockInfo);
 +
 +    // make sure the newBlockInfo is correct
 +    byte[] indices = (byte[]) Whitebox.getInternalState(newBlockInfo, "indices");
 +    for (int i = 0; i < storages.length; i += 2) {
 +      int index = newBlockInfo.findStorageInfo(storages[i]);
 +      Assert.assertEquals(i, index);
 +      Assert.assertEquals(index, indices[i]);
 +
 +      // make sure the newBlockInfo is added to the linked list of the storage
 +      Assert.assertSame(newBlockInfo, storages[i].getBlockListHeadForTesting());
 +      Assert.assertEquals(1, storages[i].numBlocks());
 +      Assert.assertNull(newBlockInfo.getNext());
 +    }
 +  }
 +
 +  @Test
 +  public void testWrite() {
 +    long blkID = 1;
 +    long numBytes = 1;
 +    long generationStamp = 1;
 +    ByteBuffer byteBuffer = ByteBuffer.allocate(Long.SIZE / Byte.SIZE * 3);
 +    byteBuffer.putLong(blkID).putLong(numBytes).putLong(generationStamp);
 +
 +    ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
 +    DataOutput out = new DataOutputStream(byteStream);
 +    BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes,
 +        generationStamp), testECPolicy);
 +
 +    try {
 +      blk.write(out);
 +    } catch(Exception ex) {
 +      fail("testWrite error:" + ex.getMessage());
 +    }
 +    assertEquals(byteBuffer.array().length, byteStream.toByteArray().length);
 +    assertArrayEquals(byteBuffer.array(), byteStream.toByteArray());
 +  }
 +}

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 0172595,cec33fe..763bc99e
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@@ -1234,11 -1269,11 +1278,11 @@@ public class TestReplicationPolicy 
      chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
      assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
  
 -    final BlockInfo info = new BlockInfoContiguous(block1, (short) 1);
 +    final BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
      final BlockCollection mbc = mock(BlockCollection.class);
+     when(mbc.getId()).thenReturn(1000L);
      when(mbc.getLastBlock()).thenReturn(info);
      when(mbc.getPreferredBlockSize()).thenReturn(block1.getNumBytes() + 1);
-     when(mbc.getPreferredBlockReplication()).thenReturn((short)1);
      when(mbc.isUnderConstruction()).thenReturn(true);
      ContentSummary cs = mock(ContentSummary.class);
      when(cs.getLength()).thenReturn((long)1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
----------------------------------------------------------------------


[46/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 0000000,7a40d73..78eaa6c
mode 000000,100755..100755
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@@ -1,0 -1,917 +1,982 @@@
+ /**
+  * 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.hdfs;
+ 
+ import java.io.FileNotFoundException;
+ import java.io.IOException;
+ import java.io.InterruptedIOException;
+ import java.nio.channels.ClosedChannelException;
+ import java.util.EnumSet;
+ import java.util.concurrent.atomic.AtomicReference;
+ 
+ import org.apache.hadoop.HadoopIllegalArgumentException;
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.crypto.CryptoProtocolVersion;
+ import org.apache.hadoop.fs.CanSetDropBehind;
+ import org.apache.hadoop.fs.CreateFlag;
+ import org.apache.hadoop.fs.FSOutputSummer;
+ import org.apache.hadoop.fs.FileAlreadyExistsException;
+ import org.apache.hadoop.fs.FileEncryptionInfo;
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.ParentNotDirectoryException;
+ import org.apache.hadoop.fs.Syncable;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+ import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+ import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+ import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
+ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+ import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
++import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+ import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException;
+ import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+ import org.apache.hadoop.hdfs.util.ByteArrayManager;
+ import org.apache.hadoop.io.EnumSetWritable;
+ import org.apache.hadoop.ipc.RemoteException;
+ import org.apache.hadoop.security.AccessControlException;
+ import org.apache.hadoop.security.token.Token;
+ import org.apache.hadoop.util.DataChecksum;
+ import org.apache.hadoop.util.DataChecksum.Type;
+ import org.apache.hadoop.util.Progressable;
+ import org.apache.hadoop.util.Time;
+ import org.apache.htrace.core.TraceScope;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.base.Preconditions;
+ 
+ 
+ /****************************************************************
+  * DFSOutputStream creates files from a stream of bytes.
+  *
+  * The client application writes data that is cached internally by
+  * this stream. Data is broken up into packets, each packet is
+  * typically 64K in size. A packet comprises of chunks. Each chunk
+  * is typically 512 bytes and has an associated checksum with it.
+  *
+  * When a client application fills up the currentPacket, it is
+  * enqueued into the dataQueue of DataStreamer. DataStreamer is a
+  * thread that picks up packets from the dataQueue and sends it to
+  * the first datanode in the pipeline.
+  *
+  ****************************************************************/
+ @InterfaceAudience.Private
+ public class DFSOutputStream extends FSOutputSummer
+     implements Syncable, CanSetDropBehind {
+   static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
+   /**
+    * Number of times to retry creating a file when there are transient 
+    * errors (typically related to encryption zones and KeyProvider operations).
+    */
+   @VisibleForTesting
+   static final int CREATE_RETRY_COUNT = 10;
+   @VisibleForTesting
+   static CryptoProtocolVersion[] SUPPORTED_CRYPTO_VERSIONS =
+       CryptoProtocolVersion.supported();
+ 
+   protected final DFSClient dfsClient;
+   protected final ByteArrayManager byteArrayManager;
+   // closed is accessed by different threads under different locks.
+   protected volatile boolean closed = false;
+ 
+   protected final String src;
+   protected final long fileId;
+   protected final long blockSize;
+   protected final int bytesPerChecksum;
+ 
+   protected DFSPacket currentPacket = null;
 -  private DataStreamer streamer;
++  protected DataStreamer streamer;
+   protected int packetSize = 0; // write packet size, not including the header.
+   protected int chunksPerPacket = 0;
+   protected long lastFlushOffset = 0; // offset when flush was invoked
+   private long initialFileSize = 0; // at time of file open
+   private final short blockReplication; // replication factor of file
+   protected boolean shouldSyncBlock = false; // force blocks to disk upon close
+   protected final AtomicReference<CachingStrategy> cachingStrategy;
+   private FileEncryptionInfo fileEncryptionInfo;
+ 
+   /** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
+   protected DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetInBlock,
+       long seqno, boolean lastPacketInBlock) throws InterruptedIOException {
+     final byte[] buf;
+     final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
+ 
+     try {
+       buf = byteArrayManager.newByteArray(bufferSize);
+     } catch (InterruptedException ie) {
+       final InterruptedIOException iioe = new InterruptedIOException(
+           "seqno=" + seqno);
+       iioe.initCause(ie);
+       throw iioe;
+     }
+ 
+     return new DFSPacket(buf, chunksPerPkt, offsetInBlock, seqno,
 -                         getChecksumSize(), lastPacketInBlock);
++        getChecksumSize(), lastPacketInBlock);
+   }
+ 
+   @Override
+   protected void checkClosed() throws IOException {
+     if (isClosed()) {
+       getStreamer().getLastException().throwException4Close();
+     }
+   }
+ 
+   //
+   // returns the list of targets, if any, that is being currently used.
+   //
+   @VisibleForTesting
+   public synchronized DatanodeInfo[] getPipeline() {
+     if (getStreamer().streamerClosed()) {
+       return null;
+     }
+     DatanodeInfo[] currentNodes = getStreamer().getNodes();
+     if (currentNodes == null) {
+       return null;
+     }
+     DatanodeInfo[] value = new DatanodeInfo[currentNodes.length];
+     for (int i = 0; i < currentNodes.length; i++) {
+       value[i] = currentNodes[i];
+     }
+     return value;
+   }
+ 
 -  /** 
++  /**
+    * @return the object for computing checksum.
+    *         The type is NULL if checksum is not computed.
+    */
+   private static DataChecksum getChecksum4Compute(DataChecksum checksum,
+       HdfsFileStatus stat) {
+     if (DataStreamer.isLazyPersist(stat) && stat.getReplication() == 1) {
+       // do not compute checksum for writing to single replica to memory
+       return DataChecksum.newDataChecksum(Type.NULL,
+           checksum.getBytesPerChecksum());
+     }
+     return checksum;
+   }
 - 
++
+   private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
+       HdfsFileStatus stat, DataChecksum checksum) throws IOException {
+     super(getChecksum4Compute(checksum, stat));
+     this.dfsClient = dfsClient;
+     this.src = src;
+     this.fileId = stat.getFileId();
+     this.blockSize = stat.getBlockSize();
+     this.blockReplication = stat.getReplication();
+     this.fileEncryptionInfo = stat.getFileEncryptionInfo();
+     this.cachingStrategy = new AtomicReference<CachingStrategy>(
+         dfsClient.getDefaultWriteCachingStrategy());
+     if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
+       DFSClient.LOG.debug(
+           "Set non-null progress callback on DFSOutputStream " + src);
+     }
 -    
++
+     this.bytesPerChecksum = checksum.getBytesPerChecksum();
+     if (bytesPerChecksum <= 0) {
+       throw new HadoopIllegalArgumentException(
+           "Invalid value: bytesPerChecksum = " + bytesPerChecksum + " <= 0");
+     }
+     if (blockSize % bytesPerChecksum != 0) {
+       throw new HadoopIllegalArgumentException("Invalid values: "
+           + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
+           + ") must divide block size (=" + blockSize + ").");
+     }
+     this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
+   }
+ 
+   /** Construct a new output stream for creating a file. */
+   protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
+       EnumSet<CreateFlag> flag, Progressable progress,
 -      DataChecksum checksum, String[] favoredNodes) throws IOException {
++      DataChecksum checksum, String[] favoredNodes, boolean createStreamer)
++      throws IOException {
+     this(dfsClient, src, progress, stat, checksum);
+     this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
+ 
+     computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
+ 
 -    streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
 -        cachingStrategy, byteArrayManager, favoredNodes);
++    if (createStreamer) {
++      streamer = new DataStreamer(stat, null, dfsClient, src, progress,
++          checksum, cachingStrategy, byteArrayManager, favoredNodes);
++    }
+   }
+ 
+   static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
+       FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
+       short replication, long blockSize, Progressable progress, int buffersize,
+       DataChecksum checksum, String[] favoredNodes) throws IOException {
+     TraceScope scope =
+         dfsClient.newPathTraceScope("newStreamForCreate", src);
+     try {
+       HdfsFileStatus stat = null;
+ 
+       // Retry the create if we get a RetryStartFileException up to a maximum
+       // number of times
+       boolean shouldRetry = true;
+       int retryCount = CREATE_RETRY_COUNT;
+       while (shouldRetry) {
+         shouldRetry = false;
+         try {
+           stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
+               new EnumSetWritable<CreateFlag>(flag), createParent, replication,
+               blockSize, SUPPORTED_CRYPTO_VERSIONS);
+           break;
+         } catch (RemoteException re) {
+           IOException e = re.unwrapRemoteException(
+               AccessControlException.class,
+               DSQuotaExceededException.class,
+               QuotaByStorageTypeExceededException.class,
+               FileAlreadyExistsException.class,
+               FileNotFoundException.class,
+               ParentNotDirectoryException.class,
+               NSQuotaExceededException.class,
+               RetryStartFileException.class,
+               SafeModeException.class,
+               UnresolvedPathException.class,
+               SnapshotAccessControlException.class,
+               UnknownCryptoProtocolVersionException.class);
+           if (e instanceof RetryStartFileException) {
+             if (retryCount > 0) {
+               shouldRetry = true;
+               retryCount--;
+             } else {
+               throw new IOException("Too many retries because of encryption" +
+                   " zone operations", e);
+             }
+           } else {
+             throw e;
+           }
+         }
+       }
+       Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!");
 -      final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
 -          flag, progress, checksum, favoredNodes);
++      final DFSOutputStream out;
++      if(stat.getErasureCodingPolicy() != null) {
++        out = new DFSStripedOutputStream(dfsClient, src, stat,
++            flag, progress, checksum, favoredNodes);
++      } else {
++        out = new DFSOutputStream(dfsClient, src, stat,
++            flag, progress, checksum, favoredNodes, true);
++      }
+       out.start();
+       return out;
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /** Construct a new output stream for append. */
+   private DFSOutputStream(DFSClient dfsClient, String src,
+       EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
+       HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
 -          throws IOException {
++      throws IOException {
+     this(dfsClient, src, progress, stat, checksum);
+     initialFileSize = stat.getLen(); // length of file when opened
+     this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK);
+ 
+     boolean toNewBlock = flags.contains(CreateFlag.NEW_BLOCK);
+ 
+     this.fileEncryptionInfo = stat.getFileEncryptionInfo();
+ 
+     // The last partial block of the file has to be filled.
+     if (!toNewBlock && lastBlock != null) {
+       // indicate that we are appending to an existing block
+       streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress, checksum,
+           cachingStrategy, byteArrayManager);
+       getStreamer().setBytesCurBlock(lastBlock.getBlockSize());
+       adjustPacketChunkSize(stat);
+       getStreamer().setPipelineInConstruction(lastBlock);
+     } else {
+       computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
+           bytesPerChecksum);
+       streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
+           dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
+           favoredNodes);
+     }
+   }
+ 
+   private void adjustPacketChunkSize(HdfsFileStatus stat) throws IOException{
+ 
+     long usedInLastBlock = stat.getLen() % blockSize;
+     int freeInLastBlock = (int)(blockSize - usedInLastBlock);
+ 
+     // calculate the amount of free space in the pre-existing
+     // last crc chunk
+     int usedInCksum = (int)(stat.getLen() % bytesPerChecksum);
+     int freeInCksum = bytesPerChecksum - usedInCksum;
+ 
+     // if there is space in the last block, then we have to
+     // append to that block
+     if (freeInLastBlock == blockSize) {
+       throw new IOException("The last block for file " +
+           src + " is full.");
+     }
+ 
+     if (usedInCksum > 0 && freeInCksum > 0) {
+       // if there is space in the last partial chunk, then
+       // setup in such a way that the next packet will have only
+       // one chunk that fills up the partial chunk.
+       //
+       computePacketChunkSize(0, freeInCksum);
+       setChecksumBufSize(freeInCksum);
+       getStreamer().setAppendChunk(true);
+     } else {
+       // if the remaining space in the block is smaller than
+       // that expected size of of a packet, then create
+       // smaller size packet.
+       //
+       computePacketChunkSize(
+           Math.min(dfsClient.getConf().getWritePacketSize(), freeInLastBlock),
+           bytesPerChecksum);
+     }
+   }
+ 
+   static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
+       EnumSet<CreateFlag> flags, int bufferSize, Progressable progress,
+       LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
+       String[] favoredNodes) throws IOException {
+     TraceScope scope =
+         dfsClient.newPathTraceScope("newStreamForAppend", src);
++    if(stat.getErasureCodingPolicy() != null) {
++      throw new IOException("Not support appending to a striping layout file yet.");
++    }
+     try {
+       final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
+           progress, lastBlock, stat, checksum, favoredNodes);
+       out.start();
+       return out;
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   protected void computePacketChunkSize(int psize, int csize) {
+     final int bodySize = psize - PacketHeader.PKT_MAX_HEADER_LEN;
+     final int chunkSize = csize + getChecksumSize();
+     chunksPerPacket = Math.max(bodySize/chunkSize, 1);
+     packetSize = chunkSize*chunksPerPacket;
+     if (DFSClient.LOG.isDebugEnabled()) {
+       DFSClient.LOG.debug("computePacketChunkSize: src=" + src +
+                 ", chunkSize=" + chunkSize +
+                 ", chunksPerPacket=" + chunksPerPacket +
+                 ", packetSize=" + packetSize);
+     }
+   }
+ 
+   protected TraceScope createWriteTraceScope() {
+     return dfsClient.newPathTraceScope("DFSOutputStream#write", src);
+   }
+ 
+   // @see FSOutputSummer#writeChunk()
+   @Override
+   protected synchronized void writeChunk(byte[] b, int offset, int len,
+       byte[] checksum, int ckoff, int cklen) throws IOException {
+     dfsClient.checkOpen();
+     checkClosed();
+ 
+     if (len > bytesPerChecksum) {
+       throw new IOException("writeChunk() buffer size is " + len +
+                             " is larger than supported  bytesPerChecksum " +
+                             bytesPerChecksum);
+     }
+     if (cklen != 0 && cklen != getChecksumSize()) {
+       throw new IOException("writeChunk() checksum size is supposed to be " +
+                             getChecksumSize() + " but found to be " + cklen);
+     }
+ 
+     if (currentPacket == null) {
+       currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
+           .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
 -      if (DFSClient.LOG.isDebugEnabled()) {
 -        DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" + 
++      if (LOG.isDebugEnabled()) {
++        LOG.debug("WriteChunk allocating new packet seqno=" +
+             currentPacket.getSeqno() +
+             ", src=" + src +
+             ", packetSize=" + packetSize +
+             ", chunksPerPacket=" + chunksPerPacket +
 -            ", bytesCurBlock=" + getStreamer().getBytesCurBlock());
++            ", bytesCurBlock=" + getStreamer().getBytesCurBlock() + ", " + this);
+       }
+     }
+ 
+     currentPacket.writeChecksum(checksum, ckoff, cklen);
+     currentPacket.writeData(b, offset, len);
+     currentPacket.incNumChunks();
+     getStreamer().incBytesCurBlock(len);
+ 
+     // If packet is full, enqueue it for transmission
 -    //
+     if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
+         getStreamer().getBytesCurBlock() == blockSize) {
+       enqueueCurrentPacketFull();
+     }
+   }
+ 
+   void enqueueCurrentPacket() throws IOException {
+     getStreamer().waitAndQueuePacket(currentPacket);
+     currentPacket = null;
+   }
+ 
+   void enqueueCurrentPacketFull() throws IOException {
+     LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
 -        + " appendChunk={}, {}", currentPacket, src, getStreamer()
 -        .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
++            + " appendChunk={}, {}", currentPacket, src, getStreamer()
++            .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
+         getStreamer());
+     enqueueCurrentPacket();
+     adjustChunkBoundary();
+     endBlock();
+   }
+ 
+   /** create an empty packet to mark the end of the block. */
+   void setCurrentPacketToEmpty() throws InterruptedIOException {
+     currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
+         getStreamer().getAndIncCurrentSeqno(), true);
+     currentPacket.setSyncBlock(shouldSyncBlock);
+   }
+ 
+   /**
+    * If the reopened file did not end at chunk boundary and the above
+    * write filled up its partial chunk. Tell the summer to generate full
+    * crc chunks from now on.
+    */
+   protected void adjustChunkBoundary() {
+     if (getStreamer().getAppendChunk() &&
+         getStreamer().getBytesCurBlock() % bytesPerChecksum == 0) {
+       getStreamer().setAppendChunk(false);
+       resetChecksumBufSize();
+     }
+ 
+     if (!getStreamer().getAppendChunk()) {
+       int psize = Math.min((int)(blockSize- getStreamer().getBytesCurBlock()),
+           dfsClient.getConf().getWritePacketSize());
+       computePacketChunkSize(psize, bytesPerChecksum);
+     }
+   }
+ 
+   /**
+    * if encountering a block boundary, send an empty packet to
+    * indicate the end of block and reset bytesCurBlock.
+    *
+    * @throws IOException
+    */
 -  protected void endBlock() throws IOException {
++  void endBlock() throws IOException {
+     if (getStreamer().getBytesCurBlock() == blockSize) {
+       setCurrentPacketToEmpty();
+       enqueueCurrentPacket();
+       getStreamer().setBytesCurBlock(0);
+       lastFlushOffset = 0;
+     }
+   }
 -  
++
+   /**
+    * Flushes out to all replicas of the block. The data is in the buffers
+    * of the DNs but not necessarily in the DN's OS buffers.
+    *
+    * It is a synchronous operation. When it returns,
+    * it guarantees that flushed data become visible to new readers. 
+    * It is not guaranteed that data has been flushed to 
+    * persistent store on the datanode. 
+    * Block allocations are persisted on namenode.
+    */
+   @Override
+   public void hflush() throws IOException {
+     TraceScope scope =
+         dfsClient.newPathTraceScope("hflush", src);
+     try {
+       flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   @Override
+   public void hsync() throws IOException {
+     TraceScope scope =
+         dfsClient.newPathTraceScope("hsync", src);
+     try {
+       flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
+     } finally {
+       scope.close();
+     }
+   }
 -  
++
+   /**
+    * The expected semantics is all data have flushed out to all replicas 
+    * and all replicas have done posix fsync equivalent - ie the OS has 
+    * flushed it to the disk device (but the disk may have it in its cache).
 -   * 
++   *
+    * Note that only the current block is flushed to the disk device.
+    * To guarantee durable sync across block boundaries the stream should
+    * be created with {@link CreateFlag#SYNC_BLOCK}.
 -   * 
++   *
+    * @param syncFlags
+    *          Indicate the semantic of the sync. Currently used to specify
+    *          whether or not to update the block length in NameNode.
+    */
+   public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
+     TraceScope scope =
+         dfsClient.newPathTraceScope("hsync", src);
+     try {
+       flushOrSync(true, syncFlags);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Flush/Sync buffered data to DataNodes.
 -   * 
++   *
+    * @param isSync
+    *          Whether or not to require all replicas to flush data to the disk
+    *          device
+    * @param syncFlags
+    *          Indicate extra detailed semantic of the flush/sync. Currently
+    *          mainly used to specify whether or not to update the file length in
+    *          the NameNode
+    * @throws IOException
+    */
+   private void flushOrSync(boolean isSync, EnumSet<SyncFlag> syncFlags)
+       throws IOException {
+     dfsClient.checkOpen();
+     checkClosed();
+     try {
+       long toWaitFor;
+       long lastBlockLength = -1L;
+       boolean updateLength = syncFlags.contains(SyncFlag.UPDATE_LENGTH);
+       boolean endBlock = syncFlags.contains(SyncFlag.END_BLOCK);
+       synchronized (this) {
+         // flush checksum buffer, but keep checksum buffer intact if we do not
+         // need to end the current block
+         int numKept = flushBuffer(!endBlock, true);
+         // bytesCurBlock potentially incremented if there was buffered data
+ 
+         if (DFSClient.LOG.isDebugEnabled()) {
+           DFSClient.LOG.debug("DFSClient flush(): "
+               + " bytesCurBlock=" + getStreamer().getBytesCurBlock()
+               + " lastFlushOffset=" + lastFlushOffset
+               + " createNewBlock=" + endBlock);
+         }
+         // Flush only if we haven't already flushed till this offset.
+         if (lastFlushOffset != getStreamer().getBytesCurBlock()) {
+           assert getStreamer().getBytesCurBlock() > lastFlushOffset;
+           // record the valid offset of this flush
+           lastFlushOffset = getStreamer().getBytesCurBlock();
+           if (isSync && currentPacket == null && !endBlock) {
+             // Nothing to send right now,
+             // but sync was requested.
+             // Send an empty packet if we do not end the block right now
+             currentPacket = createPacket(packetSize, chunksPerPacket,
+                 getStreamer().getBytesCurBlock(), getStreamer()
+                     .getAndIncCurrentSeqno(), false);
+           }
+         } else {
+           if (isSync && getStreamer().getBytesCurBlock() > 0 && !endBlock) {
+             // Nothing to send right now,
+             // and the block was partially written,
+             // and sync was requested.
+             // So send an empty sync packet if we do not end the block right
+             // now
+             currentPacket = createPacket(packetSize, chunksPerPacket,
+                 getStreamer().getBytesCurBlock(), getStreamer()
+                     .getAndIncCurrentSeqno(), false);
+           } else if (currentPacket != null) {
+             // just discard the current packet since it is already been sent.
+             currentPacket.releaseBuffer(byteArrayManager);
+             currentPacket = null;
+           }
+         }
+         if (currentPacket != null) {
+           currentPacket.setSyncBlock(isSync);
+           enqueueCurrentPacket();
+         }
+         if (endBlock && getStreamer().getBytesCurBlock() > 0) {
+           // Need to end the current block, thus send an empty packet to
+           // indicate this is the end of the block and reset bytesCurBlock
+           currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
+               getStreamer().getAndIncCurrentSeqno(), true);
+           currentPacket.setSyncBlock(shouldSyncBlock || isSync);
+           enqueueCurrentPacket();
+           getStreamer().setBytesCurBlock(0);
+           lastFlushOffset = 0;
+         } else {
+           // Restore state of stream. Record the last flush offset
+           // of the last full chunk that was flushed.
+           getStreamer().setBytesCurBlock(
+               getStreamer().getBytesCurBlock() - numKept);
+         }
+ 
+         toWaitFor = getStreamer().getLastQueuedSeqno();
+       } // end synchronized
+ 
+       getStreamer().waitForAckedSeqno(toWaitFor);
+ 
+       // update the block length first time irrespective of flag
+       if (updateLength || getStreamer().getPersistBlocks().get()) {
+         synchronized (this) {
+           if (!getStreamer().streamerClosed()
+               && getStreamer().getBlock() != null) {
+             lastBlockLength = getStreamer().getBlock().getNumBytes();
+           }
+         }
+       }
+       // If 1) any new blocks were allocated since the last flush, or 2) to
+       // update length in NN is required, then persist block locations on
+       // namenode.
+       if (getStreamer().getPersistBlocks().getAndSet(false) || updateLength) {
+         try {
+           dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
+               lastBlockLength);
+         } catch (IOException ioe) {
+           DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
+           // If we got an error here, it might be because some other thread called
+           // close before our hflush completed. In that case, we should throw an
+           // exception that the stream is closed.
+           checkClosed();
+           // If we aren't closed but failed to sync, we should expose that to the
+           // caller.
+           throw ioe;
+         }
+       }
+ 
+       synchronized(this) {
+         if (!getStreamer().streamerClosed()) {
+           getStreamer().setHflush();
+         }
+       }
+     } catch (InterruptedIOException interrupt) {
+       // This kind of error doesn't mean that the stream itself is broken - just the
+       // flushing thread got interrupted. So, we shouldn't close down the writer,
+       // but instead just propagate the error
+       throw interrupt;
+     } catch (IOException e) {
+       DFSClient.LOG.warn("Error while syncing", e);
+       synchronized (this) {
+         if (!isClosed()) {
+           getStreamer().getLastException().set(e);
+           closeThreads(true);
+         }
+       }
+       throw e;
+     }
+   }
+ 
+   /**
+    * @deprecated use {@link HdfsDataOutputStream#getCurrentBlockReplication()}.
+    */
+   @Deprecated
+   public synchronized int getNumCurrentReplicas() throws IOException {
+     return getCurrentBlockReplication();
+   }
+ 
+   /**
+    * Note that this is not a public API;
+    * use {@link HdfsDataOutputStream#getCurrentBlockReplication()} instead.
 -   * 
++   *
+    * @return the number of valid replicas of the current block
+    */
+   public synchronized int getCurrentBlockReplication() throws IOException {
+     dfsClient.checkOpen();
+     checkClosed();
+     if (getStreamer().streamerClosed()) {
+       return blockReplication; // no pipeline, return repl factor of file
+     }
+     DatanodeInfo[] currentNodes = getStreamer().getNodes();
+     if (currentNodes == null) {
+       return blockReplication; // no pipeline, return repl factor of file
+     }
+     return currentNodes.length;
+   }
 -  
++
+   /**
+    * Waits till all existing data is flushed and confirmations 
+    * received from datanodes. 
+    */
+   protected void flushInternal() throws IOException {
+     long toWaitFor;
+     synchronized (this) {
+       dfsClient.checkOpen();
+       checkClosed();
+       //
+       // If there is data in the current buffer, send it across
+       //
+       getStreamer().queuePacket(currentPacket);
+       currentPacket = null;
+       toWaitFor = getStreamer().getLastQueuedSeqno();
+     }
+ 
+     getStreamer().waitForAckedSeqno(toWaitFor);
+   }
+ 
+   protected synchronized void start() {
+     getStreamer().start();
+   }
 -  
++
+   /**
+    * Aborts this output stream and releases any system 
+    * resources associated with this stream.
+    */
+   synchronized void abort() throws IOException {
+     if (isClosed()) {
+       return;
+     }
+     getStreamer().getLastException().set(new IOException("Lease timeout of "
+         + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
+     closeThreads(true);
+     dfsClient.endFileLease(fileId);
+   }
+ 
+   boolean isClosed() {
+     return closed || getStreamer().streamerClosed();
+   }
+ 
+   void setClosed() {
+     closed = true;
+     getStreamer().release();
+   }
+ 
+   // shutdown datastreamer and responseprocessor threads.
+   // interrupt datastreamer if force is true
+   protected void closeThreads(boolean force) throws IOException {
+     try {
+       getStreamer().close(force);
+       getStreamer().join();
+       getStreamer().closeSocket();
+     } catch (InterruptedException e) {
+       throw new IOException("Failed to shutdown streamer");
+     } finally {
+       getStreamer().setSocketToNull();
+       setClosed();
+     }
+   }
 -  
++
+   /**
+    * Closes this output stream and releases any system 
+    * resources associated with this stream.
+    */
+   @Override
+   public synchronized void close() throws IOException {
+     TraceScope scope =
+         dfsClient.newPathTraceScope("DFSOutputStream#close", src);
+     try {
+       closeImpl();
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   protected synchronized void closeImpl() throws IOException {
+     if (isClosed()) {
+       getStreamer().getLastException().check(true);
+       return;
+     }
+ 
+     try {
+       flushBuffer();       // flush from all upper layers
+ 
+       if (currentPacket != null) {
+         enqueueCurrentPacket();
+       }
+ 
+       if (getStreamer().getBytesCurBlock() != 0) {
+         setCurrentPacketToEmpty();
+       }
+ 
+       flushInternal();             // flush all data to Datanodes
+       // get last block before destroying the streamer
+       ExtendedBlock lastBlock = getStreamer().getBlock();
+       closeThreads(false);
+       TraceScope scope = dfsClient.getTracer().newScope("completeFile");
+       try {
+         completeFile(lastBlock);
+       } finally {
+         scope.close();
+       }
+       dfsClient.endFileLease(fileId);
+     } catch (ClosedChannelException e) {
+     } finally {
+       setClosed();
+     }
+   }
+ 
+   // should be called holding (this) lock since setTestFilename() may 
+   // be called during unit tests
+   protected void completeFile(ExtendedBlock last) throws IOException {
+     long localstart = Time.monotonicNow();
+     final DfsClientConf conf = dfsClient.getConf();
+     long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
+     boolean fileComplete = false;
+     int retries = conf.getNumBlockWriteLocateFollowingRetry();
+     while (!fileComplete) {
+       fileComplete =
+           dfsClient.namenode.complete(src, dfsClient.clientName, last, fileId);
+       if (!fileComplete) {
+         final int hdfsTimeout = conf.getHdfsTimeout();
+         if (!dfsClient.clientRunning
+             || (hdfsTimeout > 0
+                 && localstart + hdfsTimeout < Time.monotonicNow())) {
+             String msg = "Unable to close file because dfsclient " +
+                           " was unable to contact the HDFS servers." +
+                           " clientRunning " + dfsClient.clientRunning +
+                           " hdfsTimeout " + hdfsTimeout;
+             DFSClient.LOG.info(msg);
+             throw new IOException(msg);
+         }
+         try {
+           if (retries == 0) {
+             throw new IOException("Unable to close file because the last block"
+                 + " does not have enough number of replicas.");
+           }
+           retries--;
+           Thread.sleep(sleeptime);
+           sleeptime *= 2;
+           if (Time.monotonicNow() - localstart > 5000) {
+             DFSClient.LOG.info("Could not complete " + src + " retrying...");
+           }
+         } catch (InterruptedException ie) {
+           DFSClient.LOG.warn("Caught exception ", ie);
+         }
+       }
+     }
+   }
+ 
+   @VisibleForTesting
+   public void setArtificialSlowdown(long period) {
+     getStreamer().setArtificialSlowdown(period);
+   }
+ 
+   @VisibleForTesting
+   public synchronized void setChunksPerPacket(int value) {
+     chunksPerPacket = Math.min(chunksPerPacket, value);
+     packetSize = (bytesPerChecksum + getChecksumSize()) * chunksPerPacket;
+   }
+ 
+   /**
+    * Returns the size of a file as it was when this stream was opened
+    */
+   public long getInitialLen() {
+     return initialFileSize;
+   }
+ 
+   /**
+    * @return the FileEncryptionInfo for this stream, or null if not encrypted.
+    */
+   public FileEncryptionInfo getFileEncryptionInfo() {
+     return fileEncryptionInfo;
+   }
+ 
+   /**
+    * Returns the access token currently used by streamer, for testing only
+    */
+   synchronized Token<BlockTokenIdentifier> getBlockToken() {
+     return getStreamer().getBlockToken();
+   }
+ 
+   @Override
+   public void setDropBehind(Boolean dropBehind) throws IOException {
+     CachingStrategy prevStrategy, nextStrategy;
+     // CachingStrategy is immutable.  So build a new CachingStrategy with the
+     // modifications we want, and compare-and-swap it in.
+     do {
+       prevStrategy = this.cachingStrategy.get();
+       nextStrategy = new CachingStrategy.Builder(prevStrategy).
 -                        setDropBehind(dropBehind).build();
++          setDropBehind(dropBehind).build();
+     } while (!this.cachingStrategy.compareAndSet(prevStrategy, nextStrategy));
+   }
+ 
+   @VisibleForTesting
+   ExtendedBlock getBlock() {
+     return getStreamer().getBlock();
+   }
+ 
+   @VisibleForTesting
+   public long getFileId() {
+     return fileId;
+   }
+ 
+   /**
+    * Return the source of stream.
+    */
+   String getSrc() {
+     return src;
+   }
+ 
+   /**
+    * Returns the data streamer object.
+    */
+   protected DataStreamer getStreamer() {
+     return streamer;
+   }
++
++  @Override
++  public String toString() {
++    return getClass().getSimpleName() + ":" + streamer;
++  }
++
++  static LocatedBlock addBlock(DatanodeInfo[] excludedNodes, DFSClient dfsClient,
++      String src, ExtendedBlock prevBlock, long fileId, String[] favoredNodes)
++      throws IOException {
++    final DfsClientConf conf = dfsClient.getConf();
++    int retries = conf.getNumBlockWriteLocateFollowingRetry();
++    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
++    long localstart = Time.monotonicNow();
++    while (true) {
++      try {
++        return dfsClient.namenode.addBlock(src, dfsClient.clientName, prevBlock,
++            excludedNodes, fileId, favoredNodes);
++      } catch (RemoteException e) {
++        IOException ue = e.unwrapRemoteException(FileNotFoundException.class,
++            AccessControlException.class,
++            NSQuotaExceededException.class,
++            DSQuotaExceededException.class,
++            QuotaByStorageTypeExceededException.class,
++            UnresolvedPathException.class);
++        if (ue != e) {
++          throw ue; // no need to retry these exceptions
++        }
++        if (NotReplicatedYetException.class.getName().equals(e.getClassName())) {
++          if (retries == 0) {
++            throw e;
++          } else {
++            --retries;
++            LOG.info("Exception while adding a block", e);
++            long elapsed = Time.monotonicNow() - localstart;
++            if (elapsed > 5000) {
++              LOG.info("Waiting for replication for " + (elapsed / 1000)
++                  + " seconds");
++            }
++            try {
++              LOG.warn("NotReplicatedYetException sleeping " + src
++                  + " retries left " + retries);
++              Thread.sleep(sleeptime);
++              sleeptime *= 2;
++            } catch (InterruptedException ie) {
++              LOG.warn("Caught exception", ie);
++            }
++          }
++        } else {
++          throw e;
++        }
++      }
++    }
++  }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
index 0000000,9a8ca6f..191691b
mode 000000,100755..100755
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
@@@ -1,0 -1,350 +1,364 @@@
+ /**
+  * 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.hdfs;
+ 
+ import java.io.DataOutputStream;
+ import java.io.IOException;
+ import java.nio.BufferOverflowException;
++import java.nio.ByteBuffer;
+ import java.nio.channels.ClosedChannelException;
+ import java.util.Arrays;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+ import org.apache.hadoop.hdfs.util.ByteArrayManager;
+ import org.apache.htrace.core.Span;
+ import org.apache.htrace.core.SpanId;
+ import org.apache.htrace.core.TraceScope;
+ 
+ /****************************************************************
+  * DFSPacket is used by DataStreamer and DFSOutputStream.
+  * DFSOutputStream generates packets and then ask DatStreamer
+  * to send them to datanodes.
+  ****************************************************************/
+ 
+ @InterfaceAudience.Private
 -class DFSPacket {
++public class DFSPacket {
+   public static final long HEART_BEAT_SEQNO = -1L;
+   private static SpanId[] EMPTY = new SpanId[0];
+   private final long seqno; // sequence number of buffer in block
+   private final long offsetInBlock; // offset in block
+   private boolean syncBlock; // this packet forces the current block to disk
+   private int numChunks; // number of chunks currently in packet
+   private final int maxChunks; // max chunks in packet
+   private byte[] buf;
+   private final boolean lastPacketInBlock; // is this the last packet in block?
+ 
+   /**
+    * buf is pointed into like follows:
+    *  (C is checksum data, D is payload data)
+    *
+    * [_________CCCCCCCCC________________DDDDDDDDDDDDDDDD___]
+    *           ^        ^               ^               ^
+    *           |        checksumPos     dataStart       dataPos
+    *           checksumStart
+    *
+    * Right before sending, we move the checksum data to immediately precede
+    * the actual data, and then insert the header into the buffer immediately
+    * preceding the checksum data, so we make sure to keep enough space in
+    * front of the checksum data to support the largest conceivable header.
+    */
+   private int checksumStart;
+   private int checksumPos;
+   private final int dataStart;
+   private int dataPos;
+   private SpanId[] traceParents = EMPTY;
+   private int traceParentsUsed;
+   private TraceScope scope;
+ 
+   /**
+    * Create a new packet.
+    *
+    * @param buf the buffer storing data and checksums
+    * @param chunksPerPkt maximum number of chunks per packet.
+    * @param offsetInBlock offset in bytes into the HDFS block.
+    * @param seqno the sequence number of this packet
+    * @param checksumSize the size of checksum
+    * @param lastPacketInBlock if this is the last packet
+    */
 -  DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno,
++  public DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno,
+                    int checksumSize, boolean lastPacketInBlock) {
+     this.lastPacketInBlock = lastPacketInBlock;
+     this.numChunks = 0;
+     this.offsetInBlock = offsetInBlock;
+     this.seqno = seqno;
+ 
+     this.buf = buf;
+ 
+     checksumStart = PacketHeader.PKT_MAX_HEADER_LEN;
+     checksumPos = checksumStart;
+     dataStart = checksumStart + (chunksPerPkt * checksumSize);
+     dataPos = dataStart;
+     maxChunks = chunksPerPkt;
+   }
+ 
+   /**
+    * Write data to this packet.
+    *
+    * @param inarray input array of data
+    * @param off the offset of data to write
+    * @param len the length of data to write
+    * @throws ClosedChannelException
+    */
+   synchronized void writeData(byte[] inarray, int off, int len)
+       throws ClosedChannelException {
+     checkBuffer();
+     if (dataPos + len > buf.length) {
+       throw new BufferOverflowException();
+     }
+     System.arraycopy(inarray, off, buf, dataPos, len);
+     dataPos += len;
+   }
+ 
++  public synchronized void writeData(ByteBuffer inBuffer, int len)
++      throws ClosedChannelException {
++    checkBuffer();
++    len =  len > inBuffer.remaining() ? inBuffer.remaining() : len;
++    if (dataPos + len > buf.length) {
++      throw new BufferOverflowException();
++    }
++    for (int i = 0; i < len; i++) {
++      buf[dataPos + i] = inBuffer.get();
++    }
++    dataPos += len;
++  }
++
+   /**
+    * Write checksums to this packet
+    *
+    * @param inarray input array of checksums
+    * @param off the offset of checksums to write
+    * @param len the length of checksums to write
+    * @throws ClosedChannelException
+    */
 -  synchronized void writeChecksum(byte[] inarray, int off, int len)
++  public synchronized void writeChecksum(byte[] inarray, int off, int len)
+       throws ClosedChannelException {
+     checkBuffer();
+     if (len == 0) {
+       return;
+     }
+     if (checksumPos + len > dataStart) {
+       throw new BufferOverflowException();
+     }
+     System.arraycopy(inarray, off, buf, checksumPos, len);
+     checksumPos += len;
+   }
+ 
+   /**
+    * Write the full packet, including the header, to the given output stream.
+    *
+    * @param stm
+    * @throws IOException
+    */
 -  synchronized void writeTo(DataOutputStream stm) throws IOException {
++  public synchronized void writeTo(DataOutputStream stm) throws IOException {
+     checkBuffer();
+ 
+     final int dataLen = dataPos - dataStart;
+     final int checksumLen = checksumPos - checksumStart;
+     final int pktLen = HdfsConstants.BYTES_IN_INTEGER + dataLen + checksumLen;
+ 
+     PacketHeader header = new PacketHeader(
+         pktLen, offsetInBlock, seqno, lastPacketInBlock, dataLen, syncBlock);
+ 
+     if (checksumPos != dataStart) {
+       // Move the checksum to cover the gap. This can happen for the last
+       // packet or during an hflush/hsync call.
+       System.arraycopy(buf, checksumStart, buf,
+           dataStart - checksumLen , checksumLen);
+       checksumPos = dataStart;
+       checksumStart = checksumPos - checksumLen;
+     }
+ 
+     final int headerStart = checksumStart - header.getSerializedSize();
+     assert checksumStart + 1 >= header.getSerializedSize();
+     assert headerStart >= 0;
+     assert headerStart + header.getSerializedSize() == checksumStart;
+ 
+     // Copy the header data into the buffer immediately preceding the checksum
+     // data.
+     System.arraycopy(header.getBytes(), 0, buf, headerStart,
+         header.getSerializedSize());
+ 
+     // corrupt the data for testing.
+     if (DFSClientFaultInjector.get().corruptPacket()) {
+       buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
+     }
+ 
+     // Write the now contiguous full packet to the output stream.
+     stm.write(buf, headerStart, header.getSerializedSize() + checksumLen + dataLen);
+ 
+     // undo corruption.
+     if (DFSClientFaultInjector.get().uncorruptPacket()) {
+       buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
+     }
+   }
+ 
+   private synchronized void checkBuffer() throws ClosedChannelException {
+     if (buf == null) {
+       throw new ClosedChannelException();
+     }
+   }
+ 
+   /**
+    * Release the buffer in this packet to ByteArrayManager.
+    *
+    * @param bam
+    */
+   synchronized void releaseBuffer(ByteArrayManager bam) {
+     bam.release(buf);
+     buf = null;
+   }
+ 
+   /**
+    * get the packet's last byte's offset in the block
+    *
+    * @return the packet's last byte's offset in the block
+    */
+   synchronized long getLastByteOffsetBlock() {
+     return offsetInBlock + dataPos - dataStart;
+   }
+ 
+   /**
+    * Check if this packet is a heart beat packet
+    *
+    * @return true if the sequence number is HEART_BEAT_SEQNO
+    */
+   boolean isHeartbeatPacket() {
+     return seqno == HEART_BEAT_SEQNO;
+   }
+ 
+   /**
+    * check if this packet is the last packet in block
+    *
+    * @return true if the packet is the last packet
+    */
 -  boolean isLastPacketInBlock(){
++  boolean isLastPacketInBlock() {
+     return lastPacketInBlock;
+   }
+ 
+   /**
+    * get sequence number of this packet
+    *
+    * @return the sequence number of this packet
+    */
 -  long getSeqno(){
++  long getSeqno() {
+     return seqno;
+   }
+ 
+   /**
+    * get the number of chunks this packet contains
+    *
+    * @return the number of chunks in this packet
+    */
 -  synchronized int getNumChunks(){
++  synchronized int getNumChunks() {
+     return numChunks;
+   }
+ 
+   /**
+    * increase the number of chunks by one
+    */
 -  synchronized void incNumChunks(){
++  synchronized void incNumChunks() {
+     numChunks++;
+   }
+ 
+   /**
+    * get the maximum number of packets
+    *
+    * @return the maximum number of packets
+    */
 -  int getMaxChunks(){
++  int getMaxChunks() {
+     return maxChunks;
+   }
+ 
+   /**
+    * set if to sync block
+    *
+    * @param syncBlock if to sync block
+    */
 -  synchronized void setSyncBlock(boolean syncBlock){
++  synchronized void setSyncBlock(boolean syncBlock) {
+     this.syncBlock = syncBlock;
+   }
+ 
+   @Override
+   public String toString() {
+     return "packet seqno: " + this.seqno +
+         " offsetInBlock: " + this.offsetInBlock +
+         " lastPacketInBlock: " + this.lastPacketInBlock +
+         " lastByteOffsetInBlock: " + this.getLastByteOffsetBlock();
+   }
+ 
+   /**
+    * Add a trace parent span for this packet.<p/>
+    *
+    * Trace parent spans for a packet are the trace spans responsible for
+    * adding data to that packet.  We store them as an array of longs for
+    * efficiency.<p/>
+    *
+    * Protected by the DFSOutputStream dataQueue lock.
+    */
+   public void addTraceParent(Span span) {
+     if (span == null) {
+       return;
+     }
+     addTraceParent(span.getSpanId());
+   }
+ 
+   public void addTraceParent(SpanId id) {
+     if (!id.isValid()) {
+       return;
+     }
+     if (traceParentsUsed == traceParents.length) {
+       int newLength = (traceParents.length == 0) ? 8 :
+           traceParents.length * 2;
+       traceParents = Arrays.copyOf(traceParents, newLength);
+     }
+     traceParents[traceParentsUsed] = id;
+     traceParentsUsed++;
+   }
+ 
+   /**
+    * Get the trace parent spans for this packet.<p/>
+    *
+    * Will always be non-null.<p/>
+    *
+    * Protected by the DFSOutputStream dataQueue lock.
+    */
+   public SpanId[] getTraceParents() {
+     // Remove duplicates from the array.
+     int len = traceParentsUsed;
+     Arrays.sort(traceParents, 0, len);
+     int i = 0, j = 0;
+     SpanId prevVal = SpanId.INVALID;
+     while (true) {
+       if (i == len) {
+         break;
+       }
+       SpanId val = traceParents[i];
+       if (!val.equals(prevVal)) {
+         traceParents[j] = val;
+         j++;
+         prevVal = val;
+       }
+       i++;
+     }
+     if (j < traceParents.length) {
+       traceParents = Arrays.copyOf(traceParents, j);
+       traceParentsUsed = traceParents.length;
+     }
+     return traceParents;
+   }
+ 
+   public void setTraceScope(TraceScope scope) {
+     this.scope = scope;
+   }
+ 
+   public TraceScope getTraceScope() {
+     return scope;
+   }
+ }


[15/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
index 0000000,7509da5..6be94f3
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
@@@ -1,0 -1,512 +1,517 @@@
+ /**
+  * 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.hdfs;
+ 
+ import java.io.BufferedInputStream;
+ import java.io.BufferedOutputStream;
+ import java.io.DataInputStream;
+ import java.io.DataOutputStream;
+ import java.io.IOException;
+ import java.nio.ByteBuffer;
+ import java.util.EnumSet;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.fs.FSInputChecker;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.fs.ReadOption;
+ import org.apache.hadoop.hdfs.net.Peer;
+ import org.apache.hadoop.hdfs.protocol.Block;
+ import org.apache.hadoop.hdfs.protocol.DatanodeID;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+ import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+ import org.apache.hadoop.io.IOUtils;
+ import org.apache.hadoop.net.NetUtils;
+ import org.apache.hadoop.security.token.Token;
+ import org.apache.hadoop.util.DataChecksum;
+ import org.apache.htrace.Sampler;
+ import org.apache.htrace.Trace;
+ import org.apache.htrace.TraceScope;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ 
+ /**
+  * @deprecated this is an old implementation that is being left around
+  * in case any issues spring up with the new {@link RemoteBlockReader2} implementation.
+  * It will be removed in the next release.
+  */
+ @InterfaceAudience.Private
+ @Deprecated
+ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
+   static final Logger LOG = LoggerFactory.getLogger(FSInputChecker.class);
+ 
+   private final Peer peer;
+   private final DatanodeID datanodeID;
+   private final DataInputStream in;
+   private DataChecksum checksum;
+ 
+   /** offset in block of the last chunk received */
+   private long lastChunkOffset = -1;
+   private long lastChunkLen = -1;
+   private long lastSeqNo = -1;
+ 
+   /** offset in block where reader wants to actually read */
+   private long startOffset;
+ 
+   private final long blockId;
+ 
+   /** offset in block of of first chunk - may be less than startOffset
+       if startOffset is not chunk-aligned */
+   private final long firstChunkOffset;
+ 
+   private final int bytesPerChecksum;
+   private final int checksumSize;
+ 
+   /**
+    * The total number of bytes we need to transfer from the DN.
+    * This is the amount that the user has requested plus some padding
+    * at the beginning so that the read can begin on a chunk boundary.
+    */
+   private final long bytesNeededToFinish;
+   
+   /**
+    * True if we are reading from a local DataNode.
+    */
+   private final boolean isLocal;
+ 
+   private boolean eos = false;
+   private boolean sentStatusCode = false;
+   
+   ByteBuffer checksumBytes = null;
+   /** Amount of unread data in the current received packet */
+   int dataLeft = 0;
+   
+   private final PeerCache peerCache;
+   
+   /* FSInputChecker interface */
+   
+   /* same interface as inputStream java.io.InputStream#read()
+    * used by DFSInputStream#read()
+    * This violates one rule when there is a checksum error:
+    * "Read should not modify user buffer before successful read"
+    * because it first reads the data to user buffer and then checks
+    * the checksum.
+    */
+   @Override
+   public synchronized int read(byte[] buf, int off, int len) 
+                                throws IOException {
+     
+     // This has to be set here, *before* the skip, since we can
+     // hit EOS during the skip, in the case that our entire read
+     // is smaller than the checksum chunk.
+     boolean eosBefore = eos;
+ 
+     //for the first read, skip the extra bytes at the front.
+     if (lastChunkLen < 0 && startOffset > firstChunkOffset && len > 0) {
+       // Skip these bytes. But don't call this.skip()!
+       int toSkip = (int)(startOffset - firstChunkOffset);
+       if ( super.readAndDiscard(toSkip) != toSkip ) {
+         // should never happen
+         throw new IOException("Could not skip required number of bytes");
+       }
+     }
+     
+     int nRead = super.read(buf, off, len);
+ 
+     // if eos was set in the previous read, send a status code to the DN
+     if (eos && !eosBefore && nRead >= 0) {
+       if (needChecksum()) {
+         sendReadResult(peer, Status.CHECKSUM_OK);
+       } else {
+         sendReadResult(peer, Status.SUCCESS);
+       }
+     }
+     return nRead;
+   }
+ 
+   @Override
+   public synchronized long skip(long n) throws IOException {
+     /* How can we make sure we don't throw a ChecksumException, at least
+      * in majority of the cases?. This one throws. */  
+     long nSkipped = 0;
+     while (nSkipped < n) {
+       int toSkip = (int)Math.min(n-nSkipped, Integer.MAX_VALUE);
+       int ret = readAndDiscard(toSkip);
+       if (ret <= 0) {
+         return nSkipped;
+       }
+       nSkipped += ret;
+     }
+     return nSkipped;
+   }
+ 
+   @Override
+   public int read() throws IOException {
+     throw new IOException("read() is not expected to be invoked. " +
+                           "Use read(buf, off, len) instead.");
+   }
+   
+   @Override
+   public boolean seekToNewSource(long targetPos) throws IOException {
+     /* Checksum errors are handled outside the BlockReader. 
+      * DFSInputStream does not always call 'seekToNewSource'. In the 
+      * case of pread(), it just tries a different replica without seeking.
+      */ 
+     return false;
+   }
+   
+   @Override
+   public void seek(long pos) throws IOException {
+     throw new IOException("Seek() is not supported in BlockInputChecker");
+   }
+ 
+   @Override
+   protected long getChunkPosition(long pos) {
+     throw new RuntimeException("getChunkPosition() is not supported, " +
+                                "since seek is not required");
+   }
+   
+   /**
+    * Makes sure that checksumBytes has enough capacity 
+    * and limit is set to the number of checksum bytes needed 
+    * to be read.
+    */
+   private void adjustChecksumBytes(int dataLen) {
+     int requiredSize = 
+       ((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
+     if (checksumBytes == null || requiredSize > checksumBytes.capacity()) {
+       checksumBytes =  ByteBuffer.wrap(new byte[requiredSize]);
+     } else {
+       checksumBytes.clear();
+     }
+     checksumBytes.limit(requiredSize);
+   }
+   
+   @Override
+   protected synchronized int readChunk(long pos, byte[] buf, int offset, 
+                                        int len, byte[] checksumBuf) 
+                                        throws IOException {
+     TraceScope scope =
+         Trace.startSpan("RemoteBlockReader#readChunk(" + blockId + ")",
+             Sampler.NEVER);
+     try {
+       return readChunkImpl(pos, buf, offset, len, checksumBuf);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   private synchronized int readChunkImpl(long pos, byte[] buf, int offset,
+                                      int len, byte[] checksumBuf)
+                                      throws IOException {
+     // Read one chunk.
+     if (eos) {
+       // Already hit EOF
+       return -1;
+     }
+     
+     // Read one DATA_CHUNK.
+     long chunkOffset = lastChunkOffset;
+     if ( lastChunkLen > 0 ) {
+       chunkOffset += lastChunkLen;
+     }
+     
+     // pos is relative to the start of the first chunk of the read.
+     // chunkOffset is relative to the start of the block.
+     // This makes sure that the read passed from FSInputChecker is the
+     // for the same chunk we expect to be reading from the DN.
+     if ( (pos + firstChunkOffset) != chunkOffset ) {
+       throw new IOException("Mismatch in pos : " + pos + " + " + 
+                             firstChunkOffset + " != " + chunkOffset);
+     }
+ 
+     // Read next packet if the previous packet has been read completely.
+     if (dataLeft <= 0) {
+       //Read packet headers.
+       PacketHeader header = new PacketHeader();
+       header.readFields(in);
+ 
+       if (LOG.isDebugEnabled()) {
+         LOG.debug("DFSClient readChunk got header " + header);
+       }
+ 
+       // Sanity check the lengths
+       if (!header.sanityCheck(lastSeqNo)) {
+            throw new IOException("BlockReader: error in packet header " +
+                                  header);
+       }
+ 
+       lastSeqNo = header.getSeqno();
+       dataLeft = header.getDataLen();
+       adjustChecksumBytes(header.getDataLen());
+       if (header.getDataLen() > 0) {
+         IOUtils.readFully(in, checksumBytes.array(), 0,
+                           checksumBytes.limit());
+       }
+     }
+ 
+     // Sanity checks
+     assert len >= bytesPerChecksum;
+     assert checksum != null;
+     assert checksumSize == 0 || (checksumBuf.length % checksumSize == 0);
+ 
+ 
+     int checksumsToRead, bytesToRead;
+ 
+     if (checksumSize > 0) {
+ 
+       // How many chunks left in our packet - this is a ceiling
+       // since we may have a partial chunk at the end of the file
+       int chunksLeft = (dataLeft - 1) / bytesPerChecksum + 1;
+ 
+       // How many chunks we can fit in databuffer
+       //  - note this is a floor since we always read full chunks
+       int chunksCanFit = Math.min(len / bytesPerChecksum,
+                                   checksumBuf.length / checksumSize);
+ 
+       // How many chunks should we read
+       checksumsToRead = Math.min(chunksLeft, chunksCanFit);
+       // How many bytes should we actually read
+       bytesToRead = Math.min(
+         checksumsToRead * bytesPerChecksum, // full chunks
+         dataLeft); // in case we have a partial
+     } else {
+       // no checksum
+       bytesToRead = Math.min(dataLeft, len);
+       checksumsToRead = 0;
+     }
+ 
+     if ( bytesToRead > 0 ) {
+       // Assert we have enough space
+       assert bytesToRead <= len;
+       assert checksumBytes.remaining() >= checksumSize * checksumsToRead;
+       assert checksumBuf.length >= checksumSize * checksumsToRead;
+       IOUtils.readFully(in, buf, offset, bytesToRead);
+       checksumBytes.get(checksumBuf, 0, checksumSize * checksumsToRead);
+     }
+ 
+     dataLeft -= bytesToRead;
+     assert dataLeft >= 0;
+ 
+     lastChunkOffset = chunkOffset;
+     lastChunkLen = bytesToRead;
+ 
+     // If there's no data left in the current packet after satisfying
+     // this read, and we have satisfied the client read, we expect
+     // an empty packet header from the DN to signify this.
+     // Note that pos + bytesToRead may in fact be greater since the
+     // DN finishes off the entire last chunk.
+     if (dataLeft == 0 &&
+         pos + bytesToRead >= bytesNeededToFinish) {
+ 
+       // Read header
+       PacketHeader hdr = new PacketHeader();
+       hdr.readFields(in);
+ 
+       if (!hdr.isLastPacketInBlock() ||
+           hdr.getDataLen() != 0) {
+         throw new IOException("Expected empty end-of-read packet! Header: " +
+                               hdr);
+       }
+ 
+       eos = true;
+     }
+ 
+     if ( bytesToRead == 0 ) {
+       return -1;
+     }
+ 
+     return bytesToRead;
+   }
+   
+   private RemoteBlockReader(String file, String bpid, long blockId,
+       DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
+       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
+       DatanodeID datanodeID, PeerCache peerCache) {
+     // Path is used only for printing block and file information in debug
+     super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
+                     ":" + bpid + ":of:"+ file)/*too non path-like?*/,
+           1, verifyChecksum,
+           checksum.getChecksumSize() > 0? checksum : null, 
+           checksum.getBytesPerChecksum(),
+           checksum.getChecksumSize());
+ 
+     this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
+         createSocketAddr(datanodeID.getXferAddr()));
+     
+     this.peer = peer;
+     this.datanodeID = datanodeID;
+     this.in = in;
+     this.checksum = checksum;
+     this.startOffset = Math.max( startOffset, 0 );
+     this.blockId = blockId;
+ 
+     // The total number of bytes that we need to transfer from the DN is
+     // the amount that the user wants (bytesToRead), plus the padding at
+     // the beginning in order to chunk-align. Note that the DN may elect
+     // to send more than this amount if the read starts/ends mid-chunk.
+     this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
+ 
+     this.firstChunkOffset = firstChunkOffset;
+     lastChunkOffset = firstChunkOffset;
+     lastChunkLen = -1;
+ 
+     bytesPerChecksum = this.checksum.getBytesPerChecksum();
+     checksumSize = this.checksum.getChecksumSize();
+     this.peerCache = peerCache;
+   }
+ 
+   /**
+    * Create a new BlockReader specifically to satisfy a read.
+    * This method also sends the OP_READ_BLOCK request.
+    *
+    * @param file  File location
+    * @param block  The block object
+    * @param blockToken  The block token for security
+    * @param startOffset  The read offset, relative to block head
+    * @param len  The number of bytes to read
+    * @param bufferSize  The IO buffer size (not the client buffer size)
+    * @param verifyChecksum  Whether to verify checksum
+    * @param clientName  Client name
+    * @return New BlockReader instance, or null on error.
+    */
+   public static RemoteBlockReader newBlockReader(String file,
+                                      ExtendedBlock block, 
+                                      Token<BlockTokenIdentifier> blockToken,
+                                      long startOffset, long len,
+                                      int bufferSize, boolean verifyChecksum,
+                                      String clientName, Peer peer,
+                                      DatanodeID datanodeID,
+                                      PeerCache peerCache,
+                                      CachingStrategy cachingStrategy)
+                                        throws IOException {
+     // in and out will be closed when sock is closed (by the caller)
+     final DataOutputStream out =
+         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
+     new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
+         verifyChecksum, cachingStrategy);
+     
+     //
+     // Get bytes in block, set streams
+     //
+ 
+     DataInputStream in = new DataInputStream(
+         new BufferedInputStream(peer.getInputStream(), bufferSize));
+     
+     BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
+         PBHelperClient.vintPrefixed(in));
+     RemoteBlockReader2.checkSuccess(status, peer, block, file);
+     ReadOpChecksumInfoProto checksumInfo =
+       status.getReadOpChecksumInfo();
+     DataChecksum checksum = DataTransferProtoUtil.fromProto(
+         checksumInfo.getChecksum());
+     //Warning when we get CHECKSUM_NULL?
+     
+     // Read the first chunk offset.
+     long firstChunkOffset = checksumInfo.getChunkOffset();
+     
+     if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
+         firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
+       throw new IOException("BlockReader: error in first chunk offset (" +
+                             firstChunkOffset + ") startOffset is " + 
+                             startOffset + " for file " + file);
+     }
+ 
+     return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
+         in, checksum, verifyChecksum, startOffset, firstChunkOffset, len,
+         peer, datanodeID, peerCache);
+   }
+ 
+   @Override
+   public synchronized void close() throws IOException {
+     startOffset = -1;
+     checksum = null;
+     if (peerCache != null & sentStatusCode) {
+       peerCache.put(datanodeID, peer);
+     } else {
+       peer.close();
+     }
+ 
+     // in will be closed when its Socket is closed.
+   }
+   
+   @Override
+   public void readFully(byte[] buf, int readOffset, int amtToRead)
+       throws IOException {
+     IOUtils.readFully(this, buf, readOffset, amtToRead);
+   }
+ 
+   @Override
+   public int readAll(byte[] buf, int offset, int len) throws IOException {
+     return readFully(this, buf, offset, len);
+   }
+ 
+   /**
+    * When the reader reaches end of the read, it sends a status response
+    * (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
+    * closing our connection (which we will re-open), but won't affect
+    * data correctness.
+    */
+   void sendReadResult(Peer peer, Status statusCode) {
+     assert !sentStatusCode : "already sent status code to " + peer;
+     try {
+       RemoteBlockReader2.writeReadResult(peer.getOutputStream(), statusCode);
+       sentStatusCode = true;
+     } catch (IOException e) {
+       // It's ok not to be able to send this. But something is probably wrong.
+       LOG.info("Could not send read status (" + statusCode + ") to datanode " +
+                peer.getRemoteAddressString() + ": " + e.getMessage());
+     }
+   }
+ 
+   @Override
+   public int read(ByteBuffer buf) throws IOException {
+     throw new UnsupportedOperationException("readDirect unsupported in RemoteBlockReader");
+   }
+   
+   @Override
+   public int available() throws IOException {
+     // An optimistic estimate of how much data is available
+     // to us without doing network I/O.
+     return RemoteBlockReader2.TCP_WINDOW_SIZE;
+   }
+ 
+   @Override
+   public boolean isLocal() {
+     return isLocal;
+   }
+   
+   @Override
+   public boolean isShortCircuit() {
+     return false;
+   }
+ 
+   @Override
+   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
+     return null;
+   }
++
++  @Override
++  public DataChecksum getDataChecksum() {
++    return checksum;
++  }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index 0000000,5541e6d..9699442
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@@ -1,0 -1,480 +1,485 @@@
+ /**
+  * 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.hdfs;
+ 
+ import java.io.BufferedOutputStream;
+ import java.io.DataInputStream;
+ import java.io.DataOutputStream;
+ import java.io.IOException;
+ import java.io.OutputStream;
+ import java.net.InetSocketAddress;
+ import java.nio.ByteBuffer;
+ import java.nio.channels.ReadableByteChannel;
+ import java.util.EnumSet;
+ import java.util.UUID;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.fs.ReadOption;
+ import org.apache.hadoop.hdfs.net.Peer;
+ import org.apache.hadoop.hdfs.protocol.DatanodeID;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+ import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+ import org.apache.hadoop.net.NetUtils;
+ import org.apache.hadoop.security.token.Token;
+ import org.apache.hadoop.util.DataChecksum;
+ import org.apache.htrace.Sampler;
+ import org.apache.htrace.Trace;
+ import org.apache.htrace.TraceScope;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ 
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ /**
+  * This is a wrapper around connection to datanode
+  * and understands checksum, offset etc.
+  *
+  * Terminology:
+  * <dl>
+  * <dt>block</dt>
+  *   <dd>The hdfs block, typically large (~64MB).
+  *   </dd>
+  * <dt>chunk</dt>
+  *   <dd>A block is divided into chunks, each comes with a checksum.
+  *       We want transfers to be chunk-aligned, to be able to
+  *       verify checksums.
+  *   </dd>
+  * <dt>packet</dt>
+  *   <dd>A grouping of chunks used for transport. It contains a
+  *       header, followed by checksum data, followed by real data.
+  *   </dd>
+  * </dl>
+  * Please see DataNode for the RPC specification.
+  *
+  * This is a new implementation introduced in Hadoop 0.23 which
+  * is more efficient and simpler than the older BlockReader
+  * implementation. It should be renamed to RemoteBlockReader
+  * once we are confident in it.
+  */
+ @InterfaceAudience.Private
+ public class RemoteBlockReader2  implements BlockReader {
+ 
+   static final Logger LOG = LoggerFactory.getLogger(RemoteBlockReader2.class);
+   static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB;
+ 
+   final private Peer peer;
+   final private DatanodeID datanodeID;
+   final private PeerCache peerCache;
+   final private long blockId;
+   private final ReadableByteChannel in;
+ 
+   private DataChecksum checksum;
+   private final PacketReceiver packetReceiver = new PacketReceiver(true);
+ 
+   private ByteBuffer curDataSlice = null;
+ 
+   /** offset in block of the last chunk received */
+   private long lastSeqNo = -1;
+ 
+   /** offset in block where reader wants to actually read */
+   private long startOffset;
+   private final String filename;
+ 
+   private final int bytesPerChecksum;
+   private final int checksumSize;
+ 
+   /**
+    * The total number of bytes we need to transfer from the DN.
+    * This is the amount that the user has requested plus some padding
+    * at the beginning so that the read can begin on a chunk boundary.
+    */
+   private long bytesNeededToFinish;
+ 
+   /**
+    * True if we are reading from a local DataNode.
+    */
+   private final boolean isLocal;
+ 
+   private final boolean verifyChecksum;
+ 
+   private boolean sentStatusCode = false;
+ 
+   @VisibleForTesting
+   public Peer getPeer() {
+     return peer;
+   }
+   
+   @Override
+   public synchronized int read(byte[] buf, int off, int len) 
+                                throws IOException {
+ 
+     UUID randomId = null;
+     if (LOG.isTraceEnabled()) {
+       randomId = UUID.randomUUID();
+       LOG.trace(String.format("Starting read #%s file %s from datanode %s",
+         randomId.toString(), this.filename,
+         this.datanodeID.getHostName()));
+     }
+ 
+     if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+       TraceScope scope = Trace.startSpan(
+           "RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER);
+       try {
+         readNextPacket();
+       } finally {
+         scope.close();
+       }
+     }
+ 
+     if (LOG.isTraceEnabled()) {
+       LOG.trace(String.format("Finishing read #" + randomId));
+     }
+ 
+     if (curDataSlice.remaining() == 0) {
+       // we're at EOF now
+       return -1;
+     }
+     
+     int nRead = Math.min(curDataSlice.remaining(), len);
+     curDataSlice.get(buf, off, nRead);
+     
+     return nRead;
+   }
+ 
+ 
+   @Override
+   public synchronized int read(ByteBuffer buf) throws IOException {
+     if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+       TraceScope scope = Trace.startSpan(
+           "RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER);
+       try {
+         readNextPacket();
+       } finally {
+         scope.close();
+       }
+     }
+     if (curDataSlice.remaining() == 0) {
+       // we're at EOF now
+       return -1;
+     }
+ 
+     int nRead = Math.min(curDataSlice.remaining(), buf.remaining());
+     ByteBuffer writeSlice = curDataSlice.duplicate();
+     writeSlice.limit(writeSlice.position() + nRead);
+     buf.put(writeSlice);
+     curDataSlice.position(writeSlice.position());
+ 
+     return nRead;
+   }
+ 
+   private void readNextPacket() throws IOException {
+     //Read packet headers.
+     packetReceiver.receiveNextPacket(in);
+ 
+     PacketHeader curHeader = packetReceiver.getHeader();
+     curDataSlice = packetReceiver.getDataSlice();
+     assert curDataSlice.capacity() == curHeader.getDataLen();
+     
+     if (LOG.isTraceEnabled()) {
+       LOG.trace("DFSClient readNextPacket got header " + curHeader);
+     }
+ 
+     // Sanity check the lengths
+     if (!curHeader.sanityCheck(lastSeqNo)) {
+          throw new IOException("BlockReader: error in packet header " +
+                                curHeader);
+     }
+     
+     if (curHeader.getDataLen() > 0) {
+       int chunks = 1 + (curHeader.getDataLen() - 1) / bytesPerChecksum;
+       int checksumsLen = chunks * checksumSize;
+ 
+       assert packetReceiver.getChecksumSlice().capacity() == checksumsLen :
+         "checksum slice capacity=" + packetReceiver.getChecksumSlice().capacity() + 
+           " checksumsLen=" + checksumsLen;
+       
+       lastSeqNo = curHeader.getSeqno();
+       if (verifyChecksum && curDataSlice.remaining() > 0) {
+         // N.B.: the checksum error offset reported here is actually
+         // relative to the start of the block, not the start of the file.
+         // This is slightly misleading, but preserves the behavior from
+         // the older BlockReader.
+         checksum.verifyChunkedSums(curDataSlice,
+             packetReceiver.getChecksumSlice(),
+             filename, curHeader.getOffsetInBlock());
+       }
+       bytesNeededToFinish -= curHeader.getDataLen();
+     }    
+     
+     // First packet will include some data prior to the first byte
+     // the user requested. Skip it.
+     if (curHeader.getOffsetInBlock() < startOffset) {
+       int newPos = (int) (startOffset - curHeader.getOffsetInBlock());
+       curDataSlice.position(newPos);
+     }
+ 
+     // If we've now satisfied the whole client read, read one last packet
+     // header, which should be empty
+     if (bytesNeededToFinish <= 0) {
+       readTrailingEmptyPacket();
+       if (verifyChecksum) {
+         sendReadResult(Status.CHECKSUM_OK);
+       } else {
+         sendReadResult(Status.SUCCESS);
+       }
+     }
+   }
+   
+   @Override
+   public synchronized long skip(long n) throws IOException {
+     /* How can we make sure we don't throw a ChecksumException, at least
+      * in majority of the cases?. This one throws. */
+     long skipped = 0;
+     while (skipped < n) {
+       long needToSkip = n - skipped;
+       if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+         readNextPacket();
+       }
+       if (curDataSlice.remaining() == 0) {
+         // we're at EOF now
+         break;
+       }
+ 
+       int skip = (int)Math.min(curDataSlice.remaining(), needToSkip);
+       curDataSlice.position(curDataSlice.position() + skip);
+       skipped += skip;
+     }
+     return skipped;
+   }
+ 
+   private void readTrailingEmptyPacket() throws IOException {
+     if (LOG.isTraceEnabled()) {
+       LOG.trace("Reading empty packet at end of read");
+     }
+     
+     packetReceiver.receiveNextPacket(in);
+ 
+     PacketHeader trailer = packetReceiver.getHeader();
+     if (!trailer.isLastPacketInBlock() ||
+        trailer.getDataLen() != 0) {
+       throw new IOException("Expected empty end-of-read packet! Header: " +
+                             trailer);
+     }
+   }
+ 
+   protected RemoteBlockReader2(String file, String bpid, long blockId,
+       DataChecksum checksum, boolean verifyChecksum,
+       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
+       DatanodeID datanodeID, PeerCache peerCache) {
+     this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
+         createSocketAddr(datanodeID.getXferAddr()));
+     // Path is used only for printing block and file information in debug
+     this.peer = peer;
+     this.datanodeID = datanodeID;
+     this.in = peer.getInputStreamChannel();
+     this.checksum = checksum;
+     this.verifyChecksum = verifyChecksum;
+     this.startOffset = Math.max( startOffset, 0 );
+     this.filename = file;
+     this.peerCache = peerCache;
+     this.blockId = blockId;
+ 
+     // The total number of bytes that we need to transfer from the DN is
+     // the amount that the user wants (bytesToRead), plus the padding at
+     // the beginning in order to chunk-align. Note that the DN may elect
+     // to send more than this amount if the read starts/ends mid-chunk.
+     this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
+     bytesPerChecksum = this.checksum.getBytesPerChecksum();
+     checksumSize = this.checksum.getChecksumSize();
+   }
+ 
+ 
+   @Override
+   public synchronized void close() throws IOException {
+     packetReceiver.close();
+     startOffset = -1;
+     checksum = null;
+     if (peerCache != null && sentStatusCode) {
+       peerCache.put(datanodeID, peer);
+     } else {
+       peer.close();
+     }
+ 
+     // in will be closed when its Socket is closed.
+   }
+   
+   /**
+    * When the reader reaches end of the read, it sends a status response
+    * (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
+    * closing our connection (which we will re-open), but won't affect
+    * data correctness.
+    */
+   void sendReadResult(Status statusCode) {
+     assert !sentStatusCode : "already sent status code to " + peer;
+     try {
+       writeReadResult(peer.getOutputStream(), statusCode);
+       sentStatusCode = true;
+     } catch (IOException e) {
+       // It's ok not to be able to send this. But something is probably wrong.
+       LOG.info("Could not send read status (" + statusCode + ") to datanode " +
+                peer.getRemoteAddressString() + ": " + e.getMessage());
+     }
+   }
+ 
+   /**
+    * Serialize the actual read result on the wire.
+    */
+   static void writeReadResult(OutputStream out, Status statusCode)
+       throws IOException {
+     
+     ClientReadStatusProto.newBuilder()
+       .setStatus(statusCode)
+       .build()
+       .writeDelimitedTo(out);
+ 
+     out.flush();
+   }
+   
+   /**
+    * File name to print when accessing a block directly (from servlets)
+    * @param s Address of the block location
+    * @param poolId Block pool ID of the block
+    * @param blockId Block ID of the block
+    * @return string that has a file name for debug purposes
+    */
+   public static String getFileName(final InetSocketAddress s,
+       final String poolId, final long blockId) {
+     return s.toString() + ":" + poolId + ":" + blockId;
+   }
+ 
+   @Override
+   public int readAll(byte[] buf, int offset, int len) throws IOException {
+     return BlockReaderUtil.readAll(this, buf, offset, len);
+   }
+ 
+   @Override
+   public void readFully(byte[] buf, int off, int len) throws IOException {
+     BlockReaderUtil.readFully(this, buf, off, len);
+   }
+   
+   /**
+    * Create a new BlockReader specifically to satisfy a read.
+    * This method also sends the OP_READ_BLOCK request.
+    *
+    * @param file  File location
+    * @param block  The block object
+    * @param blockToken  The block token for security
+    * @param startOffset  The read offset, relative to block head
+    * @param len  The number of bytes to read
+    * @param verifyChecksum  Whether to verify checksum
+    * @param clientName  Client name
+    * @param peer  The Peer to use
+    * @param datanodeID  The DatanodeID this peer is connected to
+    * @return New BlockReader instance, or null on error.
+    */
+   public static BlockReader newBlockReader(String file,
+                                      ExtendedBlock block,
+                                      Token<BlockTokenIdentifier> blockToken,
+                                      long startOffset, long len,
+                                      boolean verifyChecksum,
+                                      String clientName,
+                                      Peer peer, DatanodeID datanodeID,
+                                      PeerCache peerCache,
+                                      CachingStrategy cachingStrategy) throws IOException {
+     // in and out will be closed when sock is closed (by the caller)
+     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
+           peer.getOutputStream()));
+     new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
+         verifyChecksum, cachingStrategy);
+ 
+     //
+     // Get bytes in block
+     //
+     DataInputStream in = new DataInputStream(peer.getInputStream());
+ 
+     BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
+         PBHelperClient.vintPrefixed(in));
+     checkSuccess(status, peer, block, file);
+     ReadOpChecksumInfoProto checksumInfo =
+       status.getReadOpChecksumInfo();
+     DataChecksum checksum = DataTransferProtoUtil.fromProto(
+         checksumInfo.getChecksum());
+     //Warning when we get CHECKSUM_NULL?
+ 
+     // Read the first chunk offset.
+     long firstChunkOffset = checksumInfo.getChunkOffset();
+ 
+     if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
+         firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
+       throw new IOException("BlockReader: error in first chunk offset (" +
+                             firstChunkOffset + ") startOffset is " +
+                             startOffset + " for file " + file);
+     }
+ 
+     return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
+         checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
+         datanodeID, peerCache);
+   }
+ 
+   static void checkSuccess(
+       BlockOpResponseProto status, Peer peer,
+       ExtendedBlock block, String file)
+       throws IOException {
+     String logInfo = "for OP_READ_BLOCK"
+       + ", self=" + peer.getLocalAddressString()
+       + ", remote=" + peer.getRemoteAddressString()
+       + ", for file " + file
+       + ", for pool " + block.getBlockPoolId()
+       + " block " + block.getBlockId() + "_" + block.getGenerationStamp();
+     DataTransferProtoUtil.checkBlockOpStatus(status, logInfo);
+   }
+   
+   @Override
+   public int available() throws IOException {
+     // An optimistic estimate of how much data is available
+     // to us without doing network I/O.
+     return TCP_WINDOW_SIZE;
+   }
+   
+   @Override
+   public boolean isLocal() {
+     return isLocal;
+   }
+   
+   @Override
+   public boolean isShortCircuit() {
+     return false;
+   }
+ 
+   @Override
+   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
+     return null;
+   }
++
++  @Override
++  public DataChecksum getDataChecksum() {
++    return checksum;
++  }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 36da863,9f26ca3..97445a6
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@@ -961,8 -953,8 +961,8 @@@ public class ClientNamenodeProtocolServ
        RpcController controller, UpdateBlockForPipelineRequestProto req)
        throws ServiceException {
      try {
 -      LocatedBlockProto result = PBHelper.convert(server
 -          .updateBlockForPipeline(PBHelperClient.convert(req.getBlock()),
 +      LocatedBlockProto result = PBHelper.convertLocatedBlock(
-           server.updateBlockForPipeline(PBHelper.convert(req.getBlock()),
++          server.updateBlockForPipeline(PBHelperClient.convert(req.getBlock()),
                req.getClientName()));
        return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result)
            .build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index f292ee8,6f16d83..f419c46
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@@ -23,18 -23,11 +23,16 @@@ import static org.apache.hadoop.hdfs.pr
  import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CipherSuiteProto;
  import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
  
- import java.io.EOFException;
  import java.io.IOException;
- import java.io.InputStream;
  import java.util.ArrayList;
  import java.util.Arrays;
 +import java.util.Collection;
  import java.util.EnumSet;
 +import java.util.HashMap;
  import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
  
  import org.apache.hadoop.fs.CacheFlag;
  import org.apache.hadoop.fs.ContentSummary;
@@@ -134,13 -122,9 +131,14 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
++import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
@@@ -233,15 -214,11 +230,12 @@@ import org.apache.hadoop.hdfs.server.pr
  import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
  import org.apache.hadoop.hdfs.server.protocol.StorageReport;
  import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
- import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
  import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
- import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
  import org.apache.hadoop.io.EnumSetWritable;
  import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.io.erasurecode.ECSchema;
  import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
  import org.apache.hadoop.security.token.Token;
- import org.apache.hadoop.util.DataChecksum;
  
  import com.google.common.base.Preconditions;
  import com.google.common.collect.Lists;
@@@ -784,23 -726,9 +771,23 @@@ public class PBHelper 
        }
      }
  
 -    LocatedBlock lb = new LocatedBlock(PBHelperClient.convert(proto.getB()), targets,
 -        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
 -        cachedLocs.toArray(new DatanodeInfo[0]));
 +    final LocatedBlock lb;
 +    if (indices == null) {
-       lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets, storageIDs,
-           storageTypes, proto.getOffset(), proto.getCorrupt(),
++      lb = new LocatedBlock(PBHelperClient.convert(proto.getB()), targets,
++          storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
 +          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
 +    } else {
-       lb = new LocatedStripedBlock(PBHelper.convert(proto.getB()), targets,
++      lb = new LocatedStripedBlock(PBHelperClient.convert(proto.getB()), targets,
 +          storageIDs, storageTypes, indices, proto.getOffset(),
 +          proto.getCorrupt(),
 +          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
 +      List<TokenProto> tokenProtos = proto.getBlockTokensList();
 +      Token<BlockTokenIdentifier>[] blockTokens = new Token[indices.length];
 +      for (int i = 0; i < indices.length; i++) {
 +        blockTokens[i] = PBHelper.convert(tokenProtos.get(i));
 +      }
 +      ((LocatedStripedBlock) lb).setBlockTokens(blockTokens);
 +    }
      lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
  
      return lb;
@@@ -2954,192 -2860,4 +2935,192 @@@
          setLeaseId(context.getLeaseId()).
          build();
    }
 +
 +  public static ECSchema convertECSchema(ECSchemaProto schema) {
 +    List<ECSchemaOptionEntryProto> optionsList = schema.getOptionsList();
 +    Map<String, String> options = new HashMap<>(optionsList.size());
 +    for (ECSchemaOptionEntryProto option : optionsList) {
 +      options.put(option.getKey(), option.getValue());
 +    }
 +    return new ECSchema(schema.getCodecName(), schema.getDataUnits(),
 +        schema.getParityUnits(), options);
 +  }
 +
 +  public static ECSchemaProto convertECSchema(ECSchema schema) {
 +    ECSchemaProto.Builder builder = ECSchemaProto.newBuilder()
 +        .setCodecName(schema.getCodecName())
 +        .setDataUnits(schema.getNumDataUnits())
 +        .setParityUnits(schema.getNumParityUnits());
 +    Set<Entry<String, String>> entrySet = schema.getExtraOptions().entrySet();
 +    for (Entry<String, String> entry : entrySet) {
 +      builder.addOptions(ECSchemaOptionEntryProto.newBuilder()
 +          .setKey(entry.getKey()).setValue(entry.getValue()).build());
 +    }
 +    return builder.build();
 +  }
 +
 +  public static ErasureCodingPolicy convertErasureCodingPolicy(
 +      ErasureCodingPolicyProto policy) {
 +    return new ErasureCodingPolicy(policy.getName(),
 +        convertECSchema(policy.getSchema()),
 +        policy.getCellSize());
 +  }
 +
 +  public static ErasureCodingPolicyProto convertErasureCodingPolicy(
 +      ErasureCodingPolicy policy) {
 +    ErasureCodingPolicyProto.Builder builder = ErasureCodingPolicyProto
 +        .newBuilder()
 +        .setName(policy.getName())
 +        .setSchema(convertECSchema(policy.getSchema()))
 +        .setCellSize(policy.getCellSize());
 +    return builder.build();
 +  }
 +
 +  public static ErasureCodingZoneProto convertErasureCodingZone(
 +      ErasureCodingZone ecZone) {
 +    return ErasureCodingZoneProto.newBuilder().setDir(ecZone.getDir())
 +        .setEcPolicy(convertErasureCodingPolicy(ecZone.getErasureCodingPolicy()))
 +        .build();
 +  }
 +
 +  public static ErasureCodingZone convertErasureCodingZone(
 +      ErasureCodingZoneProto ecZoneProto) {
 +    return new ErasureCodingZone(ecZoneProto.getDir(),
 +        convertErasureCodingPolicy(ecZoneProto.getEcPolicy()));
 +  }
 +  
 +  public static BlockECRecoveryInfo convertBlockECRecoveryInfo(
 +      BlockECRecoveryInfoProto blockEcRecoveryInfoProto) {
 +    ExtendedBlockProto blockProto = blockEcRecoveryInfoProto.getBlock();
-     ExtendedBlock block = convert(blockProto);
++    ExtendedBlock block = PBHelperClient.convert(blockProto);
 +
 +    DatanodeInfosProto sourceDnInfosProto = blockEcRecoveryInfoProto
 +        .getSourceDnInfos();
 +    DatanodeInfo[] sourceDnInfos = convert(sourceDnInfosProto);
 +
 +    DatanodeInfosProto targetDnInfosProto = blockEcRecoveryInfoProto
 +        .getTargetDnInfos();
 +    DatanodeInfo[] targetDnInfos = convert(targetDnInfosProto);
 +
 +    StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto
 +        .getTargetStorageUuids();
 +    String[] targetStorageUuids = convert(targetStorageUuidsProto);
 +
 +    StorageTypesProto targetStorageTypesProto = blockEcRecoveryInfoProto
 +        .getTargetStorageTypes();
 +    StorageType[] convertStorageTypes = convertStorageTypes(
 +        targetStorageTypesProto.getStorageTypesList(), targetStorageTypesProto
 +            .getStorageTypesList().size());
 +
 +    List<Integer> liveBlockIndicesList = blockEcRecoveryInfoProto
 +        .getLiveBlockIndicesList();
 +    short[] liveBlkIndices = new short[liveBlockIndicesList.size()];
 +    for (int i = 0; i < liveBlockIndicesList.size(); i++) {
 +      liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue();
 +    }
 +
 +    ErasureCodingPolicy ecPolicy =
 +        convertErasureCodingPolicy(blockEcRecoveryInfoProto.getEcPolicy());
 +
 +    return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos,
 +        targetStorageUuids, convertStorageTypes, liveBlkIndices, ecPolicy);
 +  }
 +
 +  public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo(
 +      BlockECRecoveryInfo blockEcRecoveryInfo) {
 +    BlockECRecoveryInfoProto.Builder builder = BlockECRecoveryInfoProto
 +        .newBuilder();
 +    builder.setBlock(PBHelperClient.convert(
 +        blockEcRecoveryInfo.getExtendedBlock()));
 +
 +    DatanodeInfo[] sourceDnInfos = blockEcRecoveryInfo.getSourceDnInfos();
 +    builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos));
 +
 +    DatanodeInfo[] targetDnInfos = blockEcRecoveryInfo.getTargetDnInfos();
 +    builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos));
 +
 +    String[] targetStorageIDs = blockEcRecoveryInfo.getTargetStorageIDs();
 +    builder.setTargetStorageUuids(convertStorageIDs(targetStorageIDs));
 +
 +    StorageType[] targetStorageTypes = blockEcRecoveryInfo
 +        .getTargetStorageTypes();
 +    builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes));
 +
 +    short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices();
 +    builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices));
 +
 +    builder.setEcPolicy(convertErasureCodingPolicy(blockEcRecoveryInfo
 +        .getErasureCodingPolicy()));
 +
 +    return builder.build();
 +  }
 +
 +  private static List<Integer> convertIntArray(short[] liveBlockIndices) {
 +    List<Integer> liveBlockIndicesList = new ArrayList<Integer>();
 +    for (short s : liveBlockIndices) {
 +      liveBlockIndicesList.add((int) s);
 +    }
 +    return liveBlockIndicesList;
 +  }
 +
 +  private static StorageTypesProto convertStorageTypesProto(
 +      StorageType[] targetStorageTypes) {
 +    StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
 +    for (StorageType storageType : targetStorageTypes) {
 +      builder.addStorageTypes(PBHelperClient.convertStorageType(storageType));
 +    }
 +    return builder.build();
 +  }
 +
 +  private static StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) {
 +    StorageUuidsProto.Builder builder = StorageUuidsProto.newBuilder();
 +    for (String storageUuid : targetStorageIDs) {
 +      builder.addStorageUuids(storageUuid);
 +    }
 +    return builder.build();
 +  }
 +
 +  private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) {
 +    DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder();
 +    for (DatanodeInfo datanodeInfo : dnInfos) {
 +      builder.addDatanodes(PBHelperClient.convert(datanodeInfo));
 +    }
 +    return builder.build();
 +  }
 +
 +  private static String[] convert(StorageUuidsProto targetStorageUuidsProto) {
 +    List<String> storageUuidsList = targetStorageUuidsProto
 +        .getStorageUuidsList();
 +    String[] storageUuids = new String[storageUuidsList.size()];
 +    for (int i = 0; i < storageUuidsList.size(); i++) {
 +      storageUuids[i] = storageUuidsList.get(i);
 +    }
 +    return storageUuids;
 +  }
 +  
 +  public static BlockECRecoveryCommandProto convert(
 +      BlockECRecoveryCommand blkECRecoveryCmd) {
 +    BlockECRecoveryCommandProto.Builder builder = BlockECRecoveryCommandProto
 +        .newBuilder();
 +    Collection<BlockECRecoveryInfo> blockECRecoveryInfos = blkECRecoveryCmd
 +        .getECTasks();
 +    for (BlockECRecoveryInfo blkECRecoveryInfo : blockECRecoveryInfos) {
 +      builder
 +          .addBlockECRecoveryinfo(convertBlockECRecoveryInfo(blkECRecoveryInfo));
 +    }
 +    return builder.build();
 +  }
 +  
 +  public static BlockECRecoveryCommand convert(
 +      BlockECRecoveryCommandProto blkECRecoveryCmdProto) {
 +    Collection<BlockECRecoveryInfo> blkECRecoveryInfos = new ArrayList<BlockECRecoveryInfo>();
 +    List<BlockECRecoveryInfoProto> blockECRecoveryinfoList = blkECRecoveryCmdProto
 +        .getBlockECRecoveryinfoList();
 +    for (BlockECRecoveryInfoProto blockECRecoveryInfoProto : blockECRecoveryinfoList) {
 +      blkECRecoveryInfos
 +          .add(convertBlockECRecoveryInfo(blockECRecoveryInfoProto));
 +    }
 +    return new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY,
 +        blkECRecoveryInfos);
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index dc296ac,810784d..92a1135
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@@ -30,11 -31,11 +31,12 @@@ import org.apache.hadoop.util.LightWeig
  import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID;
  
  /**
 - * BlockInfo class maintains for a given block
 - * the {@link BlockCollection} it is part of and datanodes where the replicas of
 - * the block are stored.
 + * For a given block (or an erasure coding block group), BlockInfo class
 + * maintains 1) the {@link BlockCollection} it is part of, and 2) datanodes
 + * where the replicas of the block, or blocks belonging to the erasure coding
 + * block group, are stored.
   */
+ @InterfaceAudience.Private
  public abstract class BlockInfo extends Block
      implements LightWeightGSet.LinkedElement {
  
@@@ -203,17 -206,6 +205,11 @@@
     */
    abstract boolean removeStorage(DatanodeStorageInfo storage);
  
-   /**
-    * Replace the current BlockInfo with the new one in corresponding
-    * DatanodeStorageInfo's linked list
-    */
-   abstract void replaceBlock(BlockInfo newBlock);
- 
 +  public abstract boolean isStriped();
 +
 +  /** @return true if there is no datanode storage associated with the block */
 +  abstract boolean hasNoStorage();
 +
    /**
     * Find specified DatanodeStorageInfo.
     * @return DatanodeStorageInfo or null if not found.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index b9d8486,94fb222..746e298
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@@ -95,29 -104,4 +95,14 @@@ public class BlockInfoContiguous extend
      }
      return 0;
    }
 +
 +  @Override
-   void replaceBlock(BlockInfo newBlock) {
-     assert newBlock instanceof BlockInfoContiguous;
-     for (int i = this.numNodes() - 1; i >= 0; i--) {
-       final DatanodeStorageInfo storage = this.getStorageInfo(i);
-       final boolean removed = storage.removeBlock(this);
-       assert removed : "currentBlock not found.";
- 
-       final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
-           newBlock, newBlock);
-       assert result == DatanodeStorageInfo.AddBlockResult.ADDED :
-           "newBlock already exists.";
-     }
-   }
- 
-   @Override
 +  public final boolean isStriped() {
 +    return false;
 +  }
 +
 +  @Override
 +  final boolean hasNoStorage() {
 +    return getStorageInfo(0) == null;
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53358fe6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 7b21cbe,0000000..df48655
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@@ -1,253 -1,0 +1,234 @@@
 +/**
 + * 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.hdfs.server.blockmanagement;
 +
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +
 +/**
 + * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
 + *
 + * We still use triplets to store DatanodeStorageInfo for each block in the
 + * block group, as well as the previous/next block in the corresponding
 + * DatanodeStorageInfo. For a (m+k) block group, the first (m+k) triplet units
 + * are sorted and strictly mapped to the corresponding block.
 + *
 + * Normally each block belonging to group is stored in only one DataNode.
 + * However, it is possible that some block is over-replicated. Thus the triplet
 + * array's size can be larger than (m+k). Thus currently we use an extra byte
 + * array to record the block index for each triplet.
 + */
 +public class BlockInfoStriped extends BlockInfo {
 +  private final ErasureCodingPolicy ecPolicy;
 +  /**
 +   * Always the same size with triplets. Record the block index for each triplet
 +   * TODO: actually this is only necessary for over-replicated block. Thus can
 +   * be further optimized to save memory usage.
 +   */
 +  private byte[] indices;
 +
 +  public BlockInfoStriped(Block blk, ErasureCodingPolicy ecPolicy) {
 +    super(blk, (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()));
 +    indices = new byte[ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()];
 +    initIndices();
 +    this.ecPolicy = ecPolicy;
 +  }
 +
 +  public short getTotalBlockNum() {
 +    return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits());
 +  }
 +
 +  public short getDataBlockNum() {
 +    return (short) ecPolicy.getNumDataUnits();
 +  }
 +
 +  public short getParityBlockNum() {
 +    return (short) ecPolicy.getNumParityUnits();
 +  }
 +
 +  /**
 +   * If the block is committed/completed and its length is less than a full
 +   * stripe, it returns the the number of actual data blocks.
 +   * Otherwise it returns the number of data units specified by erasure coding policy.
 +   */
 +  public short getRealDataBlockNum() {
 +    if (isComplete() || getBlockUCState() == BlockUCState.COMMITTED) {
 +      return (short) Math.min(getDataBlockNum(),
 +          (getNumBytes() - 1) / BLOCK_STRIPED_CELL_SIZE + 1);
 +    } else {
 +      return getDataBlockNum();
 +    }
 +  }
 +
 +  public short getRealTotalBlockNum() {
 +    return (short) (getRealDataBlockNum() + getParityBlockNum());
 +  }
 +
 +  public ErasureCodingPolicy getErasureCodingPolicy() {
 +    return ecPolicy;
 +  }
 +
 +  private void initIndices() {
 +    for (int i = 0; i < indices.length; i++) {
 +      indices[i] = -1;
 +    }
 +  }
 +
 +  private int findSlot() {
 +    int i = getTotalBlockNum();
 +    for (; i < getCapacity(); i++) {
 +      if (getStorageInfo(i) == null) {
 +        return i;
 +      }
 +    }
 +    // need to expand the triplet size
 +    ensureCapacity(i + 1, true);
 +    return i;
 +  }
 +
 +  @Override
 +  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
 +    int blockIndex = BlockIdManager.getBlockIndex(reportedBlock);
 +    int index = blockIndex;
 +    DatanodeStorageInfo old = getStorageInfo(index);
 +    if (old != null && !old.equals(storage)) { // over replicated
 +      // check if the storage has been stored
 +      int i = findStorageInfo(storage);
 +      if (i == -1) {
 +        index = findSlot();
 +      } else {
 +        return true;
 +      }
 +    }
 +    addStorage(storage, index, blockIndex);
 +    return true;
 +  }
 +
 +  private void addStorage(DatanodeStorageInfo storage, int index,
 +      int blockIndex) {
 +    setStorageInfo(index, storage);
 +    setNext(index, null);
 +    setPrevious(index, null);
 +    indices[index] = (byte) blockIndex;
 +  }
 +
 +  private int findStorageInfoFromEnd(DatanodeStorageInfo storage) {
 +    final int len = getCapacity();
 +    for(int idx = len - 1; idx >= 0; idx--) {
 +      DatanodeStorageInfo cur = getStorageInfo(idx);
 +      if (storage.equals(cur)) {
 +        return idx;
 +      }
 +    }
 +    return -1;
 +  }
 +
 +  int getStorageBlockIndex(DatanodeStorageInfo storage) {
 +    int i = this.findStorageInfo(storage);
 +    return i == -1 ? -1 : indices[i];
 +  }
 +
 +  /**
 +   * Identify the block stored in the given datanode storage. Note that
 +   * the returned block has the same block Id with the one seen/reported by the
 +   * DataNode.
 +   */
 +  Block getBlockOnStorage(DatanodeStorageInfo storage) {
 +    int index = getStorageBlockIndex(storage);
 +    if (index < 0) {
 +      return null;
 +    } else {
 +      Block block = new Block(this);
 +      block.setBlockId(this.getBlockId() + index);
 +      return block;
 +    }
 +  }
 +
 +  @Override
 +  boolean removeStorage(DatanodeStorageInfo storage) {
 +    int dnIndex = findStorageInfoFromEnd(storage);
 +    if (dnIndex < 0) { // the node is not found
 +      return false;
 +    }
 +    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
 +        "Block is still in the list and must be removed first.";
 +    // set the triplet to null
 +    setStorageInfo(dnIndex, null);
 +    setNext(dnIndex, null);
 +    setPrevious(dnIndex, null);
 +    indices[dnIndex] = -1;
 +    return true;
 +  }
 +
 +  private void ensureCapacity(int totalSize, boolean keepOld) {
 +    if (getCapacity() < totalSize) {
 +      Object[] old = triplets;
 +      byte[] oldIndices = indices;
 +      triplets = new Object[totalSize * 3];
 +      indices = new byte[totalSize];
 +      initIndices();
 +
 +      if (keepOld) {
 +        System.arraycopy(old, 0, triplets, 0, old.length);
 +        System.arraycopy(oldIndices, 0, indices, 0, oldIndices.length);
 +      }
 +    }
 +  }
 +
-   @Override
-   void replaceBlock(BlockInfo newBlock) {
-     assert newBlock instanceof BlockInfoStriped;
-     BlockInfoStriped newBlockGroup = (BlockInfoStriped) newBlock;
-     final int size = getCapacity();
-     newBlockGroup.ensureCapacity(size, false);
-     for (int i = 0; i < size; i++) {
-       final DatanodeStorageInfo storage = this.getStorageInfo(i);
-       if (storage != null) {
-         final int blockIndex = indices[i];
-         final boolean removed = storage.removeBlock(this);
-         assert removed : "currentBlock not found.";
- 
-         newBlockGroup.addStorage(storage, i, blockIndex);
-         storage.insertToList(newBlockGroup);
-       }
-     }
-   }
- 
 +  public long spaceConsumed() {
 +    // In case striped blocks, total usage by this striped blocks should
 +    // be the total of data blocks and parity blocks because
 +    // `getNumBytes` is the total of actual data block size.
 +    return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(),
 +        ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits(),
 +        BLOCK_STRIPED_CELL_SIZE);
 +    }
 +
 +  @Override
 +  public final boolean isStriped() {
 +    return true;
 +  }
 +
 +  @Override
 +  public int numNodes() {
 +    assert this.triplets != null : "BlockInfo is not initialized";
 +    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
 +    int num = 0;
 +    for (int idx = getCapacity()-1; idx >= 0; idx--) {
 +      if (getStorageInfo(idx) != null) {
 +        num++;
 +      }
 +    }
 +    return num;
 +  }
 +
 +  @Override
 +  final boolean hasNoStorage() {
 +    final int len = getCapacity();
 +    for(int idx = 0; idx < len; idx++) {
 +      if (getStorageInfo(idx) != null) {
 +        return false;
 +      }
 +    }
 +    return true;
 +  }
 +}


[33/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 53c6cdb,28ea866..8874c4d
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@@ -23,8 -23,8 +23,9 @@@ import java.util.concurrent.TimeUnit
  import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.fs.CommonConfigurationKeys;
  import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
  import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
  import org.apache.hadoop.http.HttpConfig;
  
@@@ -171,8 -171,8 +172,10 @@@ public class DFSConfigKeys extends Comm
    public static final int     DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT = 3;
    public static final String  DFS_NAMENODE_REPLICATION_MIN_KEY = "dfs.namenode.replication.min";
    public static final int     DFS_NAMENODE_REPLICATION_MIN_DEFAULT = 1;
 +  public static final String  DFS_NAMENODE_STRIPE_MIN_KEY = "dfs.namenode.stripe.min";
 +  public static final int     DFS_NAMENODE_STRIPE_MIN_DEFAULT = 1;
+   public static final String  DFS_NAMENODE_SAFEMODE_REPLICATION_MIN_KEY =
+       "dfs.namenode.safemode.replication.min";
    public static final String  DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY = "dfs.namenode.replication.pending.timeout-sec";
    public static final int     DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT = -1;
    public static final String  DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY = "dfs.namenode.replication.max-streams";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index ac927ef,f4cf4c2..5bf52c5
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@@ -84,9 -85,7 +85,8 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
  import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
  import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
- import org.apache.hadoop.hdfs.server.namenode.NameNode;
  import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
  import org.apache.hadoop.net.NetUtils;
  import org.apache.hadoop.security.AccessControlException;
  import org.apache.hadoop.security.Credentials;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 3217484,d93277c..1e4b899
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@@ -429,10 -422,10 +429,11 @@@ public class ClientNamenodeProtocolServ
            req.getClientName(), flags);
        AppendResponseProto.Builder builder = AppendResponseProto.newBuilder();
        if (result.getLastBlock() != null) {
-         builder.setBlock(PBHelper.convertLocatedBlock(result.getLastBlock()));
 -        builder.setBlock(PBHelperClient.convert(result.getLastBlock()));
++        builder.setBlock(PBHelperClient.convertLocatedBlock(
++            result.getLastBlock()));
        }
        if (result.getFileStatus() != null) {
-         builder.setStat(PBHelper.convert(result.getFileStatus()));
+         builder.setStat(PBHelperClient.convert(result.getFileStatus()));
        }
        return builder.build();
      } catch (IOException e) {
@@@ -505,7 -498,7 +506,7 @@@
            (favor == null || favor.size() == 0) ? null : favor
                .toArray(new String[favor.size()]));
        return AddBlockResponseProto.newBuilder()
-           .setBlock(PBHelper.convertLocatedBlock(result)).build();
 -          .setBlock(PBHelperClient.convert(result)).build();
++          .setBlock(PBHelperClient.convertLocatedBlock(result)).build();
      } catch (IOException e) {
        throw new ServiceException(e);
      }
@@@ -525,11 -518,11 +526,11 @@@
                new DatanodeInfoProto[existingList.size()])),
            existingStorageIDsList.toArray(
                new String[existingStorageIDsList.size()]),
-           PBHelper.convert(excludesList.toArray(
-               new DatanodeInfoProto[excludesList.size()])), 
+           PBHelperClient.convert(excludesList.toArray(
+               new DatanodeInfoProto[excludesList.size()])),
            req.getNumAdditionalNodes(), req.getClientName());
        return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
-           PBHelper.convertLocatedBlock(result))
 -          PBHelperClient.convert(result))
++      PBHelperClient.convertLocatedBlock(result))
            .build();
      } catch (IOException e) {
        throw new ServiceException(e);
@@@ -555,7 -548,7 +556,7 @@@
        ReportBadBlocksRequestProto req) throws ServiceException {
      try {
        List<LocatedBlockProto> bl = req.getBlocksList();
-       server.reportBadBlocks(PBHelper.convertLocatedBlocks(
 -      server.reportBadBlocks(PBHelperClient.convertLocatedBlock(
++      server.reportBadBlocks(PBHelperClient.convertLocatedBlocks(
            bl.toArray(new LocatedBlockProto[bl.size()])));
      } catch (IOException e) {
        throw new ServiceException(e);
@@@ -960,8 -953,8 +961,8 @@@
        RpcController controller, UpdateBlockForPipelineRequestProto req)
        throws ServiceException {
      try {
-       LocatedBlockProto result = PBHelper.convertLocatedBlock(
 -      LocatedBlockProto result = PBHelperClient.convert(server
 -          .updateBlockForPipeline(PBHelperClient.convert(req.getBlock()),
++      LocatedBlockProto result = PBHelperClient.convertLocatedBlock(
 +          server.updateBlockForPipeline(PBHelperClient.convert(req.getBlock()),
                req.getClientName()));
        return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result)
            .build();
@@@ -1401,20 -1394,6 +1402,20 @@@
    }
  
    @Override
 +  public SetErasureCodingPolicyResponseProto setErasureCodingPolicy(
 +      RpcController controller, SetErasureCodingPolicyRequestProto req)
 +      throws ServiceException {
 +    try {
-       ErasureCodingPolicy ecPolicy = req.hasEcPolicy() ? PBHelper.convertErasureCodingPolicy(req
-           .getEcPolicy()) : null;
++      ErasureCodingPolicy ecPolicy = req.hasEcPolicy() ?
++          PBHelperClient.convertErasureCodingPolicy(req.getEcPolicy()) : null;
 +      server.setErasureCodingPolicy(req.getSrc(), ecPolicy);
 +      return SetErasureCodingPolicyResponseProto.newBuilder().build();
 +    } catch (IOException e) {
 +      throw new ServiceException(e);
 +    }
 +  }
 +
 +  @Override
    public SetXAttrResponseProto setXAttr(RpcController controller,
        SetXAttrRequestProto req) throws ServiceException {
      try {
@@@ -1535,35 -1514,4 +1536,35 @@@
        throw new ServiceException(e);
      }
    }
 +
 +  @Override
 +  public GetErasureCodingPoliciesResponseProto getErasureCodingPolicies(RpcController controller,
 +      GetErasureCodingPoliciesRequestProto request) throws ServiceException {
 +    try {
 +      ErasureCodingPolicy[] ecPolicies = server.getErasureCodingPolicies();
 +      GetErasureCodingPoliciesResponseProto.Builder resBuilder = GetErasureCodingPoliciesResponseProto
 +          .newBuilder();
 +      for (ErasureCodingPolicy ecPolicy : ecPolicies) {
-         resBuilder.addEcPolicies(PBHelper.convertErasureCodingPolicy(ecPolicy));
++        resBuilder.addEcPolicies(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
 +      }
 +      return resBuilder.build();
 +    } catch (IOException e) {
 +      throw new ServiceException(e);
 +    }
 +  }
 +
 +  @Override
 +  public GetErasureCodingPolicyResponseProto getErasureCodingPolicy(RpcController controller,
 +      GetErasureCodingPolicyRequestProto request) throws ServiceException {
 +    try {
 +      ErasureCodingPolicy ecPolicy = server.getErasureCodingPolicy(request.getSrc());
 +      GetErasureCodingPolicyResponseProto.Builder builder = GetErasureCodingPolicyResponseProto.newBuilder();
 +      if (ecPolicy != null) {
-         builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(ecPolicy));
++        builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
 +      }
 +      return builder.build();
 +    } catch (IOException e) {
 +      throw new ServiceException(e);
 +    }
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 8419244,f4ce46d..7b02691
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@@ -338,9 -327,9 +338,9 @@@ public class ClientNamenodeProtocolTran
          .build();
      try {
        AppendResponseProto res = rpcProxy.append(null, req);
-       LocatedBlock lastBlock = res.hasBlock() ? PBHelper
+       LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
 -          .convert(res.getBlock()) : null;
 +          .convertLocatedBlockProto(res.getBlock()) : null;
-       HdfsFileStatus stat = (res.hasStat()) ? PBHelper.convert(res.getStat())
+       HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
            : null;
        return new LastBlockWithStatus(lastBlock, stat);
      } catch (ServiceException e) {
@@@ -427,8 -416,7 +427,8 @@@
        req.addAllFavoredNodes(Arrays.asList(favoredNodes));
      }
      try {
-       return PBHelper.convertLocatedBlockProto(
 -      return PBHelperClient.convert(rpcProxy.addBlock(null, req.build()).getBlock());
++      return PBHelperClient.convertLocatedBlockProto(
 +          rpcProxy.addBlock(null, req.build()).getBlock());
      } catch (ServiceException e) {
        throw ProtobufHelper.getRemoteException(e);
      }
@@@ -453,8 -441,8 +453,8 @@@
          .setClientName(clientName)
          .build();
      try {
-       return PBHelper.convertLocatedBlockProto(
 -      return PBHelperClient.convert(rpcProxy.getAdditionalDatanode(null, req)
 -          .getBlock());
++      return PBHelperClient.convertLocatedBlockProto(
 +          rpcProxy.getAdditionalDatanode(null, req).getBlock());
      } catch (ServiceException e) {
        throw ProtobufHelper.getRemoteException(e);
      }
@@@ -481,7 -469,7 +481,7 @@@
    @Override
    public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
      ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
-         .addAllBlocks(Arrays.asList(PBHelper.convertLocatedBlocks(blocks)))
 -        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
++        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlocks(blocks)))
          .build();
      try {
        rpcProxy.reportBadBlocks(null, req);
@@@ -913,7 -901,7 +913,7 @@@
          .setClientName(clientName)
          .build();
      try {
-       return PBHelper.convertLocatedBlockProto(
 -      return PBHelperClient.convert(
++      return PBHelperClient.convertLocatedBlockProto(
            rpcProxy.updateBlockForPipeline(null, req).getBlock());
      } catch (ServiceException e) {
        throw ProtobufHelper.getRemoteException(e);
@@@ -1419,23 -1407,6 +1419,23 @@@
    }
  
    @Override
 +  public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
 +      throws IOException {
 +    final SetErasureCodingPolicyRequestProto.Builder builder =
 +        SetErasureCodingPolicyRequestProto.newBuilder();
 +    builder.setSrc(src);
 +    if (ecPolicy != null) {
-       builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(ecPolicy));
++      builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
 +    }
 +    SetErasureCodingPolicyRequestProto req = builder.build();
 +    try {
 +      rpcProxy.setErasureCodingPolicy(null, req);
 +    } catch (ServiceException e) {
 +      throw ProtobufHelper.getRemoteException(e);
 +    }
 +  }
 +
 +  @Override
    public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
        throws IOException {
      SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()
@@@ -1557,37 -1528,4 +1557,37 @@@
        throw ProtobufHelper.getRemoteException(e);
      }
    }
 +
 +  @Override
 +  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
 +    try {
 +      GetErasureCodingPoliciesResponseProto response = rpcProxy
 +          .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST);
 +      ErasureCodingPolicy[] ecPolicies =
 +          new ErasureCodingPolicy[response.getEcPoliciesCount()];
 +      int i = 0;
 +      for (ErasureCodingPolicyProto ecPolicyProto : response.getEcPoliciesList()) {
-         ecPolicies[i++] = PBHelper.convertErasureCodingPolicy(ecPolicyProto);
++        ecPolicies[i++] = PBHelperClient.convertErasureCodingPolicy(ecPolicyProto);
 +      }
 +      return ecPolicies;
 +    } catch (ServiceException e) {
 +      throw ProtobufHelper.getRemoteException(e);
 +    }
 +  }
 +
 +  @Override
 +  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
 +    GetErasureCodingPolicyRequestProto req = GetErasureCodingPolicyRequestProto.newBuilder()
 +        .setSrc(src).build();
 +    try {
 +      GetErasureCodingPolicyResponseProto response = rpcProxy.getErasureCodingPolicy(
 +          null, req);
 +      if (response.hasEcPolicy()) {
-         return PBHelper.convertErasureCodingPolicy(response.getEcPolicy());
++        return PBHelperClient.convertErasureCodingPolicy(response.getEcPolicy());
 +      }
 +      return null;
 +    } catch (ServiceException e) {
 +      throw ProtobufHelper.getRemoteException(e);
 +    }
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index f20e58a,18f89f8..194e563
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@@ -281,7 -281,7 +281,7 @@@ public class DatanodeProtocolClientSide
      ReportBadBlocksRequestProto.Builder builder = ReportBadBlocksRequestProto
          .newBuilder();
      for (int i = 0; i < blocks.length; i++) {
-       builder.addBlocks(i, PBHelper.convertLocatedBlock(blocks[i]));
 -      builder.addBlocks(i, PBHelperClient.convert(blocks[i]));
++      builder.addBlocks(i, PBHelperClient.convertLocatedBlock(blocks[i]));
      }
      ReportBadBlocksRequestProto req = builder.build();
      try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 1ff80b3,94d1f0c..a1ea9a6
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@@ -259,7 -259,7 +259,7 @@@ public class DatanodeProtocolServerSide
      List<LocatedBlockProto> lbps = request.getBlocksList();
      LocatedBlock [] blocks = new LocatedBlock [lbps.size()];
      for(int i=0; i<lbps.size(); i++) {
-       blocks[i] = PBHelper.convertLocatedBlockProto(lbps.get(i));
 -      blocks[i] = PBHelperClient.convert(lbps.get(i));
++      blocks[i] = PBHelperClient.convertLocatedBlockProto(lbps.get(i));
      }
      try {
        impl.reportBadBlocks(blocks);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index ce39e15,3de4513..ece9984
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@@ -17,108 -17,23 +17,25 @@@
   */
  package org.apache.hadoop.hdfs.protocolPB;
  
- import static com.google.common.base.Preconditions.checkNotNull;
- import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
-     .EncryptionZoneProto;
- import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CipherSuiteProto;
- import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
- 
- import java.io.IOException;
  import java.util.ArrayList;
  import java.util.Arrays;
 +import java.util.Collection;
- import java.util.EnumSet;
- import java.util.HashMap;
  import java.util.List;
- import java.util.Map;
- import java.util.Map.Entry;
- import java.util.Set;
- 
- import org.apache.hadoop.fs.CacheFlag;
- import org.apache.hadoop.fs.ContentSummary;
- import org.apache.hadoop.fs.CreateFlag;
- import org.apache.hadoop.fs.FsServerDefaults;
- import org.apache.hadoop.fs.Path;
+ 
+ import com.google.protobuf.ByteString;
+ 
  import org.apache.hadoop.fs.StorageType;
- import org.apache.hadoop.fs.XAttr;
- import org.apache.hadoop.fs.XAttrSetFlag;
- import org.apache.hadoop.fs.permission.AclEntry;
- import org.apache.hadoop.fs.permission.AclEntryScope;
- import org.apache.hadoop.fs.permission.AclEntryType;
- import org.apache.hadoop.fs.permission.AclStatus;
- import org.apache.hadoop.fs.permission.FsAction;
- import org.apache.hadoop.fs.permission.FsPermission;
  import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
  import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
  import org.apache.hadoop.hdfs.DFSUtilClient;
- import org.apache.hadoop.hdfs.inotify.EventBatch;
- import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
- import org.apache.hadoop.hdfs.inotify.Event;
- import org.apache.hadoop.hdfs.inotify.EventBatchList;
  import org.apache.hadoop.hdfs.protocol.Block;
- import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
- import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
- import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
- import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
- import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
- import org.apache.hadoop.hdfs.protocol.CachePoolStats;
- import org.apache.hadoop.crypto.CipherOption;
- import org.apache.hadoop.crypto.CipherSuite;
- import org.apache.hadoop.hdfs.protocol.ClientProtocol;
- import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
- import org.apache.hadoop.crypto.CryptoProtocolVersion;
  import org.apache.hadoop.hdfs.protocol.DatanodeID;
  import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
- import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
- import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
- import org.apache.hadoop.hdfs.protocol.DirectoryListing;
- import org.apache.hadoop.hdfs.protocol.EncryptionZone;
- import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
- import org.apache.hadoop.fs.FileEncryptionInfo;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
- import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
- import org.apache.hadoop.hdfs.protocol.HdfsConstants;
- import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
- import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
- import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
- import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
- import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
- import org.apache.hadoop.hdfs.protocol.LocatedBlock;
- import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
- import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
- import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
- import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
- import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
- import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
- import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
- import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryScopeProto;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.FsActionProto;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
- import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
- import org.apache.hadoop.hdfs.protocol.proto.*;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeStorageReportProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
- import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
- import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 -import org.apache.hadoop.hdfs.protocol.LocatedBlock;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 +import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECRecoveryCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
@@@ -130,14 -45,8 +47,17 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
++import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
@@@ -381,73 -168,23 +182,34 @@@ public class PBHelper 
          si, convert(reg.getRole()));
    }
  
-   // DatanodeId
-   public static DatanodeID convert(DatanodeIDProto dn) {
-     return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
-         dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
-         .getInfoSecurePort() : 0, dn.getIpcPort());
-   }
- 
-   // Arrays of DatanodeId
-   public static DatanodeIDProto[] convert(DatanodeID[] did) {
-     if (did == null)
-       return null;
-     final int len = did.length;
-     DatanodeIDProto[] result = new DatanodeIDProto[len];
-     for (int i = 0; i < len; ++i) {
-       result[i] = PBHelperClient.convert(did[i]);
-     }
-     return result;
-   }
-   
-   public static DatanodeID[] convert(DatanodeIDProto[] did) {
-     if (did == null) return null;
-     final int len = did.length;
-     DatanodeID[] result = new DatanodeID[len];
-     for (int i = 0; i < len; ++i) {
-       result[i] = convert(did[i]);
-     }
-     return result;
-   }
-   
-   // Block
-   public static BlockProto convert(Block b) {
-     return BlockProto.newBuilder().setBlockId(b.getBlockId())
-         .setGenStamp(b.getGenerationStamp()).setNumBytes(b.getNumBytes())
-         .build();
-   }
- 
-   public static Block convert(BlockProto b) {
-     return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
-   }
- 
    public static BlockWithLocationsProto convert(BlockWithLocations blk) {
 -    return BlockWithLocationsProto.newBuilder()
 -        .setBlock(PBHelperClient.convert(blk.getBlock()))
 +    BlockWithLocationsProto.Builder builder = BlockWithLocationsProto
-         .newBuilder().setBlock(convert(blk.getBlock()))
++        .newBuilder().setBlock(PBHelperClient.convert(blk.getBlock()))
          .addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids()))
          .addAllStorageUuids(Arrays.asList(blk.getStorageIDs()))
 -        .addAllStorageTypes(PBHelperClient.convertStorageTypes(blk.getStorageTypes()))
 -        .build();
 +        .addAllStorageTypes(PBHelperClient.convertStorageTypes(blk.getStorageTypes()));
 +    if (blk instanceof StripedBlockWithLocations) {
 +      StripedBlockWithLocations sblk = (StripedBlockWithLocations) blk;
-       builder.setIndices(getByteString(sblk.getIndices()));
++      builder.setIndices(PBHelperClient.getByteString(sblk.getIndices()));
 +      builder.setDataBlockNum(sblk.getDataBlockNum());
 +    }
 +    return builder.build();
    }
  
    public static BlockWithLocations convert(BlockWithLocationsProto b) {
      final List<String> datanodeUuids = b.getDatanodeUuidsList();
      final List<String> storageUuids = b.getStorageUuidsList();
      final List<StorageTypeProto> storageTypes = b.getStorageTypesList();
-     BlockWithLocations blk = new BlockWithLocations(convert(b.getBlock()),
 -    return new BlockWithLocations(PBHelperClient.convert(b.getBlock()),
++    BlockWithLocations blk = new BlockWithLocations(PBHelperClient.
++        convert(b.getBlock()),
          datanodeUuids.toArray(new String[datanodeUuids.size()]),
          storageUuids.toArray(new String[storageUuids.size()]),
-         convertStorageTypes(storageTypes, storageUuids.size()));
+         PBHelperClient.convertStorageTypes(storageTypes, storageUuids.size()));
 +    if (b.hasIndices()) {
 +      blk = new StripedBlockWithLocations(blk, b.getIndices().toByteArray(),
 +          (short) b.getDataBlockNum());
 +    }
 +    return blk;
    }
  
    public static BlocksWithLocationsProto convert(BlocksWithLocations blks) {
@@@ -596,7 -333,7 +358,7 @@@
      if (b == null) {
        return null;
      }
-     LocatedBlockProto lb = PBHelper.convertLocatedBlock(b);
 -    LocatedBlockProto lb = PBHelperClient.convert((LocatedBlock) b);
++    LocatedBlockProto lb = PBHelperClient.convertLocatedBlock(b);
      RecoveringBlockProto.Builder builder = RecoveringBlockProto.newBuilder();
      builder.setBlock(lb).setNewGenStamp(b.getNewGenerationStamp());
      if(b.getNewBlock() != null)
@@@ -606,206 -343,12 +368,12 @@@
  
    public static RecoveringBlock convert(RecoveringBlockProto b) {
      ExtendedBlock block = PBHelperClient.convert(b.getBlock().getB());
-     DatanodeInfo[] locs = convert(b.getBlock().getLocsList());
+     DatanodeInfo[] locs = PBHelperClient.convert(b.getBlock().getLocsList());
      return (b.hasTruncateBlock()) ?
-         new RecoveringBlock(block, locs, PBHelper.convert(b.getTruncateBlock())) :
+         new RecoveringBlock(block, locs, PBHelperClient.convert(b.getTruncateBlock())) :
          new RecoveringBlock(block, locs, b.getNewGenStamp());
    }
-   
-   static public DatanodeInfo convert(DatanodeInfoProto di) {
-     if (di == null) return null;
-     return new DatanodeInfo(
-         PBHelper.convert(di.getId()),
-         di.hasLocation() ? di.getLocation() : null , 
-         di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
-         di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
-         di.getLastUpdate(), di.getLastUpdateMonotonic(),
-         di.getXceiverCount(), PBHelper.convert(di.getAdminState()));
-   }
-   
-   static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
-     if (di == null) return null;
-     DatanodeInfo[] result = new DatanodeInfo[di.length];
-     for (int i = 0; i < di.length; i++) {
-       result[i] = convert(di[i]);
-     }    
-     return result;
-   }
- 
-   public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
-     DatanodeInfo[] info = new DatanodeInfo[list.size()];
-     for (int i = 0; i < info.length; i++) {
-       info[i] = convert(list.get(i));
-     }
-     return info;
-   }
- 
-   public static DatanodeStorageReportProto convertDatanodeStorageReport(
-       DatanodeStorageReport report) {
-     return DatanodeStorageReportProto.newBuilder()
-         .setDatanodeInfo(PBHelperClient.convert(report.getDatanodeInfo()))
-         .addAllStorageReports(convertStorageReports(report.getStorageReports()))
-         .build();
-   }
- 
-   public static List<DatanodeStorageReportProto> convertDatanodeStorageReports(
-       DatanodeStorageReport[] reports) {
-     final List<DatanodeStorageReportProto> protos
-         = new ArrayList<DatanodeStorageReportProto>(reports.length);
-     for(int i = 0; i < reports.length; i++) {
-       protos.add(convertDatanodeStorageReport(reports[i]));
-     }
-     return protos;
-   }
- 
-   public static DatanodeStorageReport convertDatanodeStorageReport(
-       DatanodeStorageReportProto proto) {
-     return new DatanodeStorageReport(
-         convert(proto.getDatanodeInfo()),
-         convertStorageReports(proto.getStorageReportsList()));
-   }
- 
-   public static DatanodeStorageReport[] convertDatanodeStorageReports(
-       List<DatanodeStorageReportProto> protos) {
-     final DatanodeStorageReport[] reports
-         = new DatanodeStorageReport[protos.size()];
-     for(int i = 0; i < reports.length; i++) {
-       reports[i] = convertDatanodeStorageReport(protos.get(i));
-     }
-     return reports;
-   }
- 
-   public static AdminStates convert(AdminState adminState) {
-     switch(adminState) {
-     case DECOMMISSION_INPROGRESS:
-       return AdminStates.DECOMMISSION_INPROGRESS;
-     case DECOMMISSIONED:
-       return AdminStates.DECOMMISSIONED;
-     case NORMAL:
-     default:
-       return AdminStates.NORMAL;
-     }
-   }
-   
-   public static LocatedBlockProto convertLocatedBlock(LocatedBlock b) {
-     if (b == null) return null;
-     Builder builder = LocatedBlockProto.newBuilder();
-     DatanodeInfo[] locs = b.getLocations();
-     List<DatanodeInfo> cachedLocs =
-         Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
-     for (int i = 0; i < locs.length; i++) {
-       DatanodeInfo loc = locs[i];
-       builder.addLocs(i, PBHelperClient.convert(loc));
-       boolean locIsCached = cachedLocs.contains(loc);
-       builder.addIsCached(locIsCached);
-       if (locIsCached) {
-         cachedLocs.remove(loc);
-       }
-     }
-     Preconditions.checkArgument(cachedLocs.size() == 0,
-         "Found additional cached replica locations that are not in the set of"
-         + " storage-backed locations!");
- 
-     StorageType[] storageTypes = b.getStorageTypes();
-     if (storageTypes != null) {
-       for (StorageType storageType : storageTypes) {
-         builder.addStorageTypes(PBHelperClient.convertStorageType(storageType));
-       }
-     }
-     final String[] storageIDs = b.getStorageIDs();
-     if (storageIDs != null) {
-       builder.addAllStorageIDs(Arrays.asList(storageIDs));
-     }
-     if (b instanceof LocatedStripedBlock) {
-       LocatedStripedBlock sb = (LocatedStripedBlock) b;
-       int[] indices = sb.getBlockIndices();
-       Token<BlockTokenIdentifier>[] blockTokens = sb.getBlockTokens();
-       for (int i = 0; i < indices.length; i++) {
-         builder.addBlockIndex(indices[i]);
-         builder.addBlockTokens(PBHelperClient.convert(blockTokens[i]));
-       }
-     }
- 
-     return builder.setB(PBHelperClient.convert(b.getBlock()))
-         .setBlockToken(PBHelperClient.convert(b.getBlockToken()))
-         .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
-   }
-   
-   public static LocatedBlock convertLocatedBlockProto(LocatedBlockProto proto) {
-     if (proto == null) return null;
-     List<DatanodeInfoProto> locs = proto.getLocsList();
-     DatanodeInfo[] targets = new DatanodeInfo[locs.size()];
-     for (int i = 0; i < locs.size(); i++) {
-       targets[i] = PBHelper.convert(locs.get(i));
-     }
- 
-     final StorageType[] storageTypes = convertStorageTypes(
-         proto.getStorageTypesList(), locs.size());
- 
-     final int storageIDsCount = proto.getStorageIDsCount();
-     final String[] storageIDs;
-     if (storageIDsCount == 0) {
-       storageIDs = null;
-     } else {
-       Preconditions.checkState(storageIDsCount == locs.size());
-       storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
-     }
- 
-     int[] indices = null;
-     final int indexCount = proto.getBlockIndexCount();
-     if (indexCount > 0) {
-       indices = new int[indexCount];
-       for (int i = 0; i < indexCount; i++) {
-         indices[i] = proto.getBlockIndex(i);
-       }
-     }
- 
-     // Set values from the isCached list, re-using references from loc
-     List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
-     List<Boolean> isCachedList = proto.getIsCachedList();
-     for (int i=0; i<isCachedList.size(); i++) {
-       if (isCachedList.get(i)) {
-         cachedLocs.add(targets[i]);
-       }
-     }
- 
-     final LocatedBlock lb;
-     if (indices == null) {
-       lb = new LocatedBlock(PBHelperClient.convert(proto.getB()), targets,
-           storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
-           cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
-     } else {
-       lb = new LocatedStripedBlock(PBHelperClient.convert(proto.getB()), targets,
-           storageIDs, storageTypes, indices, proto.getOffset(),
-           proto.getCorrupt(),
-           cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
-       List<TokenProto> tokenProtos = proto.getBlockTokensList();
-       Token<BlockTokenIdentifier>[] blockTokens = new Token[indices.length];
-       for (int i = 0; i < indices.length; i++) {
-         blockTokens[i] = PBHelper.convert(tokenProtos.get(i));
-       }
-       ((LocatedStripedBlock) lb).setBlockTokens(blockTokens);
-     }
-     lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
- 
-     return lb;
-   }
- 
-   public static Token<BlockTokenIdentifier> convert(
-       TokenProto blockToken) {
-     return new Token<BlockTokenIdentifier>(blockToken.getIdentifier()
-         .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
-         blockToken.getKind()), new Text(blockToken.getService()));
-   }
- 
--  
-   public static Token<DelegationTokenIdentifier> convertDelegationToken(
-       TokenProto blockToken) {
-     return new Token<DelegationTokenIdentifier>(blockToken.getIdentifier()
-         .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
-         blockToken.getKind()), new Text(blockToken.getService()));
-   }
 +
    public static ReplicaState convert(ReplicaStateProto state) {
      switch (state) {
      case RBW:
@@@ -1198,549 -704,8 +736,7 @@@
          .setCapabilities(info.getCapabilities())
          .build();
    }
-   
-   // Located Block Arrays and Lists
-   public static LocatedBlockProto[] convertLocatedBlocks(LocatedBlock[] lb) {
-     if (lb == null) return null;
-     return convertLocatedBlocks2(Arrays.asList(lb))
-         .toArray(new LocatedBlockProto[lb.length]);
-   }
-   
-   public static LocatedBlock[] convertLocatedBlocks(LocatedBlockProto[] lb) {
-     if (lb == null) return null;
-     return convertLocatedBlocks(Arrays.asList(lb))
-         .toArray(new LocatedBlock[lb.length]);
-   }
-   
-   public static List<LocatedBlock> convertLocatedBlocks(
-       List<LocatedBlockProto> lb) {
-     if (lb == null) return null;
-     final int len = lb.size();
-     List<LocatedBlock> result = new ArrayList<>(len);
-     for (LocatedBlockProto aLb : lb) {
-       result.add(PBHelper.convertLocatedBlockProto(aLb));
-     }
-     return result;
-   }
-   
-   public static List<LocatedBlockProto> convertLocatedBlocks2(
-       List<LocatedBlock> lb) {
-     if (lb == null) return null;
-     final int len = lb.size();
-     List<LocatedBlockProto> result = new ArrayList<>(len);
-     for (LocatedBlock aLb : lb) {
-       result.add(PBHelper.convertLocatedBlock(aLb));
-     }
-     return result;
-   }
-   
-   
-   // LocatedBlocks
-   public static LocatedBlocks convert(LocatedBlocksProto lb) {
-     return new LocatedBlocks(
-         lb.getFileLength(), lb.getUnderConstruction(),
-         PBHelper.convertLocatedBlocks(lb.getBlocksList()),
-         lb.hasLastBlock() ?
-             PBHelper.convertLocatedBlockProto(lb.getLastBlock()) : null,
-         lb.getIsLastBlockComplete(),
-         lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null,
-         lb.hasEcPolicy() ? convertErasureCodingPolicy(lb.getEcPolicy()) : null);
-   }
-   
-   public static LocatedBlocksProto convert(LocatedBlocks lb) {
-     if (lb == null) {
-       return null;
-     }
-     LocatedBlocksProto.Builder builder = 
-         LocatedBlocksProto.newBuilder();
-     if (lb.getLastLocatedBlock() != null) {
-       builder.setLastBlock(
-           PBHelper.convertLocatedBlock(lb.getLastLocatedBlock()));
-     }
-     if (lb.getFileEncryptionInfo() != null) {
-       builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
-     }
-     if (lb.getErasureCodingPolicy() != null) {
-       builder.setEcPolicy(convertErasureCodingPolicy(lb.getErasureCodingPolicy()));
-     }
-     return builder.setFileLength(lb.getFileLength())
-         .setUnderConstruction(lb.isUnderConstruction())
-         .addAllBlocks(PBHelper.convertLocatedBlocks2(lb.getLocatedBlocks()))
-         .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
-   }
-   
-   // DataEncryptionKey
-   public static DataEncryptionKey convert(DataEncryptionKeyProto bet) {
-     String encryptionAlgorithm = bet.getEncryptionAlgorithm();
-     return new DataEncryptionKey(bet.getKeyId(),
-         bet.getBlockPoolId(),
-         bet.getNonce().toByteArray(),
-         bet.getEncryptionKey().toByteArray(),
-         bet.getExpiryDate(),
-         encryptionAlgorithm.isEmpty() ? null : encryptionAlgorithm);
-   }
-   
-   public static DataEncryptionKeyProto convert(DataEncryptionKey bet) {
-     DataEncryptionKeyProto.Builder b = DataEncryptionKeyProto.newBuilder()
-         .setKeyId(bet.keyId)
-         .setBlockPoolId(bet.blockPoolId)
-         .setNonce(ByteString.copyFrom(bet.nonce))
-         .setEncryptionKey(ByteString.copyFrom(bet.encryptionKey))
-         .setExpiryDate(bet.expiryDate);
-     if (bet.encryptionAlgorithm != null) {
-       b.setEncryptionAlgorithm(bet.encryptionAlgorithm);
-     }
-     return b.build();
-   }
-   
-   public static FsServerDefaults convert(FsServerDefaultsProto fs) {
-     if (fs == null) return null;
-     return new FsServerDefaults(
-         fs.getBlockSize(), fs.getBytesPerChecksum(), 
-         fs.getWritePacketSize(), (short) fs.getReplication(),
-         fs.getFileBufferSize(),
-         fs.getEncryptDataTransfer(),
-         fs.getTrashInterval(),
-         PBHelperClient.convert(fs.getChecksumType()));
-   }
-   
-   public static FsServerDefaultsProto convert(FsServerDefaults fs) {
-     if (fs == null) return null;
-     return FsServerDefaultsProto.newBuilder().
-       setBlockSize(fs.getBlockSize()).
-       setBytesPerChecksum(fs.getBytesPerChecksum()).
-       setWritePacketSize(fs.getWritePacketSize())
-       .setReplication(fs.getReplication())
-       .setFileBufferSize(fs.getFileBufferSize())
-       .setEncryptDataTransfer(fs.getEncryptDataTransfer())
-       .setTrashInterval(fs.getTrashInterval())
-       .setChecksumType(PBHelperClient.convert(fs.getChecksumType()))
-       .build();
-   }
-   
-   public static FsPermissionProto convert(FsPermission p) {
-     return FsPermissionProto.newBuilder().setPerm(p.toExtendedShort()).build();
-   }
-   
-   public static FsPermission convert(FsPermissionProto p) {
-     return new FsPermissionExtension((short)p.getPerm());
-   }
-   
-   
-   // The creatFlag field in PB is a bitmask whose values are the same a the 
-   // emum values of CreateFlag
-   public static int convertCreateFlag(EnumSetWritable<CreateFlag> flag) {
-     int value = 0;
-     if (flag.contains(CreateFlag.APPEND)) {
-       value |= CreateFlagProto.APPEND.getNumber();
-     }
-     if (flag.contains(CreateFlag.CREATE)) {
-       value |= CreateFlagProto.CREATE.getNumber();
-     }
-     if (flag.contains(CreateFlag.OVERWRITE)) {
-       value |= CreateFlagProto.OVERWRITE.getNumber();
-     }
-     if (flag.contains(CreateFlag.LAZY_PERSIST)) {
-       value |= CreateFlagProto.LAZY_PERSIST.getNumber();
-     }
-     if (flag.contains(CreateFlag.NEW_BLOCK)) {
-       value |= CreateFlagProto.NEW_BLOCK.getNumber();
-     }
-     return value;
-   }
-   
-   public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
-     EnumSet<CreateFlag> result = 
-        EnumSet.noneOf(CreateFlag.class);   
-     if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
-       result.add(CreateFlag.APPEND);
-     }
-     if ((flag & CreateFlagProto.CREATE_VALUE) == CreateFlagProto.CREATE_VALUE) {
-       result.add(CreateFlag.CREATE);
-     }
-     if ((flag & CreateFlagProto.OVERWRITE_VALUE) 
-         == CreateFlagProto.OVERWRITE_VALUE) {
-       result.add(CreateFlag.OVERWRITE);
-     }
-     if ((flag & CreateFlagProto.LAZY_PERSIST_VALUE)
-         == CreateFlagProto.LAZY_PERSIST_VALUE) {
-       result.add(CreateFlag.LAZY_PERSIST);
-     }
-     if ((flag & CreateFlagProto.NEW_BLOCK_VALUE)
-         == CreateFlagProto.NEW_BLOCK_VALUE) {
-       result.add(CreateFlag.NEW_BLOCK);
-     }
-     return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
-   }
- 
-   public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
-     int value = 0;
-     if (flags.contains(CacheFlag.FORCE)) {
-       value |= CacheFlagProto.FORCE.getNumber();
-     }
-     return value;
-   }
- 
-   public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
-     EnumSet<CacheFlag> result = EnumSet.noneOf(CacheFlag.class);
-     if ((flags & CacheFlagProto.FORCE_VALUE) == CacheFlagProto.FORCE_VALUE) {
-       result.add(CacheFlag.FORCE);
-     }
-     return result;
-   }
- 
-   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
-     if (fs == null)
-       return null;
-     return new HdfsLocatedFileStatus(
-         fs.getLength(), fs.getFileType().equals(FileType.IS_DIR), 
-         fs.getBlockReplication(), fs.getBlocksize(),
-         fs.getModificationTime(), fs.getAccessTime(),
-         PBHelper.convert(fs.getPermission()), fs.getOwner(), fs.getGroup(), 
-         fs.getFileType().equals(FileType.IS_SYMLINK) ? 
-             fs.getSymlink().toByteArray() : null,
-         fs.getPath().toByteArray(),
-         fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
-         fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null,
-         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
-         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
-         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
-             : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-         fs.hasEcPolicy() ? PBHelper.convertErasureCodingPolicy(fs.getEcPolicy()) : null);
-   }
- 
-   public static SnapshottableDirectoryStatus convert(
-       SnapshottableDirectoryStatusProto sdirStatusProto) {
-     if (sdirStatusProto == null) {
-       return null;
-     }
-     final HdfsFileStatusProto status = sdirStatusProto.getDirStatus();
-     return new SnapshottableDirectoryStatus(
-         status.getModificationTime(),
-         status.getAccessTime(),
-         PBHelper.convert(status.getPermission()),
-         status.getOwner(),
-         status.getGroup(),
-         status.getPath().toByteArray(),
-         status.getFileId(),
-         status.getChildrenNum(),
-         sdirStatusProto.getSnapshotNumber(),
-         sdirStatusProto.getSnapshotQuota(),
-         sdirStatusProto.getParentFullpath().toByteArray());
-   }
-   
-   public static HdfsFileStatusProto convert(HdfsFileStatus fs) {
-     if (fs == null)
-       return null;
-     FileType fType = FileType.IS_FILE;
-     if (fs.isDir()) {
-       fType = FileType.IS_DIR;
-     } else if (fs.isSymlink()) {
-       fType = FileType.IS_SYMLINK;
-     }
- 
-     HdfsFileStatusProto.Builder builder = 
-      HdfsFileStatusProto.newBuilder().
-       setLength(fs.getLen()).
-       setFileType(fType).
-       setBlockReplication(fs.getReplication()).
-       setBlocksize(fs.getBlockSize()).
-       setModificationTime(fs.getModificationTime()).
-       setAccessTime(fs.getAccessTime()).
-       setPermission(PBHelper.convert(fs.getPermission())).
-       setOwner(fs.getOwner()).
-       setGroup(fs.getGroup()).
-       setFileId(fs.getFileId()).
-       setChildrenNum(fs.getChildrenNum()).
-       setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
-       setStoragePolicy(fs.getStoragePolicy());
-     if (fs.isSymlink())  {
-       builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
-     }
-     if (fs.getFileEncryptionInfo() != null) {
-       builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo()));
-     }
-     if (fs instanceof HdfsLocatedFileStatus) {
-       final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
-       LocatedBlocks locations = lfs.getBlockLocations();
-       if (locations != null) {
-         builder.setLocations(PBHelper.convert(locations));
-       }
-     }
-     if(fs.getErasureCodingPolicy() != null) {
-       builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(fs.getErasureCodingPolicy()));
-     }
-     return builder.build();
-   }
-   
-   public static SnapshottableDirectoryStatusProto convert(
-       SnapshottableDirectoryStatus status) {
-     if (status == null) {
-       return null;
-     }
-     int snapshotNumber = status.getSnapshotNumber();
-     int snapshotQuota = status.getSnapshotQuota();
-     byte[] parentFullPath = status.getParentFullPath();
-     ByteString parentFullPathBytes = ByteString.copyFrom(
-         parentFullPath == null ? DFSUtilClient.EMPTY_BYTES : parentFullPath);
-     HdfsFileStatusProto fs = convert(status.getDirStatus());
-     SnapshottableDirectoryStatusProto.Builder builder = 
-         SnapshottableDirectoryStatusProto
-         .newBuilder().setSnapshotNumber(snapshotNumber)
-         .setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
-         .setDirStatus(fs);
-     return builder.build();
-   }
-   
-   public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
-     if (fs == null) return null;
-     final int len = fs.length;
-     HdfsFileStatusProto[] result = new HdfsFileStatusProto[len];
-     for (int i = 0; i < len; ++i) {
-       result[i] = PBHelper.convert(fs[i]);
-     }
-     return result;
-   }
-   
-   public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
-     if (fs == null) return null;
-     final int len = fs.length;
-     HdfsFileStatus[] result = new HdfsFileStatus[len];
-     for (int i = 0; i < len; ++i) {
-       result[i] = PBHelper.convert(fs[i]);
-     }
-     return result;
-   }
-   
-   public static DirectoryListing convert(DirectoryListingProto dl) {
-     if (dl == null)
-       return null;
-     List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
-     return new DirectoryListing( 
-         partList.isEmpty() ? new HdfsLocatedFileStatus[0] 
-           : PBHelper.convert(
-               partList.toArray(new HdfsFileStatusProto[partList.size()])),
-         dl.getRemainingEntries());
-   }
- 
-   public static DirectoryListingProto convert(DirectoryListing d) {
-     if (d == null)
-       return null;
-     return DirectoryListingProto.newBuilder().
-         addAllPartialListing(Arrays.asList(
-             PBHelper.convert(d.getPartialListing()))).
-         setRemainingEntries(d.getRemainingEntries()).
-         build();
-   }
- 
-   public static long[] convert(GetFsStatsResponseProto res) {
-     long[] result = new long[7];
-     result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
-     result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
-     result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
-     result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
-     result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
-     result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
-     result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
-         res.getMissingReplOneBlocks();
-     return result;
-   }
-   
-   public static GetFsStatsResponseProto convert(long[] fsStats) {
-     GetFsStatsResponseProto.Builder result = GetFsStatsResponseProto
-         .newBuilder();
-     if (fsStats.length >= ClientProtocol.GET_STATS_CAPACITY_IDX + 1)
-       result.setCapacity(fsStats[ClientProtocol.GET_STATS_CAPACITY_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_USED_IDX + 1)
-       result.setUsed(fsStats[ClientProtocol.GET_STATS_USED_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_REMAINING_IDX + 1)
-       result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
-       result.setUnderReplicated(
-               fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
-       result.setCorruptBlocks(
-           fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
-       result.setMissingBlocks(
-           fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
-     if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
-       result.setMissingReplOneBlocks(
-           fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
-     return result.build();
-   }
-   
-   public static DatanodeReportTypeProto
-     convert(DatanodeReportType t) {
-     switch (t) {
-     case ALL: return DatanodeReportTypeProto.ALL;
-     case LIVE: return DatanodeReportTypeProto.LIVE;
-     case DEAD: return DatanodeReportTypeProto.DEAD;
-     case DECOMMISSIONING: return DatanodeReportTypeProto.DECOMMISSIONING;
-     default: 
-       throw new IllegalArgumentException("Unexpected data type report:" + t);
-     }
-   }
-   
-   public static DatanodeReportType 
-     convert(DatanodeReportTypeProto t) {
-     switch (t) {
-     case ALL: return DatanodeReportType.ALL;
-     case LIVE: return DatanodeReportType.LIVE;
-     case DEAD: return DatanodeReportType.DEAD;
-     case DECOMMISSIONING: return DatanodeReportType.DECOMMISSIONING;
-     default: 
-       throw new IllegalArgumentException("Unexpected data type report:" + t);
-     }
-   }
- 
-   public static SafeModeActionProto convert(
-       SafeModeAction a) {
-     switch (a) {
-     case SAFEMODE_LEAVE:
-       return SafeModeActionProto.SAFEMODE_LEAVE;
-     case SAFEMODE_ENTER:
-       return SafeModeActionProto.SAFEMODE_ENTER;
-     case SAFEMODE_GET:
-       return SafeModeActionProto.SAFEMODE_GET;
-     default:
-       throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
-     }
-   }
-   
-   public static SafeModeAction convert(
-       ClientNamenodeProtocolProtos.SafeModeActionProto a) {
-     switch (a) {
-     case SAFEMODE_LEAVE:
-       return SafeModeAction.SAFEMODE_LEAVE;
-     case SAFEMODE_ENTER:
-       return SafeModeAction.SAFEMODE_ENTER;
-     case SAFEMODE_GET:
-       return SafeModeAction.SAFEMODE_GET;
-     default:
-       throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
-     }
-   }
-   
-   public static RollingUpgradeActionProto convert(RollingUpgradeAction a) {
-     switch (a) {
-     case QUERY:
-       return RollingUpgradeActionProto.QUERY;
-     case PREPARE:
-       return RollingUpgradeActionProto.START;
-     case FINALIZE:
-       return RollingUpgradeActionProto.FINALIZE;
-     default:
-       throw new IllegalArgumentException("Unexpected value: " + a);
-     }
-   }
-   
-   public static RollingUpgradeAction convert(RollingUpgradeActionProto a) {
-     switch (a) {
-     case QUERY:
-       return RollingUpgradeAction.QUERY;
-     case START:
-       return RollingUpgradeAction.PREPARE;
-     case FINALIZE:
-       return RollingUpgradeAction.FINALIZE;
-     default:
-       throw new IllegalArgumentException("Unexpected value: " + a);
-     }
-   }
- 
-   public static RollingUpgradeStatusProto convertRollingUpgradeStatus(
-       RollingUpgradeStatus status) {
-     return RollingUpgradeStatusProto.newBuilder()
-         .setBlockPoolId(status.getBlockPoolId())
-         .setFinalized(status.isFinalized())
-         .build();
-   }
- 
-   public static RollingUpgradeStatus convert(RollingUpgradeStatusProto proto) {
-     return new RollingUpgradeStatus(proto.getBlockPoolId(),
-         proto.getFinalized());
-   }
- 
-   public static RollingUpgradeInfoProto convert(RollingUpgradeInfo info) {
-     return RollingUpgradeInfoProto.newBuilder()
-         .setStatus(convertRollingUpgradeStatus(info))
-         .setCreatedRollbackImages(info.createdRollbackImages())
-         .setStartTime(info.getStartTime())
-         .setFinalizeTime(info.getFinalizeTime())
-         .build();
-   }
- 
-   public static RollingUpgradeInfo convert(RollingUpgradeInfoProto proto) {
-     RollingUpgradeStatusProto status = proto.getStatus();
-     return new RollingUpgradeInfo(status.getBlockPoolId(),
-         proto.getCreatedRollbackImages(),
-         proto.getStartTime(), proto.getFinalizeTime());
-   }
- 
-   public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
-     if (c == null)
-       return null;
-     List<String> fileList = c.getFilesList();
-     return new CorruptFileBlocks(fileList.toArray(new String[fileList.size()]),
-         c.getCookie());
-   }
- 
-   public static CorruptFileBlocksProto convert(CorruptFileBlocks c) {
-     if (c == null)
-       return null;
-     return CorruptFileBlocksProto.newBuilder().
-         addAllFiles(Arrays.asList(c.getFiles())).
-         setCookie(c.getCookie()).
-         build();
-   }
-   
-   public static ContentSummary convert(ContentSummaryProto cs) {
-     if (cs == null) return null;
-     ContentSummary.Builder builder = new ContentSummary.Builder();
-     builder.length(cs.getLength()).
-         fileCount(cs.getFileCount()).
-         directoryCount(cs.getDirectoryCount()).
-         quota(cs.getQuota()).
-         spaceConsumed(cs.getSpaceConsumed()).
-         spaceQuota(cs.getSpaceQuota());
-     if (cs.hasTypeQuotaInfos()) {
-       for (HdfsProtos.StorageTypeQuotaInfoProto info :
-           cs.getTypeQuotaInfos().getTypeQuotaInfoList()) {
-         StorageType type = PBHelperClient.convertStorageType(info.getType());
-         builder.typeConsumed(type, info.getConsumed());
-         builder.typeQuota(type, info.getQuota());
-       }
-     }
-     return builder.build();
-   }
-   
-   public static ContentSummaryProto convert(ContentSummary cs) {
-     if (cs == null) return null;
-     ContentSummaryProto.Builder builder = ContentSummaryProto.newBuilder();
-         builder.setLength(cs.getLength()).
-         setFileCount(cs.getFileCount()).
-         setDirectoryCount(cs.getDirectoryCount()).
-         setQuota(cs.getQuota()).
-         setSpaceConsumed(cs.getSpaceConsumed()).
-         setSpaceQuota(cs.getSpaceQuota());
- 
-     if (cs.isTypeQuotaSet() || cs.isTypeConsumedAvailable()) {
-       HdfsProtos.StorageTypeQuotaInfosProto.Builder isb =
-           HdfsProtos.StorageTypeQuotaInfosProto.newBuilder();
-       for (StorageType t: StorageType.getTypesSupportingQuota()) {
-         HdfsProtos.StorageTypeQuotaInfoProto info =
-             HdfsProtos.StorageTypeQuotaInfoProto.newBuilder().
-                 setType(PBHelperClient.convertStorageType(t)).
-                 setConsumed(cs.getTypeConsumed(t)).
-                 setQuota(cs.getTypeQuota(t)).
-                 build();
-         isb.addTypeQuotaInfo(info);
-       }
-       builder.setTypeQuotaInfos(isb);
-     }
-     return builder.build();
-   }
  
 -
    public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
      if (s == null) return null;
      switch (s.getState()) {
@@@ -2933,179 -790,4 +821,140 @@@
          setLeaseId(context.getLeaseId()).
          build();
    }
 +
-   public static ECSchema convertECSchema(ECSchemaProto schema) {
-     List<ECSchemaOptionEntryProto> optionsList = schema.getOptionsList();
-     Map<String, String> options = new HashMap<>(optionsList.size());
-     for (ECSchemaOptionEntryProto option : optionsList) {
-       options.put(option.getKey(), option.getValue());
++  private static List<Integer> convertIntArray(short[] liveBlockIndices) {
++    List<Integer> liveBlockIndicesList = new ArrayList<>();
++    for (short s : liveBlockIndices) {
++      liveBlockIndicesList.add((int) s);
 +    }
-     return new ECSchema(schema.getCodecName(), schema.getDataUnits(),
-         schema.getParityUnits(), options);
-   }
- 
-   public static ECSchemaProto convertECSchema(ECSchema schema) {
-     ECSchemaProto.Builder builder = ECSchemaProto.newBuilder()
-         .setCodecName(schema.getCodecName())
-         .setDataUnits(schema.getNumDataUnits())
-         .setParityUnits(schema.getNumParityUnits());
-     Set<Entry<String, String>> entrySet = schema.getExtraOptions().entrySet();
-     for (Entry<String, String> entry : entrySet) {
-       builder.addOptions(ECSchemaOptionEntryProto.newBuilder()
-           .setKey(entry.getKey()).setValue(entry.getValue()).build());
++    return liveBlockIndicesList;
++  }
++
++  private static StorageTypesProto convertStorageTypesProto(
++      StorageType[] targetStorageTypes) {
++    StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
++    for (StorageType storageType : targetStorageTypes) {
++      builder.addStorageTypes(PBHelperClient.convertStorageType(storageType));
 +    }
 +    return builder.build();
 +  }
 +
-   public static ErasureCodingPolicy convertErasureCodingPolicy(
-       ErasureCodingPolicyProto policy) {
-     return new ErasureCodingPolicy(policy.getName(),
-         convertECSchema(policy.getSchema()),
-         policy.getCellSize());
++  private static HdfsProtos.StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) {
++    HdfsProtos.StorageUuidsProto.Builder builder = HdfsProtos.StorageUuidsProto.newBuilder();
++    for (String storageUuid : targetStorageIDs) {
++      builder.addStorageUuids(storageUuid);
++    }
++    return builder.build();
 +  }
 +
-   public static ErasureCodingPolicyProto convertErasureCodingPolicy(
-       ErasureCodingPolicy policy) {
-     ErasureCodingPolicyProto.Builder builder = ErasureCodingPolicyProto
-         .newBuilder()
-         .setName(policy.getName())
-         .setSchema(convertECSchema(policy.getSchema()))
-         .setCellSize(policy.getCellSize());
++  private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) {
++    DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder();
++    for (DatanodeInfo datanodeInfo : dnInfos) {
++      builder.addDatanodes(PBHelperClient.convert(datanodeInfo));
++    }
 +    return builder.build();
 +  }
-   
++
++  private static String[] convert(HdfsProtos.StorageUuidsProto targetStorageUuidsProto) {
++    List<String> storageUuidsList = targetStorageUuidsProto
++        .getStorageUuidsList();
++    String[] storageUuids = new String[storageUuidsList.size()];
++    for (int i = 0; i < storageUuidsList.size(); i++) {
++      storageUuids[i] = storageUuidsList.get(i);
++    }
++    return storageUuids;
++  }
++
 +  public static BlockECRecoveryInfo convertBlockECRecoveryInfo(
 +      BlockECRecoveryInfoProto blockEcRecoveryInfoProto) {
 +    ExtendedBlockProto blockProto = blockEcRecoveryInfoProto.getBlock();
 +    ExtendedBlock block = PBHelperClient.convert(blockProto);
 +
 +    DatanodeInfosProto sourceDnInfosProto = blockEcRecoveryInfoProto
 +        .getSourceDnInfos();
-     DatanodeInfo[] sourceDnInfos = convert(sourceDnInfosProto);
++    DatanodeInfo[] sourceDnInfos = PBHelperClient.convert(sourceDnInfosProto);
 +
 +    DatanodeInfosProto targetDnInfosProto = blockEcRecoveryInfoProto
 +        .getTargetDnInfos();
-     DatanodeInfo[] targetDnInfos = convert(targetDnInfosProto);
++    DatanodeInfo[] targetDnInfos = PBHelperClient.convert(targetDnInfosProto);
 +
-     StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto
++    HdfsProtos.StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto
 +        .getTargetStorageUuids();
 +    String[] targetStorageUuids = convert(targetStorageUuidsProto);
 +
 +    StorageTypesProto targetStorageTypesProto = blockEcRecoveryInfoProto
 +        .getTargetStorageTypes();
-     StorageType[] convertStorageTypes = convertStorageTypes(
++    StorageType[] convertStorageTypes = PBHelperClient.convertStorageTypes(
 +        targetStorageTypesProto.getStorageTypesList(), targetStorageTypesProto
 +            .getStorageTypesList().size());
 +
 +    List<Integer> liveBlockIndicesList = blockEcRecoveryInfoProto
 +        .getLiveBlockIndicesList();
 +    short[] liveBlkIndices = new short[liveBlockIndicesList.size()];
 +    for (int i = 0; i < liveBlockIndicesList.size(); i++) {
 +      liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue();
 +    }
 +
 +    ErasureCodingPolicy ecPolicy =
-         convertErasureCodingPolicy(blockEcRecoveryInfoProto.getEcPolicy());
++        PBHelperClient.convertErasureCodingPolicy(
++            blockEcRecoveryInfoProto.getEcPolicy());
 +
 +    return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos,
 +        targetStorageUuids, convertStorageTypes, liveBlkIndices, ecPolicy);
 +  }
 +
 +  public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo(
 +      BlockECRecoveryInfo blockEcRecoveryInfo) {
 +    BlockECRecoveryInfoProto.Builder builder = BlockECRecoveryInfoProto
 +        .newBuilder();
 +    builder.setBlock(PBHelperClient.convert(
 +        blockEcRecoveryInfo.getExtendedBlock()));
 +
 +    DatanodeInfo[] sourceDnInfos = blockEcRecoveryInfo.getSourceDnInfos();
 +    builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos));
 +
 +    DatanodeInfo[] targetDnInfos = blockEcRecoveryInfo.getTargetDnInfos();
 +    builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos));
 +
 +    String[] targetStorageIDs = blockEcRecoveryInfo.getTargetStorageIDs();
 +    builder.setTargetStorageUuids(convertStorageIDs(targetStorageIDs));
 +
 +    StorageType[] targetStorageTypes = blockEcRecoveryInfo
 +        .getTargetStorageTypes();
 +    builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes));
 +
 +    short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices();
 +    builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices));
 +
-     builder.setEcPolicy(convertErasureCodingPolicy(blockEcRecoveryInfo
-         .getErasureCodingPolicy()));
- 
-     return builder.build();
-   }
- 
-   private static List<Integer> convertIntArray(short[] liveBlockIndices) {
-     List<Integer> liveBlockIndicesList = new ArrayList<Integer>();
-     for (short s : liveBlockIndices) {
-       liveBlockIndicesList.add((int) s);
-     }
-     return liveBlockIndicesList;
-   }
++    builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(
++        blockEcRecoveryInfo.getErasureCodingPolicy()));
 +
-   private static StorageTypesProto convertStorageTypesProto(
-       StorageType[] targetStorageTypes) {
-     StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
-     for (StorageType storageType : targetStorageTypes) {
-       builder.addStorageTypes(PBHelperClient.convertStorageType(storageType));
-     }
 +    return builder.build();
 +  }
 +
-   private static StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) {
-     StorageUuidsProto.Builder builder = StorageUuidsProto.newBuilder();
-     for (String storageUuid : targetStorageIDs) {
-       builder.addStorageUuids(storageUuid);
-     }
-     return builder.build();
-   }
- 
-   private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) {
-     DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder();
-     for (DatanodeInfo datanodeInfo : dnInfos) {
-       builder.addDatanodes(PBHelperClient.convert(datanodeInfo));
-     }
-     return builder.build();
-   }
- 
-   private static String[] convert(StorageUuidsProto targetStorageUuidsProto) {
-     List<String> storageUuidsList = targetStorageUuidsProto
-         .getStorageUuidsList();
-     String[] storageUuids = new String[storageUuidsList.size()];
-     for (int i = 0; i < storageUuidsList.size(); i++) {
-       storageUuids[i] = storageUuidsList.get(i);
-     }
-     return storageUuids;
-   }
-   
 +  public static BlockECRecoveryCommandProto convert(
 +      BlockECRecoveryCommand blkECRecoveryCmd) {
 +    BlockECRecoveryCommandProto.Builder builder = BlockECRecoveryCommandProto
 +        .newBuilder();
 +    Collection<BlockECRecoveryInfo> blockECRecoveryInfos = blkECRecoveryCmd
 +        .getECTasks();
 +    for (BlockECRecoveryInfo blkECRecoveryInfo : blockECRecoveryInfos) {
 +      builder
 +          .addBlockECRecoveryinfo(convertBlockECRecoveryInfo(blkECRecoveryInfo));
 +    }
 +    return builder.build();
 +  }
-   
++
 +  public static BlockECRecoveryCommand convert(
 +      BlockECRecoveryCommandProto blkECRecoveryCmdProto) {
-     Collection<BlockECRecoveryInfo> blkECRecoveryInfos = new ArrayList<BlockECRecoveryInfo>();
++    Collection<BlockECRecoveryInfo> blkECRecoveryInfos = new ArrayList<>();
 +    List<BlockECRecoveryInfoProto> blockECRecoveryinfoList = blkECRecoveryCmdProto
 +        .getBlockECRecoveryinfoList();
 +    for (BlockECRecoveryInfoProto blockECRecoveryInfoProto : blockECRecoveryinfoList) {
 +      blkECRecoveryInfos
 +          .add(convertBlockECRecoveryInfo(blockECRecoveryInfoProto));
 +    }
 +    return new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY,
 +        blkECRecoveryInfos);
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 1211169,b0a11fe..e7f9262
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@@ -1160,8 -1107,8 +1176,8 @@@ public class BlockManager implements Bl
     * Adds block to list of blocks which will be invalidated on all its
     * datanodes.
     */
 -  private void addToInvalidates(Block b) {
 +  private void addToInvalidates(BlockInfo storedBlock) {
-     if (!namesystem.isPopulatingReplQueues()) {
+     if (!isPopulatingReplQueues()) {
        return;
      }
      StringBuilder datanodes = new StringBuilder();
@@@ -1287,8 -1215,8 +1303,8 @@@
      if (hasEnoughLiveReplicas || hasMoreCorruptReplicas
          || corruptedDuringWrite) {
        // the block is over-replicated so invalidate the replicas immediately
 -      invalidateBlock(b, node);
 +      invalidateBlock(b, node, numberOfReplicas);
-     } else if (namesystem.isPopulatingReplQueues()) {
+     } else if (isPopulatingReplQueues()) {
        // add the block to neededReplication
        updateNeededReplications(b.getStored(), -1, 0);
      }
@@@ -2654,9 -2488,9 +2670,9 @@@
        DatanodeStorageInfo storageInfo)
    throws IOException {
      assert (storedBlock != null && namesystem.hasWriteLock());
--    if (!namesystem.isInStartupSafeMode() 
-         || namesystem.isPopulatingReplQueues()) {
++    if (!namesystem.isInStartupSafeMode()
+         || isPopulatingReplQueues()) {
 -      addStoredBlock(storedBlock, storageInfo, null, false);
 +      addStoredBlock(storedBlock, reported, storageInfo, null, false);
        return;
      }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index a80bfd6,6d199d7..fb86ff3
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@@ -542,12 -546,12 +542,12 @@@ public class DecommissionManager 
          if (blockManager.isNeededReplication(block, liveReplicas)) {
            if (!blockManager.neededReplications.contains(block) &&
                blockManager.pendingReplications.getNumReplicas(block) == 0 &&
-               namesystem.isPopulatingReplQueues()) {
+               blockManager.isPopulatingReplQueues()) {
              // Process these blocks only when active NN is out of safe mode.
              blockManager.neededReplications.add(block,
 -                curReplicas,
 +                liveReplicas,
                  num.decommissionedAndDecommissioning(),
 -                block.getReplication());
 +                blockManager.getExpectedReplicaNum(block));
            }
          }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 82a0f62,2aad83d..9228bec
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@@ -1156,8 -1175,7 +1179,9 @@@ public class DataNode extends Reconfigu
      saslClient = new SaslDataTransferClient(dnConf.conf, 
          dnConf.saslPropsResolver, dnConf.trustedChannelResolver);
      saslServer = new SaslDataTransferServer(dnConf, blockPoolTokenSecretManager);
 +    // Initialize ErasureCoding worker
 +    ecWorker = new ErasureCodingWorker(conf, this);
+     startMetricsLogger(conf);
    }
  
    /**
@@@ -3264,9 -3256,72 +3291,76 @@@
      checkSuperuserPrivilege();
      spanReceiverHost.removeSpanReceiver(id);
    }
 +  
 +  public ErasureCodingWorker getErasureCodingWorker(){
 +    return ecWorker;
-     
++  }
+ 
+   /**
+    * Get timeout value of each OOB type from configuration
+    */
+   private void initOOBTimeout() {
+     final int oobStart = Status.OOB_RESTART_VALUE; // the first OOB type
+     final int oobEnd = Status.OOB_RESERVED3_VALUE; // the last OOB type
+     final int numOobTypes = oobEnd - oobStart + 1;
+     oobTimeouts = new long[numOobTypes];
+ 
+     final String[] ele = conf.get(DFS_DATANODE_OOB_TIMEOUT_KEY,
+         DFS_DATANODE_OOB_TIMEOUT_DEFAULT).split(",");
+     for (int i = 0; i < numOobTypes; i++) {
+       oobTimeouts[i] = (i < ele.length) ? Long.parseLong(ele[i]) : 0;
+     }
+   }
+ 
+   /**
+    * Get the timeout to be used for transmitting the OOB type
+    * @return the timeout in milliseconds
+    */
+   public long getOOBTimeout(Status status)
+       throws IOException {
+     if (status.getNumber() < Status.OOB_RESTART_VALUE ||
+         status.getNumber() > Status.OOB_RESERVED3_VALUE) {
+       // Not an OOB.
+       throw new IOException("Not an OOB status: " + status);
+     }
+ 
+     return oobTimeouts[status.getNumber() - Status.OOB_RESTART_VALUE];
+   }
+ 
+   /**
+    * Start a timer to periodically write DataNode metrics to the log file. This
+    * behavior can be disabled by configuration.
+    *
+    * @param metricConf
+    */
+   protected void startMetricsLogger(Configuration metricConf) {
+     long metricsLoggerPeriodSec = metricConf.getInt(
+         DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_KEY,
+         DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_DEFAULT);
+ 
+     if (metricsLoggerPeriodSec <= 0) {
+       return;
+     }
+ 
+     MetricsLoggerTask.makeMetricsLoggerAsync(METRICS_LOG);
+ 
+     // Schedule the periodic logging.
+     metricsLoggerTimer = new ScheduledThreadPoolExecutor(1);
+     metricsLoggerTimer.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
+     metricsLoggerTimer.scheduleWithFixedDelay(new MetricsLoggerTask(METRICS_LOG,
+         "DataNode", (short) 0), metricsLoggerPeriodSec, metricsLoggerPeriodSec,
+         TimeUnit.SECONDS);
+   }
+ 
+   protected void stopMetricsLogger() {
+     if (metricsLoggerTimer != null) {
+       metricsLoggerTimer.shutdown();
+       metricsLoggerTimer = null;
+     }
+   }
+ 
+   @VisibleForTesting
+   ScheduledThreadPoolExecutor getMetricsLoggerTimer() {
+     return metricsLoggerTimer;
    }
  }


[06/50] [abbrv] hadoop git commit: Merge commit '456e901a4c5c639267ee87b8e5f1319f256d20c2' (HDFS-6407. Add sorting and pagination in the datanode tab of the NN Web UI. Contributed by Haohui Mai.) into HDFS-7285-merge

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 0000000,8528999..7756bb9
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@@ -1,0 -1,1486 +1,1511 @@@
+ /**
+  * 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.hdfs.protocol;
+ 
+ import java.io.IOException;
+ import java.util.EnumSet;
+ import java.util.List;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.crypto.CryptoProtocolVersion;
+ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+ import org.apache.hadoop.fs.CacheFlag;
+ import org.apache.hadoop.fs.ContentSummary;
+ import org.apache.hadoop.fs.CreateFlag;
+ import org.apache.hadoop.fs.FsServerDefaults;
+ import org.apache.hadoop.fs.Options;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.fs.XAttr;
+ import org.apache.hadoop.fs.XAttrSetFlag;
+ import org.apache.hadoop.fs.permission.AclEntry;
+ import org.apache.hadoop.fs.permission.AclStatus;
+ import org.apache.hadoop.fs.permission.FsAction;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hdfs.inotify.EventBatchList;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+ import org.apache.hadoop.io.EnumSetWritable;
+ import org.apache.hadoop.io.Text;
+ import org.apache.hadoop.io.retry.AtMostOnce;
+ import org.apache.hadoop.io.retry.Idempotent;
+ import org.apache.hadoop.security.KerberosInfo;
+ import org.apache.hadoop.security.token.Token;
+ import org.apache.hadoop.security.token.TokenInfo;
+ 
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
+ 
+ /**********************************************************************
+  * ClientProtocol is used by user code via the DistributedFileSystem class to
+  * communicate with the NameNode.  User code can manipulate the directory
+  * namespace, as well as open/close file streams, etc.
+  *
+  **********************************************************************/
+ @InterfaceAudience.Private
+ @InterfaceStability.Evolving
+ @KerberosInfo(
+     serverPrincipal = DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
+ @TokenInfo(DelegationTokenSelector.class)
+ public interface ClientProtocol {
+ 
+   /**
+    * Until version 69, this class ClientProtocol served as both
+    * the client interface to the NN AND the RPC protocol used to
+    * communicate with the NN.
+    *
+    * This class is used by both the DFSClient and the
+    * NN server side to insulate from the protocol serialization.
+    *
+    * If you are adding/changing this interface then you need to
+    * change both this class and ALSO related protocol buffer
+    * wire protocol definition in ClientNamenodeProtocol.proto.
+    *
+    * For more details on protocol buffer wire protocol, please see
+    * .../org/apache/hadoop/hdfs/protocolPB/overview.html
+    *
+    * The log of historical changes can be retrieved from the svn).
+    * 69: Eliminate overloaded method names.
+    *
+    * 69L is the last version id when this class was used for protocols
+    *  serialization. DO not update this version any further.
+    */
+   long versionID = 69L;
+ 
+   ///////////////////////////////////////
+   // File contents
+   ///////////////////////////////////////
+   /**
+    * Get locations of the blocks of the specified file
+    * within the specified range.
+    * DataNode locations for each block are sorted by
+    * the proximity to the client.
+    * <p>
+    * Return {@link LocatedBlocks} which contains
+    * file length, blocks and their locations.
+    * DataNode locations for each block are sorted by
+    * the distance to the client's address.
+    * <p>
+    * The client will then have to contact
+    * one of the indicated DataNodes to obtain the actual data.
+    *
+    * @param src file name
+    * @param offset range start offset
+    * @param length range length
+    *
+    * @return file length and array of blocks with their locations
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws java.io.FileNotFoundException If file <code>src</code> does not
+    *           exist
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   LocatedBlocks getBlockLocations(String src, long offset, long length)
+       throws IOException;
+ 
+   /**
+    * Get server default values for a number of configuration params.
+    * @return a set of server default configuration values
+    * @throws IOException
+    */
+   @Idempotent
+   FsServerDefaults getServerDefaults() throws IOException;
+ 
+   /**
+    * Create a new file entry in the namespace.
+    * <p>
+    * This will create an empty file specified by the source path.
+    * The path should reflect a full path originated at the root.
+    * The name-node does not have a notion of "current" directory for a client.
+    * <p>
+    * Once created, the file is visible and available for read to other clients.
+    * Although, other clients cannot {@link #delete(String, boolean)}, re-create
+    * or {@link #rename(String, String)} it until the file is completed
+    * or explicitly as a result of lease expiration.
+    * <p>
+    * Blocks have a maximum size.  Clients that intend to create
+    * multi-block files must also use
+    * {@link #addBlock}
+    *
+    * @param src path of the file being created.
+    * @param masked masked permission.
+    * @param clientName name of the current client.
+    * @param flag indicates whether the file should be
+    * overwritten if it already exists or create if it does not exist or append.
+    * @param createParent create missing parent directory if true
+    * @param replication block replication factor.
+    * @param blockSize maximum block size.
+    * @param supportedVersions CryptoProtocolVersions supported by the client
+    *
+    * @return the status of the created file, it could be null if the server
+    *           doesn't support returning the file status
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws AlreadyBeingCreatedException if the path does not exist.
+    * @throws DSQuotaExceededException If file creation violates disk space
+    *           quota restriction
+    * @throws org.apache.hadoop.fs.FileAlreadyExistsException If file
+    *           <code>src</code> already exists
+    * @throws java.io.FileNotFoundException If parent of <code>src</code> does
+    *           not exist and <code>createParent</code> is false
+    * @throws org.apache.hadoop.fs.ParentNotDirectoryException If parent of
+    *           <code>src</code> is not a directory.
+    * @throws NSQuotaExceededException If file creation violates name space
+    *           quota restriction
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException create not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred
+    *
+    * RuntimeExceptions:
+    * @throws org.apache.hadoop.fs.InvalidPathException Path <code>src</code> is
+    *           invalid
+    * <p>
+    * <em>Note that create with {@link CreateFlag#OVERWRITE} is idempotent.</em>
+    */
+   @AtMostOnce
+   HdfsFileStatus create(String src, FsPermission masked,
+       String clientName, EnumSetWritable<CreateFlag> flag,
+       boolean createParent, short replication, long blockSize,
+       CryptoProtocolVersion[] supportedVersions)
+       throws IOException;
+ 
+   /**
+    * Append to the end of the file.
+    * @param src path of the file being created.
+    * @param clientName name of the current client.
+    * @param flag indicates whether the data is appended to a new block.
+    * @return wrapper with information about the last partial block and file
+    *    status if any
+    * @throws org.apache.hadoop.security.AccessControlException if permission to
+    * append file is denied by the system. As usually on the client side the
+    * exception will be wrapped into
+    * {@link org.apache.hadoop.ipc.RemoteException}.
+    * Allows appending to an existing file if the server is
+    * configured with the parameter dfs.support.append set to true, otherwise
+    * throws an IOException.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If permission to
+    *           append to file is denied
+    * @throws java.io.FileNotFoundException If file <code>src</code> is not found
+    * @throws DSQuotaExceededException If append violates disk space quota
+    *           restriction
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException append not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred.
+    *
+    * RuntimeExceptions:
+    * @throws UnsupportedOperationException if append is not supported
+    */
+   @AtMostOnce
+   LastBlockWithStatus append(String src, String clientName,
+       EnumSetWritable<CreateFlag> flag) throws IOException;
+ 
+   /**
+    * Set replication for an existing file.
+    * <p>
+    * The NameNode sets replication to the new value and returns.
+    * The actual block replication is not expected to be performed during
+    * this method call. The blocks will be populated or removed in the
+    * background as the result of the routine block maintenance procedures.
+    *
+    * @param src file name
+    * @param replication new replication
+    *
+    * @return true if successful;
+    *         false if file does not exist or is a directory
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws DSQuotaExceededException If replication violates disk space
+    *           quota restriction
+    * @throws java.io.FileNotFoundException If file <code>src</code> is not found
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>src</code>
+    *           contains a symlink
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   boolean setReplication(String src, short replication)
+       throws IOException;
+ 
+   /**
+    * Get all the available block storage policies.
+    * @return All the in-use block storage policies currently.
+    */
+   @Idempotent
+   BlockStoragePolicy[] getStoragePolicies() throws IOException;
+ 
+   /**
+    * Set the storage policy for a file/directory.
+    * @param src Path of an existing file/directory.
+    * @param policyName The name of the storage policy
+    * @throws SnapshotAccessControlException If access is denied
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>src</code>
+    *           contains a symlink
+    * @throws java.io.FileNotFoundException If file/dir <code>src</code> is not
+    *           found
+    * @throws QuotaExceededException If changes violate the quota restriction
+    */
+   @Idempotent
+   void setStoragePolicy(String src, String policyName)
+       throws IOException;
+ 
+   /**
+    * Get the storage policy for a file/directory.
+    * @param path
+    *          Path of an existing file/directory.
+    * @throws AccessControlException
+    *           If access is denied
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException
+    *           if <code>src</code> contains a symlink
+    * @throws java.io.FileNotFoundException
+    *           If file/dir <code>src</code> is not found
+    */
+   @Idempotent
+   BlockStoragePolicy getStoragePolicy(String path) throws IOException;
+ 
+   /**
+    * Set permissions for an existing file/directory.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws java.io.FileNotFoundException If file <code>src</code> is not found
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   void setPermission(String src, FsPermission permission)
+       throws IOException;
+ 
+   /**
+    * Set Owner of a path (i.e. a file or a directory).
+    * The parameters username and groupname cannot both be null.
+    * @param src file path
+    * @param username If it is null, the original username remains unchanged.
+    * @param groupname If it is null, the original groupname remains unchanged.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws java.io.FileNotFoundException If file <code>src</code> is not found
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   void setOwner(String src, String username, String groupname)
+       throws IOException;
+ 
+   /**
+    * The client can give up on a block by calling abandonBlock().
+    * The client can then either obtain a new block, or complete or abandon the
+    * file.
+    * Any partial writes to the block will be discarded.
+    *
+    * @param b         Block to abandon
+    * @param fileId    The id of the file where the block resides.  Older clients
+    *                    will pass GRANDFATHER_INODE_ID here.
+    * @param src       The path of the file where the block resides.
+    * @param holder    Lease holder.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws java.io.FileNotFoundException file <code>src</code> is not found
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   void abandonBlock(ExtendedBlock b, long fileId,
+       String src, String holder)
+       throws IOException;
+ 
+   /**
+    * A client that wants to write an additional block to the
+    * indicated filename (which must currently be open for writing)
+    * should call addBlock().
+    *
+    * addBlock() allocates a new block and datanodes the block data
+    * should be replicated to.
+    *
+    * addBlock() also commits the previous block by reporting
+    * to the name-node the actual generation stamp and the length
+    * of the block that the client has transmitted to data-nodes.
+    *
+    * @param src the file being created
+    * @param clientName the name of the client that adds the block
+    * @param previous  previous block
+    * @param excludeNodes a list of nodes that should not be
+    * allocated for the current block
+    * @param fileId the id uniquely identifying a file
+    * @param favoredNodes the list of nodes where the client wants the blocks.
+    *          Nodes are identified by either host name or address.
+    *
+    * @return LocatedBlock allocated block information.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws java.io.FileNotFoundException If file <code>src</code> is not found
+    * @throws org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException
+    *           previous blocks of the file are not replicated yet.
+    *           Blocks cannot be added until replication completes.
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException create not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   LocatedBlock addBlock(String src, String clientName,
+       ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
+       String[] favoredNodes)
+       throws IOException;
+ 
+   /**
+    * Get a datanode for an existing pipeline.
+    *
+    * @param src the file being written
+    * @param fileId the ID of the file being written
+    * @param blk the block being written
+    * @param existings the existing nodes in the pipeline
+    * @param excludes the excluded nodes
+    * @param numAdditionalNodes number of additional datanodes
+    * @param clientName the name of the client
+    *
+    * @return the located block.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws java.io.FileNotFoundException If file <code>src</code> is not found
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException create not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   LocatedBlock getAdditionalDatanode(final String src,
+       final long fileId, final ExtendedBlock blk,
+       final DatanodeInfo[] existings,
+       final String[] existingStorageIDs,
+       final DatanodeInfo[] excludes,
+       final int numAdditionalNodes, final String clientName
+       ) throws IOException;
+ 
+   /**
+    * The client is done writing data to the given filename, and would
+    * like to complete it.
+    *
+    * The function returns whether the file has been closed successfully.
+    * If the function returns false, the caller should try again.
+    *
+    * close() also commits the last block of file by reporting
+    * to the name-node the actual generation stamp and the length
+    * of the block that the client has transmitted to data-nodes.
+    *
+    * A call to complete() will not return true until all the file's
+    * blocks have been replicated the minimum number of times.  Thus,
+    * DataNode failures may cause a client to call complete() several
+    * times before succeeding.
+    *
+    * @param src the file being created
+    * @param clientName the name of the client that adds the block
+    * @param last the last block info
+    * @param fileId the id uniquely identifying a file
+    *
+    * @return true if all file blocks are minimally replicated or false otherwise
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws java.io.FileNotFoundException If file <code>src</code> is not found
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException create not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   boolean complete(String src, String clientName,
+                           ExtendedBlock last, long fileId)
+       throws IOException;
+ 
+   /**
+    * The client wants to report corrupted blocks (blocks with specified
+    * locations on datanodes).
+    * @param blocks Array of located blocks to report
+    */
+   @Idempotent
+   void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
+ 
+   ///////////////////////////////////////
+   // Namespace management
+   ///////////////////////////////////////
+   /**
+    * Rename an item in the file system namespace.
+    * @param src existing file or directory name.
+    * @param dst new name.
+    * @return true if successful, or false if the old name does not exist
+    * or if the new name already belongs to the namespace.
+    *
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException an I/O error occurred
+    */
+   @AtMostOnce
+   boolean rename(String src, String dst)
+       throws IOException;
+ 
+   /**
+    * Moves blocks from srcs to trg and delete srcs.
+    *
+    * @param trg existing file
+    * @param srcs - list of existing files (same block size, same replication)
+    * @throws IOException if some arguments are invalid
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>trg</code> or
+    *           <code>srcs</code> contains a symlink
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    */
+   @AtMostOnce
+   void concat(String trg, String[] srcs)
+       throws IOException;
+ 
+   /**
+    * Rename src to dst.
+    * <ul>
+    * <li>Fails if src is a file and dst is a directory.
+    * <li>Fails if src is a directory and dst is a file.
+    * <li>Fails if the parent of dst does not exist or is a file.
+    * </ul>
+    * <p>
+    * Without OVERWRITE option, rename fails if the dst already exists.
+    * With OVERWRITE option, rename overwrites the dst, if it is a file
+    * or an empty directory. Rename fails if dst is a non-empty directory.
+    * <p>
+    * This implementation of rename is atomic.
+    * <p>
+    * @param src existing file or directory name.
+    * @param dst new name.
+    * @param options Rename options
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws DSQuotaExceededException If rename violates disk space
+    *           quota restriction
+    * @throws org.apache.hadoop.fs.FileAlreadyExistsException If <code>dst</code>
+    *           already exists and <code>options</code> has
+    *           {@link org.apache.hadoop.fs.Options.Rename#OVERWRITE} option
+    *           false.
+    * @throws java.io.FileNotFoundException If <code>src</code> does not exist
+    * @throws NSQuotaExceededException If rename violates namespace
+    *           quota restriction
+    * @throws org.apache.hadoop.fs.ParentNotDirectoryException If parent of
+    *           <code>dst</code> is not a directory
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException rename not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code> or
+    *           <code>dst</code> contains a symlink
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred
+    */
+   @AtMostOnce
+   void rename2(String src, String dst, Options.Rename... options)
+       throws IOException;
+ 
+   /**
+    * Truncate file src to new size.
+    * <ul>
+    * <li>Fails if src is a directory.
+    * <li>Fails if src does not exist.
+    * <li>Fails if src is not closed.
+    * <li>Fails if new size is greater than current size.
+    * </ul>
+    * <p>
+    * This implementation of truncate is purely a namespace operation if truncate
+    * occurs at a block boundary. Requires DataNode block recovery otherwise.
+    * <p>
+    * @param src  existing file
+    * @param newLength  the target size
+    *
+    * @return true if client does not need to wait for block recovery,
+    * false if client needs to wait for block recovery.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws java.io.FileNotFoundException If file <code>src</code> is not found
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException truncate
+    *           not allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   boolean truncate(String src, long newLength, String clientName)
+       throws IOException;
+ 
+   /**
+    * Delete the given file or directory from the file system.
+    * <p>
+    * same as delete but provides a way to avoid accidentally
+    * deleting non empty directories programmatically.
+    * @param src existing name
+    * @param recursive if true deletes a non empty directory recursively,
+    * else throws an exception.
+    * @return true only if the existing file or directory was actually removed
+    * from the file system.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws java.io.FileNotFoundException If file <code>src</code> is not found
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException create not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred
+    */
+   @AtMostOnce
+   boolean delete(String src, boolean recursive)
+       throws IOException;
+ 
+   /**
+    * Create a directory (or hierarchy of directories) with the given
+    * name and permission.
+    *
+    * @param src The path of the directory being created
+    * @param masked The masked permission of the directory being created
+    * @param createParent create missing parent directory if true
+    *
+    * @return True if the operation success.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException If access is
+    *           denied
+    * @throws org.apache.hadoop.fs.FileAlreadyExistsException If <code>src</code>
+    *           already exists
+    * @throws java.io.FileNotFoundException If parent of <code>src</code> does
+    *           not exist and <code>createParent</code> is false
+    * @throws NSQuotaExceededException If file creation violates quota
+    *           restriction
+    * @throws org.apache.hadoop.fs.ParentNotDirectoryException If parent of
+    *           <code>src</code> is not a directory
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException create not
+    *           allowed in safemode
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred.
+    *
+    * RunTimeExceptions:
+    * @throws org.apache.hadoop.fs.InvalidPathException If <code>src</code> is
+    *           invalid
+    */
+   @Idempotent
+   boolean mkdirs(String src, FsPermission masked, boolean createParent)
+       throws IOException;
+ 
+   /**
+    * Get a partial listing of the indicated directory.
+    *
+    * @param src the directory name
+    * @param startAfter the name to start listing after encoded in java UTF8
+    * @param needLocation if the FileStatus should contain block locations
+    *
+    * @return a partial listing starting after startAfter
+    *
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws java.io.FileNotFoundException file <code>src</code> is not found
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException If <code>src</code>
+    *           contains a symlink
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   DirectoryListing getListing(String src, byte[] startAfter,
+       boolean needLocation) throws IOException;
+ 
+   /**
+    * Get listing of all the snapshottable directories.
+    *
+    * @return Information about all the current snapshottable directory
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+       throws IOException;
+ 
+   ///////////////////////////////////////
+   // System issues and management
+   ///////////////////////////////////////
+ 
+   /**
+    * Client programs can cause stateful changes in the NameNode
+    * that affect other clients.  A client may obtain a file and
+    * neither abandon nor complete it.  A client might hold a series
+    * of locks that prevent other clients from proceeding.
+    * Clearly, it would be bad if a client held a bunch of locks
+    * that it never gave up.  This can happen easily if the client
+    * dies unexpectedly.
+    * <p>
+    * So, the NameNode will revoke the locks and live file-creates
+    * for clients that it thinks have died.  A client tells the
+    * NameNode that it is still alive by periodically calling
+    * renewLease().  If a certain amount of time passes since
+    * the last call to renewLease(), the NameNode assumes the
+    * client has died.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   void renewLease(String clientName) throws IOException;
+ 
+   /**
+    * Start lease recovery.
+    * Lightweight NameNode operation to trigger lease recovery
+    *
+    * @param src path of the file to start lease recovery
+    * @param clientName name of the current client
+    * @return true if the file is already closed
+    * @throws IOException
+    */
+   @Idempotent
+   boolean recoverLease(String src, String clientName) throws IOException;
+ 
+   int GET_STATS_CAPACITY_IDX = 0;
+   int GET_STATS_USED_IDX = 1;
+   int GET_STATS_REMAINING_IDX = 2;
+   int GET_STATS_UNDER_REPLICATED_IDX = 3;
+   int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
+   int GET_STATS_MISSING_BLOCKS_IDX = 5;
+   int GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX = 6;
+ 
+   /**
+    * Get a set of statistics about the filesystem.
+    * Right now, only seven values are returned.
+    * <ul>
+    * <li> [0] contains the total storage capacity of the system, in bytes.</li>
+    * <li> [1] contains the total used space of the system, in bytes.</li>
+    * <li> [2] contains the available storage of the system, in bytes.</li>
+    * <li> [3] contains number of under replicated blocks in the system.</li>
+    * <li> [4] contains number of blocks with a corrupt replica. </li>
+    * <li> [5] contains number of blocks without any good replicas left. </li>
+    * <li> [6] contains number of blocks which have replication factor
+    *          1 and have lost the only replica. </li>
+    * </ul>
+    * Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of
+    * actual numbers to index into the array.
+    */
+   @Idempotent
+   long[] getStats() throws IOException;
+ 
+   /**
+    * Get a report on the system's current datanodes.
+    * One DatanodeInfo object is returned for each DataNode.
+    * Return live datanodes if type is LIVE; dead datanodes if type is DEAD;
+    * otherwise all datanodes if type is ALL.
+    */
+   @Idempotent
+   DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
+       throws IOException;
+ 
+   /**
+    * Get a report on the current datanode storages.
+    */
+   @Idempotent
+   DatanodeStorageReport[] getDatanodeStorageReport(
+       HdfsConstants.DatanodeReportType type) throws IOException;
+ 
+   /**
+    * Get the block size for the given file.
+    * @param filename The name of the file
+    * @return The number of bytes in each block
+    * @throws IOException
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if the path contains
+    *           a symlink.
+    */
+   @Idempotent
+   long getPreferredBlockSize(String filename)
+       throws IOException;
+ 
+   /**
+    * Enter, leave or get safe mode.
+    * <p>
+    * Safe mode is a name node state when it
+    * <ol><li>does not accept changes to name space (read-only), and</li>
+    * <li>does not replicate or delete blocks.</li></ol>
+    *
+    * <p>
+    * Safe mode is entered automatically at name node startup.
+    * Safe mode can also be entered manually using
+    * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean)
+    * setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}.
+    * <p>
+    * At startup the name node accepts data node reports collecting
+    * information about block locations.
+    * In order to leave safe mode it needs to collect a configurable
+    * percentage called threshold of blocks, which satisfy the minimal
+    * replication condition.
+    * The minimal replication condition is that each block must have at least
+    * <tt>dfs.namenode.replication.min</tt> replicas.
+    * When the threshold is reached the name node extends safe mode
+    * for a configurable amount of time
+    * to let the remaining data nodes to check in before it
+    * will start replicating missing blocks.
+    * Then the name node leaves safe mode.
+    * <p>
+    * If safe mode is turned on manually using
+    * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean)
+    * setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}
+    * then the name node stays in safe mode until it is manually turned off
+    * using {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean)
+    * setSafeMode(SafeModeAction.SAFEMODE_LEAVE,false)}.
+    * Current state of the name node can be verified using
+    * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean)
+    * setSafeMode(SafeModeAction.SAFEMODE_GET,false)}
+    * <h4>Configuration parameters:</h4>
+    * <tt>dfs.safemode.threshold.pct</tt> is the threshold parameter.<br>
+    * <tt>dfs.safemode.extension</tt> is the safe mode extension parameter.<br>
+    * <tt>dfs.namenode.replication.min</tt> is the minimal replication parameter.
+    *
+    * <h4>Special cases:</h4>
+    * The name node does not enter safe mode at startup if the threshold is
+    * set to 0 or if the name space is empty.<br>
+    * If the threshold is set to 1 then all blocks need to have at least
+    * minimal replication.<br>
+    * If the threshold value is greater than 1 then the name node will not be
+    * able to turn off safe mode automatically.<br>
+    * Safe mode can always be turned off manually.
+    *
+    * @param action  <ul> <li>0 leave safe mode;</li>
+    *                <li>1 enter safe mode;</li>
+    *                <li>2 get safe mode state.</li></ul>
+    * @param isChecked If true then action will be done only in ActiveNN.
+    *
+    * @return <ul><li>0 if the safe mode is OFF or</li>
+    *         <li>1 if the safe mode is ON.</li></ul>
+    *
+    * @throws IOException
+    */
+   @Idempotent
+   boolean setSafeMode(HdfsConstants.SafeModeAction action, boolean isChecked)
+       throws IOException;
+ 
+   /**
+    * Save namespace image.
+    * <p>
+    * Saves current namespace into storage directories and reset edits log.
+    * Requires superuser privilege and safe mode.
+    *
+    * @param timeWindow NameNode does a checkpoint if the latest checkpoint was
+    *                   done beyond the given time period (in seconds).
+    * @param txGap NameNode does a checkpoint if the gap between the latest
+    *              checkpoint and the latest transaction id is greater this gap.
+    * @return whether an extra checkpoint has been done
+    *
+    * @throws IOException if image creation failed.
+    */
+   @AtMostOnce
+   boolean saveNamespace(long timeWindow, long txGap) throws IOException;
+ 
+   /**
+    * Roll the edit log.
+    * Requires superuser privileges.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException if the superuser
+    *           privilege is violated
+    * @throws IOException if log roll fails
+    * @return the txid of the new segment
+    */
+   @Idempotent
+   long rollEdits() throws IOException;
+ 
+   /**
+    * Enable/Disable restore failed storage.
+    * <p>
+    * sets flag to enable restore of failed storage replicas
+    *
+    * @throws org.apache.hadoop.security.AccessControlException if the superuser
+    *           privilege is violated.
+    */
+   @Idempotent
+   boolean restoreFailedStorage(String arg) throws IOException;
+ 
+   /**
+    * Tells the namenode to reread the hosts and exclude files.
+    * @throws IOException
+    */
+   @Idempotent
+   void refreshNodes() throws IOException;
+ 
+   /**
+    * Finalize previous upgrade.
+    * Remove file system state saved during the upgrade.
+    * The upgrade will become irreversible.
+    *
+    * @throws IOException
+    */
+   @Idempotent
+   void finalizeUpgrade() throws IOException;
+ 
+   /**
+    * Rolling upgrade operations.
+    * @param action either query, prepare or finalize.
+    * @return rolling upgrade information. On query, if no upgrade is in
+    * progress, returns null.
+    */
+   @Idempotent
+   RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
+       throws IOException;
+ 
+   /**
+    * @return CorruptFileBlocks, containing a list of corrupt files (with
+    *         duplicates if there is more than one corrupt block in a file)
+    *         and a cookie
+    * @throws IOException
+    *
+    * Each call returns a subset of the corrupt files in the system. To obtain
+    * all corrupt files, call this method repeatedly and each time pass in the
+    * cookie returned from the previous call.
+    */
+   @Idempotent
+   CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+       throws IOException;
+ 
+   /**
+    * Dumps namenode data structures into specified file. If the file
+    * already exists, then append.
+    *
+    * @throws IOException
+    */
+   @Idempotent
+   void metaSave(String filename) throws IOException;
+ 
+   /**
+    * Tell all datanodes to use a new, non-persistent bandwidth value for
+    * dfs.balance.bandwidthPerSec.
+    *
+    * @param bandwidth Blanacer bandwidth in bytes per second for this datanode.
+    * @throws IOException
+    */
+   @Idempotent
+   void setBalancerBandwidth(long bandwidth) throws IOException;
+ 
+   /**
+    * Get the file info for a specific file or directory.
+    * @param src The string representation of the path to the file
+    *
+    * @return object containing information regarding the file
+    *         or null if file not found
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws java.io.FileNotFoundException file <code>src</code> is not found
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if the path contains
+    *           a symlink.
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   HdfsFileStatus getFileInfo(String src) throws IOException;
+ 
+   /**
+    * Get the close status of a file.
+    * @param src The string representation of the path to the file
+    *
+    * @return return true if file is closed
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws java.io.FileNotFoundException file <code>src</code> is not found
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if the path contains
+    *           a symlink.
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   boolean isFileClosed(String src) throws IOException;
+ 
+   /**
+    * Get the file info for a specific file or directory. If the path
+    * refers to a symlink then the FileStatus of the symlink is returned.
+    * @param src The string representation of the path to the file
+    *
+    * @return object containing information regarding the file
+    *         or null if file not found
+    *
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>src</code>
+    *           contains a symlink
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   HdfsFileStatus getFileLinkInfo(String src) throws IOException;
+ 
+   /**
+    * Get {@link ContentSummary} rooted at the specified directory.
+    * @param path The string representation of the path
+    *
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws java.io.FileNotFoundException file <code>path</code> is not found
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>path</code>
+    *           contains a symlink.
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   ContentSummary getContentSummary(String path) throws IOException;
+ 
+   /**
+    * Set the quota for a directory.
+    * @param path  The string representation of the path to the directory
+    * @param namespaceQuota Limit on the number of names in the tree rooted
+    *                       at the directory
+    * @param storagespaceQuota Limit on storage space occupied all the files
+    *                       under this directory.
+    * @param type StorageType that the space quota is intended to be set on.
+    *             It may be null when called by traditional space/namespace
+    *             quota. When type is is not null, the storagespaceQuota
+    *             parameter is for type specified and namespaceQuota must be
+    *             {@link HdfsConstants#QUOTA_DONT_SET}.
+    *
+    * <br><br>
+    *
+    * The quota can have three types of values : (1) 0 or more will set
+    * the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET}  implies
+    * the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET}
+    * implies the quota will be reset. Any other value is a runtime error.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws java.io.FileNotFoundException file <code>path</code> is not found
+    * @throws QuotaExceededException if the directory size
+    *           is greater than the given quota
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if the
+    *           <code>path</code> contains a symlink.
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+       StorageType type) throws IOException;
+ 
+   /**
+    * Write all metadata for this file into persistent storage.
+    * The file must be currently open for writing.
+    * @param src The string representation of the path
+    * @param inodeId The inode ID, or GRANDFATHER_INODE_ID if the client is
+    *                too old to support fsync with inode IDs.
+    * @param client The string representation of the client
+    * @param lastBlockLength The length of the last block (under construction)
+    *                        to be reported to NameNode
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws java.io.FileNotFoundException file <code>src</code> is not found
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>src</code>
+    *           contains a symlink.
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   void fsync(String src, long inodeId, String client, long lastBlockLength)
+       throws IOException;
+ 
+   /**
+    * Sets the modification and access time of the file to the specified time.
+    * @param src The string representation of the path
+    * @param mtime The number of milliseconds since Jan 1, 1970.
+    *              Setting mtime to -1 means that modification time should not
+    *              be set by this call.
+    * @param atime The number of milliseconds since Jan 1, 1970.
+    *              Setting atime to -1 means that access time should not be set
+    *              by this call.
+    *
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws java.io.FileNotFoundException file <code>src</code> is not found
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>src</code>
+    *           contains a symlink.
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred
+    */
+   @Idempotent
+   void setTimes(String src, long mtime, long atime) throws IOException;
+ 
+   /**
+    * Create symlink to a file or directory.
+    * @param target The path of the destination that the
+    *               link points to.
+    * @param link The path of the link being created.
+    * @param dirPerm permissions to use when creating parent directories
+    * @param createParent - if true then missing parent dirs are created
+    *                       if false then parent must exist
+    *
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws org.apache.hadoop.fs.FileAlreadyExistsException If file
+    *           <code>link</code> already exists
+    * @throws java.io.FileNotFoundException If parent of <code>link</code> does
+    *           not exist and <code>createParent</code> is false
+    * @throws org.apache.hadoop.fs.ParentNotDirectoryException If parent of
+    *           <code>link</code> is not a directory.
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>link</code>
+    *           contains a symlink.
+    * @throws SnapshotAccessControlException if path is in RO snapshot
+    * @throws IOException If an I/O error occurred
+    */
+   @AtMostOnce
+   void createSymlink(String target, String link, FsPermission dirPerm,
+       boolean createParent) throws IOException;
+ 
+   /**
+    * Return the target of the given symlink. If there is an intermediate
+    * symlink in the path (ie a symlink leading up to the final path component)
+    * then the given path is returned with this symlink resolved.
+    *
+    * @param path The path with a link that needs resolution.
+    * @return The path after resolving the first symbolic link in the path.
+    * @throws org.apache.hadoop.security.AccessControlException permission denied
+    * @throws java.io.FileNotFoundException If <code>path</code> does not exist
+    * @throws IOException If the given path does not refer to a symlink
+    *           or an I/O error occurred
+    */
+   @Idempotent
+   String getLinkTarget(String path) throws IOException;
+ 
+   /**
+    * Get a new generation stamp together with an access token for
+    * a block under construction
+    *
+    * This method is called only when a client needs to recover a failed
+    * pipeline or set up a pipeline for appending to a block.
+    *
+    * @param block a block
+    * @param clientName the name of the client
+    * @return a located block with a new generation stamp and an access token
+    * @throws IOException if any error occurs
+    */
+   @Idempotent
+   LocatedBlock updateBlockForPipeline(ExtendedBlock block,
+       String clientName) throws IOException;
+ 
+   /**
+    * Update a pipeline for a block under construction.
+    *
+    * @param clientName the name of the client
+    * @param oldBlock the old block
+    * @param newBlock the new block containing new generation stamp and length
+    * @param newNodes datanodes in the pipeline
+    * @throws IOException if any error occurs
+    */
+   @AtMostOnce
+   void updatePipeline(String clientName, ExtendedBlock oldBlock,
+       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
+       throws IOException;
+ 
+   /**
+    * Get a valid Delegation Token.
+    *
+    * @param renewer the designated renewer for the token
+    * @return Token<DelegationTokenIdentifier>
+    * @throws IOException
+    */
+   @Idempotent
+   Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+       throws IOException;
+ 
+   /**
+    * Renew an existing delegation token.
+    *
+    * @param token delegation token obtained earlier
+    * @return the new expiration time
+    * @throws IOException
+    */
+   @Idempotent
+   long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+       throws IOException;
+ 
+   /**
+    * Cancel an existing delegation token.
+    *
+    * @param token delegation token
+    * @throws IOException
+    */
+   @Idempotent
+   void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+       throws IOException;
+ 
+   /**
+    * @return encryption key so a client can encrypt data sent via the
+    *         DataTransferProtocol to/from DataNodes.
+    * @throws IOException
+    */
+   @Idempotent
+   DataEncryptionKey getDataEncryptionKey() throws IOException;
+ 
+   /**
+    * Create a snapshot.
+    * @param snapshotRoot the path that is being snapshotted
+    * @param snapshotName name of the snapshot created
+    * @return the snapshot path.
+    * @throws IOException
+    */
+   @AtMostOnce
+   String createSnapshot(String snapshotRoot, String snapshotName)
+       throws IOException;
+ 
+   /**
+    * Delete a specific snapshot of a snapshottable directory.
+    * @param snapshotRoot  The snapshottable directory
+    * @param snapshotName Name of the snapshot for the snapshottable directory
+    * @throws IOException
+    */
+   @AtMostOnce
+   void deleteSnapshot(String snapshotRoot, String snapshotName)
+       throws IOException;
+ 
+   /**
+    * Rename a snapshot.
+    * @param snapshotRoot the directory path where the snapshot was taken
+    * @param snapshotOldName old name of the snapshot
+    * @param snapshotNewName new name of the snapshot
+    * @throws IOException
+    */
+   @AtMostOnce
+   void renameSnapshot(String snapshotRoot, String snapshotOldName,
+       String snapshotNewName) throws IOException;
+ 
+   /**
+    * Allow snapshot on a directory.
+    * @param snapshotRoot the directory to be snapped
+    * @throws IOException on error
+    */
+   @Idempotent
+   void allowSnapshot(String snapshotRoot)
+       throws IOException;
+ 
+   /**
+    * Disallow snapshot on a directory.
+    * @param snapshotRoot the directory to disallow snapshot
+    * @throws IOException on error
+    */
+   @Idempotent
+   void disallowSnapshot(String snapshotRoot)
+       throws IOException;
+ 
+   /**
+    * Get the difference between two snapshots, or between a snapshot and the
+    * current tree of a directory.
+    *
+    * @param snapshotRoot
+    *          full path of the directory where snapshots are taken
+    * @param fromSnapshot
+    *          snapshot name of the from point. Null indicates the current
+    *          tree
+    * @param toSnapshot
+    *          snapshot name of the to point. Null indicates the current
+    *          tree.
+    * @return The difference report represented as a {@link SnapshotDiffReport}.
+    * @throws IOException on error
+    */
+   @Idempotent
+   SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+       String fromSnapshot, String toSnapshot) throws IOException;
+ 
+   /**
+    * Add a CacheDirective to the CacheManager.
+    *
+    * @param directive A CacheDirectiveInfo to be added
+    * @param flags {@link CacheFlag}s to use for this operation.
+    * @return A CacheDirectiveInfo associated with the added directive
+    * @throws IOException if the directive could not be added
+    */
+   @AtMostOnce
+   long addCacheDirective(CacheDirectiveInfo directive,
+       EnumSet<CacheFlag> flags) throws IOException;
+ 
+   /**
+    * Modify a CacheDirective in the CacheManager.
+    *
+    * @param flags {@link CacheFlag}s to use for this operation.
+    * @throws IOException if the directive could not be modified
+    */
+   @AtMostOnce
+   void modifyCacheDirective(CacheDirectiveInfo directive,
+       EnumSet<CacheFlag> flags) throws IOException;
+ 
+   /**
+    * Remove a CacheDirectiveInfo from the CacheManager.
+    *
+    * @param id of a CacheDirectiveInfo
+    * @throws IOException if the cache directive could not be removed
+    */
+   @AtMostOnce
+   void removeCacheDirective(long id) throws IOException;
+ 
+   /**
+    * List the set of cached paths of a cache pool. Incrementally fetches results
+    * from the server.
+    *
+    * @param prevId The last listed entry ID, or -1 if this is the first call to
+    *               listCacheDirectives.
+    * @param filter Parameters to use to filter the list results,
+    *               or null to display all directives visible to us.
+    * @return A batch of CacheDirectiveEntry objects.
+    */
+   @Idempotent
+   BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
+       long prevId, CacheDirectiveInfo filter) throws IOException;
+ 
+   /**
+    * Add a new cache pool.
+    *
+    * @param info Description of the new cache pool
+    * @throws IOException If the request could not be completed.
+    */
+   @AtMostOnce
+   void addCachePool(CachePoolInfo info) throws IOException;
+ 
+   /**
+    * Modify an existing cache pool.
+    *
+    * @param req
+    *          The request to modify a cache pool.
+    * @throws IOException
+    *          If the request could not be completed.
+    */
+   @AtMostOnce
+   void modifyCachePool(CachePoolInfo req) throws IOException;
+ 
+   /**
+    * Remove a cache pool.
+    *
+    * @param pool name of the cache pool to remove.
+    * @throws IOException if the cache pool did not exist, or could not be
+    *           removed.
+    */
+   @AtMostOnce
+   void removeCachePool(String pool) throws IOException;
+ 
+   /**
+    * List the set of cache pools. Incrementally fetches results from the server.
+    *
+    * @param prevPool name of the last pool listed, or the empty string if this
+    *          is the first invocation of listCachePools
+    * @return A batch of CachePoolEntry objects.
+    */
+   @Idempotent
+   BatchedEntries<CachePoolEntry> listCachePools(String prevPool)
+       throws IOException;
+ 
+   /**
+    * Modifies ACL entries of files and directories.  This method can add new ACL
+    * entries or modify the permissions on existing ACL entries.  All existing
+    * ACL entries that are not specified in this call are retained without
+    * changes.  (Modifications are merged into the current ACL.)
+    */
+   @Idempotent
+   void modifyAclEntries(String src, List<AclEntry> aclSpec)
+       throws IOException;
+ 
+   /**
+    * Removes ACL entries from files and directories.  Other ACL entries are
+    * retained.
+    */
+   @Idempotent
+   void removeAclEntries(String src, List<AclEntry> aclSpec)
+       throws IOException;
+ 
+   /**
+    * Removes all default ACL entries from files and directories.
+    */
+   @Idempotent
+   void removeDefaultAcl(String src) throws IOException;
+ 
+   /**
+    * Removes all but the base ACL entries of files and directories.  The entries
+    * for user, group, and others are retained for compatibility with permission
+    * bits.
+    */
+   @Idempotent
+   void removeAcl(String src) throws IOException;
+ 
+   /**
+    * Fully replaces ACL of files and directories, discarding all existing
+    * entries.
+    */
+   @Idempotent
+   void setAcl(String src, List<AclEntry> aclSpec) throws IOException;
+ 
+   /**
+    * Gets the ACLs of files and directories.
+    */
+   @Idempotent
+   AclStatus getAclStatus(String src) throws IOException;
+ 
+   /**
+    * Create an encryption zone.
+    */
+   @AtMostOnce
+   void createEncryptionZone(String src, String keyName)
+     throws IOException;
+ 
+   /**
+    * Get the encryption zone for a path.
+    */
+   @Idempotent
+   EncryptionZone getEZForPath(String src)
+     throws IOException;
+ 
+   /**
+    * Used to implement cursor-based batched listing of {@EncryptionZone}s.
+    *
+    * @param prevId ID of the last item in the previous batch. If there is no
+    *               previous batch, a negative value can be used.
+    * @return Batch of encryption zones.
+    */
+   @Idempotent
+   BatchedEntries<EncryptionZone> listEncryptionZones(
+       long prevId) throws IOException;
+ 
+   /**
+    * Set xattr of a file or directory.
+    * The name must be prefixed with the namespace followed by ".". For example,
+    * "user.attr".
+    * <p/>
+    * Refer to the HDFS extended attributes user documentation for details.
+    *
+    * @param src file or directory
+    * @param xAttr <code>XAttr</code> to set
+    * @param flag set flag
+    * @throws IOException
+    */
+   @AtMostOnce
+   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
+       throws IOException;
+ 
+   /**
+    * Get xattrs of a file or directory. Values in xAttrs parameter are ignored.
+    * If xAttrs is null or empty, this is the same as getting all xattrs of the
+    * file or directory.  Only those xattrs for which the logged-in user has
+    * permissions to view are returned.
+    * <p/>
+    * Refer to the HDFS extended attributes user documentation for details.
+    *
+    * @param src file or directory
+    * @param xAttrs xAttrs to get
+    * @return List<XAttr> <code>XAttr</code> list
+    * @throws IOException
+    */
+   @Idempotent
+   List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
+       throws IOException;
+ 
+   /**
+    * List the xattrs names for a file or directory.
+    * Only the xattr names for which the logged in user has the permissions to
+    * access will be returned.
+    * <p/>
+    * Refer to the HDFS extended attributes user documentation for details.
+    *
+    * @param src file or directory
+    * @return List<XAttr> <code>XAttr</code> list
+    * @throws IOException
+    */
+   @Idempotent
+   List<XAttr> listXAttrs(String src)
+       throws IOException;
+ 
+   /**
+    * Remove xattr of a file or directory.Value in xAttr parameter is ignored.
+    * The name must be prefixed with the namespace followed by ".". For example,
+    * "user.attr".
+    * <p/>
+    * Refer to the HDFS extended attributes user documentation for details.
+    *
+    * @param src file or directory
+    * @param xAttr <code>XAttr</code> to remove
+    * @throws IOException
+    */
+   @AtMostOnce
+   void removeXAttr(String src, XAttr xAttr) throws IOException;
+ 
+   /**
+    * Checks if the user can access a path.  The mode specifies which access
+    * checks to perform.  If the requested permissions are granted, then the
+    * method returns normally.  If access is denied, then the method throws an
+    * {@link org.apache.hadoop.security.AccessControlException}.
+    * In general, applications should avoid using this method, due to the risk of
+    * time-of-check/time-of-use race conditions.  The permissions on a file may
+    * change immediately after the access call returns.
+    *
+    * @param path Path to check
+    * @param mode type of access to check
+    * @throws org.apache.hadoop.security.AccessControlException if access is
+    *           denied
+    * @throws java.io.FileNotFoundException if the path does not exist
+    * @throws IOException see specific implementation
+    */
+   @Idempotent
+   void checkAccess(String path, FsAction mode) throws IOException;
+ 
+   /**
+    * Get the highest txid the NameNode knows has been written to the edit
+    * log, or -1 if the NameNode's edit log is not yet open for write. Used as
+    * the starting point for the inotify event stream.
+    */
+   @Idempotent
+   long getCurrentEditLogTxid() throws IOException;
+ 
+   /**
+    * Get an ordered list of batches of events corresponding to the edit log
+    * transactions for txids equal to or greater than txid.
+    */
+   @Idempotent
+   EventBatchList getEditsFromTxid(long txid) throws IOException;
++
++  /**
++   * Create an erasure coding zone with specified policy, if any, otherwise
++   * default
++   */
++  @AtMostOnce
++  void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy)
++      throws IOException;
++
++  /**
++   * Get the erasure coding policies loaded in Namenode
++   *
++   * @throws IOException
++   */
++  @Idempotent
++  ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException;
++
++  /**
++   * Get the information about the EC zone for the path
++   * 
++   * @param src path to get the info for
++   * @throws IOException
++   */
++  @Idempotent
++  ErasureCodingZone getErasureCodingZone(String src) throws IOException;
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
index 0000000,0000000..533b630
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
@@@ -1,0 -1,0 +1,54 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one or more
++ * contributor license agreements. See the NOTICE file distributed with this
++ * work for additional information regarding copyright ownership. The ASF
++ * licenses this file to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance with the License.
++ * You may obtain a copy of the License at
++ * 
++ * http://www.apache.org/licenses/LICENSE-2.0
++ * 
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
++ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
++ * License for the specific language governing permissions and limitations under
++ * the License.
++ */
++package org.apache.hadoop.hdfs.protocol;
++
++/**
++ * Information about the EC Zone at the specified path.
++ */
++public class ErasureCodingZone {
++
++  private String dir;
++  private ErasureCodingPolicy ecPolicy;
++
++  public ErasureCodingZone(String dir, ErasureCodingPolicy ecPolicy) {
++    this.dir = dir;
++    this.ecPolicy = ecPolicy;
++  }
++
++  /**
++   * Get directory of the EC zone.
++   * 
++   * @return
++   */
++  public String getDir() {
++    return dir;
++  }
++
++  /**
++   * Get the erasure coding policy for the EC Zone
++   * 
++   * @return
++   */
++  public ErasureCodingPolicy getErasureCodingPolicy() {
++    return ecPolicy;
++  }
++
++  @Override
++  public String toString() {
++    return "Dir: " + getDir() + ", Policy: " + ecPolicy;
++  }
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------


[12/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 6093776,0000000..7b21cbe
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@@ -1,258 -1,0 +1,253 @@@
 +/**
 + * 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.hdfs.server.blockmanagement;
 +
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +
 +/**
 + * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
 + *
 + * We still use triplets to store DatanodeStorageInfo for each block in the
 + * block group, as well as the previous/next block in the corresponding
 + * DatanodeStorageInfo. For a (m+k) block group, the first (m+k) triplet units
 + * are sorted and strictly mapped to the corresponding block.
 + *
 + * Normally each block belonging to group is stored in only one DataNode.
 + * However, it is possible that some block is over-replicated. Thus the triplet
 + * array's size can be larger than (m+k). Thus currently we use an extra byte
 + * array to record the block index for each triplet.
 + */
 +public class BlockInfoStriped extends BlockInfo {
 +  private final ErasureCodingPolicy ecPolicy;
 +  /**
 +   * Always the same size with triplets. Record the block index for each triplet
 +   * TODO: actually this is only necessary for over-replicated block. Thus can
 +   * be further optimized to save memory usage.
 +   */
 +  private byte[] indices;
 +
 +  public BlockInfoStriped(Block blk, ErasureCodingPolicy ecPolicy) {
 +    super(blk, (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()));
 +    indices = new byte[ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()];
 +    initIndices();
 +    this.ecPolicy = ecPolicy;
 +  }
 +
-   BlockInfoStriped(BlockInfoStriped b) {
-     this(b, b.getErasureCodingPolicy());
-     this.setBlockCollection(b.getBlockCollection());
-   }
- 
 +  public short getTotalBlockNum() {
 +    return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits());
 +  }
 +
 +  public short getDataBlockNum() {
 +    return (short) ecPolicy.getNumDataUnits();
 +  }
 +
 +  public short getParityBlockNum() {
 +    return (short) ecPolicy.getNumParityUnits();
 +  }
 +
 +  /**
 +   * If the block is committed/completed and its length is less than a full
 +   * stripe, it returns the the number of actual data blocks.
 +   * Otherwise it returns the number of data units specified by erasure coding policy.
 +   */
 +  public short getRealDataBlockNum() {
 +    if (isComplete() || getBlockUCState() == BlockUCState.COMMITTED) {
 +      return (short) Math.min(getDataBlockNum(),
 +          (getNumBytes() - 1) / BLOCK_STRIPED_CELL_SIZE + 1);
 +    } else {
 +      return getDataBlockNum();
 +    }
 +  }
 +
 +  public short getRealTotalBlockNum() {
 +    return (short) (getRealDataBlockNum() + getParityBlockNum());
 +  }
 +
 +  public ErasureCodingPolicy getErasureCodingPolicy() {
 +    return ecPolicy;
 +  }
 +
 +  private void initIndices() {
 +    for (int i = 0; i < indices.length; i++) {
 +      indices[i] = -1;
 +    }
 +  }
 +
 +  private int findSlot() {
 +    int i = getTotalBlockNum();
 +    for (; i < getCapacity(); i++) {
 +      if (getStorageInfo(i) == null) {
 +        return i;
 +      }
 +    }
 +    // need to expand the triplet size
 +    ensureCapacity(i + 1, true);
 +    return i;
 +  }
 +
 +  @Override
 +  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
 +    int blockIndex = BlockIdManager.getBlockIndex(reportedBlock);
 +    int index = blockIndex;
 +    DatanodeStorageInfo old = getStorageInfo(index);
 +    if (old != null && !old.equals(storage)) { // over replicated
 +      // check if the storage has been stored
 +      int i = findStorageInfo(storage);
 +      if (i == -1) {
 +        index = findSlot();
 +      } else {
 +        return true;
 +      }
 +    }
 +    addStorage(storage, index, blockIndex);
 +    return true;
 +  }
 +
 +  private void addStorage(DatanodeStorageInfo storage, int index,
 +      int blockIndex) {
 +    setStorageInfo(index, storage);
 +    setNext(index, null);
 +    setPrevious(index, null);
 +    indices[index] = (byte) blockIndex;
 +  }
 +
 +  private int findStorageInfoFromEnd(DatanodeStorageInfo storage) {
 +    final int len = getCapacity();
 +    for(int idx = len - 1; idx >= 0; idx--) {
 +      DatanodeStorageInfo cur = getStorageInfo(idx);
 +      if (storage.equals(cur)) {
 +        return idx;
 +      }
 +    }
 +    return -1;
 +  }
 +
 +  int getStorageBlockIndex(DatanodeStorageInfo storage) {
 +    int i = this.findStorageInfo(storage);
 +    return i == -1 ? -1 : indices[i];
 +  }
 +
 +  /**
 +   * Identify the block stored in the given datanode storage. Note that
 +   * the returned block has the same block Id with the one seen/reported by the
 +   * DataNode.
 +   */
 +  Block getBlockOnStorage(DatanodeStorageInfo storage) {
 +    int index = getStorageBlockIndex(storage);
 +    if (index < 0) {
 +      return null;
 +    } else {
 +      Block block = new Block(this);
 +      block.setBlockId(this.getBlockId() + index);
 +      return block;
 +    }
 +  }
 +
 +  @Override
 +  boolean removeStorage(DatanodeStorageInfo storage) {
 +    int dnIndex = findStorageInfoFromEnd(storage);
 +    if (dnIndex < 0) { // the node is not found
 +      return false;
 +    }
 +    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
 +        "Block is still in the list and must be removed first.";
 +    // set the triplet to null
 +    setStorageInfo(dnIndex, null);
 +    setNext(dnIndex, null);
 +    setPrevious(dnIndex, null);
 +    indices[dnIndex] = -1;
 +    return true;
 +  }
 +
 +  private void ensureCapacity(int totalSize, boolean keepOld) {
 +    if (getCapacity() < totalSize) {
 +      Object[] old = triplets;
 +      byte[] oldIndices = indices;
 +      triplets = new Object[totalSize * 3];
 +      indices = new byte[totalSize];
 +      initIndices();
 +
 +      if (keepOld) {
 +        System.arraycopy(old, 0, triplets, 0, old.length);
 +        System.arraycopy(oldIndices, 0, indices, 0, oldIndices.length);
 +      }
 +    }
 +  }
 +
 +  @Override
 +  void replaceBlock(BlockInfo newBlock) {
 +    assert newBlock instanceof BlockInfoStriped;
 +    BlockInfoStriped newBlockGroup = (BlockInfoStriped) newBlock;
 +    final int size = getCapacity();
 +    newBlockGroup.ensureCapacity(size, false);
 +    for (int i = 0; i < size; i++) {
 +      final DatanodeStorageInfo storage = this.getStorageInfo(i);
 +      if (storage != null) {
 +        final int blockIndex = indices[i];
 +        final boolean removed = storage.removeBlock(this);
 +        assert removed : "currentBlock not found.";
 +
 +        newBlockGroup.addStorage(storage, i, blockIndex);
 +        storage.insertToList(newBlockGroup);
 +      }
 +    }
 +  }
 +
 +  public long spaceConsumed() {
 +    // In case striped blocks, total usage by this striped blocks should
 +    // be the total of data blocks and parity blocks because
 +    // `getNumBytes` is the total of actual data block size.
 +    return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(),
 +        ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits(),
 +        BLOCK_STRIPED_CELL_SIZE);
 +    }
 +
 +  @Override
 +  public final boolean isStriped() {
 +    return true;
 +  }
 +
 +  @Override
 +  public int numNodes() {
 +    assert this.triplets != null : "BlockInfo is not initialized";
 +    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
 +    int num = 0;
 +    for (int idx = getCapacity()-1; idx >= 0; idx--) {
 +      if (getStorageInfo(idx) != null) {
 +        num++;
 +      }
 +    }
 +    return num;
 +  }
 +
 +  @Override
 +  final boolean hasNoStorage() {
 +    final int len = getCapacity();
 +    for(int idx = 0; idx < len; idx++) {
 +      if (getStorageInfo(idx) != null) {
 +        return false;
 +      }
 +    }
 +    return true;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index ae08825,95933d2..6c6d758
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@@ -89,12 -84,8 +88,13 @@@ import org.apache.hadoop.hdfs.server.pr
  import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
  import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
  import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+ import org.apache.hadoop.hdfs.util.LightWeightHashSet;
  import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
 +
  import org.apache.hadoop.metrics2.util.MBeans;
  import org.apache.hadoop.net.Node;
  import org.apache.hadoop.security.UserGroupInformation;
@@@ -216,8 -202,8 +216,8 @@@ public class BlockManager implements Bl
     * Maps a StorageID to the set of blocks that are "extra" for this
     * DataNode. We'll eventually remove these extras.
     */
 -  public final Map<String, LightWeightLinkedSet<Block>> excessReplicateMap =
 +  public final Map<String, LightWeightLinkedSet<BlockInfo>> excessReplicateMap =
-     new TreeMap<>();
+     new HashMap<>();
  
    /**
     * Store set of Blocks that need to be replicated 1 or more times.
@@@ -689,26 -662,20 +689,25 @@@
     */
    private BlockInfo completeBlock(final BlockCollection bc,
        final int blkIndex, boolean force) throws IOException {
 -    if(blkIndex < 0)
 +    if (blkIndex < 0) {
        return null;
 +    }
      BlockInfo curBlock = bc.getBlocks()[blkIndex];
 -    if(curBlock.isComplete())
 +    if (curBlock.isComplete()) {
        return curBlock;
 +    }
  
      int numNodes = curBlock.numNodes();
 -    if (!force && numNodes < minReplication)
 +    if (!force && !hasMinStorage(curBlock, numNodes)) {
        throw new IOException("Cannot complete block: " +
            "block does not satisfy minimal replication requirement.");
 -    if(!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED)
 +    }
 +    if (!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
        throw new IOException(
            "Cannot complete block: block has not been COMMITTED by the client");
 -    BlockInfo completeBlock = curBlock.convertToCompleteBlock();
 +    }
 +
 +    final BlockInfo completeBlock = curBlock.convertToCompleteBlock();
- 
      // replace penultimate block in file
      bc.setBlock(blkIndex, completeBlock);
      
@@@ -1276,9 -1186,8 +1276,9 @@@
            " corrupt as it does not belong to any file", b);
        addToInvalidates(b.corrupted, node);
        return;
--    } 
 -    short expectedReplicas = b.corrupted.getReplication();
++    }
 +    short expectedReplicas =
-         getExpectedReplicaNum(b.stored.getBlockCollection(), b.stored);
++        getExpectedReplicaNum(b.stored);
  
      // Add replica to the data-node if it is not already there
      if (storageInfo != null) {
@@@ -1446,10 -1350,10 +1446,10 @@@
      namesystem.writeLock();
      try {
        synchronized (neededReplications) {
 -        for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
 -          for (BlockInfo block : blocksToReplicate.get(priority)) {
 +        for (int priority = 0; priority < blocksToRecover.size(); priority++) {
 +          for (BlockInfo block : blocksToRecover.get(priority)) {
              // block should belong to a file
-             bc = blocksMap.getBlockCollection(block);
+             bc = getBlockCollection(block);
              // abandoned block or block reopened for append
              if (bc == null
                  || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
@@@ -1458,20 -1362,17 +1458,20 @@@
                continue;
              }
  
-             requiredReplication = getExpectedReplicaNum(bc, block);
+             requiredReplication = getExpectedReplicaNum(block);
  
              // get a source data-node
 -            containingNodes = new ArrayList<DatanodeDescriptor>();
 -            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<DatanodeStorageInfo>();
 +            containingNodes = new ArrayList<>();
 +            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
              NumberReplicas numReplicas = new NumberReplicas();
 -            srcNode = chooseSourceDatanode(
 -                block, containingNodes, liveReplicaNodes, numReplicas,
 -                priority);
 -            if(srcNode == null) { // block can not be replicated from any node
 -              LOG.debug("Block " + block + " cannot be repl from any node");
 +            List<Short> liveBlockIndices = new ArrayList<>();
 +            final DatanodeDescriptor[] srcNodes = chooseSourceDatanodes(block,
 +                containingNodes, liveReplicaNodes, numReplicas,
 +                liveBlockIndices, priority);
 +            if(srcNodes == null || srcNodes.length == 0) {
 +              // block can not be replicated from any node
 +              LOG.debug("Block " + block + " cannot be recovered " +
 +                  "from any node");
                continue;
              }
  
@@@ -1588,32 -1474,7 +1588,32 @@@
            }
  
            // Add block to the to be replicated list
 -          rw.srcNode.addBlockToBeReplicated(block, targets);
 +          if (block.isStriped()) {
 +            assert rw instanceof ErasureCodingWork;
 +            assert rw.targets.length > 0;
-             String src = block.getBlockCollection().getName();
++            String src = getBlockCollection(block).getName();
 +            ErasureCodingZone ecZone = null;
 +            try {
 +              ecZone = namesystem.getErasureCodingZoneForPath(src);
 +            } catch (IOException e) {
 +              blockLog
 +                  .warn("Failed to get the EC zone for the file {} ", src);
 +            }
 +            if (ecZone == null) {
 +              blockLog.warn("No erasure coding policy found for the file {}. "
 +                  + "So cannot proceed for recovery", src);
 +              // TODO: we may have to revisit later for what we can do better to
 +              // handle this case.
 +              continue;
 +            }
 +            rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
 +                new ExtendedBlock(namesystem.getBlockPoolId(), block),
 +                rw.srcNodes, rw.targets,
 +                ((ErasureCodingWork) rw).liveBlockIndicies,
 +                ecZone.getErasureCodingPolicy());
 +          } else {
 +            rw.srcNodes[0].addBlockToBeReplicated(block, targets);
 +          }
            scheduledWork++;
            DatanodeStorageInfo.incrementBlocksScheduled(targets);
  
@@@ -2079,8 -1924,8 +2080,8 @@@
    private void removeZombieReplicas(BlockReportContext context,
        DatanodeStorageInfo zombie) {
      LOG.warn("processReport 0x{}: removing zombie storage {}, which no " +
--             "longer exists on the DataNode.",
--              Long.toHexString(context.getReportId()), zombie.getStorageID());
++            "longer exists on the DataNode.",
++        Long.toHexString(context.getReportId()), zombie.getStorageID());
      assert(namesystem.hasWriteLock());
      Iterator<BlockInfo> iter = zombie.getBlockIterator();
      int prevBlocks = zombie.numBlocks();
@@@ -2324,10 -2164,10 +2325,10 @@@
          // OpenFileBlocks only inside snapshots also will be added to safemode
          // threshold. So we need to update such blocks to safemode
          // refer HDFS-5283
-         if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) {
+         if (namesystem.isInSnapshot(storedBlock)) {
            int numOfReplicas = storedBlock.getUnderConstructionFeature()
                .getNumExpectedLocations();
 -          namesystem.incrementSafeBlockCount(numOfReplicas);
 +          namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock);
          }
          //and fall through to next clause
        }      
@@@ -2720,8 -2541,8 +2721,8 @@@
      // Now check for completion of blocks and safe block count
      int numCurrentReplica = countLiveNodes(storedBlock);
      if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
 -        && numCurrentReplica >= minReplication) {
 +        && hasMinStorage(storedBlock, numCurrentReplica)) {
-       completeBlock(storedBlock.getBlockCollection(), storedBlock, false);
+       completeBlock(getBlockCollection(storedBlock), storedBlock, false);
      } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
        // check whether safe replication is reached for the block
        // only complete blocks are counted towards that.
@@@ -2760,10 -2580,11 +2761,11 @@@
        // it will happen in next block report otherwise.
        return block;
      }
-     BlockCollection bc = storedBlock.getBlockCollection();
+     BlockCollection bc = getBlockCollection(storedBlock);
+     assert bc != null : "Block must belong to a file";
  
      // add block to the datanode
 -    AddBlockResult result = storageInfo.addBlock(storedBlock);
 +    AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock);
  
      int curReplicaDelta;
      if (result == AddBlockResult.ADDED) {
@@@ -3817,9 -3491,9 +3809,9 @@@
     */
    public void checkReplication(BlockCollection bc) {
      for (BlockInfo block : bc.getBlocks()) {
-       short expected = getExpectedReplicaNum(bc, block);
 -      final short expected = block.getReplication();
++      short expected = getExpectedReplicaNum(block);
        final NumberReplicas n = countNodes(block);
-       if (isNeededReplication(block, expected, n.liveReplicas())) { 
+       if (isNeededReplication(block, n.liveReplicas())) {
          neededReplications.add(block, n.liveReplicas(),
              n.decommissionedAndDecommissioning(), expected);
        } else if (n.liveReplicas() > expected) {
@@@ -3978,18 -3605,15 +3968,17 @@@
     * A block needs replication if the number of replicas is less than expected
     * or if it does not have enough racks.
     */
-   boolean isNeededReplication(BlockInfo storedBlock, int expected, int current) {
+   boolean isNeededReplication(BlockInfo storedBlock, int current) {
 -    int expected = storedBlock.getReplication();
 -    return current < expected || !blockHasEnoughRacks(storedBlock);
++    int expected = getExpectedReplicaNum(storedBlock);
 +    return current < expected || !blockHasEnoughRacks(storedBlock, expected);
    }
-   
-   public short getExpectedReplicaNum(BlockCollection bc, BlockInfo block) {
-     if (block.isStriped()) {
-       return ((BlockInfoStriped) block).getRealTotalBlockNum();
-     } else {
-       return bc.getPreferredBlockReplication();
-     }
+ 
+   public short getExpectedReplicaNum(BlockInfo block) {
 -    return block.getReplication();
++    return block.isStriped() ?
++        ((BlockInfoStriped) block).getRealTotalBlockNum() :
++        block.getReplication();
    }
 -  
 +
    public long getMissingBlocksCount() {
      // not locking
      return this.neededReplications.getCorruptBlockSize();
@@@ -4005,22 -3629,13 +3994,22 @@@
      return blocksMap.addBlockCollection(block, bc);
    }
  
 -  public BlockCollection getBlockCollection(BlockInfo b) {
 -    return namesystem.getBlockCollection(b.getBlockCollectionId());
 +  /**
 +   * Do some check when adding a block to blocksmap.
 +   * For HDFS-7994 to check whether then block is a NonEcBlockUsingStripedID.
 +   *
 +   */
 +  public BlockInfo addBlockCollectionWithCheck(
 +      BlockInfo block, BlockCollection bc) {
 +    if (!hasNonEcBlockUsingStripedID && !block.isStriped() &&
 +        BlockIdManager.isStripedBlockID(block.getBlockId())) {
 +      hasNonEcBlockUsingStripedID = true;
 +    }
 +    return addBlockCollection(block, bc);
    }
  
-   public BlockCollection getBlockCollection(Block b) {
-     return blocksMap.getBlockCollection(b);
 -  /** @return an iterator of the datanodes. */
 -  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
 -    return blocksMap.getStorages(block);
++  public BlockCollection getBlockCollection(BlockInfo b) {
++    return namesystem.getBlockCollection(b.getBlockCollectionId());
    }
  
    public int numCorruptReplicas(Block block) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
index 58b455e,88cf06d..0e92779
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
@@@ -55,11 -58,11 +58,11 @@@ public class BlockUnderConstructionFeat
    private Block truncateBlock;
  
    public BlockUnderConstructionFeature(Block blk,
 -      BlockUCState state, DatanodeStorageInfo[] targets) {
 +      BlockUCState state, DatanodeStorageInfo[] targets, boolean isStriped) {
      assert getBlockUCState() != COMPLETE :
-       "BlockUnderConstructionFeature cannot be in COMPLETE state";
+         "BlockUnderConstructionFeature cannot be in COMPLETE state";
      this.blockUCState = state;
 -    setExpectedLocations(blk, targets);
 +    setExpectedLocations(blk, targets, isStriped);
    }
  
    /** Set expected locations */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index 5bfae42,33c68f3..51d62c1
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@@ -129,18 -118,14 +125,18 @@@ class BlocksMap 
      if (blockInfo == null)
        return;
  
-     blockInfo.setBlockCollection(null);
-     final int size = blockInfo instanceof BlockInfoContiguous ?
-         blockInfo.numNodes() : blockInfo.getCapacity();
+     blockInfo.setBlockCollectionId(INodeId.INVALID_INODE_ID);
 -    for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
++    final int size = blockInfo.isStriped() ?
++        blockInfo.getCapacity() : blockInfo.numNodes();
 +    for(int idx = size - 1; idx >= 0; idx--) {
        DatanodeDescriptor dn = blockInfo.getDatanode(idx);
 -      dn.removeBlock(blockInfo); // remove from the list and wipe the location
 +      if (dn != null) {
 +        dn.removeBlock(blockInfo); // remove from the list and wipe the location
 +      }
      }
    }
 -  
 -  /** Returns the block object it it exists in the map. */
 +
 +  /** Returns the block object if it exists in the map. */
    BlockInfo getStoredBlock(Block b) {
      return blocks.get(b);
    }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index a4d5442,7e3c59b..29e541c
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@@ -38,11 -39,10 +38,12 @@@ import org.apache.hadoop.fs.StorageType
  import org.apache.hadoop.hdfs.protocol.Block;
  import org.apache.hadoop.hdfs.protocol.DatanodeID;
  import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 +import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
  import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
  import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
  import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
  import org.apache.hadoop.hdfs.server.protocol.StorageReport;
  import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
  import org.apache.hadoop.hdfs.util.EnumCounters;
@@@ -223,12 -222,8 +224,11 @@@ public class DatanodeDescriptor extend
    /** A queue of blocks to be replicated by this datanode */
    private final BlockQueue<BlockTargetPair> replicateBlocks =
        new BlockQueue<>();
 +  /** A queue of blocks to be erasure coded by this datanode */
 +  private final BlockQueue<BlockECRecoveryInfo> erasurecodeBlocks =
 +      new BlockQueue<>();
    /** A queue of blocks to be recovered by this datanode */
-   private final BlockQueue<BlockInfo> recoverBlocks =
-       new BlockQueue<>();
+   private final BlockQueue<BlockInfo> recoverBlocks = new BlockQueue<>();
    /** A set of blocks to be invalidated by this datanode */
    private final LightWeightHashSet<Block> invalidateBlocks =
        new LightWeightHashSet<>();
@@@ -696,24 -662,27 +696,34 @@@
      }
    }
  
 +  @VisibleForTesting
 +  public boolean containsInvalidateBlock(Block block) {
 +    synchronized (invalidateBlocks) {
 +      return invalidateBlocks.contains(block);
 +    }
 +  }
 +
    /**
-    * @return Approximate number of blocks currently scheduled to be written 
+    * Return the sum of remaining spaces of the specified type. If the remaining
+    * space of a storage is less than minSize, it won't be counted toward the
+    * sum.
+    *
+    * @param t The storage type. If null, the type is ignored.
+    * @param minSize The minimum free space required.
+    * @return the sum of remaining spaces that are bigger than minSize.
     */
-   public long getRemaining(StorageType t) {
+   public long getRemaining(StorageType t, long minSize) {
      long remaining = 0;
-     for(DatanodeStorageInfo s : getStorageInfos()) {
-       if (s.getStorageType() == t) {
-         remaining += s.getRemaining();
+     for (DatanodeStorageInfo s : getStorageInfos()) {
+       if (s.getState() == State.NORMAL &&
+           (t == null || s.getStorageType() == t)) {
+         long r = s.getRemaining();
+         if (r >= minSize) {
+           remaining += r;
+         }
        }
      }
-     return remaining;    
+     return remaining;
    }
  
    /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 5e3cac2,1a20ab0..a80bfd6
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@@ -233,16 -234,16 +234,16 @@@ public class DecommissionManager 
    }
  
    /**
 -   * Checks whether a block is sufficiently replicated for decommissioning.
 -   * Full-strength replication is not always necessary, hence "sufficient".
 +   * Checks whether a block is sufficiently replicated/stored for
 +   * decommissioning. For replicated blocks or striped blocks, full-strength
 +   * replication or storage is not always necessary, hence "sufficient".
     * @return true if sufficient, else false.
     */
 -  private boolean isSufficientlyReplicated(BlockInfo block,
 -      BlockCollection bc,
 +  private boolean isSufficient(BlockInfo block, BlockCollection bc,
        NumberReplicas numberReplicas) {
-     final int numExpected = blockManager.getExpectedReplicaNum(bc, block);
 -    final int numExpected = block.getReplication();
++    final int numExpected = blockManager.getExpectedReplicaNum(block);
      final int numLive = numberReplicas.liveReplicas();
-     if (!blockManager.isNeededReplication(block, numExpected, numLive)) {
+     if (!blockManager.isNeededReplication(block, numLive)) {
        // Block doesn't need replication. Skip.
        LOG.trace("Block {} does not need replication.", block);
        return true;
@@@ -274,11 -274,12 +275,12 @@@
      return false;
    }
  
-   private void logBlockReplicationInfo(BlockInfo block, BlockCollection bc,
 -  private static void logBlockReplicationInfo(BlockInfo block,
++  private void logBlockReplicationInfo(BlockInfo block,
+       BlockCollection bc,
        DatanodeDescriptor srcNode, NumberReplicas num,
        Iterable<DatanodeStorageInfo> storages) {
      int curReplicas = num.liveReplicas();
-     int curExpectedReplicas = blockManager.getExpectedReplicaNum(bc, block);
 -    int curExpectedReplicas = block.getReplication();
++    int curExpectedReplicas = blockManager.getExpectedReplicaNum(block);
      StringBuilder nodeList = new StringBuilder();
      for (DatanodeStorageInfo storage : storages) {
        final DatanodeDescriptor node = storage.getDatanodeDescriptor();
@@@ -530,8 -536,10 +533,9 @@@
            continue;
          }
  
+         BlockCollection bc = namesystem.getBlockCollection(bcId);
          final NumberReplicas num = blockManager.countNodes(block);
          final int liveReplicas = num.liveReplicas();
 -        final int curReplicas = liveReplicas;
  
          // Schedule under-replicated blocks for replication if not already
          // pending
@@@ -542,9 -549,9 +545,9 @@@
                namesystem.isPopulatingReplQueues()) {
              // Process these blocks only when active NN is out of safe mode.
              blockManager.neededReplications.add(block,
 -                curReplicas,
 +                liveReplicas,
                  num.decommissionedAndDecommissioning(),
-                 blockManager.getExpectedReplicaNum(bc, block));
 -                block.getReplication());
++                blockManager.getExpectedReplicaNum(block));
            }
          }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
index 479ee4c,0000000..7a52273
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
@@@ -1,85 -1,0 +1,86 @@@
 +/**
 + * 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.hdfs.server.blockmanagement;
 +
 +import org.apache.hadoop.classification.InterfaceAudience;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.util.SequentialNumber;
 +
 +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BLOCK_GROUP_INDEX_MASK;
 +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_BLOCKS_IN_GROUP;
 +
 +/**
 + * Generate the next valid block group ID by incrementing the maximum block
 + * group ID allocated so far, with the first 2^10 block group IDs reserved.
 + * HDFS-EC introduces a hierarchical protocol to name blocks and groups:
 + * Contiguous: {reserved block IDs | flag | block ID}
 + * Striped: {reserved block IDs | flag | block group ID | index in group}
 + *
 + * Following n bits of reserved block IDs, The (n+1)th bit in an ID
 + * distinguishes contiguous (0) and striped (1) blocks. For a striped block,
 + * bits (n+2) to (64-m) represent the ID of its block group, while the last m
 + * bits represent its index of the group. The value m is determined by the
 + * maximum number of blocks in a group (MAX_BLOCKS_IN_GROUP).
 + *
 + * Note that the {@link #nextValue()} methods requires external lock to
 + * guarantee IDs have no conflicts.
 + */
 +@InterfaceAudience.Private
 +public class SequentialBlockGroupIdGenerator extends SequentialNumber {
 +
 +  private final BlockManager blockManager;
 +
 +  SequentialBlockGroupIdGenerator(BlockManager blockManagerRef) {
 +    super(Long.MIN_VALUE);
 +    this.blockManager = blockManagerRef;
 +  }
 +
 +  @Override // NumberGenerator
 +  public long nextValue() {
 +    skipTo((getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK) + MAX_BLOCKS_IN_GROUP);
 +    // Make sure there's no conflict with existing random block IDs
 +    final Block b = new Block(getCurrentValue());
 +    while (hasValidBlockInRange(b)) {
 +      skipTo(getCurrentValue() + MAX_BLOCKS_IN_GROUP);
 +      b.setBlockId(getCurrentValue());
 +    }
 +    if (b.getBlockId() >= 0) {
 +      throw new IllegalStateException("All negative block group IDs are used, "
 +          + "growing into positive IDs, "
 +          + "which might conflict with non-erasure coded blocks.");
 +    }
 +    return getCurrentValue();
 +  }
 +
 +  /**
 +   * @param b A block object whose id is set to the starting point for check
 +   * @return true if any ID in the range
-    *      {id, id+HdfsConstants.MAX_BLOCKS_IN_GROUP} is pointed-to by a file
++   *      {id, id+HdfsConstants.MAX_BLOCKS_IN_GROUP} is pointed-to by a stored
++   *      block.
 +   */
 +  private boolean hasValidBlockInRange(Block b) {
 +    final long id = b.getBlockId();
 +    for (int i = 0; i < MAX_BLOCKS_IN_GROUP; i++) {
 +      b.setBlockId(id + i);
-       if (blockManager.getBlockCollection(b) != null) {
++      if (blockManager.getStoredBlock(b) != null) {
 +        return true;
 +      }
 +    }
 +    return false;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java
index 6074784,f053b7b..631b435
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java
@@@ -19,6 -19,8 +19,7 @@@ package org.apache.hadoop.hdfs.server.b
  
  import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.hdfs.protocol.Block;
 -import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+ import org.apache.hadoop.hdfs.server.namenode.INodeId;
  import org.apache.hadoop.util.SequentialNumber;
  
  /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 1b695e3,5bc50b0..82a0f62
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@@ -2158,11 -2103,14 +2128,11 @@@ public class DataNode extends Reconfigu
          //
          // Header info
          //
 -        Token<BlockTokenIdentifier> accessToken = BlockTokenSecretManager.DUMMY_TOKEN;
 -        if (isBlockTokenEnabled) {
 -          accessToken = blockPoolTokenSecretManager.generateToken(b, 
 -              EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
 -        }
 +        Token<BlockTokenIdentifier> accessToken = getBlockAccessToken(b, 
 +            EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
  
          long writeTimeout = dnConf.socketWriteTimeout + 
-                             HdfsServerConstants.WRITE_TIMEOUT_EXTENSION * (targets.length-1);
+                             HdfsConstants.WRITE_TIMEOUT_EXTENSION * (targets.length-1);
          OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
          InputStream unbufIn = NetUtils.getInputStream(sock);
          DataEncryptionKeyFactory keyFactory =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
index 22d821f,0000000..a0ac033
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
@@@ -1,163 -1,0 +1,163 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hdfs.server.namenode;
 +
 +import com.google.common.base.Preconditions;
 +import com.google.common.collect.Lists;
 +
 +import org.apache.hadoop.fs.XAttr;
 +import org.apache.hadoop.fs.XAttrSetFlag;
 +import org.apache.hadoop.hdfs.XAttrHelper;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 +import org.apache.hadoop.io.IOUtils;
 +import org.apache.hadoop.io.WritableUtils;
 +
 +import java.io.ByteArrayInputStream;
 +import java.io.ByteArrayOutputStream;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.EnumSet;
 +import java.util.List;
 +
 +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_ZONE;
 +
 +/**
 + * Manages the list of erasure coding zones in the filesystem.
 + * <p/>
 + * The ErasureCodingZoneManager has its own lock, but relies on the FSDirectory
 + * lock being held for many operations. The FSDirectory lock should not be
 + * taken if the manager lock is already held.
 + * TODO: consolidate zone logic w/ encrypt. zones {@link EncryptionZoneManager}
 + */
 +public class ErasureCodingZoneManager {
 +  private final FSDirectory dir;
 +
 +  /**
 +   * Construct a new ErasureCodingZoneManager.
 +   *
 +   * @param dir Enclosing FSDirectory
 +   */
 +  public ErasureCodingZoneManager(FSDirectory dir) {
 +    this.dir = dir;
 +  }
 +
 +  ErasureCodingPolicy getErasureCodingPolicy(INodesInPath iip) throws IOException {
 +    ErasureCodingZone ecZone = getErasureCodingZone(iip);
 +    return ecZone == null ? null : ecZone.getErasureCodingPolicy();
 +  }
 +
 +  ErasureCodingZone getErasureCodingZone(INodesInPath iip) throws IOException {
 +    assert dir.hasReadLock();
 +    Preconditions.checkNotNull(iip, "INodes cannot be null");
 +    List<INode> inodes = iip.getReadOnlyINodes();
 +    for (int i = inodes.size() - 1; i >= 0; i--) {
 +      final INode inode = inodes.get(i);
 +      if (inode == null) {
 +        continue;
 +      }
 +      // We don't allow symlinks in an EC zone, or pointing to a file/dir in
 +      // an EC. Therefore if a symlink is encountered, the dir shouldn't have
 +      // EC
 +      // TODO: properly support symlinks in EC zones
 +      if (inode.isSymlink()) {
 +        return null;
 +      }
 +      final List<XAttr> xAttrs = inode.getXAttrFeature() == null ?
 +          new ArrayList<XAttr>(0)
 +          : inode.getXAttrFeature().getXAttrs();
 +      for (XAttr xAttr : xAttrs) {
-         if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
++        if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixedName(xAttr))) {
 +          ByteArrayInputStream bIn=new ByteArrayInputStream(xAttr.getValue());
 +          DataInputStream dIn=new DataInputStream(bIn);
 +          String ecPolicyName = WritableUtils.readString(dIn);
 +          ErasureCodingPolicy ecPolicy = dir.getFSNamesystem()
 +              .getErasureCodingPolicyManager().getPolicy(ecPolicyName);
 +          return new ErasureCodingZone(dir.getInode(inode.getId())
 +              .getFullPathName(), ecPolicy);
 +        }
 +      }
 +    }
 +    return null;
 +  }
 +
 +  List<XAttr> createErasureCodingZone(final INodesInPath srcIIP,
 +      ErasureCodingPolicy ecPolicy) throws IOException {
 +    assert dir.hasWriteLock();
 +    Preconditions.checkNotNull(srcIIP, "INodes cannot be null");
 +    String src = srcIIP.getPath();
 +    if (dir.isNonEmptyDirectory(srcIIP)) {
 +      throw new IOException(
 +          "Attempt to create an erasure coding zone for a " +
 +              "non-empty directory " + src);
 +    }
 +    if (srcIIP.getLastINode() != null &&
 +        !srcIIP.getLastINode().isDirectory()) {
 +      throw new IOException("Attempt to create an erasure coding zone " +
 +          "for a file " + src);
 +    }
 +    if (getErasureCodingPolicy(srcIIP) != null) {
 +      throw new IOException("Directory " + src + " is already in an " +
 +          "erasure coding zone.");
 +    }
 +
 +    // System default erasure coding policy will be used since no specified.
 +    if (ecPolicy == null) {
 +      ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 +    }
 +
 +    final XAttr ecXAttr;
 +    DataOutputStream dOut = null;
 +    try {
 +      ByteArrayOutputStream bOut = new ByteArrayOutputStream();
 +      dOut = new DataOutputStream(bOut);
 +      WritableUtils.writeString(dOut, ecPolicy.getName());
 +      ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE,
 +          bOut.toByteArray());
 +    } finally {
 +      IOUtils.closeStream(dOut);
 +    }
 +    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
 +    xattrs.add(ecXAttr);
 +    FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs,
 +        EnumSet.of(XAttrSetFlag.CREATE));
 +    return xattrs;
 +  }
 +
 +  void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src)
 +      throws IOException {
 +    assert dir.hasReadLock();
 +    final ErasureCodingZone srcZone = getErasureCodingZone(srcIIP);
 +    final ErasureCodingZone dstZone = getErasureCodingZone(dstIIP);
 +    if (srcZone != null && srcZone.getDir().equals(src) && dstZone == null) {
 +      return;
 +    }
 +    final ErasureCodingPolicy srcECPolicy =
 +        srcZone != null ? srcZone.getErasureCodingPolicy() : null;
 +    final ErasureCodingPolicy dstECPolicy =
 +        dstZone != null ? dstZone.getErasureCodingPolicy() : null;
 +    if (srcECPolicy != null && !srcECPolicy.equals(dstECPolicy) ||
 +        dstECPolicy != null && !dstECPolicy.equals(srcECPolicy)) {
 +      throw new IOException(
 +          src + " can't be moved because the source and destination have " +
 +              "different erasure coding policies.");
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 9b08092,df0bc20..4bed13e
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@@ -122,7 -122,7 +122,7 @@@ public class FSDirAttrOp 
                                              " does not exist.");
        }
        boolean changed = unprotectedSetTimes(fsd, inode, mtime, atime, true,
--                                            iip.getLatestSnapshotId());
++          iip.getLatestSnapshotId());
        if (changed) {
          fsd.getEditLog().logTimes(src, mtime, atime);
        }
@@@ -399,30 -397,26 +397,30 @@@
    }
  
    static BlockInfo[] unprotectedSetReplication(
-       FSDirectory fsd, String src, short replication, short[] blockRepls)
+       FSDirectory fsd, String src, short replication)
        throws QuotaExceededException, UnresolvedLinkException,
 -             SnapshotAccessControlException {
 +      SnapshotAccessControlException, UnsupportedActionException {
      assert fsd.hasWriteLock();
  
+     final BlockManager bm = fsd.getBlockManager();
      final INodesInPath iip = fsd.getINodesInPath4Write(src, true);
      final INode inode = iip.getLastINode();
      if (inode == null || !inode.isFile()) {
        return null;
      }
      INodeFile file = inode.asFile();
 +    if (file.isStriped()) {
 +      throw new UnsupportedActionException(
 +          "Cannot set replication to a file with striped blocks");
 +    }
  
-     final short oldBR = file.getPreferredBlockReplication();
+     // Make sure the directory has sufficient quotas
+     short oldBR = file.getPreferredBlockReplication();
  
-     // before setFileReplication, check for increasing block replication.
-     // if replication > oldBR, then newBR == replication.
-     // if replication < oldBR, we don't know newBR yet.
-     if (replication > oldBR) {
-       long dsDelta = file.storagespaceConsumed(null).getStorageSpace() / oldBR;
-       fsd.updateCount(iip, 0L, dsDelta, oldBR, replication, true);
+     // Ensure the quota does not exceed
+     if (oldBR < replication) {
+       long size = file.computeFileSize(true, true);
+       fsd.updateCount(iip, 0L, size, oldBR, replication, true);
      }
  
      file.setFileReplication(replication, iip.getLatestSnapshotId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 68aef76,e9d0806..e480959
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@@ -46,7 -44,6 +46,8 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
++
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
  import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@@ -529,32 -516,15 +530,32 @@@ class FSDirWriteFileOp 
        final INodeFile fileINode = inodesInPath.getLastINode().asFile();
        Preconditions.checkState(fileINode.isUnderConstruction());
  
 -      // check quota limits and updated space consumed
 -      fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
 -          fileINode.getFileReplication(), true);
 -
        // associate new last block for the file
 -      BlockInfo blockInfo = new BlockInfoContiguous(block,
 -          fileINode.getFileReplication());
 -      blockInfo.convertToBlockUnderConstruction(
 -          HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
 +      final BlockInfo blockInfo;
 +      if (isStriped) {
 +        ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
 +            fsd.getFSNamesystem(), inodesInPath);
 +        ErasureCodingPolicy ecPolicy = ecZone.getErasureCodingPolicy();
 +        short numDataUnits = (short) ecPolicy.getNumDataUnits();
 +        short numParityUnits = (short) ecPolicy.getNumParityUnits();
 +        short numLocations = (short) (numDataUnits + numParityUnits);
 +
 +        // check quota limits and updated space consumed
 +        fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
 +            numLocations, true);
 +        blockInfo = new BlockInfoStriped(block, ecPolicy);
 +        blockInfo.convertToBlockUnderConstruction(
 +            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
 +      } else {
 +        // check quota limits and updated space consumed
 +        fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
-             fileINode.getPreferredBlockReplication(), true);
++            fileINode.getFileReplication(), true);
 +
 +        short numLocations = fileINode.getFileReplication();
 +        blockInfo = new BlockInfoContiguous(block, numLocations);
 +        blockInfo.convertToBlockUnderConstruction(
 +            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
 +      }
        fsd.getBlockManager().addBlockCollection(blockInfo, fileINode);
        fileINode.addBlock(blockInfo);
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index a61161f,f22762c..7203316
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@@ -1077,29 -1048,19 +1077,29 @@@ public class FSEditLogLoader 
            // TODO: shouldn't this only be true for the last block?
            // what about an old-version fsync() where fsync isn't called
            // until several blocks in?
 -          newBI = new BlockInfoContiguous(newBlock,
 -              file.getPreferredBlockReplication());
 +          if (isStriped) {
 +            newBI = new BlockInfoStriped(newBlock,
 +                ecZone.getErasureCodingPolicy());
 +          } else {
 +            newBI = new BlockInfoContiguous(newBlock,
 +                file.getPreferredBlockReplication());
 +          }
-           newBI.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
-               null);
+           newBI.convertToBlockUnderConstruction(
 -              HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
++              BlockUCState.UNDER_CONSTRUCTION, null);
          } else {
            // OP_CLOSE should add finalized blocks. This code path
            // is only executed when loading edits written by prior
            // versions of Hadoop. Current versions always log
            // OP_ADD operations as each block is allocated.
 -          newBI = new BlockInfoContiguous(newBlock,
 -              file.getFileReplication());
 +          if (isStriped) {
 +            newBI = new BlockInfoStriped(newBlock,
 +                ErasureCodingPolicyManager.getSystemDefaultPolicy());
 +          } else {
 +            newBI = new BlockInfoContiguous(newBlock,
-                 file.getPreferredBlockReplication());
++                file.getFileReplication());
 +          }
          }
 -        fsNamesys.getBlockManager().addBlockCollection(newBI, file);
 +        fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);
          file.addBlock(newBI);
          fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
        }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index ffaf86b,ac88919..a115138
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@@ -43,9 -42,9 +43,10 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
  import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+ import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
  import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 0d9d427,f4952f7..5f39446
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@@ -7423,11 -7287,30 +7435,35 @@@ public class FSNamesystem implements Na
        logger.addAppender(asyncAppender);        
      }
    }
 -
+   /**
+    * Return total number of Sync Operations on FSEditLog.
+    */
+   @Override
+   @Metric({"TotalSyncCount",
+               "Total number of sync operations performed on edit logs"})
+   public long getTotalSyncCount() {
+     return fsImage.editLog.getTotalSyncCount();
+   }
+ 
+   /**
+    * Return total time spent doing sync operations on FSEditLog.
+    */
+   @Override
+   @Metric({"TotalSyncTimes",
+               "Total time spend in sync operation on various edit logs"})
+   public String getTotalSyncTimes() {
+     JournalSet journalSet = fsImage.editLog.getJournalSet();
+     if (journalSet != null) {
+       return journalSet.getSyncTimes();
+     } else {
+       return "";
+     }
+   }
 +
 +  @Override
 +  public ErasureCodingZone getErasureCodingZoneForPath(String src)
 +      throws IOException {
 +    return FSDirErasureCodingOp.getErasureCodingZone(this, src);
 +  }
  }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 4fa457d,d546905..ae9b0d2
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@@ -38,7 -37,7 +38,8 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
  import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 9d43c15,7ebe859..e3363a4
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@@ -258,8 -254,7 +258,8 @@@ public class NamenodeFsck implements Da
        NumberReplicas numberReplicas= bm.countNodes(blockInfo);
        out.println("Block Id: " + blockId);
        out.println("Block belongs to: "+iNode.getFullPathName());
 -      out.println("No. of Expected Replica: " + blockInfo.getReplication());
 +      out.println("No. of Expected Replica: " +
-           bm.getExpectedReplicaNum(bc, blockInfo));
++          bm.getExpectedReplicaNum(blockInfo));
        out.println("No. of live Replica: " + numberReplicas.liveReplicas());
        out.println("No. of excess Replica: " + numberReplicas.excessReplicas());
        out.println("No. of stale Replica: " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index bae033b,4a208d8..923a335
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@@ -17,11 -17,9 +17,12 @@@
   */
  package org.apache.hadoop.hdfs.server.namenode;
  
 +import java.io.IOException;
 +
  import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
  import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
  import org.apache.hadoop.hdfs.util.RwLock;
@@@ -48,17 -48,7 +51,17 @@@ public interface Namesystem extends RwL
  
    void checkOperation(OperationCategory read) throws StandbyException;
  
-   boolean isInSnapshot(BlockCollection bc);
- 
 +  /**
 +   * Gets the ECZone for path
 +   * @param src
 +   *          - path
 +   * @return {@link ErasureCodingZone}
 +   * @throws IOException
 +   */
 +  ErasureCodingZone getErasureCodingZoneForPath(String src)
 +      throws IOException;
 +
+   boolean isInSnapshot(BlockInfo blockUC);
+ 
    CacheManager getCacheManager();
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index 0162f85,91ebaaf..450d981
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@@ -245,11 -245,10 +247,11 @@@ public class FSImageFormatPBSnapshot 
          BlockInfo[] blocks = new BlockInfo[bpl.size()];
          for(int j = 0, e = bpl.size(); j < e; ++j) {
            Block blk = PBHelper.convert(bpl.get(j));
-           BlockInfo storedBlock =  fsn.getBlockManager().getStoredBlock(blk);
+           BlockInfo storedBlock = bm.getStoredBlock(blk);
            if(storedBlock == null) {
 -            storedBlock = bm.addBlockCollection(
 -                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
 +            storedBlock = (BlockInfoContiguous) fsn.getBlockManager()
 +                .addBlockCollectionWithCheck(new BlockInfoContiguous(blk,
 +                    copy.getFileReplication()), file);
            }
            blocks[j] = storedBlock;
          }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index fcf2bc1,96776e4..0db56dd
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@@ -2413,24 -2403,13 +2403,34 @@@
  </property>
  
  <property>
 +  <name>dfs.datanode.stripedread.threshold.millis</name>
 +  <value>5000</value>
 +  <description>datanode striped read threshold in millisecond.
 +  </description>
 +</property>
 +
 +<property>
 +  <name>dfs.datanode.stripedread.threads</name>
 +  <value>20</value>
 +  <description>datanode striped read thread pool size.
 +  </description>
 +</property>
 +
 +<property>
 +  <name>dfs.datanode.stripedread.buffer.size</name>
 +  <value>262144</value>
 +  <description>datanode striped read buffer size.
 +  </description>
 +</property>
 +
++<property>
+   <name>dfs.namenode.quota.init-threads</name>
+   <value>4</value>
+   <description>
+     The number of concurrent threads to be used in quota initialization. The
+     speed of quota initialization also affects the namenode fail-over latency.
+     If the size of name space is big, try increasing this.
+   </description>
+ </property>
+ 
  </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 59daba4,24e0965..4bb5c64
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@@ -119,8 -119,8 +119,9 @@@ import org.apache.hadoop.security.UserG
  import org.apache.hadoop.security.authorize.ProxyUsers;
  import org.apache.hadoop.test.GenericTestUtils;
  import org.apache.hadoop.util.ExitUtil;
+ import org.apache.hadoop.util.ShutdownHookManager;
  import org.apache.hadoop.util.StringUtils;
 +import org.apache.hadoop.util.ShutdownHookManager;
  import org.apache.hadoop.util.ToolRunner;
  
  import com.google.common.base.Joiner;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index ad8f204,c1ed758..eb24fb0
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@@ -955,9 -939,9 +958,9 @@@ public class TestBalancer 
    void testBalancer1Internal(Configuration conf) throws Exception {
      initConf(conf);
      testUnevenDistribution(conf,
--        new long[] {50*CAPACITY/100, 10*CAPACITY/100},
++        new long[]{50 * CAPACITY / 100, 10 * CAPACITY / 100},
          new long[]{CAPACITY, CAPACITY},
--        new String[] {RACK0, RACK1});
++        new String[]{RACK0, RACK1});
    }
    
    @Test(expected=HadoopIllegalArgumentException.class)
@@@ -971,7 -955,7 +974,7 @@@
    public void testBalancerWithNonZeroThreadsForMove() throws Exception {
      Configuration conf = new HdfsConfiguration();
      conf.setInt(DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, 8);
--    testBalancer1Internal (conf);
++    testBalancer1Internal(conf);
    }
    
    @Test(timeout=100000)
@@@ -981,8 -965,8 +984,8 @@@
    
    void testBalancer2Internal(Configuration conf) throws Exception {
      initConf(conf);
--    testBalancerDefaultConstructor(conf, new long[] { CAPACITY, CAPACITY },
--        new String[] { RACK0, RACK1 }, CAPACITY, RACK2);
++    testBalancerDefaultConstructor(conf, new long[]{CAPACITY, CAPACITY},
++        new String[]{RACK0, RACK1}, CAPACITY, RACK2);
    }
  
    private void testBalancerDefaultConstructor(Configuration conf,
@@@ -1555,75 -1540,116 +1559,183 @@@
      }
    }
  
+   /** Balancer should not move blocks with size < minBlockSize. */
+   @Test(timeout=60000)
+   public void testMinBlockSizeAndSourceNodes() throws Exception {
+     final Configuration conf = new HdfsConfiguration();
+     initConf(conf);
+  
+     final short replication = 3;
+     final long[] lengths = {10, 10, 10, 10}; 
+     final long[] capacities = new long[replication];
+     final long totalUsed = capacities.length * sum(lengths);
+     Arrays.fill(capacities, 1000);
+ 
+     cluster = new MiniDFSCluster.Builder(conf)
+         .numDataNodes(capacities.length)
+         .simulatedCapacities(capacities)
+         .build();
+     final DistributedFileSystem dfs = cluster.getFileSystem();
+ 
+     try {
+       cluster.waitActive();
+       client = NameNodeProxies.createProxy(conf, dfs.getUri(),
+           ClientProtocol.class).getProxy();
+       
+       // fill up the cluster to be 80% full
+       for(int i = 0; i < lengths.length; i++) {
+         final long size = lengths[i];
+         final Path p = new Path("/file" + i + "_size" + size);
+         try(final OutputStream out = dfs.create(p)) {
+           for(int j = 0; j < size; j++) {
+             out.write(j);
+           }
+         }
+       }
+       
+       // start up an empty node with the same capacity
+       cluster.startDataNodes(conf, capacities.length, true, null, null, capacities);
+       LOG.info("capacities    = " + Arrays.toString(capacities));
+       LOG.info("totalUsedSpace= " + totalUsed);
+       LOG.info("lengths       = " + Arrays.toString(lengths) + ", #=" + lengths.length);
+       waitForHeartBeat(totalUsed, 2*capacities[0]*capacities.length, client, cluster);
+       
+       final Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+ 
+       { // run Balancer with min-block-size=50
+         final Parameters p = new Parameters(
+             BalancingPolicy.Node.INSTANCE, 1,
+             NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
+             Collections.<String> emptySet(), Collections.<String> emptySet(),
+             Collections.<String> emptySet(), false);
+ 
+         conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 50);
+         final int r = Balancer.run(namenodes, p, conf);
+         assertEquals(ExitStatus.NO_MOVE_PROGRESS.getExitCode(), r);
+       }
+       
+       conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
+ 
+       { // run Balancer with empty nodes as source nodes
+         final Set<String> sourceNodes = new HashSet<>();
+         final List<DataNode> datanodes = cluster.getDataNodes();
+         for(int i = capacities.length; i < datanodes.size(); i++) {
+           sourceNodes.add(datanodes.get(i).getDisplayName());
+         }
+         final Parameters p = new Parameters(
+           BalancingPolicy.Node.INSTANCE, 1,
+           NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
+           Collections.<String> emptySet(), Collections.<String> emptySet(),
+           sourceNodes, false);
+ 
+         conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 50);
+         final int r = Balancer.run(namenodes, p, conf);
+         assertEquals(ExitStatus.NO_MOVE_BLOCK.getExitCode(), r);
+       }
+ 
+       { // run Balancer with a filled node as a source node
+         final Set<String> sourceNodes = new HashSet<>();
+         final List<DataNode> datanodes = cluster.getDataNodes();
+         sourceNodes.add(datanodes.get(0).getDisplayName());
+         final Parameters p = new Parameters(
+           BalancingPolicy.Node.INSTANCE, 1,
+           NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
+           Collections.<String> emptySet(), Collections.<String> emptySet(),
+           sourceNodes, false);
+ 
+         conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
+         final int r = Balancer.run(namenodes, p, conf);
+         assertEquals(ExitStatus.NO_MOVE_BLOCK.getExitCode(), r);
+       }
+ 
+       { // run Balancer with all filled node as source nodes
+         final Set<String> sourceNodes = new HashSet<>();
+         final List<DataNode> datanodes = cluster.getDataNodes();
+         for(int i = 0; i < capacities.length; i++) {
+           sourceNodes.add(datanodes.get(i).getDisplayName());
+         }
+         final Parameters p = new Parameters(
+           BalancingPolicy.Node.INSTANCE, 1,
+           NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
+           Collections.<String> emptySet(), Collections.<String> emptySet(),
+           sourceNodes, false);
+ 
+         conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
+         final int r = Balancer.run(namenodes, p, conf);
+         assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
+       }
+     } finally {
+       cluster.shutdown();
+     }
+   }
 -  
 +  public void integrationTestWithStripedFile(Configuration conf) throws Exception {
 +    initConfWithStripe(conf);
 +    doTestBalancerWithStripedFile(conf);
 +  }
 +
 +  @Test(timeout = 100000)
 +  public void testBalancerWithStripedFile() throws Exception {
 +    Configuration conf = new Configuration();
 +    initConfWithStripe(conf);
 +    doTestBalancerWithStripedFile(conf);
 +  }
 +
 +  private void doTestBalancerWithStripedFile(Configuration conf) throws Exception {
 +    int numOfDatanodes = dataBlocks + parityBlocks + 2;
 +    int numOfRacks = dataBlocks;
 +    long capacity = 20 * DEFAULT_STRIPE_BLOCK_SIZE;
 +    long[] capacities = new long[numOfDatanodes];
 +    for (int i = 0; i < capacities.length; i++) {
 +      capacities[i] = capacity;
 +    }
 +    String[] racks = new String[numOfDatanodes];
 +    for (int i = 0; i < numOfDatanodes; i++) {
 +      racks[i] = "/rack" + (i % numOfRacks);
 +    }
 +    cluster = new MiniDFSCluster.Builder(conf)
 +        .numDataNodes(numOfDatanodes)
 +        .racks(racks)
 +        .simulatedCapacities(capacities)
 +        .build();
 +
 +    try {
 +      cluster.waitActive();
 +      client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(),
 +          ClientProtocol.class).getProxy();
 +      client.createErasureCodingZone("/", null);
 +
 +      long totalCapacity = sum(capacities);
 +
 +      // fill up the cluster with 30% data. It'll be 45% full plus parity.
 +      long fileLen = totalCapacity * 3 / 10;
 +      long totalUsedSpace = fileLen * (dataBlocks + parityBlocks) / dataBlocks;
 +      FileSystem fs = cluster.getFileSystem(0);
 +      DFSTestUtil.createFile(fs, filePath, fileLen, (short) 3, r.nextLong());
 +
 +      // verify locations of striped blocks
 +      LocatedBlocks locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
 +      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
 +
 +      // add one datanode
 +      String newRack = "/rack" + (++numOfRacks);
 +      cluster.startDataNodes(conf, 1, true, null,
 +          new String[]{newRack}, null, new long[]{capacity});
 +      totalCapacity += capacity;
 +      cluster.triggerHeartbeats();
 +
 +      // run balancer and validate results
 +      Balancer.Parameters p = Balancer.Parameters.DEFAULT;
 +      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
 +      runBalancer(conf, totalUsedSpace, totalCapacity, p, 0);
 +
 +      // verify locations of striped blocks
 +      locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
 +      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
- 
 +    } finally {
 +      cluster.shutdown();
 +    }
 +  }
 +
    /**
     * @param args
     */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab56fcdb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
index bae4f1d,ceef9f2..d6213ff
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
@@@ -70,18 -71,18 +71,6 @@@ public class TestBlockInfo 
    }
  
    @Test
--  public void testCopyConstructor() {
-     BlockInfo old = new BlockInfoContiguous((short) 3);
 -    BlockInfoContiguous old = new BlockInfoContiguous((short) 3);
--    try {
-       BlockInfo copy = new BlockInfoContiguous((BlockInfoContiguous)old);
-       assertEquals(old.getBlockCollection(), copy.getBlockCollection());
 -      BlockInfoContiguous copy = new BlockInfoContiguous(old);
 -      assertEquals(old.getBlockCollectionId(), copy.getBlockCollectionId());
--      assertEquals(old.getCapacity(), copy.getCapacity());
--    } catch (Exception e) {
--      Assert.fail("Copy constructor throws exception: " + e);
--    }
--  }
--
--  @Test
    public void testReplaceStorage() throws Exception {
  
      // Create two dummy storages.


[48/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 0000000,92d117c..be346a4
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@@ -1,0 -1,3135 +1,3240 @@@
+ /**
+  * 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.hdfs;
+ 
+ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT;
+ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
+ 
+ import java.io.BufferedOutputStream;
+ import java.io.DataInputStream;
+ import java.io.DataOutputStream;
+ import java.io.FileNotFoundException;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.net.InetAddress;
+ import java.net.InetSocketAddress;
+ import java.net.Socket;
+ import java.net.SocketAddress;
+ import java.net.URI;
+ import java.net.UnknownHostException;
+ import java.security.GeneralSecurityException;
+ import java.util.ArrayList;
+ import java.util.EnumSet;
+ import java.util.HashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Random;
+ import java.util.concurrent.SynchronousQueue;
+ import java.util.concurrent.ThreadLocalRandom;
+ import java.util.concurrent.ThreadPoolExecutor;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicInteger;
+ 
+ import javax.net.SocketFactory;
+ 
+ import org.apache.hadoop.HadoopIllegalArgumentException;
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.crypto.CipherSuite;
+ import org.apache.hadoop.crypto.CryptoCodec;
+ import org.apache.hadoop.crypto.CryptoInputStream;
+ import org.apache.hadoop.crypto.CryptoOutputStream;
+ import org.apache.hadoop.crypto.CryptoProtocolVersion;
+ import org.apache.hadoop.crypto.key.KeyProvider;
+ import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
+ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
+ import org.apache.hadoop.fs.BlockLocation;
+ import org.apache.hadoop.fs.CacheFlag;
+ import org.apache.hadoop.fs.ContentSummary;
+ import org.apache.hadoop.fs.CreateFlag;
+ import org.apache.hadoop.fs.FileAlreadyExistsException;
+ import org.apache.hadoop.fs.FileEncryptionInfo;
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.FsServerDefaults;
+ import org.apache.hadoop.fs.FsStatus;
+ import org.apache.hadoop.fs.FsTracer;
+ import org.apache.hadoop.fs.HdfsBlockLocation;
+ import org.apache.hadoop.fs.InvalidPathException;
+ import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
+ import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+ import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+ import org.apache.hadoop.fs.Options;
+ import org.apache.hadoop.fs.Options.ChecksumOpt;
+ import org.apache.hadoop.fs.ParentNotDirectoryException;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.fs.RemoteIterator;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.fs.UnresolvedLinkException;
+ import org.apache.hadoop.fs.XAttr;
+ import org.apache.hadoop.fs.XAttrSetFlag;
+ import org.apache.hadoop.fs.permission.AclEntry;
+ import org.apache.hadoop.fs.permission.AclStatus;
+ import org.apache.hadoop.fs.permission.FsAction;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+ import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
+ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+ import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+ import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
+ import org.apache.hadoop.hdfs.net.Peer;
+ import org.apache.hadoop.hdfs.protocol.AclException;
+ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
+ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+ import org.apache.hadoop.hdfs.protocol.CachePoolIterator;
+ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+ import org.apache.hadoop.hdfs.protocol.DatanodeID;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+ import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+ import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+ import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
+ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+ import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
+ import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+ import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+ import org.apache.hadoop.hdfs.util.IOUtilsClient;
+ import org.apache.hadoop.io.DataOutputBuffer;
+ import org.apache.hadoop.io.EnumSetWritable;
+ import org.apache.hadoop.io.IOUtils;
+ import org.apache.hadoop.io.MD5Hash;
+ import org.apache.hadoop.io.Text;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+ import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
+ import org.apache.hadoop.ipc.RPC;
+ import org.apache.hadoop.ipc.RemoteException;
+ import org.apache.hadoop.net.DNS;
+ import org.apache.hadoop.net.NetUtils;
+ import org.apache.hadoop.security.AccessControlException;
+ import org.apache.hadoop.security.UserGroupInformation;
+ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+ import org.apache.hadoop.security.token.Token;
+ import org.apache.hadoop.security.token.TokenRenewer;
+ import org.apache.hadoop.util.Daemon;
+ import org.apache.hadoop.util.DataChecksum;
+ import org.apache.hadoop.util.DataChecksum.Type;
+ import org.apache.hadoop.util.Progressable;
+ import org.apache.hadoop.util.Time;
+ import org.apache.htrace.core.TraceScope;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.base.Joiner;
+ import com.google.common.base.Preconditions;
+ import com.google.common.collect.Lists;
+ import com.google.common.net.InetAddresses;
+ import org.apache.htrace.core.Tracer;
+ 
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ /********************************************************
+  * DFSClient can connect to a Hadoop Filesystem and 
+  * perform basic file tasks.  It uses the ClientProtocol
+  * to communicate with a NameNode daemon, and connects 
+  * directly to DataNodes to read/write block data.
+  *
+  * Hadoop DFS users should obtain an instance of 
+  * DistributedFileSystem, which uses DFSClient to handle
+  * filesystem tasks.
+  *
+  ********************************************************/
+ @InterfaceAudience.Private
+ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
+     DataEncryptionKeyFactory {
+   public static final Logger LOG = LoggerFactory.getLogger(DFSClient.class);
+   public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
+ 
+   private final Configuration conf;
+   private final Tracer tracer;
+   private final DfsClientConf dfsClientConf;
+   final ClientProtocol namenode;
+   /* The service used for delegation tokens */
+   private Text dtService;
+ 
+   final UserGroupInformation ugi;
+   volatile boolean clientRunning = true;
+   volatile long lastLeaseRenewal;
+   private volatile FsServerDefaults serverDefaults;
+   private volatile long serverDefaultsLastUpdate;
+   final String clientName;
+   final SocketFactory socketFactory;
+   final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
+   final FileSystem.Statistics stats;
+   private final String authority;
+   private final Random r = new Random();
+   private SocketAddress[] localInterfaceAddrs;
+   private DataEncryptionKey encryptionKey;
+   final SaslDataTransferClient saslClient;
+   private final CachingStrategy defaultReadCachingStrategy;
+   private final CachingStrategy defaultWriteCachingStrategy;
+   private final ClientContext clientContext;
+ 
+   private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
+       new DFSHedgedReadMetrics();
+   private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
++  private static volatile ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
+   private final int smallBufferSize;
+ 
+   public DfsClientConf getConf() {
+     return dfsClientConf;
+   }
+ 
+   Configuration getConfiguration() {
+     return conf;
+   }
+ 
+   /**
+    * A map from file names to {@link DFSOutputStream} objects
+    * that are currently being written by this client.
+    * Note that a file can only be written by a single client.
+    */
+   private final Map<Long, DFSOutputStream> filesBeingWritten
+       = new HashMap<Long, DFSOutputStream>();
+ 
+   /**
+    * Same as this(NameNode.getNNAddress(conf), conf);
+    * @see #DFSClient(InetSocketAddress, Configuration)
+    * @deprecated Deprecated at 0.21
+    */
+   @Deprecated
+   public DFSClient(Configuration conf) throws IOException {
+     this(DFSUtilClient.getNNAddress(conf), conf);
+   }
+   
+   public DFSClient(InetSocketAddress address, Configuration conf) throws IOException {
+     this(DFSUtilClient.getNNUri(address), conf);
+   }
+ 
+   /**
+    * Same as this(nameNodeUri, conf, null);
+    * @see #DFSClient(URI, Configuration, FileSystem.Statistics)
+    */
+   public DFSClient(URI nameNodeUri, Configuration conf
+       ) throws IOException {
+     this(nameNodeUri, conf, null);
+   }
+ 
+   /**
+    * Same as this(nameNodeUri, null, conf, stats);
+    * @see #DFSClient(URI, ClientProtocol, Configuration, FileSystem.Statistics) 
+    */
+   public DFSClient(URI nameNodeUri, Configuration conf,
+                    FileSystem.Statistics stats)
+     throws IOException {
+     this(nameNodeUri, null, conf, stats);
+   }
+   
+   /** 
+    * Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
+    * If HA is enabled and a positive value is set for
+    * {@link HdfsClientConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY}
+    * in the configuration, the DFSClient will use
+    * {@link LossyRetryInvocationHandler} as its RetryInvocationHandler.
+    * Otherwise one of nameNodeUri or rpcNamenode must be null.
+    */
+   @VisibleForTesting
+   public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
+       Configuration conf, FileSystem.Statistics stats)
+     throws IOException {
+     // Copy only the required DFSClient configuration
+     this.tracer = FsTracer.get(conf);
+     this.dfsClientConf = new DfsClientConf(conf);
+     this.conf = conf;
+     this.stats = stats;
+     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
+     this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
+     this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf);
+ 
+     this.ugi = UserGroupInformation.getCurrentUser();
+     
+     this.authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
+     this.clientName = "DFSClient_" + dfsClientConf.getTaskId() + "_" + 
+         ThreadLocalRandom.current().nextInt()  + "_" +
+         Thread.currentThread().getId();
+     int numResponseToDrop = conf.getInt(
+         HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
+         HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
+     ProxyAndInfo<ClientProtocol> proxyInfo = null;
+     AtomicBoolean nnFallbackToSimpleAuth = new AtomicBoolean(false);
+ 
+     if (numResponseToDrop > 0) {
+       // This case is used for testing.
+       LOG.warn(HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
+           + " is set to " + numResponseToDrop
+           + ", this hacked client will proactively drop responses");
+       proxyInfo = NameNodeProxiesClient.createProxyWithLossyRetryHandler(conf,
+           nameNodeUri, ClientProtocol.class, numResponseToDrop,
+           nnFallbackToSimpleAuth);
+     }
+     
+     if (proxyInfo != null) {
+       this.dtService = proxyInfo.getDelegationTokenService();
+       this.namenode = proxyInfo.getProxy();
+     } else if (rpcNamenode != null) {
+       // This case is used for testing.
+       Preconditions.checkArgument(nameNodeUri == null);
+       this.namenode = rpcNamenode;
+       dtService = null;
+     } else {
+       Preconditions.checkArgument(nameNodeUri != null,
+           "null URI");
+       proxyInfo = NameNodeProxiesClient.createProxyWithClientProtocol(conf,
+           nameNodeUri, nnFallbackToSimpleAuth);
+       this.dtService = proxyInfo.getDelegationTokenService();
+       this.namenode = proxyInfo.getProxy();
+     }
+ 
+     String localInterfaces[] =
+       conf.getTrimmedStrings(HdfsClientConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
+     localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
+     if (LOG.isDebugEnabled() && 0 != localInterfaces.length) {
+       LOG.debug("Using local interfaces [" +
+       Joiner.on(',').join(localInterfaces)+ "] with addresses [" +
+       Joiner.on(',').join(localInterfaceAddrs) + "]");
+     }
+     
+     Boolean readDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_READS) == null) ?
+         null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
+     Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
+         null : conf.getLong(DFS_CLIENT_CACHE_READAHEAD, 0);
+     Boolean writeDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES) == null) ?
+         null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES, false);
+     this.defaultReadCachingStrategy =
+         new CachingStrategy(readDropBehind, readahead);
+     this.defaultWriteCachingStrategy =
+         new CachingStrategy(writeDropBehind, readahead);
+     this.clientContext = ClientContext.get(
+         conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
+         dfsClientConf);
+ 
+     if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) {
 -      this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize());
++      this.initThreadsNumForHedgedReads(dfsClientConf.
++          getHedgedReadThreadpoolSize());
+     }
++
++    this.initThreadsNumForStripedReads(dfsClientConf.
++        getStripedReadThreadpoolSize());
+     this.saslClient = new SaslDataTransferClient(
+       conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf),
+       TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth);
+   }
+   
+   /**
+    * Return the socket addresses to use with each configured
+    * local interface. Local interfaces may be specified by IP
+    * address, IP address range using CIDR notation, interface
+    * name (e.g. eth0) or sub-interface name (e.g. eth0:0).
+    * The socket addresses consist of the IPs for the interfaces
+    * and the ephemeral port (port 0). If an IP, IP range, or
+    * interface name matches an interface with sub-interfaces
+    * only the IP of the interface is used. Sub-interfaces can
+    * be used by specifying them explicitly (by IP or name).
+    * 
+    * @return SocketAddresses for the configured local interfaces,
+    *    or an empty array if none are configured
+    * @throws UnknownHostException if a given interface name is invalid
+    */
+   private static SocketAddress[] getLocalInterfaceAddrs(
+       String interfaceNames[]) throws UnknownHostException {
+     List<SocketAddress> localAddrs = new ArrayList<SocketAddress>();
+     for (String interfaceName : interfaceNames) {
+       if (InetAddresses.isInetAddress(interfaceName)) {
+         localAddrs.add(new InetSocketAddress(interfaceName, 0));
+       } else if (NetUtils.isValidSubnet(interfaceName)) {
+         for (InetAddress addr : NetUtils.getIPs(interfaceName, false)) {
+           localAddrs.add(new InetSocketAddress(addr, 0));
+         }
+       } else {
+         for (String ip : DNS.getIPs(interfaceName, false)) {
+           localAddrs.add(new InetSocketAddress(ip, 0));
+         }
+       }
+     }
+     return localAddrs.toArray(new SocketAddress[localAddrs.size()]);
+   }
+ 
+   /**
+    * Select one of the configured local interfaces at random. We use a random
+    * interface because other policies like round-robin are less effective
+    * given that we cache connections to datanodes.
+    *
+    * @return one of the local interface addresses at random, or null if no
+    *    local interfaces are configured
+    */
+   SocketAddress getRandomLocalInterfaceAddr() {
+     if (localInterfaceAddrs.length == 0) {
+       return null;
+     }
+     final int idx = r.nextInt(localInterfaceAddrs.length);
+     final SocketAddress addr = localInterfaceAddrs[idx];
+     if (LOG.isDebugEnabled()) {
+       LOG.debug("Using local interface " + addr);
+     }
+     return addr;
+   }
+ 
+   /**
+    * Return the timeout that clients should use when writing to datanodes.
+    * @param numNodes the number of nodes in the pipeline.
+    */
+   int getDatanodeWriteTimeout(int numNodes) {
+     final int t = dfsClientConf.getDatanodeSocketWriteTimeout();
+     return t > 0? t + HdfsConstants.WRITE_TIMEOUT_EXTENSION*numNodes: 0;
+   }
+ 
+   int getDatanodeReadTimeout(int numNodes) {
+     final int t = dfsClientConf.getSocketTimeout();
+     return t > 0? HdfsConstants.READ_TIMEOUT_EXTENSION*numNodes + t: 0;
+   }
+   
+   @VisibleForTesting
+   public String getClientName() {
+     return clientName;
+   }
+ 
+   void checkOpen() throws IOException {
+     if (!clientRunning) {
+       IOException result = new IOException("Filesystem closed");
+       throw result;
+     }
+   }
+ 
+   /** Return the lease renewer instance. The renewer thread won't start
+    *  until the first output stream is created. The same instance will
+    *  be returned until all output streams are closed.
+    */
+   public LeaseRenewer getLeaseRenewer() throws IOException {
+       return LeaseRenewer.getInstance(authority, ugi, this);
+   }
+ 
+   /** Get a lease and start automatic renewal */
+   private void beginFileLease(final long inodeId, final DFSOutputStream out)
+       throws IOException {
+     getLeaseRenewer().put(inodeId, out, this);
+   }
+ 
+   /** Stop renewal of lease for the file. */
+   void endFileLease(final long inodeId) throws IOException {
+     getLeaseRenewer().closeFile(inodeId, this);
+   }
+     
+ 
+   /** Put a file. Only called from LeaseRenewer, where proper locking is
+    *  enforced to consistently update its local dfsclients array and 
+    *  client's filesBeingWritten map.
+    */
+   public void putFileBeingWritten(final long inodeId, final DFSOutputStream out) {
+     synchronized(filesBeingWritten) {
+       filesBeingWritten.put(inodeId, out);
+       // update the last lease renewal time only when there was no
+       // writes. once there is one write stream open, the lease renewer
+       // thread keeps it updated well with in anyone's expiration time.
+       if (lastLeaseRenewal == 0) {
+         updateLastLeaseRenewal();
+       }
+     }
+   }
+ 
+   /** Remove a file. Only called from LeaseRenewer. */
+   public void removeFileBeingWritten(final long inodeId) {
+     synchronized(filesBeingWritten) {
+       filesBeingWritten.remove(inodeId);
+       if (filesBeingWritten.isEmpty()) {
+         lastLeaseRenewal = 0;
+       }
+     }
+   }
+ 
+   /** Is file-being-written map empty? */
+   public boolean isFilesBeingWrittenEmpty() {
+     synchronized(filesBeingWritten) {
+       return filesBeingWritten.isEmpty();
+     }
+   }
+   
+   /** @return true if the client is running */
+   public boolean isClientRunning() {
+     return clientRunning;
+   }
+ 
+   long getLastLeaseRenewal() {
+     return lastLeaseRenewal;
+   }
+ 
+   void updateLastLeaseRenewal() {
+     synchronized(filesBeingWritten) {
+       if (filesBeingWritten.isEmpty()) {
+         return;
+       }
+       lastLeaseRenewal = Time.monotonicNow();
+     }
+   }
+ 
+   /**
+    * Renew leases.
+    * @return true if lease was renewed. May return false if this
+    * client has been closed or has no files open.
+    **/
+   public boolean renewLease() throws IOException {
+     if (clientRunning && !isFilesBeingWrittenEmpty()) {
+       try {
+         namenode.renewLease(clientName);
+         updateLastLeaseRenewal();
+         return true;
+       } catch (IOException e) {
+         // Abort if the lease has already expired. 
+         final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
+         if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
+           LOG.warn("Failed to renew lease for " + clientName + " for "
+               + (elapsed/1000) + " seconds (>= hard-limit ="
+               + (HdfsConstants.LEASE_HARDLIMIT_PERIOD / 1000) + " seconds.) "
+               + "Closing all files being written ...", e);
+           closeAllFilesBeingWritten(true);
+         } else {
+           // Let the lease renewer handle it and retry.
+           throw e;
+         }
+       }
+     }
+     return false;
+   }
+   
+   /**
+    * Close connections the Namenode.
+    */
+   void closeConnectionToNamenode() {
+     RPC.stopProxy(namenode);
+   }
+ 
+   /** Close/abort all files being written. */
+   public void closeAllFilesBeingWritten(final boolean abort) {
+     for(;;) {
+       final long inodeId;
+       final DFSOutputStream out;
+       synchronized(filesBeingWritten) {
+         if (filesBeingWritten.isEmpty()) {
+           return;
+         }
+         inodeId = filesBeingWritten.keySet().iterator().next();
+         out = filesBeingWritten.remove(inodeId);
+       }
+       if (out != null) {
+         try {
+           if (abort) {
+             out.abort();
+           } else {
+             out.close();
+           }
+         } catch(IOException ie) {
+           LOG.error("Failed to " + (abort ? "abort" : "close") + " file: "
+               + out.getSrc() + " with inode: " + inodeId, ie);
+         }
+       }
+     }
+   }
+ 
+   /**
+    * Close the file system, abandoning all of the leases and files being
+    * created and close connections to the namenode.
+    */
+   @Override
+   public synchronized void close() throws IOException {
+     if(clientRunning) {
+       closeAllFilesBeingWritten(false);
+       clientRunning = false;
+       getLeaseRenewer().closeClient(this);
+       // close connections to the namenode
+       closeConnectionToNamenode();
+     }
+   }
+ 
+   /**
+    * Close all open streams, abandoning all of the leases and files being
+    * created.
+    * @param abort whether streams should be gracefully closed
+    */
+   public void closeOutputStreams(boolean abort) {
+     if (clientRunning) {
+       closeAllFilesBeingWritten(abort);
+     }
+   }
+ 
+   /**
+    * @see ClientProtocol#getPreferredBlockSize(String)
+    */
+   public long getBlockSize(String f) throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("getBlockSize", f);
+     try {
+       return namenode.getPreferredBlockSize(f);
+     } catch (IOException ie) {
+       LOG.warn("Problem getting block size", ie);
+       throw ie;
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Get server default values for a number of configuration params.
+    * @see ClientProtocol#getServerDefaults()
+    */
+   public FsServerDefaults getServerDefaults() throws IOException {
+     checkOpen();
+     long now = Time.monotonicNow();
+     if ((serverDefaults == null) ||
+         (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD)) {
+       serverDefaults = namenode.getServerDefaults();
+       serverDefaultsLastUpdate = now;
+     }
+     assert serverDefaults != null;
+     return serverDefaults;
+   }
+   
+   /**
+    * Get a canonical token service name for this client's tokens.  Null should
+    * be returned if the client is not using tokens.
+    * @return the token service for the client
+    */
+   @InterfaceAudience.LimitedPrivate( { "HDFS" }) 
+   public String getCanonicalServiceName() {
+     return (dtService != null) ? dtService.toString() : null;
+   }
+   
+   /**
+    * @see ClientProtocol#getDelegationToken(Text)
+    */
+   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+       throws IOException {
+     assert dtService != null;
+     TraceScope scope = tracer.newScope("getDelegationToken");
+     try {
+       Token<DelegationTokenIdentifier> token =
+         namenode.getDelegationToken(renewer);
+       if (token != null) {
+         token.setService(this.dtService);
+         LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
+       } else {
+         LOG.info("Cannot get delegation token from " + renewer);
+       }
+       return token;
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Renew a delegation token
+    * @param token the token to renew
+    * @return the new expiration time
+    * @throws InvalidToken
+    * @throws IOException
+    * @deprecated Use Token.renew instead.
+    */
+   @Deprecated
+   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+       throws InvalidToken, IOException {
+     LOG.info("Renewing " + DelegationTokenIdentifier.stringifyToken(token));
+     try {
+       return token.renew(conf);
+     } catch (InterruptedException ie) {                                       
+       throw new RuntimeException("caught interrupted", ie);
+     } catch (RemoteException re) {
+       throw re.unwrapRemoteException(InvalidToken.class,
+                                      AccessControlException.class);
+     }
+   }
+   
+   /**
+    * Cancel a delegation token
+    * @param token the token to cancel
+    * @throws InvalidToken
+    * @throws IOException
+    * @deprecated Use Token.cancel instead.
+    */
+   @Deprecated
+   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+       throws InvalidToken, IOException {
+     LOG.info("Cancelling " + DelegationTokenIdentifier.stringifyToken(token));
+     try {
+       token.cancel(conf);
+      } catch (InterruptedException ie) {                                       
+       throw new RuntimeException("caught interrupted", ie);
+     } catch (RemoteException re) {
+       throw re.unwrapRemoteException(InvalidToken.class,
+                                      AccessControlException.class);
+     }
+   }
+   
+   @InterfaceAudience.Private
+   public static class Renewer extends TokenRenewer {
+     
+     static {
+       //Ensure that HDFS Configuration files are loaded before trying to use
+       // the renewer.
+       HdfsConfigurationLoader.init();
+     }
+     
+     @Override
+     public boolean handleKind(Text kind) {
+       return DelegationTokenIdentifier.HDFS_DELEGATION_KIND.equals(kind);
+     }
+ 
+     @SuppressWarnings("unchecked")
+     @Override
+     public long renew(Token<?> token, Configuration conf) throws IOException {
+       Token<DelegationTokenIdentifier> delToken = 
+         (Token<DelegationTokenIdentifier>) token;
+       ClientProtocol nn = getNNProxy(delToken, conf);
+       try {
+         return nn.renewDelegationToken(delToken);
+       } catch (RemoteException re) {
+         throw re.unwrapRemoteException(InvalidToken.class, 
+                                        AccessControlException.class);
+       }
+     }
+ 
+     @SuppressWarnings("unchecked")
+     @Override
+     public void cancel(Token<?> token, Configuration conf) throws IOException {
+       Token<DelegationTokenIdentifier> delToken = 
+           (Token<DelegationTokenIdentifier>) token;
+       LOG.info("Cancelling " + 
+                DelegationTokenIdentifier.stringifyToken(delToken));
+       ClientProtocol nn = getNNProxy(delToken, conf);
+       try {
+         nn.cancelDelegationToken(delToken);
+       } catch (RemoteException re) {
+         throw re.unwrapRemoteException(InvalidToken.class,
+             AccessControlException.class);
+       }
+     }
+     
+     private static ClientProtocol getNNProxy(
+         Token<DelegationTokenIdentifier> token, Configuration conf)
+         throws IOException {
+       URI uri = HAUtilClient.getServiceUriFromToken(
+           HdfsConstants.HDFS_URI_SCHEME, token);
+       if (HAUtilClient.isTokenForLogicalUri(token) &&
+           !HAUtilClient.isLogicalUri(conf, uri)) {
+         // If the token is for a logical nameservice, but the configuration
+         // we have disagrees about that, we can't actually renew it.
+         // This can be the case in MR, for example, if the RM doesn't
+         // have all of the HA clusters configured in its configuration.
+         throw new IOException("Unable to map logical nameservice URI '" +
+             uri + "' to a NameNode. Local configuration does not have " +
+             "a failover proxy provider configured.");
+       }
+       
+       ProxyAndInfo<ClientProtocol> info =
+         NameNodeProxiesClient.createProxyWithClientProtocol(conf, uri, null);
+       assert info.getDelegationTokenService().equals(token.getService()) :
+         "Returned service '" + info.getDelegationTokenService().toString() +
+         "' doesn't match expected service '" +
+         token.getService().toString() + "'";
+         
+       return info.getProxy();
+     }
+ 
+     @Override
+     public boolean isManaged(Token<?> token) throws IOException {
+       return true;
+     }
+     
+   }
+ 
+   /**
+    * Report corrupt blocks that were discovered by the client.
+    * @see ClientProtocol#reportBadBlocks(LocatedBlock[])
+    */
+   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+     checkOpen();
+     namenode.reportBadBlocks(blocks);
+   }
+   
+   public LocatedBlocks getLocatedBlocks(String src, long start)
+       throws IOException {
+     return getLocatedBlocks(src, start, dfsClientConf.getPrefetchSize());
+   }
+ 
+   /*
+    * This is just a wrapper around callGetBlockLocations, but non-static so that
+    * we can stub it out for tests.
+    */
+   @VisibleForTesting
+   public LocatedBlocks getLocatedBlocks(String src, long start, long length)
+       throws IOException {
+     TraceScope scope = newPathTraceScope("getBlockLocations", src);
+     try {
+       return callGetBlockLocations(namenode, src, start, length);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * @see ClientProtocol#getBlockLocations(String, long, long)
+    */
+   static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
+       String src, long start, long length) 
+       throws IOException {
+     try {
+       return namenode.getBlockLocations(src, start, length);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileNotFoundException.class,
+                                      UnresolvedPathException.class);
+     }
+   }
+ 
+   /**
+    * Recover a file's lease
+    * @param src a file's path
+    * @return true if the file is already closed
+    * @throws IOException
+    */
+   boolean recoverLease(String src) throws IOException {
+     checkOpen();
+ 
+     TraceScope scope = newPathTraceScope("recoverLease", src);
+     try {
+       return namenode.recoverLease(src, clientName);
+     } catch (RemoteException re) {
+       throw re.unwrapRemoteException(FileNotFoundException.class,
+                                      AccessControlException.class,
+                                      UnresolvedPathException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Get block location info about file
+    * 
+    * getBlockLocations() returns a list of hostnames that store 
+    * data for a specific file region.  It returns a set of hostnames
+    * for every block within the indicated region.
+    *
+    * This function is very useful when writing code that considers
+    * data-placement when performing operations.  For example, the
+    * MapReduce system tries to schedule tasks on the same machines
+    * as the data-block the task processes. 
+    */
+   public BlockLocation[] getBlockLocations(String src, long start, 
+         long length) throws IOException, UnresolvedLinkException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("getBlockLocations", src);
+     try {
+       LocatedBlocks blocks = getLocatedBlocks(src, start, length);
+       BlockLocation[] locations =  DFSUtilClient.locatedBlocks2Locations(blocks);
+       HdfsBlockLocation[] hdfsLocations = new HdfsBlockLocation[locations.length];
+       for (int i = 0; i < locations.length; i++) {
+         hdfsLocations[i] = new HdfsBlockLocation(locations[i], blocks.get(i));
+       }
+       return hdfsLocations;
+     } finally {
+       scope.close();
+     }
+   }
+   
+   /**
+    * Decrypts a EDEK by consulting the KeyProvider.
+    */
+   private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
+       feInfo) throws IOException {
+     TraceScope scope = tracer.newScope("decryptEDEK");
+     try {
+       KeyProvider provider = getKeyProvider();
+       if (provider == null) {
+         throw new IOException("No KeyProvider is configured, cannot access" +
+             " an encrypted file");
+       }
+       EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
+           feInfo.getKeyName(), feInfo.getEzKeyVersionName(), feInfo.getIV(),
+           feInfo.getEncryptedDataEncryptionKey());
+       try {
+         KeyProviderCryptoExtension cryptoProvider = KeyProviderCryptoExtension
+             .createKeyProviderCryptoExtension(provider);
+         return cryptoProvider.decryptEncryptedKey(ekv);
+       } catch (GeneralSecurityException e) {
+         throw new IOException(e);
+       }
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Obtain the crypto protocol version from the provided FileEncryptionInfo,
+    * checking to see if this version is supported by.
+    *
+    * @param feInfo FileEncryptionInfo
+    * @return CryptoProtocolVersion from the feInfo
+    * @throws IOException if the protocol version is unsupported.
+    */
+   private static CryptoProtocolVersion getCryptoProtocolVersion
+       (FileEncryptionInfo feInfo) throws IOException {
+     final CryptoProtocolVersion version = feInfo.getCryptoProtocolVersion();
+     if (!CryptoProtocolVersion.supports(version)) {
+       throw new IOException("Client does not support specified " +
+           "CryptoProtocolVersion " + version.getDescription() + " version " +
+           "number" + version.getVersion());
+     }
+     return version;
+   }
+ 
+   /**
+    * Obtain a CryptoCodec based on the CipherSuite set in a FileEncryptionInfo
+    * and the available CryptoCodecs configured in the Configuration.
+    *
+    * @param conf   Configuration
+    * @param feInfo FileEncryptionInfo
+    * @return CryptoCodec
+    * @throws IOException if no suitable CryptoCodec for the CipherSuite is
+    *                     available.
+    */
+   private static CryptoCodec getCryptoCodec(Configuration conf,
+       FileEncryptionInfo feInfo) throws IOException {
+     final CipherSuite suite = feInfo.getCipherSuite();
+     if (suite.equals(CipherSuite.UNKNOWN)) {
+       throw new IOException("NameNode specified unknown CipherSuite with ID "
+           + suite.getUnknownValue() + ", cannot instantiate CryptoCodec.");
+     }
+     final CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
+     if (codec == null) {
+       throw new UnknownCipherSuiteException(
+           "No configuration found for the cipher suite "
+           + suite.getConfigSuffix() + " prefixed with "
+           + HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
+           + ". Please see the example configuration "
+           + "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
+           + "at core-default.xml for details.");
+     }
+     return codec;
+   }
+ 
+   /**
+    * Wraps the stream in a CryptoInputStream if the underlying file is
+    * encrypted.
+    */
+   public HdfsDataInputStream createWrappedInputStream(DFSInputStream dfsis)
+       throws IOException {
+     final FileEncryptionInfo feInfo = dfsis.getFileEncryptionInfo();
+     if (feInfo != null) {
+       // File is encrypted, wrap the stream in a crypto stream.
+       // Currently only one version, so no special logic based on the version #
+       getCryptoProtocolVersion(feInfo);
+       final CryptoCodec codec = getCryptoCodec(conf, feInfo);
+       final KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
+       final CryptoInputStream cryptoIn =
+           new CryptoInputStream(dfsis, codec, decrypted.getMaterial(),
+               feInfo.getIV());
+       return new HdfsDataInputStream(cryptoIn);
+     } else {
+       // No FileEncryptionInfo so no encryption.
+       return new HdfsDataInputStream(dfsis);
+     }
+   }
+ 
+   /**
+    * Wraps the stream in a CryptoOutputStream if the underlying file is
+    * encrypted.
+    */
+   public HdfsDataOutputStream createWrappedOutputStream(DFSOutputStream dfsos,
+       FileSystem.Statistics statistics) throws IOException {
+     return createWrappedOutputStream(dfsos, statistics, 0);
+   }
+ 
+   /**
+    * Wraps the stream in a CryptoOutputStream if the underlying file is
+    * encrypted.
+    */
+   public HdfsDataOutputStream createWrappedOutputStream(DFSOutputStream dfsos,
+       FileSystem.Statistics statistics, long startPos) throws IOException {
+     final FileEncryptionInfo feInfo = dfsos.getFileEncryptionInfo();
+     if (feInfo != null) {
+       // File is encrypted, wrap the stream in a crypto stream.
+       // Currently only one version, so no special logic based on the version #
+       getCryptoProtocolVersion(feInfo);
+       final CryptoCodec codec = getCryptoCodec(conf, feInfo);
+       KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
+       final CryptoOutputStream cryptoOut =
+           new CryptoOutputStream(dfsos, codec,
+               decrypted.getMaterial(), feInfo.getIV(), startPos);
+       return new HdfsDataOutputStream(cryptoOut, statistics, startPos);
+     } else {
+       // No FileEncryptionInfo present so no encryption.
+       return new HdfsDataOutputStream(dfsos, statistics, startPos);
+     }
+   }
+ 
+   public DFSInputStream open(String src) 
+       throws IOException, UnresolvedLinkException {
+     return open(src, dfsClientConf.getIoBufferSize(), true, null);
+   }
+ 
+   /**
+    * Create an input stream that obtains a nodelist from the
+    * namenode, and then reads from all the right places.  Creates
+    * inner subclass of InputStream that does the right out-of-band
+    * work.
+    * @deprecated Use {@link #open(String, int, boolean)} instead.
+    */
+   @Deprecated
+   public DFSInputStream open(String src, int buffersize, boolean verifyChecksum,
+                              FileSystem.Statistics stats)
+       throws IOException, UnresolvedLinkException {
+     return open(src, buffersize, verifyChecksum);
+   }
+   
+ 
+   /**
+    * Create an input stream that obtains a nodelist from the
+    * namenode, and then reads from all the right places.  Creates
+    * inner subclass of InputStream that does the right out-of-band
+    * work.
+    */
+   public DFSInputStream open(String src, int buffersize, boolean verifyChecksum)
+       throws IOException, UnresolvedLinkException {
+     checkOpen();
+     //    Get block info from namenode
+     TraceScope scope = newPathTraceScope("newDFSInputStream", src);
+     try {
 -      return new DFSInputStream(this, src, verifyChecksum, null);
++      LocatedBlocks locatedBlocks = getLocatedBlocks(src, 0);
++      if (locatedBlocks != null) {
++        ErasureCodingPolicy ecPolicy = locatedBlocks.getErasureCodingPolicy();
++        if (ecPolicy != null) {
++          return new DFSStripedInputStream(this, src, verifyChecksum, ecPolicy,
++              locatedBlocks);
++        }
++        return new DFSInputStream(this, src, verifyChecksum, locatedBlocks);
++      } else {
++        throw new IOException("Cannot open filename " + src);
++      }
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Get the namenode associated with this DFSClient object
+    * @return the namenode associated with this DFSClient object
+    */
+   public ClientProtocol getNamenode() {
+     return namenode;
+   }
+   
+   /**
+    * Call {@link #create(String, boolean, short, long, Progressable)} with
+    * default <code>replication</code> and <code>blockSize<code> and null <code>
+    * progress</code>.
+    */
+   public OutputStream create(String src, boolean overwrite) 
+       throws IOException {
+     return create(src, overwrite, dfsClientConf.getDefaultReplication(),
+         dfsClientConf.getDefaultBlockSize(), null);
+   }
+     
+   /**
+    * Call {@link #create(String, boolean, short, long, Progressable)} with
+    * default <code>replication</code> and <code>blockSize<code>.
+    */
+   public OutputStream create(String src, 
+                              boolean overwrite,
+                              Progressable progress) throws IOException {
+     return create(src, overwrite, dfsClientConf.getDefaultReplication(),
+         dfsClientConf.getDefaultBlockSize(), progress);
+   }
+     
+   /**
+    * Call {@link #create(String, boolean, short, long, Progressable)} with
+    * null <code>progress</code>.
+    */
+   public OutputStream create(String src, 
+                              boolean overwrite, 
+                              short replication,
+                              long blockSize) throws IOException {
+     return create(src, overwrite, replication, blockSize, null);
+   }
+ 
+   /**
+    * Call {@link #create(String, boolean, short, long, Progressable, int)}
+    * with default bufferSize.
+    */
+   public OutputStream create(String src, boolean overwrite, short replication,
+       long blockSize, Progressable progress) throws IOException {
+     return create(src, overwrite, replication, blockSize, progress,
+         dfsClientConf.getIoBufferSize());
+   }
+ 
+   /**
+    * Call {@link #create(String, FsPermission, EnumSet, short, long, 
+    * Progressable, int, ChecksumOpt)} with default <code>permission</code>
+    * {@link FsPermission#getFileDefault()}.
+    * 
+    * @param src File name
+    * @param overwrite overwrite an existing file if true
+    * @param replication replication factor for the file
+    * @param blockSize maximum block size
+    * @param progress interface for reporting client progress
+    * @param buffersize underlying buffersize
+    * 
+    * @return output stream
+    */
+   public OutputStream create(String src,
+                              boolean overwrite,
+                              short replication,
+                              long blockSize,
+                              Progressable progress,
+                              int buffersize)
+       throws IOException {
+     return create(src, FsPermission.getFileDefault(),
+         overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
+             : EnumSet.of(CreateFlag.CREATE), replication, blockSize, progress,
+         buffersize, null);
+   }
+ 
+   /**
+    * Call {@link #create(String, FsPermission, EnumSet, boolean, short, 
+    * long, Progressable, int, ChecksumOpt)} with <code>createParent</code>
+    *  set to true.
+    */
+   public DFSOutputStream create(String src, 
+                              FsPermission permission,
+                              EnumSet<CreateFlag> flag, 
+                              short replication,
+                              long blockSize,
+                              Progressable progress,
+                              int buffersize,
+                              ChecksumOpt checksumOpt)
+       throws IOException {
+     return create(src, permission, flag, true,
+         replication, blockSize, progress, buffersize, checksumOpt, null);
+   }
+ 
+   /**
+    * Create a new dfs file with the specified block replication 
+    * with write-progress reporting and return an output stream for writing
+    * into the file.  
+    * 
+    * @param src File name
+    * @param permission The permission of the directory being created.
+    *          If null, use default permission {@link FsPermission#getFileDefault()}
+    * @param flag indicates create a new file or create/overwrite an
+    *          existing file or append to an existing file
+    * @param createParent create missing parent directory if true
+    * @param replication block replication
+    * @param blockSize maximum block size
+    * @param progress interface for reporting client progress
+    * @param buffersize underlying buffer size 
+    * @param checksumOpt checksum options
+    * 
+    * @return output stream
+    *
+    * @see ClientProtocol#create for detailed description of exceptions thrown
+    */
+   public DFSOutputStream create(String src, 
+                              FsPermission permission,
+                              EnumSet<CreateFlag> flag, 
+                              boolean createParent,
+                              short replication,
+                              long blockSize,
+                              Progressable progress,
+                              int buffersize,
+                              ChecksumOpt checksumOpt) throws IOException {
 -    return create(src, permission, flag, createParent, replication, blockSize, 
++    return create(src, permission, flag, createParent, replication, blockSize,
+         progress, buffersize, checksumOpt, null);
+   }
+ 
+   private FsPermission applyUMask(FsPermission permission) {
+     if (permission == null) {
+       permission = FsPermission.getFileDefault();
+     }
+     return permission.applyUMask(dfsClientConf.getUMask());
+   }
+ 
+   /**
+    * Same as {@link #create(String, FsPermission, EnumSet, boolean, short, long,
+    * Progressable, int, ChecksumOpt)} with the addition of favoredNodes that is
+    * a hint to where the namenode should place the file blocks.
+    * The favored nodes hint is not persisted in HDFS. Hence it may be honored
+    * at the creation time only. HDFS could move the blocks during balancing or
+    * replication, to move the blocks from favored nodes. A value of null means
+    * no favored nodes for this create
+    */
+   public DFSOutputStream create(String src, 
+                              FsPermission permission,
+                              EnumSet<CreateFlag> flag, 
+                              boolean createParent,
+                              short replication,
+                              long blockSize,
+                              Progressable progress,
+                              int buffersize,
+                              ChecksumOpt checksumOpt,
+                              InetSocketAddress[] favoredNodes) throws IOException {
+     checkOpen();
+     final FsPermission masked = applyUMask(permission);
+     if(LOG.isDebugEnabled()) {
+       LOG.debug(src + ": masked=" + masked);
+     }
+     final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
+         src, masked, flag, createParent, replication, blockSize, progress,
+         buffersize, dfsClientConf.createChecksum(checksumOpt),
+         getFavoredNodesStr(favoredNodes));
+     beginFileLease(result.getFileId(), result);
+     return result;
+   }
+ 
+   private String[] getFavoredNodesStr(InetSocketAddress[] favoredNodes) {
+     String[] favoredNodeStrs = null;
+     if (favoredNodes != null) {
+       favoredNodeStrs = new String[favoredNodes.length];
+       for (int i = 0; i < favoredNodes.length; i++) {
+         favoredNodeStrs[i] = 
+             favoredNodes[i].getHostName() + ":" 
+                          + favoredNodes[i].getPort();
+       }
+     }
+     return favoredNodeStrs;
+   }
+   
+   /**
+    * Append to an existing file if {@link CreateFlag#APPEND} is present
+    */
+   private DFSOutputStream primitiveAppend(String src, EnumSet<CreateFlag> flag,
+       int buffersize, Progressable progress) throws IOException {
+     if (flag.contains(CreateFlag.APPEND)) {
+       HdfsFileStatus stat = getFileInfo(src);
+       if (stat == null) { // No file to append to
+         // New file needs to be created if create option is present
+         if (!flag.contains(CreateFlag.CREATE)) {
+           throw new FileNotFoundException("failed to append to non-existent file "
+               + src + " on client " + clientName);
+         }
+         return null;
+       }
+       return callAppend(src, buffersize, flag, progress, null);
+     }
+     return null;
+   }
+   
+   /**
+    * Same as {{@link #create(String, FsPermission, EnumSet, short, long,
+    *  Progressable, int, ChecksumOpt)} except that the permission
+    *  is absolute (ie has already been masked with umask.
+    */
+   public DFSOutputStream primitiveCreate(String src, 
+                              FsPermission absPermission,
+                              EnumSet<CreateFlag> flag,
+                              boolean createParent,
+                              short replication,
+                              long blockSize,
+                              Progressable progress,
+                              int buffersize,
+                              ChecksumOpt checksumOpt)
+       throws IOException, UnresolvedLinkException {
+     checkOpen();
+     CreateFlag.validate(flag);
+     DFSOutputStream result = primitiveAppend(src, flag, buffersize, progress);
+     if (result == null) {
+       DataChecksum checksum = dfsClientConf.createChecksum(checksumOpt);
+       result = DFSOutputStream.newStreamForCreate(this, src, absPermission,
+           flag, createParent, replication, blockSize, progress, buffersize,
+           checksum, null);
+     }
+     beginFileLease(result.getFileId(), result);
+     return result;
+   }
+   
+   /**
+    * Creates a symbolic link.
+    * 
+    * @see ClientProtocol#createSymlink(String, String,FsPermission, boolean) 
+    */
+   public void createSymlink(String target, String link, boolean createParent)
+       throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("createSymlink", target);
+     try {
+       final FsPermission dirPerm = applyUMask(null);
+       namenode.createSymlink(target, link, dirPerm, createParent);
+     } catch (RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileAlreadyExistsException.class, 
+                                      FileNotFoundException.class,
+                                      ParentNotDirectoryException.class,
+                                      NSQuotaExceededException.class, 
+                                      DSQuotaExceededException.class,
+                                      QuotaByStorageTypeExceededException.class,
+                                      UnresolvedPathException.class,
+                                      SnapshotAccessControlException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Resolve the *first* symlink, if any, in the path.
+    * 
+    * @see ClientProtocol#getLinkTarget(String)
+    */
+   public String getLinkTarget(String path) throws IOException { 
+     checkOpen();
+     TraceScope scope = newPathTraceScope("getLinkTarget", path);
+     try {
+       return namenode.getLinkTarget(path);
+     } catch (RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileNotFoundException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /** Method to get stream returned by append call */
+   private DFSOutputStream callAppend(String src, int buffersize,
+       EnumSet<CreateFlag> flag, Progressable progress, String[] favoredNodes)
+       throws IOException {
+     CreateFlag.validateForAppend(flag);
+     try {
+       LastBlockWithStatus blkWithStatus = namenode.append(src, clientName,
+           new EnumSetWritable<>(flag, CreateFlag.class));
+       return DFSOutputStream.newStreamForAppend(this, src, flag, buffersize,
+           progress, blkWithStatus.getLastBlock(),
+           blkWithStatus.getFileStatus(), dfsClientConf.createChecksum(null),
+           favoredNodes);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileNotFoundException.class,
+                                      SafeModeException.class,
+                                      DSQuotaExceededException.class,
+                                      QuotaByStorageTypeExceededException.class,
+                                      UnsupportedOperationException.class,
+                                      UnresolvedPathException.class,
+                                      SnapshotAccessControlException.class);
+     }
+   }
+   
+   /**
+    * Append to an existing HDFS file.  
+    * 
+    * @param src file name
+    * @param buffersize buffer size
+    * @param flag indicates whether to append data to a new block instead of
+    *             the last block
+    * @param progress for reporting write-progress; null is acceptable.
+    * @param statistics file system statistics; null is acceptable.
+    * @return an output stream for writing into the file
+    * 
+    * @see ClientProtocol#append(String, String, EnumSetWritable)
+    */
+   public HdfsDataOutputStream append(final String src, final int buffersize,
+       EnumSet<CreateFlag> flag, final Progressable progress,
+       final FileSystem.Statistics statistics) throws IOException {
+     final DFSOutputStream out = append(src, buffersize, flag, null, progress);
+     return createWrappedOutputStream(out, statistics, out.getInitialLen());
+   }
+ 
+   /**
+    * Append to an existing HDFS file.
+    * 
+    * @param src file name
+    * @param buffersize buffer size
+    * @param flag indicates whether to append data to a new block instead of the
+    *          last block
+    * @param progress for reporting write-progress; null is acceptable.
+    * @param statistics file system statistics; null is acceptable.
+    * @param favoredNodes FavoredNodes for new blocks
+    * @return an output stream for writing into the file
+    * @see ClientProtocol#append(String, String, EnumSetWritable)
+    */
+   public HdfsDataOutputStream append(final String src, final int buffersize,
+       EnumSet<CreateFlag> flag, final Progressable progress,
+       final FileSystem.Statistics statistics,
+       final InetSocketAddress[] favoredNodes) throws IOException {
+     final DFSOutputStream out = append(src, buffersize, flag,
+         getFavoredNodesStr(favoredNodes), progress);
+     return createWrappedOutputStream(out, statistics, out.getInitialLen());
+   }
+ 
+   private DFSOutputStream append(String src, int buffersize,
+       EnumSet<CreateFlag> flag, String[] favoredNodes, Progressable progress)
+       throws IOException {
+     checkOpen();
+     final DFSOutputStream result = callAppend(src, buffersize, flag, progress,
+         favoredNodes);
+     beginFileLease(result.getFileId(), result);
+     return result;
+   }
+ 
+   /**
+    * Set replication for an existing file.
+    * @param src file name
+    * @param replication replication to set the file to
+    * 
+    * @see ClientProtocol#setReplication(String, short)
+    */
+   public boolean setReplication(String src, short replication)
+       throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("setReplication", src);
+     try {
+       return namenode.setReplication(src, replication);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileNotFoundException.class,
+                                      SafeModeException.class,
+                                      DSQuotaExceededException.class,
+                                      QuotaByStorageTypeExceededException.class,
+                                      UnresolvedPathException.class,
+                                      SnapshotAccessControlException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Set storage policy for an existing file/directory
+    * @param src file/directory name
+    * @param policyName name of the storage policy
+    */
+   public void setStoragePolicy(String src, String policyName)
+       throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("setStoragePolicy", src);
+     try {
+       namenode.setStoragePolicy(src, policyName);
+     } catch (RemoteException e) {
+       throw e.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     SafeModeException.class,
+                                     NSQuotaExceededException.class,
+                                     UnresolvedPathException.class,
+                                     SnapshotAccessControlException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * @param path file/directory name
+    * @return Get the storage policy for specified path
+    */
+   public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("getStoragePolicy", path);
+     try {
+       return namenode.getStoragePolicy(path);
+     } catch (RemoteException e) {
+       throw e.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     SafeModeException.class,
+                                     UnresolvedPathException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * @return All the existing storage policies
+    */
+   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+     checkOpen();
+     TraceScope scope = tracer.newScope("getStoragePolicies");
+     try {
+       return namenode.getStoragePolicies();
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Rename file or directory.
+    * @see ClientProtocol#rename(String, String)
+    * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
+    */
+   @Deprecated
+   public boolean rename(String src, String dst) throws IOException {
+     checkOpen();
+     TraceScope scope = newSrcDstTraceScope("rename", src, dst);
+     try {
+       return namenode.rename(src, dst);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      NSQuotaExceededException.class,
+                                      DSQuotaExceededException.class,
+                                      QuotaByStorageTypeExceededException.class,
+                                      UnresolvedPathException.class,
+                                      SnapshotAccessControlException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Move blocks from src to trg and delete src
+    * See {@link ClientProtocol#concat}.
+    */
+   public void concat(String trg, String [] srcs) throws IOException {
+     checkOpen();
+     TraceScope scope = tracer.newScope("concat");
+     try {
+       namenode.concat(trg, srcs);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      UnresolvedPathException.class,
+                                      SnapshotAccessControlException.class);
+     } finally {
+       scope.close();
+     }
+   }
+   /**
+    * Rename file or directory.
+    * @see ClientProtocol#rename2(String, String, Options.Rename...)
+    */
+   public void rename(String src, String dst, Options.Rename... options)
+       throws IOException {
+     checkOpen();
+     TraceScope scope = newSrcDstTraceScope("rename2", src, dst);
+     try {
+       namenode.rename2(src, dst, options);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      DSQuotaExceededException.class,
+                                      QuotaByStorageTypeExceededException.class,
+                                      FileAlreadyExistsException.class,
+                                      FileNotFoundException.class,
+                                      ParentNotDirectoryException.class,
+                                      SafeModeException.class,
+                                      NSQuotaExceededException.class,
+                                      UnresolvedPathException.class,
+                                      SnapshotAccessControlException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Truncate a file to an indicated size
+    * See {@link ClientProtocol#truncate}.
+    */
+   public boolean truncate(String src, long newLength) throws IOException {
+     checkOpen();
+     if (newLength < 0) {
+       throw new HadoopIllegalArgumentException(
+           "Cannot truncate to a negative file size: " + newLength + ".");
+     }
+     TraceScope scope = newPathTraceScope("truncate", src);
+     try {
+       return namenode.truncate(src, newLength, clientName);
+     } catch (RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+           UnresolvedPathException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Delete file or directory.
+    * See {@link ClientProtocol#delete(String, boolean)}. 
+    */
+   @Deprecated
+   public boolean delete(String src) throws IOException {
+     checkOpen();
+     return delete(src, true);
+   }
+ 
+   /**
+    * delete file or directory.
+    * delete contents of the directory if non empty and recursive 
+    * set to true
+    *
+    * @see ClientProtocol#delete(String, boolean)
+    */
+   public boolean delete(String src, boolean recursive) throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("delete", src);
+     try {
+       return namenode.delete(src, recursive);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileNotFoundException.class,
+                                      SafeModeException.class,
+                                      UnresolvedPathException.class,
+                                      SnapshotAccessControlException.class);
+     } finally {
+       scope.close();
+     }
+   }
+   
+   /** Implemented using getFileInfo(src)
+    */
+   public boolean exists(String src) throws IOException {
+     checkOpen();
+     return getFileInfo(src) != null;
+   }
+ 
+   /**
+    * Get a partial listing of the indicated directory
+    * No block locations need to be fetched
+    */
+   public DirectoryListing listPaths(String src,  byte[] startAfter)
+     throws IOException {
+     return listPaths(src, startAfter, false);
+   }
+   
+   /**
+    * Get a partial listing of the indicated directory
+    *
+    * Recommend to use HdfsFileStatus.EMPTY_NAME as startAfter
+    * if the application wants to fetch a listing starting from
+    * the first entry in the directory
+    *
+    * @see ClientProtocol#getListing(String, byte[], boolean)
+    */
+   public DirectoryListing listPaths(String src,  byte[] startAfter,
+       boolean needLocation) throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("listPaths", src);
+     try {
+       return namenode.getListing(src, startAfter, needLocation);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileNotFoundException.class,
+                                      UnresolvedPathException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Get the file info for a specific file or directory.
+    * @param src The string representation of the path to the file
+    * @return object containing information regarding the file
+    *         or null if file not found
+    *         
+    * @see ClientProtocol#getFileInfo(String) for description of exceptions
+    */
+   public HdfsFileStatus getFileInfo(String src) throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("getFileInfo", src);
+     try {
+       return namenode.getFileInfo(src);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileNotFoundException.class,
+                                      UnresolvedPathException.class);
+     } finally {
+       scope.close();
+     }
+   }
+   
+   /**
+    * Close status of a file
+    * @return true if file is already closed
+    */
+   public boolean isFileClosed(String src) throws IOException{
+     checkOpen();
+     TraceScope scope = newPathTraceScope("isFileClosed", src);
+     try {
+       return namenode.isFileClosed(src);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileNotFoundException.class,
+                                      UnresolvedPathException.class);
+     } finally {
+       scope.close();
+     }
+   }
+   
+   /**
+    * Get the file info for a specific file or directory. If src
+    * refers to a symlink then the FileStatus of the link is returned.
+    * @param src path to a file or directory.
+    * 
+    * For description of exceptions thrown 
+    * @see ClientProtocol#getFileLinkInfo(String)
+    */
+   public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("getFileLinkInfo", src);
+     try {
+       return namenode.getFileLinkInfo(src);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      UnresolvedPathException.class);
+     } finally {
+       scope.close();
+     }
+    }
+   
+   @InterfaceAudience.Private
+   public void clearDataEncryptionKey() {
+     LOG.debug("Clearing encryption key");
+     synchronized (this) {
+       encryptionKey = null;
+     }
+   }
+   
+   /**
+    * @return true if data sent between this client and DNs should be encrypted,
+    *         false otherwise.
+    * @throws IOException in the event of error communicating with the NN
+    */
+   boolean shouldEncryptData() throws IOException {
+     FsServerDefaults d = getServerDefaults();
+     return d == null ? false : d.getEncryptDataTransfer();
+   }
+   
+   @Override
+   public DataEncryptionKey newDataEncryptionKey() throws IOException {
+     if (shouldEncryptData()) {
+       synchronized (this) {
+         if (encryptionKey == null ||
+             encryptionKey.expiryDate < Time.now()) {
+           LOG.debug("Getting new encryption token from NN");
+           encryptionKey = namenode.getDataEncryptionKey();
+         }
+         return encryptionKey;
+       }
+     } else {
+       return null;
+     }
+   }
+ 
+   /**
+    * Get the checksum of the whole file of a range of the file. Note that the
+    * range always starts from the beginning of the file.
+    * @param src The file path
+    * @param length the length of the range, i.e., the range is [0, length]
+    * @return The checksum 
+    * @see DistributedFileSystem#getFileChecksum(Path)
+    */
+   public MD5MD5CRC32FileChecksum getFileChecksum(String src, long length)
+       throws IOException {
+     checkOpen();
+     Preconditions.checkArgument(length >= 0);
+     //get block locations for the file range
+     LocatedBlocks blockLocations = callGetBlockLocations(namenode, src, 0,
+         length);
+     if (null == blockLocations) {
+       throw new FileNotFoundException("File does not exist: " + src);
+     }
+     if (blockLocations.isUnderConstruction()) {
+       throw new IOException("Fail to get checksum, since file " + src
+           + " is under construction.");
+     }
+     List<LocatedBlock> locatedblocks = blockLocations.getLocatedBlocks();
+     final DataOutputBuffer md5out = new DataOutputBuffer();
+     int bytesPerCRC = -1;
+     DataChecksum.Type crcType = DataChecksum.Type.DEFAULT;
+     long crcPerBlock = 0;
+     boolean refetchBlocks = false;
+     int lastRetriedIndex = -1;
+ 
+     // get block checksum for each block
+     long remaining = length;
+     if (src.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) {
+       remaining = Math.min(length, blockLocations.getFileLength());
+     }
+     for(int i = 0; i < locatedblocks.size() && remaining > 0; i++) {
+       if (refetchBlocks) {  // refetch to get fresh tokens
+         blockLocations = callGetBlockLocations(namenode, src, 0, length);
+         if (null == blockLocations) {
+           throw new FileNotFoundException("File does not exist: " + src);
+         }
+         if (blockLocations.isUnderConstruction()) {
+           throw new IOException("Fail to get checksum, since file " + src
+               + " is under construction.");
+         }
+         locatedblocks = blockLocations.getLocatedBlocks();
+         refetchBlocks = false;
+       }
+       LocatedBlock lb = locatedblocks.get(i);
+       final ExtendedBlock block = lb.getBlock();
+       if (remaining < block.getNumBytes()) {
+         block.setNumBytes(remaining);
+       }
+       remaining -= block.getNumBytes();
+       final DatanodeInfo[] datanodes = lb.getLocations();
+       
+       //try each datanode location of the block
+       final int timeout = 3000*datanodes.length + dfsClientConf.getSocketTimeout();
+       boolean done = false;
+       for(int j = 0; !done && j < datanodes.length; j++) {
+         DataOutputStream out = null;
+         DataInputStream in = null;
+         
+         try {
+           //connect to a datanode
+           IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
+           out = new DataOutputStream(new BufferedOutputStream(pair.out,
+               smallBufferSize));
+           in = new DataInputStream(pair.in);
+ 
+           if (LOG.isDebugEnabled()) {
+             LOG.debug("write to " + datanodes[j] + ": "
+                 + Op.BLOCK_CHECKSUM + ", block=" + block);
+           }
+           // get block MD5
+           new Sender(out).blockChecksum(block, lb.getBlockToken());
+ 
+           final BlockOpResponseProto reply =
+             BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
+ 
+           String logInfo = "for block " + block + " from datanode " + datanodes[j];
+           DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
+ 
+           OpBlockChecksumResponseProto checksumData =
+             reply.getChecksumResponse();
+ 
+           //read byte-per-checksum
+           final int bpc = checksumData.getBytesPerCrc();
+           if (i == 0) { //first block
+             bytesPerCRC = bpc;
+           }
+           else if (bpc != bytesPerCRC) {
+             throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
+                 + " but bytesPerCRC=" + bytesPerCRC);
+           }
+           
+           //read crc-per-block
+           final long cpb = checksumData.getCrcPerBlock();
+           if (locatedblocks.size() > 1 && i == 0) {
+             crcPerBlock = cpb;
+           }
+ 
+           //read md5
+           final MD5Hash md5 = new MD5Hash(
+               checksumData.getMd5().toByteArray());
+           md5.write(md5out);
+           
+           // read crc-type
+           final DataChecksum.Type ct;
+           if (checksumData.hasCrcType()) {
+             ct = PBHelperClient.convert(checksumData
+                 .getCrcType());
+           } else {
+             LOG.debug("Retrieving checksum from an earlier-version DataNode: " +
+                       "inferring checksum by reading first byte");
+             ct = inferChecksumTypeByReading(lb, datanodes[j]);
+           }
+ 
+           if (i == 0) { // first block
+             crcType = ct;
+           } else if (crcType != DataChecksum.Type.MIXED
+               && crcType != ct) {
+             // if crc types are mixed in a file
+             crcType = DataChecksum.Type.MIXED;
+           }
+ 
+           done = true;
+ 
+           if (LOG.isDebugEnabled()) {
+             if (i == 0) {
+               LOG.debug("set bytesPerCRC=" + bytesPerCRC
+                   + ", crcPerBlock=" + crcPerBlock);
+             }
+             LOG.debug("got reply from " + datanodes[j] + ": md5=" + md5);
+           }
+         } catch (InvalidBlockTokenException ibte) {
+           if (i > lastRetriedIndex) {
+             if (LOG.isDebugEnabled()) {
+               LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
+                   + "for file " + src + " for block " + block
+                   + " from datanode " + datanodes[j]
+                   + ". Will retry the block once.");
+             }
+             lastRetriedIndex = i;
+             done = true; // actually it's not done; but we'll retry
+             i--; // repeat at i-th block
+             refetchBlocks = true;
+             break;
+           }
+         } catch (IOException ie) {
+           LOG.warn("src=" + src + ", datanodes["+j+"]=" + datanodes[j], ie);
+         } finally {
+           IOUtils.closeStream(in);
+           IOUtils.closeStream(out);
+         }
+       }
+ 
+       if (!done) {
+         throw new IOException("Fail to get block MD5 for " + block);
+       }
+     }
+ 
+     //compute file MD5
+     final MD5Hash fileMD5 = MD5Hash.digest(md5out.getData()); 
+     switch (crcType) {
+       case CRC32:
+         return new MD5MD5CRC32GzipFileChecksum(bytesPerCRC,
+             crcPerBlock, fileMD5);
+       case CRC32C:
+         return new MD5MD5CRC32CastagnoliFileChecksum(bytesPerCRC,
+             crcPerBlock, fileMD5);
+       default:
+         // If there is no block allocated for the file,
+         // return one with the magic entry that matches what previous
+         // hdfs versions return.
+         if (locatedblocks.size() == 0) {
+           return new MD5MD5CRC32GzipFileChecksum(0, 0, fileMD5);
+         }
+ 
+         // we should never get here since the validity was checked
+         // when getCrcType() was called above.
+         return null;
+     }
+   }
+ 
+   /**
+    * Connect to the given datanode's datantrasfer port, and return
+    * the resulting IOStreamPair. This includes encryption wrapping, etc.
+    */
+   private IOStreamPair connectToDN(DatanodeInfo dn, int timeout,
+       LocatedBlock lb) throws IOException {
+     boolean success = false;
+     Socket sock = null;
+     try {
+       sock = socketFactory.createSocket();
+       String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
+       if (LOG.isDebugEnabled()) {
+         LOG.debug("Connecting to datanode " + dnAddr);
+       }
+       NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
+       sock.setSoTimeout(timeout);
+   
+       OutputStream unbufOut = NetUtils.getOutputStream(sock);
+       InputStream unbufIn = NetUtils.getInputStream(sock);
+       IOStreamPair ret = saslClient.newSocketSend(sock, unbufOut, unbufIn, this,
+         lb.getBlockToken(), dn);
+       success = true;
+       return ret;
+     } finally {
+       if (!success) {
+         IOUtils.closeSocket(sock);
+       }
+     }
+   }
+   
+   /**
+    * Infer the checksum type for a replica by sending an OP_READ_BLOCK
+    * for the first byte of that replica. This is used for compatibility
+    * with older HDFS versions which did not include the checksum type in
+    * OpBlockChecksumResponseProto.
+    *
+    * @param lb the located block
+    * @param dn the connected datanode
+    * @return the inferred checksum type
+    * @throws IOException if an error occurs
+    */
+   private Type inferChecksumTypeByReading(LocatedBlock lb, DatanodeInfo dn)
+       throws IOException {
+     IOStreamPair pair = connectToDN(dn, dfsClientConf.getSocketTimeout(), lb);
+ 
+     try {
+       DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
+           smallBufferSize));
+       DataInputStream in = new DataInputStream(pair.in);
+   
+       new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,
+           0, 1, true, CachingStrategy.newDefaultStrategy());
+       final BlockOpResponseProto reply =
+           BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
+       String logInfo = "trying to read " + lb.getBlock() + " from datanode " + dn;
+       DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
+ 
+       return PBHelperClient.convert(reply.getReadOpChecksumInfo().getChecksum().getType());
+     } finally {
+       IOUtilsClient.cleanup(null, pair.in, pair.out);
+     }
+   }
+ 
+   /**
+    * Set permissions to a file or directory.
+    * @param src path name.
+    * @param permission permission to set to
+    * 
+    * @see ClientProtocol#setPermission(String, FsPermission)
+    */
+   public void setPermission(String src, FsPermission permission)
+       throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("setPermission", src);
+     try {
+       namenode.setPermission(src, permission);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileNotFoundException.class,
+                                      SafeModeException.class,
+                                      UnresolvedPathException.class,
+                                      SnapshotAccessControlException.class);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Set file or directory owner.
+    * @param src path name.
+    * @param username user id.
+    * @param groupname user group.
+    * 
+    * @see ClientProtocol#setOwner(String, String, String)
+    */
+   public void setOwner(String src, String username, String groupname)
+       throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("setOwner", src);
+     try {
+       namenode.setOwner(src, username, groupname);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException(AccessControlException.class,
+                                      FileNotFoundException.class,
+                                      SafeModeException.class,
+                                      UnresolvedPathException.class,
+                                      SnapshotAccessControlException.class);                                   
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   private long[] callGetStats() throws IOException {
+     checkOpen();
+     TraceScope scope = tracer.newScope("getStats");
+     try {
+       return namenode.getStats();
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * @see ClientProtocol#getStats()
+    */
+   public FsStatus getDiskStatus() throws IOException {
+     long rawNums[] = callGetStats();
+     return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
+   }
+ 
+   /**
+    * Returns count of blocks with no good replicas left. Normally should be 
+    * zero.
+    * @throws IOException
+    */ 
+   public long getMissingBlocksCount() throws IOException {
+     return callGetStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
+   }
+   
+   /**
+    * Returns count of blocks with replication factor 1 and have
+    * lost the only replica.
+    * @throws IOException
+    */
+   public long getMissingReplOneBlocksCount() throws IOException {
+     return callGetStats()[ClientProtocol.
+         GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX];
+   }
+ 
+   /**
+    * Returns count of blocks with one of more replica missing.
+    * @throws IOException
+    */ 
+   public long getUnderReplicatedBlocksCount() throws IOException {
+     return callGetStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
+   }
+   
+   /**
+    * Returns count of blocks with at least one replica marked corrupt. 
+    * @throws IOException
+    */ 
+   public long getCorruptBlocksCount() throws IOException {
+     return callGetStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
+   }
+   
+   /**
+    * @return a list in which each entry describes a corrupt file/block
+    * @throws IOException
+    */
+   public CorruptFileBlocks listCorruptFileBlocks(String path,
+                                                  String cookie)
+         throws IOException {
+     checkOpen();
+     TraceScope scope = newPathTraceScope("listCorruptFileBlocks", path);
+     try {
+       return namenode.listCorruptFileBlocks(path, cookie);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   public DatanodeInfo[] datanodeReport(DatanodeReportType type)
+       throws IOException {
+     checkOpen();
+     TraceScope scope = tracer.newScope("datanodeReport");
+     try {
+       return namenode.getDatanodeReport(type);
+     } finally {
+       scope.close();
+     }
+   }
+     
+   public DatanodeStorageReport[] getDatanodeStorageReport(
+       DatanodeReportType type) throws IOException {
+     checkOpen();
+     TraceScope scope =
+         tracer.newScope("datanodeStorageReport");
+     try {
+       return namenode.getDatanodeStorageReport(type);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   /**
+    * Enter, leave or get safe mode.
+    * 
+    * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction,boolean)
+    */
+   public boolean setSafeMode(SafeModeAction action) throws IOException {
+     checkOpen();
+     return setSafeMode(action, false);
+   }
+   
+   /**
+    * Enter, leave or get safe mode.
+    * 
+    * @param action
+    *          One of SafeModeAction.GET, SafeModeAction.ENTER and
+    *          SafeModeActiob.LEAVE
+    * @param isChecked
+    *          If true, then check only active namenode's safemode status, else
+    *          check first namenode's status.
+    * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
+    */
+   public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
+     TraceScope scope =
+         tracer.newScope("setSafeMode");
+     try {
+       return namenode.setSafeMode(action, isChecked);
+     } finally {
+       scope.close();
+     }
+   }
+  
+   /**
+    * Create one snapshot.
+    * 
+    * @param snapshotRoot The directory where the snapshot is to be taken
+    * @param snapshotName Name of the snapshot
+    * @return the snapshot path.
+    * @see ClientProtocol#createSnapshot(String, String)
+    */
+   public String createSnapshot(String snapshotRoot, String snapshotName)
+       throws IOException {
+     checkOpen();
+     TraceScope scope = tracer.newScope("createSnapshot");
+     try {
+       return namenode.createSnapshot(snapshotRoot, snapshotName);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException();
+     } finally {
+       scope.close();
+     }
+   }
+   
+   /**
+    * Delete a snapshot of a snapshottable directory.
+    * 
+    * @param snapshotRoot The snapshottable directory that the 
+    *                    to-be-deleted snapshot belongs to
+    * @param snapshotName The name of the to-be-deleted snapshot
+    * @throws IOException
+    * @see ClientProtocol#deleteSnapshot(String, String)
+    */
+   public void deleteSnapshot(String snapshotRoot, String snapshotName)
+       throws IOException {
+     checkOpen();
+     TraceScope scope = tracer.newScope("deleteSnapshot");
+     try {
+       namenode.deleteSnapshot(snapshotRoot, snapshotName);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException();
+     } finally {
+       scope.close();
+     }
+   }
+   
+   /**
+    * Rename a snapshot.
+    * @param snapshotDir The directory path where the snapshot was taken
+    * @param snapshotOldName Old name of the snapshot
+    * @param snapshotNewName New name of the snapshot
+    * @throws IOException
+    * @see ClientProtocol#renameSnapshot(String, String, String)
+    */
+   public void renameSnapshot(String snapshotDir, String snapshotOldName,
+       String snapshotNewName) throws IOException {
+     checkOpen();
+     TraceScope scope = tracer.newScope("renameSnapshot");
+     try {
+       namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
+     } catch(RemoteException re) {
+       throw re.unwrapRemoteException();
+     } finally {
+       scope.close();
+     }
+   }
+   
+   /**
+    * Get all the current snapshottable directories.
+    * @return All the c

<TRUNCATED>

[47/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 0000000,7101753..d9f409c
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@@ -1,0 -1,1917 +1,1889 @@@
+ /**
+  * 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.hdfs;
+ 
+ import java.io.EOFException;
+ import java.io.IOException;
+ import java.net.InetSocketAddress;
+ import java.nio.ByteBuffer;
+ import java.util.AbstractMap;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.EnumSet;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Map.Entry;
+ import java.util.Set;
+ import java.util.concurrent.Callable;
+ import java.util.concurrent.CancellationException;
+ import java.util.concurrent.CompletionService;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.concurrent.ExecutionException;
+ import java.util.concurrent.ExecutorCompletionService;
+ import java.util.concurrent.Future;
+ import java.util.concurrent.ThreadLocalRandom;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ 
 -import com.google.common.base.Preconditions;
+ import org.apache.commons.io.IOUtils;
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.fs.ByteBufferReadable;
+ import org.apache.hadoop.fs.ByteBufferUtil;
+ import org.apache.hadoop.fs.CanSetDropBehind;
+ import org.apache.hadoop.fs.CanSetReadahead;
+ import org.apache.hadoop.fs.CanUnbuffer;
+ import org.apache.hadoop.fs.ChecksumException;
+ import org.apache.hadoop.fs.FSInputStream;
+ import org.apache.hadoop.fs.FileEncryptionInfo;
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
+ import org.apache.hadoop.fs.ReadOption;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.fs.UnresolvedLinkException;
+ import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+ import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+ import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+ import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
+ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+ import org.apache.hadoop.io.ByteBufferPool;
+ import org.apache.hadoop.ipc.RPC;
+ import org.apache.hadoop.ipc.RemoteException;
+ import org.apache.hadoop.net.NetUtils;
+ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+ import org.apache.hadoop.security.token.Token;
+ import org.apache.hadoop.util.IdentityHashStore;
+ import org.apache.htrace.core.SpanId;
+ import org.apache.htrace.core.TraceScope;
+ import org.apache.htrace.core.Tracer;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ 
+ /****************************************************************
+  * DFSInputStream provides bytes from a named file.  It handles 
+  * negotiation of the namenode and various datanodes as necessary.
+  ****************************************************************/
+ @InterfaceAudience.Private
+ public class DFSInputStream extends FSInputStream
+ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
+     HasEnhancedByteBufferAccess, CanUnbuffer {
+   @VisibleForTesting
+   public static boolean tcpReadsDisabledForTesting = false;
+   private long hedgedReadOpsLoopNumForTesting = 0;
+   protected final DFSClient dfsClient;
+   protected AtomicBoolean closed = new AtomicBoolean(false);
+   protected final String src;
+   protected final boolean verifyChecksum;
+ 
+   // state by stateful read only:
+   // (protected by lock on this)
+   /////
+   private DatanodeInfo currentNode = null;
+   protected LocatedBlock currentLocatedBlock = null;
+   protected long pos = 0;
+   protected long blockEnd = -1;
+   private BlockReader blockReader = null;
+   ////
+ 
+   // state shared by stateful and positional read:
+   // (protected by lock on infoLock)
+   ////
+   protected LocatedBlocks locatedBlocks = null;
+   private long lastBlockBeingWrittenLength = 0;
+   private FileEncryptionInfo fileEncryptionInfo = null;
+   protected CachingStrategy cachingStrategy;
+   ////
+ 
+   protected final ReadStatistics readStatistics = new ReadStatistics();
+   // lock for state shared between read and pread
+   // Note: Never acquire a lock on <this> with this lock held to avoid deadlocks
+   //       (it's OK to acquire this lock when the lock on <this> is held)
+   protected final Object infoLock = new Object();
+ 
+   /**
+    * Track the ByteBuffers that we have handed out to readers.
+    * 
+    * The value type can be either ByteBufferPool or ClientMmap, depending on
+    * whether we this is a memory-mapped buffer or not.
+    */
+   private IdentityHashStore<ByteBuffer, Object> extendedReadBuffers;
+ 
+   private synchronized IdentityHashStore<ByteBuffer, Object>
+         getExtendedReadBuffers() {
+     if (extendedReadBuffers == null) {
+       extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
+     }
+     return extendedReadBuffers;
+   }
+ 
+   public static class ReadStatistics {
+     public ReadStatistics() {
+       clear();
+     }
+ 
+     public ReadStatistics(ReadStatistics rhs) {
+       this.totalBytesRead = rhs.getTotalBytesRead();
+       this.totalLocalBytesRead = rhs.getTotalLocalBytesRead();
+       this.totalShortCircuitBytesRead = rhs.getTotalShortCircuitBytesRead();
+       this.totalZeroCopyBytesRead = rhs.getTotalZeroCopyBytesRead();
+     }
+ 
+     /**
+      * @return The total bytes read.  This will always be at least as
+      * high as the other numbers, since it includes all of them.
+      */
+     public long getTotalBytesRead() {
+       return totalBytesRead;
+     }
+ 
+     /**
+      * @return The total local bytes read.  This will always be at least
+      * as high as totalShortCircuitBytesRead, since all short-circuit
+      * reads are also local.
+      */
+     public long getTotalLocalBytesRead() {
+       return totalLocalBytesRead;
+     }
+ 
+     /**
+      * @return The total short-circuit local bytes read.
+      */
+     public long getTotalShortCircuitBytesRead() {
+       return totalShortCircuitBytesRead;
+     }
+     
+     /**
+      * @return The total number of zero-copy bytes read.
+      */
+     public long getTotalZeroCopyBytesRead() {
+       return totalZeroCopyBytesRead;
+     }
+ 
+     /**
+      * @return The total number of bytes read which were not local.
+      */
+     public long getRemoteBytesRead() {
+       return totalBytesRead - totalLocalBytesRead;
+     }
+     
+     void addRemoteBytes(long amt) {
+       this.totalBytesRead += amt;
+     }
+ 
+     void addLocalBytes(long amt) {
+       this.totalBytesRead += amt;
+       this.totalLocalBytesRead += amt;
+     }
+ 
+     void addShortCircuitBytes(long amt) {
+       this.totalBytesRead += amt;
+       this.totalLocalBytesRead += amt;
+       this.totalShortCircuitBytesRead += amt;
+     }
+ 
+     void addZeroCopyBytes(long amt) {
+       this.totalBytesRead += amt;
+       this.totalLocalBytesRead += amt;
+       this.totalShortCircuitBytesRead += amt;
+       this.totalZeroCopyBytesRead += amt;
+     }
+ 
+     void clear() {
+       this.totalBytesRead = 0;
+       this.totalLocalBytesRead = 0;
+       this.totalShortCircuitBytesRead = 0;
+       this.totalZeroCopyBytesRead = 0;
+     }
+     
+     private long totalBytesRead;
+ 
+     private long totalLocalBytesRead;
+ 
+     private long totalShortCircuitBytesRead;
+ 
+     private long totalZeroCopyBytesRead;
+   }
+   
+   /**
+    * This variable tracks the number of failures since the start of the
+    * most recent user-facing operation. That is to say, it should be reset
+    * whenever the user makes a call on this stream, and if at any point
+    * during the retry logic, the failure count exceeds a threshold,
+    * the errors will be thrown back to the operation.
+    *
+    * Specifically this counts the number of times the client has gone
+    * back to the namenode to get a new list of block locations, and is
+    * capped at maxBlockAcquireFailures
+    */
+   protected int failures = 0;
+ 
+   /* XXX Use of CocurrentHashMap is temp fix. Need to fix 
+    * parallel accesses to DFSInputStream (through ptreads) properly */
+   private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
+              new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
+ 
+   private byte[] oneByteBuf; // used for 'int read()'
+ 
+   void addToDeadNodes(DatanodeInfo dnInfo) {
+     deadNodes.put(dnInfo, dnInfo);
+   }
+   
+   DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
+       LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
+     this.dfsClient = dfsClient;
+     this.verifyChecksum = verifyChecksum;
+     this.src = src;
+     synchronized (infoLock) {
+       this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy();
+     }
+     this.locatedBlocks = locatedBlocks;
+     openInfo(false);
+   }
+ 
+   /**
+    * Grab the open-file info from namenode
+    * @param refreshLocatedBlocks whether to re-fetch locatedblocks
+    */
+   void openInfo(boolean refreshLocatedBlocks) throws IOException,
+       UnresolvedLinkException {
+     final DfsClientConf conf = dfsClient.getConf();
+     synchronized(infoLock) {
+       lastBlockBeingWrittenLength =
+           fetchLocatedBlocksAndGetLastBlockLength(refreshLocatedBlocks);
+       int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength();
+       while (retriesForLastBlockLength > 0) {
+         // Getting last block length as -1 is a special case. When cluster
+         // restarts, DNs may not report immediately. At this time partial block
+         // locations will not be available with NN for getting the length. Lets
+         // retry for 3 times to get the length.
+         if (lastBlockBeingWrittenLength == -1) {
+           DFSClient.LOG.warn("Last block locations not available. "
+               + "Datanodes might not have reported blocks completely."
+               + " Will retry for " + retriesForLastBlockLength + " times");
+           waitFor(conf.getRetryIntervalForGetLastBlockLength());
+           lastBlockBeingWrittenLength =
+               fetchLocatedBlocksAndGetLastBlockLength(true);
+         } else {
+           break;
+         }
+         retriesForLastBlockLength--;
+       }
+       if (retriesForLastBlockLength == 0) {
+         throw new IOException("Could not obtain the last block locations.");
+       }
+     }
+   }
+ 
+   private void waitFor(int waitTime) throws IOException {
+     try {
+       Thread.sleep(waitTime);
+     } catch (InterruptedException e) {
+       throw new IOException(
+           "Interrupted while getting the last block length.");
+     }
+   }
+ 
+   private long fetchLocatedBlocksAndGetLastBlockLength(boolean refresh)
+       throws IOException {
+     LocatedBlocks newInfo = locatedBlocks;
+     if (locatedBlocks == null || refresh) {
+       newInfo = dfsClient.getLocatedBlocks(src, 0);
+     }
+     if (DFSClient.LOG.isDebugEnabled()) {
+       DFSClient.LOG.debug("newInfo = " + newInfo);
+     }
+     if (newInfo == null) {
+       throw new IOException("Cannot open filename " + src);
+     }
+ 
+     if (locatedBlocks != null) {
+       Iterator<LocatedBlock> oldIter = locatedBlocks.getLocatedBlocks().iterator();
+       Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator();
+       while (oldIter.hasNext() && newIter.hasNext()) {
+         if (! oldIter.next().getBlock().equals(newIter.next().getBlock())) {
+           throw new IOException("Blocklist for " + src + " has changed!");
+         }
+       }
+     }
+     locatedBlocks = newInfo;
+     long lastBlockBeingWrittenLength = 0;
+     if (!locatedBlocks.isLastBlockComplete()) {
+       final LocatedBlock last = locatedBlocks.getLastLocatedBlock();
+       if (last != null) {
+         if (last.getLocations().length == 0) {
+           if (last.getBlockSize() == 0) {
+             // if the length is zero, then no data has been written to
+             // datanode. So no need to wait for the locations.
+             return 0;
+           }
+           return -1;
+         }
+         final long len = readBlockLength(last);
+         last.getBlock().setNumBytes(len);
+         lastBlockBeingWrittenLength = len; 
+       }
+     }
+ 
+     fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo();
+ 
+     return lastBlockBeingWrittenLength;
+   }
+ 
+   /** Read the block length from one of the datanodes. */
+   private long readBlockLength(LocatedBlock locatedblock) throws IOException {
+     assert locatedblock != null : "LocatedBlock cannot be null";
+     int replicaNotFoundCount = locatedblock.getLocations().length;
+     
+     final DfsClientConf conf = dfsClient.getConf();
+     for(DatanodeInfo datanode : locatedblock.getLocations()) {
+       ClientDatanodeProtocol cdp = null;
+       
+       try {
+         cdp = DFSUtilClient.createClientDatanodeProtocolProxy(datanode,
+             dfsClient.getConfiguration(), conf.getSocketTimeout(),
+             conf.isConnectToDnViaHostname(), locatedblock);
+         
+         final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
+         
+         if (n >= 0) {
+           return n;
+         }
+       }
+       catch(IOException ioe) {
+         if (ioe instanceof RemoteException &&
+           (((RemoteException) ioe).unwrapRemoteException() instanceof
+             ReplicaNotFoundException)) {
+           // special case : replica might not be on the DN, treat as 0 length
+           replicaNotFoundCount--;
+         }
+         
+         if (DFSClient.LOG.isDebugEnabled()) {
+           DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode "
+               + datanode + " for block " + locatedblock.getBlock(), ioe);
+         }
+       } finally {
+         if (cdp != null) {
+           RPC.stopProxy(cdp);
+         }
+       }
+     }
+ 
+     // Namenode told us about these locations, but none know about the replica
+     // means that we hit the race between pipeline creation start and end.
+     // we require all 3 because some other exception could have happened
+     // on a DN that has it.  we want to report that error
+     if (replicaNotFoundCount == 0) {
+       return 0;
+     }
+ 
+     throw new IOException("Cannot obtain block length for " + locatedblock);
+   }
+   
+   public long getFileLength() {
+     synchronized(infoLock) {
+       return locatedBlocks == null? 0:
+           locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
+     }
+   }
+ 
+   // Short circuit local reads are forbidden for files that are
+   // under construction.  See HDFS-2757.
+   boolean shortCircuitForbidden() {
+     synchronized(infoLock) {
+       return locatedBlocks.isUnderConstruction();
+     }
+   }
+ 
+   /**
+    * Returns the datanode from which the stream is currently reading.
+    */
+   public synchronized DatanodeInfo getCurrentDatanode() {
+     return currentNode;
+   }
+ 
+   /**
+    * Returns the block containing the target position. 
+    */
+   synchronized public ExtendedBlock getCurrentBlock() {
+     if (currentLocatedBlock == null){
+       return null;
+     }
+     return currentLocatedBlock.getBlock();
+   }
+ 
+   /**
+    * Return collection of blocks that has already been located.
+    */
+   public List<LocatedBlock> getAllBlocks() throws IOException {
+     return getBlockRange(0, getFileLength());
+   }
+ 
+   /**
+    * Get block at the specified position.
+    * Fetch it from the namenode if not cached.
+    * 
+    * @param offset block corresponding to this offset in file is returned
+    * @return located block
+    * @throws IOException
+    */
+   protected LocatedBlock getBlockAt(long offset) throws IOException {
+     synchronized(infoLock) {
+       assert (locatedBlocks != null) : "locatedBlocks is null";
+ 
+       final LocatedBlock blk;
+ 
+       //check offset
+       if (offset < 0 || offset >= getFileLength()) {
+         throw new IOException("offset < 0 || offset >= getFileLength(), offset="
+             + offset
+             + ", locatedBlocks=" + locatedBlocks);
+       }
+       else if (offset >= locatedBlocks.getFileLength()) {
+         // offset to the portion of the last block,
+         // which is not known to the name-node yet;
+         // getting the last block
+         blk = locatedBlocks.getLastLocatedBlock();
+       }
+       else {
+         // search cached blocks first
+         int targetBlockIdx = locatedBlocks.findBlock(offset);
+         if (targetBlockIdx < 0) { // block is not cached
+           targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
+           // fetch more blocks
+           final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
+           assert (newBlocks != null) : "Could not find target position " + offset;
+           locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
+         }
+         blk = locatedBlocks.get(targetBlockIdx);
+       }
+       return blk;
+     }
+   }
+ 
+   /** Fetch a block from namenode and cache it */
+   protected void fetchBlockAt(long offset) throws IOException {
+     synchronized(infoLock) {
+       int targetBlockIdx = locatedBlocks.findBlock(offset);
+       if (targetBlockIdx < 0) { // block is not cached
+         targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
+       }
+       // fetch blocks
+       final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
+       if (newBlocks == null) {
+         throw new IOException("Could not find target position " + offset);
+       }
+       locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
+     }
+   }
+ 
+   /**
+    * Get blocks in the specified range.
+    * Fetch them from the namenode if not cached. This function
+    * will not get a read request beyond the EOF.
+    * @param offset starting offset in file
+    * @param length length of data
+    * @return consequent segment of located blocks
+    * @throws IOException
+    */
+   private List<LocatedBlock> getBlockRange(long offset,
+       long length)  throws IOException {
+     // getFileLength(): returns total file length
+     // locatedBlocks.getFileLength(): returns length of completed blocks
+     if (offset >= getFileLength()) {
+       throw new IOException("Offset: " + offset +
+         " exceeds file length: " + getFileLength());
+     }
+     synchronized(infoLock) {
+       final List<LocatedBlock> blocks;
+       final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
+       final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk;
+       final boolean readLengthPastCompleteBlk = offset + length > lengthOfCompleteBlk;
+ 
+       if (readOffsetWithinCompleteBlk) {
+         //get the blocks of finalized (completed) block range
+         blocks = getFinalizedBlockRange(offset,
+           Math.min(length, lengthOfCompleteBlk - offset));
+       } else {
+         blocks = new ArrayList<LocatedBlock>(1);
+       }
+ 
+       // get the blocks from incomplete block range
+       if (readLengthPastCompleteBlk) {
+          blocks.add(locatedBlocks.getLastLocatedBlock());
+       }
+ 
+       return blocks;
+     }
+   }
+ 
+   /**
+    * Get blocks in the specified range.
+    * Includes only the complete blocks.
+    * Fetch them from the namenode if not cached.
+    */
+   private List<LocatedBlock> getFinalizedBlockRange(
+       long offset, long length) throws IOException {
+     synchronized(infoLock) {
+       assert (locatedBlocks != null) : "locatedBlocks is null";
+       List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
+       // search cached blocks first
+       int blockIdx = locatedBlocks.findBlock(offset);
+       if (blockIdx < 0) { // block is not cached
+         blockIdx = LocatedBlocks.getInsertIndex(blockIdx);
+       }
+       long remaining = length;
+       long curOff = offset;
+       while(remaining > 0) {
+         LocatedBlock blk = null;
+         if(blockIdx < locatedBlocks.locatedBlockCount())
+           blk = locatedBlocks.get(blockIdx);
+         if (blk == null || curOff < blk.getStartOffset()) {
+           LocatedBlocks newBlocks;
+           newBlocks = dfsClient.getLocatedBlocks(src, curOff, remaining);
+           locatedBlocks.insertRange(blockIdx, newBlocks.getLocatedBlocks());
+           continue;
+         }
+         assert curOff >= blk.getStartOffset() : "Block not found";
+         blockRange.add(blk);
+         long bytesRead = blk.getStartOffset() + blk.getBlockSize() - curOff;
+         remaining -= bytesRead;
+         curOff += bytesRead;
+         blockIdx++;
+       }
+       return blockRange;
+     }
+   }
+ 
+   /**
+    * Open a DataInputStream to a DataNode so that it can be read from.
+    * We get block ID and the IDs of the destinations at startup, from the namenode.
+    */
+   private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
+     if (target >= getFileLength()) {
+       throw new IOException("Attempted to read past end of file");
+     }
+ 
+     // Will be getting a new BlockReader.
+     closeCurrentBlockReaders();
+ 
+     //
+     // Connect to best DataNode for desired Block, with potential offset
+     //
+     DatanodeInfo chosenNode = null;
+     int refetchToken = 1; // only need to get a new access token once
+     int refetchEncryptionKey = 1; // only need to get a new encryption key once
+     
+     boolean connectFailedOnce = false;
+ 
+     while (true) {
+       //
+       // Compute desired block
+       //
+       LocatedBlock targetBlock = getBlockAt(target);
+ 
+       // update current position
+       this.pos = target;
+       this.blockEnd = targetBlock.getStartOffset() +
+             targetBlock.getBlockSize() - 1;
+       this.currentLocatedBlock = targetBlock;
+ 
+       long offsetIntoBlock = target - targetBlock.getStartOffset();
+ 
+       DNAddrPair retval = chooseDataNode(targetBlock, null);
+       chosenNode = retval.info;
+       InetSocketAddress targetAddr = retval.addr;
+       StorageType storageType = retval.storageType;
+ 
+       try {
+         blockReader = getBlockReader(targetBlock, offsetIntoBlock,
+             targetBlock.getBlockSize() - offsetIntoBlock, targetAddr,
+             storageType, chosenNode);
+         if(connectFailedOnce) {
+           DFSClient.LOG.info("Successfully connected to " + targetAddr +
+                              " for " + targetBlock.getBlock());
+         }
+         return chosenNode;
+       } catch (IOException ex) {
+         if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+           DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+               + "encryption key was invalid when connecting to " + targetAddr
+               + " : " + ex);
+           // The encryption key used is invalid.
+           refetchEncryptionKey--;
+           dfsClient.clearDataEncryptionKey();
+         } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
+           refetchToken--;
+           fetchBlockAt(target);
+         } else {
+           connectFailedOnce = true;
+           DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
+             + ", add to deadNodes and continue. " + ex, ex);
+           // Put chosen node into dead list, continue
+           addToDeadNodes(chosenNode);
+         }
+       }
+     }
+   }
+ 
+   protected BlockReader getBlockReader(LocatedBlock targetBlock,
+       long offsetInBlock, long length, InetSocketAddress targetAddr,
+       StorageType storageType, DatanodeInfo datanode) throws IOException {
+     ExtendedBlock blk = targetBlock.getBlock();
+     Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
+     CachingStrategy curCachingStrategy;
+     boolean shortCircuitForbidden;
+     synchronized (infoLock) {
+       curCachingStrategy = cachingStrategy;
+       shortCircuitForbidden = shortCircuitForbidden();
+     }
+     return new BlockReaderFactory(dfsClient.getConf()).
+         setInetSocketAddress(targetAddr).
+         setRemotePeerFactory(dfsClient).
+         setDatanodeInfo(datanode).
+         setStorageType(storageType).
+         setFileName(src).
+         setBlock(blk).
+         setBlockToken(accessToken).
+         setStartOffset(offsetInBlock).
+         setVerifyChecksum(verifyChecksum).
+         setClientName(dfsClient.clientName).
+         setLength(length).
+         setCachingStrategy(curCachingStrategy).
+         setAllowShortCircuitLocalReads(!shortCircuitForbidden).
+         setClientCacheContext(dfsClient.getClientContext()).
+         setUserGroupInformation(dfsClient.ugi).
+         setConfiguration(dfsClient.getConfiguration()).
+         setTracer(dfsClient.getTracer()).
+         build();
+   }
+ 
+   /**
+    * Close it down!
+    */
+   @Override
+   public synchronized void close() throws IOException {
+     if (!closed.compareAndSet(false, true)) {
+       DFSClient.LOG.debug("DFSInputStream has been closed already");
+       return;
+     }
+     dfsClient.checkOpen();
+ 
+     if ((extendedReadBuffers != null) && (!extendedReadBuffers.isEmpty())) {
+       final StringBuilder builder = new StringBuilder();
+       extendedReadBuffers.visitAll(new IdentityHashStore.Visitor<ByteBuffer, Object>() {
+         private String prefix = "";
+         @Override
+         public void accept(ByteBuffer k, Object v) {
+           builder.append(prefix).append(k);
+           prefix = ", ";
+         }
+       });
+       DFSClient.LOG.warn("closing file " + src + ", but there are still " +
+           "unreleased ByteBuffers allocated by read().  " +
+           "Please release " + builder.toString() + ".");
+     }
+     closeCurrentBlockReaders();
+     super.close();
+   }
+ 
+   @Override
+   public synchronized int read() throws IOException {
+     if (oneByteBuf == null) {
+       oneByteBuf = new byte[1];
+     }
+     int ret = read( oneByteBuf, 0, 1 );
+     return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
+   }
+ 
+   /**
+    * Wraps different possible read implementations so that readBuffer can be
+    * strategy-agnostic.
+    */
+   interface ReaderStrategy {
+     public int doRead(BlockReader blockReader, int off, int len)
+         throws ChecksumException, IOException;
+ 
+     /**
+      * Copy data from the src ByteBuffer into the read buffer.
+      * @param src The src buffer where the data is copied from
+      * @param offset Useful only when the ReadStrategy is based on a byte array.
+      *               Indicate the offset of the byte array for copy.
+      * @param length Useful only when the ReadStrategy is based on a byte array.
+      *               Indicate the length of the data to copy.
+      */
+     public int copyFrom(ByteBuffer src, int offset, int length);
+   }
+ 
+   protected void updateReadStatistics(ReadStatistics readStatistics,
+         int nRead, BlockReader blockReader) {
+     if (nRead <= 0) return;
+     synchronized(infoLock) {
+       if (blockReader.isShortCircuit()) {
+         readStatistics.addShortCircuitBytes(nRead);
+       } else if (blockReader.isLocal()) {
+         readStatistics.addLocalBytes(nRead);
+       } else {
+         readStatistics.addRemoteBytes(nRead);
+       }
+     }
+   }
+   
+   /**
+    * Used to read bytes into a byte[]
+    */
+   private class ByteArrayStrategy implements ReaderStrategy {
+     final byte[] buf;
+ 
+     public ByteArrayStrategy(byte[] buf) {
+       this.buf = buf;
+     }
+ 
+     @Override
+     public int doRead(BlockReader blockReader, int off, int len)
+           throws ChecksumException, IOException {
+       int nRead = blockReader.read(buf, off, len);
+       updateReadStatistics(readStatistics, nRead, blockReader);
+       return nRead;
+     }
+ 
+     @Override
+     public int copyFrom(ByteBuffer src, int offset, int length) {
+       ByteBuffer writeSlice = src.duplicate();
+       writeSlice.get(buf, offset, length);
+       return length;
+     }
+   }
+ 
+   /**
+    * Used to read bytes into a user-supplied ByteBuffer
+    */
+   protected class ByteBufferStrategy implements ReaderStrategy {
+     final ByteBuffer buf;
+     ByteBufferStrategy(ByteBuffer buf) {
+       this.buf = buf;
+     }
+ 
+     @Override
+     public int doRead(BlockReader blockReader, int off, int len)
+         throws ChecksumException, IOException {
+       int oldpos = buf.position();
+       int oldlimit = buf.limit();
+       boolean success = false;
+       try {
+         int ret = blockReader.read(buf);
+         success = true;
+         updateReadStatistics(readStatistics, ret, blockReader);
+         if (ret == 0) {
+           DFSClient.LOG.warn("zero");
+         }
+         return ret;
+       } finally {
+         if (!success) {
+           // Reset to original state so that retries work correctly.
+           buf.position(oldpos);
+           buf.limit(oldlimit);
+         }
+       } 
+     }
+ 
+     @Override
+     public int copyFrom(ByteBuffer src, int offset, int length) {
+       ByteBuffer writeSlice = src.duplicate();
+       int remaining = Math.min(buf.remaining(), writeSlice.remaining());
+       writeSlice.limit(writeSlice.position() + remaining);
+       buf.put(writeSlice);
+       return remaining;
+     }
+   }
+ 
+   /* This is a used by regular read() and handles ChecksumExceptions.
+    * name readBuffer() is chosen to imply similarity to readBuffer() in
+    * ChecksumFileSystem
+    */ 
+   private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
+       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+       throws IOException {
+     IOException ioe;
+     
+     /* we retry current node only once. So this is set to true only here.
+      * Intention is to handle one common case of an error that is not a
+      * failure on datanode or client : when DataNode closes the connection
+      * since client is idle. If there are other cases of "non-errors" then
+      * then a datanode might be retried by setting this to true again.
+      */
+     boolean retryCurrentNode = true;
+ 
+     while (true) {
+       // retry as many times as seekToNewSource allows.
+       try {
+         return reader.doRead(blockReader, off, len);
+       } catch ( ChecksumException ce ) {
+         DFSClient.LOG.warn("Found Checksum error for "
+             + getCurrentBlock() + " from " + currentNode
+             + " at " + ce.getPos());        
+         ioe = ce;
+         retryCurrentNode = false;
+         // we want to remember which block replicas we have tried
+         addIntoCorruptedBlockMap(getCurrentBlock(), currentNode,
+             corruptedBlockMap);
+       } catch ( IOException e ) {
+         if (!retryCurrentNode) {
+           DFSClient.LOG.warn("Exception while reading from "
+               + getCurrentBlock() + " of " + src + " from "
+               + currentNode, e);
+         }
+         ioe = e;
+       }
+       boolean sourceFound = false;
+       if (retryCurrentNode) {
+         /* possibly retry the same node so that transient errors don't
+          * result in application level failures (e.g. Datanode could have
+          * closed the connection because the client is idle for too long).
+          */ 
+         sourceFound = seekToBlockSource(pos);
+       } else {
+         addToDeadNodes(currentNode);
+         sourceFound = seekToNewSource(pos);
+       }
+       if (!sourceFound) {
+         throw ioe;
+       }
+       retryCurrentNode = false;
+     }
+   }
+ 
+   protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
+     dfsClient.checkOpen();
+     if (closed.get()) {
+       throw new IOException("Stream closed");
+     }
+     Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap 
+       = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
+     failures = 0;
+     if (pos < getFileLength()) {
+       int retries = 2;
+       while (retries > 0) {
+         try {
+           // currentNode can be left as null if previous read had a checksum
+           // error on the same block. See HDFS-3067
+           if (pos > blockEnd || currentNode == null) {
+             currentNode = blockSeekTo(pos);
+           }
+           int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
+           synchronized(infoLock) {
+             if (locatedBlocks.isLastBlockComplete()) {
+               realLen = (int) Math.min(realLen,
+                   locatedBlocks.getFileLength() - pos);
+             }
+           }
+           int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
+           
+           if (result >= 0) {
+             pos += result;
+           } else {
+             // got a EOS from reader though we expect more data on it.
+             throw new IOException("Unexpected EOS from the reader");
+           }
+           if (dfsClient.stats != null) {
+             dfsClient.stats.incrementBytesRead(result);
+           }
+           return result;
+         } catch (ChecksumException ce) {
+           throw ce;            
+         } catch (IOException e) {
+           if (retries == 1) {
+             DFSClient.LOG.warn("DFS Read", e);
+           }
+           blockEnd = -1;
+           if (currentNode != null) { addToDeadNodes(currentNode); }
+           if (--retries == 0) {
+             throw e;
+           }
+         } finally {
+           // Check if need to report block replicas corruption either read
+           // was successful or ChecksumException occured.
+           reportCheckSumFailure(corruptedBlockMap, 
+               currentLocatedBlock.getLocations().length);
+         }
+       }
+     }
+     return -1;
+   }
+ 
+   /**
+    * Read the entire buffer.
+    */
+   @Override
+   public synchronized int read(final byte buf[], int off, int len) throws IOException {
+     ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
+     TraceScope scope =
+         dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src);
+     try {
+       return readWithStrategy(byteArrayReader, off, len);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   @Override
+   public synchronized int read(final ByteBuffer buf) throws IOException {
+     ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
+     TraceScope scope =
+         dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src);
+     try {
+       return readWithStrategy(byteBufferReader, 0, buf.remaining());
+     } finally {
+       scope.close();
+     }
+   }
+ 
+ 
+   /**
+    * Add corrupted block replica into map.
+    */
+   protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
+       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
+     Set<DatanodeInfo> dnSet = null;
+     if((corruptedBlockMap.containsKey(blk))) {
+       dnSet = corruptedBlockMap.get(blk);
+     }else {
+       dnSet = new HashSet<DatanodeInfo>();
+     }
+     if (!dnSet.contains(node)) {
+       dnSet.add(node);
+       corruptedBlockMap.put(blk, dnSet);
+     }
+   }
+ 
+   private DNAddrPair chooseDataNode(LocatedBlock block,
+       Collection<DatanodeInfo> ignoredNodes) throws IOException {
+     while (true) {
+       DNAddrPair result = getBestNodeDNAddrPair(block, ignoredNodes);
+       if (result != null) {
+         return result;
+       } else {
+         String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
+           deadNodes, ignoredNodes);
+         String blockInfo = block.getBlock() + " file=" + src;
+         if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
+           String description = "Could not obtain block: " + blockInfo;
+           DFSClient.LOG.warn(description + errMsg
+               + ". Throwing a BlockMissingException");
+           throw new BlockMissingException(src, description,
+               block.getStartOffset());
+         }
+ 
+         DatanodeInfo[] nodes = block.getLocations();
+         if (nodes == null || nodes.length == 0) {
+           DFSClient.LOG.info("No node available for " + blockInfo);
+         }
+         DFSClient.LOG.info("Could not obtain " + block.getBlock()
+             + " from any node: " + errMsg
+             + ". Will get new block locations from namenode and retry...");
+         try {
+           // Introducing a random factor to the wait time before another retry.
+           // The wait time is dependent on # of failures and a random factor.
+           // At the first time of getting a BlockMissingException, the wait time
+           // is a random number between 0..3000 ms. If the first retry
+           // still fails, we will wait 3000 ms grace period before the 2nd retry.
+           // Also at the second retry, the waiting window is expanded to 6000 ms
+           // alleviating the request rate from the server. Similarly the 3rd retry
+           // will wait 6000ms grace period before retry and the waiting window is
+           // expanded to 9000ms. 
+           final int timeWindow = dfsClient.getConf().getTimeWindow();
+           double waitTime = timeWindow * failures +       // grace period for the last round of attempt
+               // expanding time window for each failure
+               timeWindow * (failures + 1) *
+               ThreadLocalRandom.current().nextDouble();
+           DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
+           Thread.sleep((long)waitTime);
+         } catch (InterruptedException iex) {
+         }
+         deadNodes.clear(); //2nd option is to remove only nodes[blockId]
+         openInfo(true);
+         block = refreshLocatedBlock(block);
+         failures++;
+       }
+     }
+   }
+ 
+   /**
+    * Get the best node from which to stream the data.
+    * @param block LocatedBlock, containing nodes in priority order.
+    * @param ignoredNodes Do not choose nodes in this array (may be null)
+    * @return The DNAddrPair of the best node. Null if no node can be chosen.
+    */
+   protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block,
+       Collection<DatanodeInfo> ignoredNodes) {
+     DatanodeInfo[] nodes = block.getLocations();
+     StorageType[] storageTypes = block.getStorageTypes();
+     DatanodeInfo chosenNode = null;
+     StorageType storageType = null;
+     if (nodes != null) {
+       for (int i = 0; i < nodes.length; i++) {
+         if (!deadNodes.containsKey(nodes[i])
+             && (ignoredNodes == null || !ignoredNodes.contains(nodes[i]))) {
+           chosenNode = nodes[i];
+           // Storage types are ordered to correspond with nodes, so use the same
+           // index to get storage type.
+           if (storageTypes != null && i < storageTypes.length) {
+             storageType = storageTypes[i];
+           }
+           break;
+         }
+       }
+     }
+     if (chosenNode == null) {
 -      DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() +
 -          " after checking nodes = " + Arrays.toString(nodes) +
 -          ", ignoredNodes = " + ignoredNodes);
++      reportLostBlock(block, ignoredNodes);
+       return null;
+     }
+     final String dnAddr =
+         chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
+     if (DFSClient.LOG.isDebugEnabled()) {
+       DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
+     }
+     InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
+     return new DNAddrPair(chosenNode, targetAddr, storageType);
+   }
+ 
++  /**
++   * Warn the user of a lost block
++   */
++  protected void reportLostBlock(LocatedBlock lostBlock,
++      Collection<DatanodeInfo> ignoredNodes) {
++    DatanodeInfo[] nodes = lostBlock.getLocations();
++    DFSClient.LOG.warn("No live nodes contain block " + lostBlock.getBlock() +
++        " after checking nodes = " + Arrays.toString(nodes) +
++        ", ignoredNodes = " + ignoredNodes);
++  }
++
+   private static String getBestNodeDNAddrPairErrorString(
+       DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
+       DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {
+     StringBuilder errMsgr = new StringBuilder(
+         " No live nodes contain current block ");
+     errMsgr.append("Block locations:");
+     for (DatanodeInfo datanode : nodes) {
+       errMsgr.append(" ");
+       errMsgr.append(datanode.toString());
+     }
+     errMsgr.append(" Dead nodes: ");
+     for (DatanodeInfo datanode : deadNodes.keySet()) {
+       errMsgr.append(" ");
+       errMsgr.append(datanode.toString());
+     }
+     if (ignoredNodes != null) {
+       errMsgr.append(" Ignored nodes: ");
+       for (DatanodeInfo datanode : ignoredNodes) {
+         errMsgr.append(" ");
+         errMsgr.append(datanode.toString());
+       }
+     }
+     return errMsgr.toString();
+   }
+ 
+   protected void fetchBlockByteRange(LocatedBlock block, long start, long end,
+       byte[] buf, int offset,
+       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+       throws IOException {
+     block = refreshLocatedBlock(block);
+     while (true) {
+       DNAddrPair addressPair = chooseDataNode(block, null);
+       try {
+         actualGetFromOneDataNode(addressPair, block, start, end,
+             buf, offset, corruptedBlockMap);
+         return;
+       } catch (IOException e) {
+         // Ignore. Already processed inside the function.
+         // Loop through to try the next node.
+       }
+     }
+   }
+ 
+   private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
+       final LocatedBlock block, final long start, final long end,
+       final ByteBuffer bb,
+       final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
+       final int hedgedReadId) {
+     final SpanId parentSpanId = Tracer.getCurrentSpanId();
+     return new Callable<ByteBuffer>() {
+       @Override
+       public ByteBuffer call() throws Exception {
+         byte[] buf = bb.array();
+         int offset = bb.position();
+         TraceScope scope = dfsClient.getTracer().
+             newScope("hedgedRead" + hedgedReadId, parentSpanId);
+         try {
+           actualGetFromOneDataNode(datanode, block, start, end, buf,
+               offset, corruptedBlockMap);
+           return bb;
+         } finally {
+           scope.close();
+         }
+       }
+     };
+   }
+ 
+   /**
 -   * Used when reading contiguous blocks
 -   */
 -  private void actualGetFromOneDataNode(final DNAddrPair datanode,
 -      LocatedBlock block, final long start, final long end, byte[] buf,
 -      int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
 -      throws IOException {
 -    final int length = (int) (end - start + 1);
 -    actualGetFromOneDataNode(datanode, block, start, end, buf,
 -        new int[]{offset}, new int[]{length}, corruptedBlockMap);
 -  }
 -
 -  /**
+    * Read data from one DataNode.
 -   * @param datanode the datanode from which to read data
 -   * @param block the located block containing the requested data
 -   * @param startInBlk the startInBlk offset of the block
 -   * @param endInBlk the endInBlk offset of the block
 -   * @param buf the given byte array into which the data is read
 -   * @param offsets the data may be read into multiple segments of the buf
 -   *                (when reading a striped block). this array indicates the
 -   *                offset of each buf segment.
 -   * @param lengths the length of each buf segment
++   *
++   * @param datanode          the datanode from which to read data
++   * @param block             the located block containing the requested data
++   * @param startInBlk        the startInBlk offset of the block
++   * @param endInBlk          the endInBlk offset of the block
++   * @param buf               the given byte array into which the data is read
++   * @param offset            the offset in buf
+    * @param corruptedBlockMap map recording list of datanodes with corrupted
+    *                          block replica
+    */
 -  void actualGetFromOneDataNode(final DNAddrPair datanode,
 -      LocatedBlock block, final long startInBlk, final long endInBlk,
 -      byte[] buf, int[] offsets, int[] lengths,
++  void actualGetFromOneDataNode(final DNAddrPair datanode, LocatedBlock block,
++      final long startInBlk, final long endInBlk, byte[] buf, int offset,
+       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+       throws IOException {
+     DFSClientFaultInjector.get().startFetchFromDatanode();
+     int refetchToken = 1; // only need to get a new access token once
+     int refetchEncryptionKey = 1; // only need to get a new encryption key once
+     final int len = (int) (endInBlk - startInBlk + 1);
 -    checkReadPortions(offsets, lengths, len);
+ 
+     while (true) {
+       // cached block locations may have been updated by chooseDataNode()
+       // or fetchBlockAt(). Always get the latest list of locations at the
+       // start of the loop.
+       block = refreshLocatedBlock(block);
+       BlockReader reader = null;
+       try {
+         DFSClientFaultInjector.get().fetchFromDatanodeException();
+         reader = getBlockReader(block, startInBlk, len, datanode.addr,
+             datanode.storageType, datanode.info);
 -        for (int i = 0; i < offsets.length; i++) {
 -          int nread = reader.readAll(buf, offsets[i], lengths[i]);
 -          updateReadStatistics(readStatistics, nread, reader);
 -          if (nread != lengths[i]) {
 -            throw new IOException("truncated return from reader.read(): " +
 -                "excpected " + lengths[i] + ", got " + nread);
 -          }
++        int nread = reader.readAll(buf, offset, len);
++        updateReadStatistics(readStatistics, nread, reader);
++        if (nread != len) {
++          throw new IOException("truncated return from reader.read(): " +
++              "excpected " + len + ", got " + nread);
+         }
+         DFSClientFaultInjector.get().readFromDatanodeDelay();
+         return;
+       } catch (ChecksumException e) {
+         String msg = "fetchBlockByteRange(). Got a checksum exception for "
+             + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
+             + datanode.info;
+         DFSClient.LOG.warn(msg);
+         // we want to remember what we have tried
+         addIntoCorruptedBlockMap(block.getBlock(), datanode.info,
+             corruptedBlockMap);
+         addToDeadNodes(datanode.info);
+         throw new IOException(msg);
+       } catch (IOException e) {
+         if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
 -          DFSClient.LOG.info("Will fetch a new encryption key and retry, " 
++          DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+               + "encryption key was invalid when connecting to " + datanode.addr
+               + " : " + e);
+           // The encryption key used is invalid.
+           refetchEncryptionKey--;
+           dfsClient.clearDataEncryptionKey();
+         } else if (refetchToken > 0 && tokenRefetchNeeded(e, datanode.addr)) {
+           refetchToken--;
+           try {
+             fetchBlockAt(block.getStartOffset());
+           } catch (IOException fbae) {
+             // ignore IOE, since we can retry it later in a loop
+           }
+         } else {
+           String msg = "Failed to connect to " + datanode.addr + " for file "
+               + src + " for block " + block.getBlock() + ":" + e;
+           DFSClient.LOG.warn("Connection failure: " + msg, e);
+           addToDeadNodes(datanode.info);
+           throw new IOException(msg);
+         }
+       } finally {
+         if (reader != null) {
+           reader.close();
+         }
+       }
+     }
+   }
+ 
+   /**
+    * Refresh cached block locations.
+    * @param block The currently cached block locations
+    * @return Refreshed block locations
+    * @throws IOException
+    */
+   protected LocatedBlock refreshLocatedBlock(LocatedBlock block)
+       throws IOException {
+     return getBlockAt(block.getStartOffset());
+   }
+ 
+   /**
 -   * This method verifies that the read portions are valid and do not overlap
 -   * with each other.
 -   */
 -  private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) {
 -    Preconditions.checkArgument(offsets.length == lengths.length && offsets.length > 0);
 -    int sum = 0;
 -    for (int i = 0; i < lengths.length; i++) {
 -      if (i > 0) {
 -        int gap = offsets[i] - offsets[i - 1];
 -        // make sure read portions do not overlap with each other
 -        Preconditions.checkArgument(gap >= lengths[i - 1]);
 -      }
 -      sum += lengths[i];
 -    }
 -    Preconditions.checkArgument(sum == totalLen);
 -  }
 -
 -  /**
+    * Like {@link #fetchBlockByteRange}except we start up a second, parallel,
+    * 'hedged' read if the first read is taking longer than configured amount of
+    * time. We then wait on which ever read returns first.
+    */
+   private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
+       long end, byte[] buf, int offset,
+       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+       throws IOException {
+     final DfsClientConf conf = dfsClient.getConf();
+     ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
+     CompletionService<ByteBuffer> hedgedService =
+         new ExecutorCompletionService<ByteBuffer>(
+         dfsClient.getHedgedReadsThreadPool());
+     ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
+     ByteBuffer bb = null;
+     int len = (int) (end - start + 1);
+     int hedgedReadId = 0;
+     block = refreshLocatedBlock(block);
+     while (true) {
+       // see HDFS-6591, this metric is used to verify/catch unnecessary loops
+       hedgedReadOpsLoopNumForTesting++;
+       DNAddrPair chosenNode = null;
+       // there is no request already executing.
+       if (futures.isEmpty()) {
+         // chooseDataNode is a commitment. If no node, we go to
+         // the NN to reget block locations. Only go here on first read.
+         chosenNode = chooseDataNode(block, ignored);
+         bb = ByteBuffer.wrap(buf, offset, len);
+         Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
+             chosenNode, block, start, end, bb,
+             corruptedBlockMap, hedgedReadId++);
+         Future<ByteBuffer> firstRequest = hedgedService
+             .submit(getFromDataNodeCallable);
+         futures.add(firstRequest);
+         try {
+           Future<ByteBuffer> future = hedgedService.poll(
+               conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
+           if (future != null) {
+             future.get();
+             return;
+           }
+           if (DFSClient.LOG.isDebugEnabled()) {
+             DFSClient.LOG.debug("Waited " + conf.getHedgedReadThresholdMillis()
+                 + "ms to read from " + chosenNode.info
+                 + "; spawning hedged read");
+           }
+           // Ignore this node on next go around.
+           ignored.add(chosenNode.info);
+           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
+           continue; // no need to refresh block locations
+         } catch (InterruptedException e) {
+           // Ignore
+         } catch (ExecutionException e) {
+           // Ignore already logged in the call.
+         }
+       } else {
+         // We are starting up a 'hedged' read. We have a read already
+         // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
+         // If no nodes to do hedged reads against, pass.
+         try {
+           chosenNode = getBestNodeDNAddrPair(block, ignored);
+           if (chosenNode == null) {
+             chosenNode = chooseDataNode(block, ignored);
+           }
+           bb = ByteBuffer.allocate(len);
+           Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
+               chosenNode, block, start, end, bb,
+               corruptedBlockMap, hedgedReadId++);
+           Future<ByteBuffer> oneMoreRequest = hedgedService
+               .submit(getFromDataNodeCallable);
+           futures.add(oneMoreRequest);
+         } catch (IOException ioe) {
+           if (DFSClient.LOG.isDebugEnabled()) {
+             DFSClient.LOG.debug("Failed getting node for hedged read: "
+                 + ioe.getMessage());
+           }
+         }
+         // if not succeeded. Submit callables for each datanode in a loop, wait
+         // for a fixed interval and get the result from the fastest one.
+         try {
+           ByteBuffer result = getFirstToComplete(hedgedService, futures);
+           // cancel the rest.
+           cancelAll(futures);
+           if (result.array() != buf) { // compare the array pointers
+             dfsClient.getHedgedReadMetrics().incHedgedReadWins();
+             System.arraycopy(result.array(), result.position(), buf, offset,
+                 len);
+           } else {
+             dfsClient.getHedgedReadMetrics().incHedgedReadOps();
+           }
+           return;
+         } catch (InterruptedException ie) {
+           // Ignore and retry
+         }
+         // We got here if exception. Ignore this node on next go around IFF
+         // we found a chosenNode to hedge read against.
+         if (chosenNode != null && chosenNode.info != null) {
+           ignored.add(chosenNode.info);
+         }
+       }
+     }
+   }
+ 
+   @VisibleForTesting
+   public long getHedgedReadOpsLoopNumForTesting() {
+     return hedgedReadOpsLoopNumForTesting;
+   }
+ 
+   private ByteBuffer getFirstToComplete(
+       CompletionService<ByteBuffer> hedgedService,
+       ArrayList<Future<ByteBuffer>> futures) throws InterruptedException {
+     if (futures.isEmpty()) {
+       throw new InterruptedException("let's retry");
+     }
+     Future<ByteBuffer> future = null;
+     try {
+       future = hedgedService.take();
+       ByteBuffer bb = future.get();
+       futures.remove(future);
+       return bb;
+     } catch (ExecutionException e) {
+       // already logged in the Callable
+       futures.remove(future);
+     } catch (CancellationException ce) {
+       // already logged in the Callable
+       futures.remove(future);
+     }
+ 
+     throw new InterruptedException("let's retry");
+   }
+ 
+   private void cancelAll(List<Future<ByteBuffer>> futures) {
+     for (Future<ByteBuffer> future : futures) {
+       // Unfortunately, hdfs reads do not take kindly to interruption.
+       // Threads return a variety of interrupted-type exceptions but
+       // also complaints about invalid pbs -- likely because read
+       // is interrupted before gets whole pb.  Also verbose WARN
+       // logging.  So, for now, do not interrupt running read.
+       future.cancel(false);
+     }
+   }
+ 
+   /**
+    * Should the block access token be refetched on an exception
+    * 
+    * @param ex Exception received
+    * @param targetAddr Target datanode address from where exception was received
+    * @return true if block access token has expired or invalid and it should be
+    *         refetched
+    */
+   protected static boolean tokenRefetchNeeded(IOException ex,
+       InetSocketAddress targetAddr) {
+     /*
+      * Get a new access token and retry. Retry is needed in 2 cases. 1)
+      * When both NN and DN re-started while DFSClient holding a cached
+      * access token. 2) In the case that NN fails to update its
+      * access key at pre-set interval (by a wide margin) and
+      * subsequently restarts. In this case, DN re-registers itself with
+      * NN and receives a new access key, but DN will delete the old
+      * access key from its memory since it's considered expired based on
+      * the estimated expiration date.
+      */
+     if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
+       DFSClient.LOG.info("Access token was invalid when connecting to "
+           + targetAddr + " : " + ex);
+       return true;
+     }
+     return false;
+   }
+ 
+   /**
+    * Read bytes starting from the specified position.
+    * 
+    * @param position start read from this position
+    * @param buffer read buffer
+    * @param offset offset into buffer
+    * @param length number of bytes to read
+    * 
+    * @return actual number of bytes read
+    */
+   @Override
+   public int read(long position, byte[] buffer, int offset, int length)
+       throws IOException {
+     TraceScope scope = dfsClient.
+         newPathTraceScope("DFSInputStream#byteArrayPread", src);
+     try {
+       return pread(position, buffer, offset, length);
+     } finally {
+       scope.close();
+     }
+   }
+ 
+   private int pread(long position, byte[] buffer, int offset, int length)
+       throws IOException {
+     // sanity checks
+     dfsClient.checkOpen();
+     if (closed.get()) {
+       throw new IOException("Stream closed");
+     }
+     failures = 0;
+     long filelen = getFileLength();
+     if ((position < 0) || (position >= filelen)) {
+       return -1;
+     }
+     int realLen = length;
+     if ((position + length) > filelen) {
+       realLen = (int)(filelen - position);
+     }
+     
+     // determine the block and byte range within the block
+     // corresponding to position and realLen
+     List<LocatedBlock> blockRange = getBlockRange(position, realLen);
+     int remaining = realLen;
+     Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap 
+       = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
+     for (LocatedBlock blk : blockRange) {
+       long targetStart = position - blk.getStartOffset();
+       long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
+       try {
 -        if (dfsClient.isHedgedReadsEnabled()) {
++        if (dfsClient.isHedgedReadsEnabled() && !blk.isStriped()) {
+           hedgedFetchBlockByteRange(blk, targetStart,
+               targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
+         } else {
+           fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
+               buffer, offset, corruptedBlockMap);
+         }
+       } finally {
+         // Check and report if any block replicas are corrupted.
+         // BlockMissingException may be caught if all block replicas are
+         // corrupted.
+         reportCheckSumFailure(corruptedBlockMap, blk.getLocations().length);
+       }
+ 
+       remaining -= bytesToRead;
+       position += bytesToRead;
+       offset += bytesToRead;
+     }
+     assert remaining == 0 : "Wrong number of bytes read.";
+     if (dfsClient.stats != null) {
+       dfsClient.stats.incrementBytesRead(realLen);
+     }
+     return realLen;
+   }
+   
+   /**
+    * DFSInputStream reports checksum failure.
+    * Case I : client has tried multiple data nodes and at least one of the
+    * attempts has succeeded. We report the other failures as corrupted block to
+    * namenode. 
+    * Case II: client has tried out all data nodes, but all failed. We
+    * only report if the total number of replica is 1. We do not
+    * report otherwise since this maybe due to the client is a handicapped client
+    * (who can not read).
+    * @param corruptedBlockMap map of corrupted blocks
+    * @param dataNodeCount number of data nodes who contains the block replicas
+    */
+   protected void reportCheckSumFailure(
+       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap, 
+       int dataNodeCount) {
+     if (corruptedBlockMap.isEmpty()) {
+       return;
+     }
+     Iterator<Entry<ExtendedBlock, Set<DatanodeInfo>>> it = corruptedBlockMap
+         .entrySet().iterator();
+     Entry<ExtendedBlock, Set<DatanodeInfo>> entry = it.next();
+     ExtendedBlock blk = entry.getKey();
+     Set<DatanodeInfo> dnSet = entry.getValue();
+     if (((dnSet.size() < dataNodeCount) && (dnSet.size() > 0))
+         || ((dataNodeCount == 1) && (dnSet.size() == dataNodeCount))) {
+       DatanodeInfo[] locs = new DatanodeInfo[dnSet.size()];
+       int i = 0;
+       for (DatanodeInfo dn:dnSet) {
+         locs[i++] = dn;
+       }
+       LocatedBlock [] lblocks = { new LocatedBlock(blk, locs) };
+       dfsClient.reportChecksumFailure(src, lblocks);
+     }
+     corruptedBlockMap.clear();
+   }
+ 
+   @Override
+   public long skip(long n) throws IOException {
+     if ( n > 0 ) {
+       long curPos = getPos();
+       long fileLen = getFileLength();
+       if( n+curPos > fileLen ) {
+         n = fileLen - curPos;
+       }
+       seek(curPos+n);
+       return n;
+     }
+     return n < 0 ? -1 : 0;
+   }
+ 
+   /**
+    * Seek to a new arbitrary location
+    */
+   @Override
+   public synchronized void seek(long targetPos) throws IOException {
+     if (targetPos > getFileLength()) {
+       throw new EOFException("Cannot seek after EOF");
+     }
+     if (targetPos < 0) {
+       throw new EOFException("Cannot seek to negative offset");
+     }
+     if (closed.get()) {
+       throw new IOException("Stream is closed!");
+     }
+     boolean done = false;
+     if (pos <= targetPos && targetPos <= blockEnd) {
+       //
+       // If this seek is to a positive position in the current
+       // block, and this piece of data might already be lying in
+       // the TCP buffer, then just eat up the intervening data.
+       //
+       int diff = (int)(targetPos - pos);
+       if (diff <= blockReader.available()) {
+         try {
+           pos += blockReader.skip(diff);
+           if (pos == targetPos) {
+             done = true;
+           } else {
+             // The range was already checked. If the block reader returns
+             // something unexpected instead of throwing an exception, it is
+             // most likely a bug. 
+             String errMsg = "BlockReader failed to seek to " + 
+                 targetPos + ". Instead, it seeked to " + pos + ".";
+             DFSClient.LOG.warn(errMsg);
+             throw new IOException(errMsg);
+           }
+         } catch (IOException e) {//make following read to retry
+           if(DFSClient.LOG.isDebugEnabled()) {
+             DFSClient.LOG.debug("Exception while seek to " + targetPos
+                 + " from " + getCurrentBlock() + " of " + src + " from "
+                 + currentNode, e);
+           }
+         }
+       }
+     }
+     if (!done) {
+       pos = targetPos;
+       blockEnd = -1;
+     }
+   }
+ 
+   /**
+    * Same as {@link #seekToNewSource(long)} except that it does not exclude
+    * the current datanode and might connect to the same node.
+    */
+   private boolean seekToBlockSource(long targetPos)
+                                                  throws IOException {
+     currentNode = blockSeekTo(targetPos);
+     return true;
+   }
+   
+   /**
+    * Seek to given position on a node other than the current node.  If
+    * a node other than the current node is found, then returns true. 
+    * If another node could not be found, then returns false.
+    */
+   @Override
+   public synchronized boolean seekToNewSource(long targetPos) throws IOException {
+     if (currentNode == null) {
+       return seekToBlockSource(targetPos);
+     }
+     boolean markedDead = deadNodes.containsKey(currentNode);
+     addToDeadNodes(currentNode);
+     DatanodeInfo oldNode = currentNode;
+     DatanodeInfo newNode = blockSeekTo(targetPos);
+     if (!markedDead) {
+       /* remove it from deadNodes. blockSeekTo could have cleared 
+        * deadNodes and added currentNode again. Thats ok. */
+       deadNodes.remove(oldNode);
+     }
+     if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) {
+       currentNode = newNode;
+       return true;
+     } else {
+       return false;
+     }
+   }
+       
+   /**
+    */
+   @Override
+   public synchronized long getPos() {
+     return pos;
+   }
+ 
+   /** Return the size of the remaining available bytes
+    * if the size is less than or equal to {@link Integer#MAX_VALUE},
+    * otherwise, return {@link Integer#MAX_VALUE}.
+    */
+   @Override
+   public synchronized int available() throws IOException {
+     if (closed.get()) {
+       throw new IOException("Stream closed");
+     }
+ 
+     final long remaining = getFileLength() - pos;
+     return remaining <= Integer.MAX_VALUE? (int)remaining: Integer.MAX_VALUE;
+   }
+ 
+   /**
+    * We definitely don't support marks
+    */
+   @Override
+   public boolean markSupported() {
+     return false;
+   }
+   @Override
+   public void mark(int readLimit) {
+   }
+   @Override
+   public void reset() throws IOException {
+     throw new IOException("Mark/reset not supported");
+   }
+ 
+   /** Utility class to encapsulate data node info and its address. */
+   static final class DNAddrPair {
+     final DatanodeInfo info;
+     final InetSocketAddress addr;
+     final StorageType storageType;
+ 
+     DNAddrPair(DatanodeInfo info, InetSocketAddress addr,
+         StorageType storageType) {
+       this.info = info;
+       this.addr = addr;
+       this.storageType = storageType;
+     }
+   }
+ 
+   /**
+    * Get statistics about the reads which this DFSInputStream has done.
+    */
+   public ReadStatistics getReadStatistics() {
+     synchronized(infoLock) {
+       return new ReadStatistics(readStatistics);
+     }
+   }
+ 
+   /**
+    * Clear statistics about the reads which this DFSInputStream has done.
+    */
+   public void clearReadStatistics() {
+     synchronized(infoLock) {
+       readStatistics.clear();
+     }
+   }
+ 
+   public FileEncryptionInfo getFileEncryptionInfo() {
+     synchronized(infoLock) {
+       return fileEncryptionInfo;
+     }
+   }
+ 
+   protected void closeCurrentBlockReaders() {
+     if (blockReader == null) return;
+     // Close the current block reader so that the new caching settings can 
+     // take effect immediately.
+     try {
+       blockReader.close();
+     } catch (IOException e) {
+       DFSClient.LOG.error("error closing blockReader", e);
+     }
+     blockReader = null;
+     blockEnd = -1;
+   }
+ 
+   @Override
+   public synchronized void setReadahead(Long readahead)
+       throws IOException {
+     synchronized (infoLock) {
+       this.cachingStrategy =
+           new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
+     }
+     closeCurrentBlockReaders();
+   }
+ 
+   @Override
+   public synchronized void setDropBehind(Boolean dropBehind)
+       throws IOException {
+     synchronized (infoLock) {
+       this.cachingStrategy =
+           new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
+     }
+     closeCurrentBlockReaders();
+   }
+ 
+   /**
+    * The immutable empty buffer we return when we reach EOF when doing a
+    * zero-copy read.
+    */
+   private static final ByteBuffer EMPTY_BUFFER =
+     ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
+ 
+   @Override
+   public synchronized ByteBuffer read(ByteBufferPool bufferPool,
+       int maxLength, EnumSet<ReadOption> opts) 
+           throws IOException, UnsupportedOperationException {
+     if (maxLength == 0) {
+       return EMPTY_BUFFER;
+     } else if (maxLength < 0) {
+       throw new IllegalArgumentException("can't read a negative " +
+           "number of bytes.");
+     }
+     if ((blockReader == null) || (blockEnd == -1)) {
+       if (pos >= getFileLength()) {
+         return null;
+       }
+       /*
+        * If we don't have a blockReader, or the one we have has no more bytes
+        * left to read, we call seekToBlockSource to get a new blockReader and
+        * recalculate blockEnd.  Note that we assume we're not at EOF here
+        * (we check this above).
+        */
+       if ((!seekToBlockSource(pos)) || (blockReader == null)) {
+         throw new IOException("failed to allocate new BlockReader " +
+             "at position " + pos);
+       }
+     }
+     ByteBuffer buffer = null;
+     if (dfsClient.getConf().getShortCircuitConf().isShortCircuitMmapEnabled()) {
+       buffer = tryReadZeroCopy(maxLength, opts);
+     }
+     if (buffer != null) {
+       return buffer;
+     }
+     buffer = ByteBufferUtil.fallbackRead(this, bufferPool, maxLength);
+     if (buffer != null) {
+       getExtendedReadBuffers().put(buffer, bufferPool);
+     }
+     return buffer;
+   }
+ 
+   private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
+       EnumSet<ReadOption> opts) throws IOException {
+     // Copy 'pos' and 'blockEnd' to local variables to make it easier for the
+     // JVM to optimize this function.
+     final long curPos = pos;
+     final long curEnd = blockEnd;
+     final long blockStartInFile = currentLocatedBlock.getStartOffset();
+     final long blockPos = curPos - blockStartInFile;
+ 
+     // Shorten this read if the end of the block is nearby.
+     long length63;
+     if ((curPos + maxLength) <= (curEnd + 1)) {
+       length63 = maxLength;
+     } else {
+       length63 = 1 + curEnd - curPos;
+       if (length63 <= 0) {
+         if (DFSClient.LOG.isDebugEnabled()) {
+           DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
+             curPos + " of " + src + "; " + length63 + " bytes left in block.  " +
+             "blockPos=" + blockPos + "; curPos=" + curPos +
+             "; curEnd=" + curEnd);
+         }
+         return null;
+       }
+       if (DFSClient.LOG.isDebugEnabled()) {
+         DFSClient.LOG.debug("Reducing read length from " + maxLength +
+             " to " + length63 + " to avoid going more than one byte " +
+             "past the end of the block.  blockPos=" + blockPos +
+             "; curPos=" + curPos + "; curEnd=" + curEnd);
+       }
+     }
+     // Make sure that don't go beyond 31-bit offsets in the MappedByteBuffer.
+     int length;
+     if (blockPos + length63 <= Integer.MAX_VALUE) {
+       length = (int)length63;
+     } else {
+       long length31 = Integer.MAX_VALUE - blockPos;
+       if (length31 <= 0) {
+         // Java ByteBuffers can't be longer than 2 GB, because they use
+         // 4-byte signed integers to represent capacity, etc.
+         // So we can't mmap the parts of the block higher than the 2 GB offset.
+         // FIXME: we could work around this with multiple memory maps.
+         // See HDFS-5101.
+         if (DFSClient.LOG.isDebugEnabled()) {
+           DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
+             curPos + " of " + src + "; 31-bit MappedByteBuffer limit " +
+             "exceeded.  blockPos=" + blockPos + ", curEnd=" + curEnd);
+         }
+         return null;
+       }
+       length = (int)length31;
+       if (DFSClient.LOG.isDebugEnabled()) {
+         DFSClient.LOG.debug("Reducing read length from " + maxLength +
+             " to " + length + " to avoid 31-bit limit.  " +
+             "blockPos=" + blockPos + "; curPos=" + curPos +
+             "; curEnd=" + curEnd);
+       }
+     }
+     final ClientMmap clientMmap = blockReader.getClientMmap(opts);
+     if (clientMmap == null) {
+       if (DFSClient.LOG.isDebugEnabled()) {
+         DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
+           curPos + " of " + src + "; BlockReader#getClientMmap returned " +
+           "null.");
+       }
+       return null;
+     }
+     boolean success = false;
+     ByteBuffer buffer;
+     try {
+       seek(curPos + length);
+       buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
+       buffer.position((int)blockPos);
+       buffer.limit((int)(blockPos + length));
+       getExtendedReadBuffers().put(buffer, clientMmap);
+       synchronized (infoLock) {
+         readStatistics.addZeroCopyBytes(length);
+       }
+       if (DFSClient.LOG.isDebugEnabled()) {
+         DFSClient.LOG.debug("readZeroCopy read " + length + 
+             " bytes from offset " + curPos + " via the zero-copy read " +
+             "path.  blockEnd = " + blockEnd);
+       }
+       success = true;
+     } finally {
+       if (!success) {
+         IOUtils.closeQuietly(clientMmap);
+       }
+     }
+     return buffer;
+   }
+ 
+   @Override
+   public synchronized void releaseBuffer(ByteBuffer buffer) {
+     if (buffer == EMPTY_BUFFER) return;
+     Object val = getExtendedReadBuffers().remove(buffer);
+     if (val == null) {
+       throw new IllegalArgumentException("tried to release a buffer " +
+           "that was not created by this stream, " + buffer);
+     }
+     if (val instanceof ClientMmap) {
+       IOUtils.closeQuietly((ClientMmap)val);
+     } else if (val instanceof ByteBufferPool) {
+       ((ByteBufferPool)val).putBuffer(buffer);
+     }
+   }
+ 
+   @Override
+   public synchronized void unbuffer() {
+     closeCurrentBlockReaders();
+   }
+ }


[38/50] [abbrv] hadoop git commit: HDFS-9040. Erasure coding: coordinate data streamers in DFSStripedOutputStream. Contributed by Jing Zhao and Walter Su.

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
index 8d4a0cf..12453fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
@@ -20,23 +20,35 @@ package org.apache.hadoop.hdfs;
 import com.google.common.base.Joiner;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+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.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.web.ByteRangeInputStream;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.junit.Assert;
 
 import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Random;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.junit.Assert.assertEquals;
+
 public class StripedFileTestUtil {
   public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class);
   /*
@@ -50,8 +62,8 @@ public class StripedFileTestUtil {
   static final int stripesPerBlock = 4;
   static final int blockSize = BLOCK_STRIPED_CELL_SIZE * stripesPerBlock;
   static final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS + 2;
+  static final int BLOCK_GROUP_SIZE = blockSize * NUM_DATA_BLOCKS;
 
-  static final Random random = new Random();
 
   static byte[] generateBytes(int cnt) {
     byte[] bytes = new byte[cnt];
@@ -61,6 +73,11 @@ public class StripedFileTestUtil {
     return bytes;
   }
 
+  static byte getByte(long pos) {
+    final int mod = 29;
+    return (byte) (pos % mod + 1);
+  }
+
   static int readAll(FSDataInputStream in, byte[] buf) throws IOException {
     int readLen = 0;
     int ret;
@@ -71,15 +88,10 @@ public class StripedFileTestUtil {
     return readLen;
   }
 
-  static byte getByte(long pos) {
-    final int mod = 29;
-    return (byte) (pos % mod + 1);
-  }
-
   static void verifyLength(FileSystem fs, Path srcPath, int fileLength)
       throws IOException {
     FileStatus status = fs.getFileStatus(srcPath);
-    Assert.assertEquals("File length should be the same", fileLength, status.getLen());
+    assertEquals("File length should be the same", fileLength, status.getLen());
   }
 
   static void verifyPread(FileSystem fs, Path srcPath,  int fileLength,
@@ -101,9 +113,7 @@ public class StripedFileTestUtil {
           offset += target;
         }
         for (int i = 0; i < fileLength - startOffset; i++) {
-          Assert.assertEquals("Byte at " + (startOffset + i) + " is different, "
-                  + "the startOffset is " + startOffset,
-              expected[startOffset + i], result[i]);
+          assertEquals("Byte at " + (startOffset + i) + " is different, " + "the startOffset is " + startOffset, expected[startOffset + i], result[i]);
         }
       }
     }
@@ -119,8 +129,7 @@ public class StripedFileTestUtil {
         System.arraycopy(buf, 0, result, readLen, ret);
         readLen += ret;
       }
-      Assert.assertEquals("The length of file should be the same to write size",
-          fileLength, readLen);
+      assertEquals("The length of file should be the same to write size", fileLength, readLen);
       Assert.assertArrayEquals(expected, result);
     }
   }
@@ -137,8 +146,7 @@ public class StripedFileTestUtil {
         result.put(buf);
         buf.clear();
       }
-      Assert.assertEquals("The length of file should be the same to write size",
-          fileLength, readLen);
+      assertEquals("The length of file should be the same to write size", fileLength, readLen);
       Assert.assertArrayEquals(expected, result.array());
     }
   }
@@ -199,10 +207,9 @@ public class StripedFileTestUtil {
     fsdis.seek(pos);
     byte[] buf = new byte[writeBytes];
     int readLen = StripedFileTestUtil.readAll(fsdis, buf);
-    Assert.assertEquals(readLen, writeBytes - pos);
+    assertEquals(readLen, writeBytes - pos);
     for (int i = 0; i < readLen; i++) {
-      Assert.assertEquals("Byte at " + i + " should be the same",
-          StripedFileTestUtil.getByte(pos + i), buf[i]);
+      assertEquals("Byte at " + i + " should be the same", StripedFileTestUtil.getByte(pos + i), buf[i]);
     }
   }
 
@@ -210,6 +217,7 @@ public class StripedFileTestUtil {
       final int dnIndex, final AtomicInteger pos) {
     final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex);
     final DatanodeInfo datanode = getDatanodes(s);
+    assert datanode != null;
     LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos);
     cluster.stopDataNode(datanode.getXferAddr());
   }
@@ -218,7 +226,7 @@ public class StripedFileTestUtil {
     for(;;) {
       final DatanodeInfo[] datanodes = streamer.getNodes();
       if (datanodes != null) {
-        Assert.assertEquals(1, datanodes.length);
+        assertEquals(1, datanodes.length);
         Assert.assertNotNull(datanodes[0]);
         return datanodes[0];
       }
@@ -287,7 +295,6 @@ public class StripedFileTestUtil {
    * @param min minimum of the range
    * @param max maximum of the range
    * @param n number to be generated
-   * @return
    */
   public static int[] randomArray(int min, int max, int n){
     if (n > (max - min + 1) || max < min || min < 0 || max < 0) {
@@ -315,4 +322,170 @@ public class StripedFileTestUtil {
     }
     return result;
   }
+
+  /**
+   * Verify that blocks in striped block group are on different nodes, and every
+   * internal blocks exists.
+   */
+  public static void verifyLocatedStripedBlocks(LocatedBlocks lbs, int groupSize) {
+    for (LocatedBlock lb : lbs.getLocatedBlocks()) {
+      assert lb instanceof LocatedStripedBlock;
+      HashSet<DatanodeInfo> locs = new HashSet<>();
+      Collections.addAll(locs, lb.getLocations());
+      assertEquals(groupSize, lb.getLocations().length);
+      assertEquals(groupSize, locs.size());
+
+      // verify that every internal blocks exists
+      int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices();
+      assertEquals(groupSize, blockIndices.length);
+      HashSet<Integer> found = new HashSet<>();
+      for (int index : blockIndices) {
+        assert index >=0;
+        found.add(index);
+      }
+      assertEquals(groupSize, found.size());
+    }
+  }
+
+  static void checkData(DistributedFileSystem dfs, Path srcPath, int length,
+      int[] killedDnIndex, long oldGS) throws IOException {
+
+    StripedFileTestUtil.verifyLength(dfs, srcPath, length);
+    Arrays.sort(killedDnIndex);
+    List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
+    LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(srcPath.toString(), 0L,
+        Long.MAX_VALUE);
+    int expectedNumGroup = 0;
+    if (length > 0) {
+      expectedNumGroup = (length - 1) / BLOCK_GROUP_SIZE + 1;
+    }
+    assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size());
+
+    for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
+      Assert.assertTrue(firstBlock instanceof LocatedStripedBlock);
+
+      final long gs = firstBlock.getBlock().getGenerationStamp();
+      final String s = "gs=" + gs + ", oldGS=" + oldGS;
+      LOG.info(s);
+      Assert.assertTrue(s, gs >= oldGS);
+
+      LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
+          (LocatedStripedBlock) firstBlock, BLOCK_STRIPED_CELL_SIZE,
+          NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+      blockGroupList.add(Arrays.asList(blocks));
+    }
+
+    // test each block group
+    for (int group = 0; group < blockGroupList.size(); group++) {
+      final boolean isLastGroup = group == blockGroupList.size() - 1;
+      final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE
+          : length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE;
+      final int numCellInGroup = (groupSize - 1)/BLOCK_STRIPED_CELL_SIZE + 1;
+      final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS;
+      final int lastCellSize = groupSize - (numCellInGroup - 1)*BLOCK_STRIPED_CELL_SIZE;
+
+      //get the data of this block
+      List<LocatedBlock> blockList = blockGroupList.get(group);
+      byte[][] dataBlockBytes = new byte[NUM_DATA_BLOCKS][];
+      byte[][] parityBlockBytes = new byte[NUM_PARITY_BLOCKS][];
+
+      // for each block, use BlockReader to read data
+      for (int i = 0; i < blockList.size(); i++) {
+        final int j = i >= NUM_DATA_BLOCKS? 0: i;
+        final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS
+            + (j <= lastCellIndex? 1: 0);
+        final int blockSize = numCellInBlock*BLOCK_STRIPED_CELL_SIZE
+            + (isLastGroup && j == lastCellIndex? lastCellSize - BLOCK_STRIPED_CELL_SIZE: 0);
+
+        final byte[] blockBytes = new byte[blockSize];
+        if (i < NUM_DATA_BLOCKS) {
+          dataBlockBytes[i] = blockBytes;
+        } else {
+          parityBlockBytes[i - NUM_DATA_BLOCKS] = blockBytes;
+        }
+
+        final LocatedBlock lb = blockList.get(i);
+        LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock
+            + ", blockSize=" + blockSize + ", lb=" + lb);
+        if (lb == null) {
+          continue;
+        }
+        final ExtendedBlock block = lb.getBlock();
+        assertEquals(blockSize, block.getNumBytes());
+
+        if (block.getNumBytes() == 0) {
+          continue;
+        }
+
+        if (Arrays.binarySearch(killedDnIndex, i) < 0) {
+          final BlockReader blockReader = BlockReaderTestUtil.getBlockReader(
+              dfs, lb, 0, block.getNumBytes());
+          blockReader.readAll(blockBytes, 0, (int) block.getNumBytes());
+          blockReader.close();
+        }
+      }
+
+      // check data
+      final int groupPosInFile = group*BLOCK_GROUP_SIZE;
+      for (int i = 0; i < dataBlockBytes.length; i++) {
+        boolean killed = false;
+        if (Arrays.binarySearch(killedDnIndex, i) >= 0){
+          killed = true;
+        }
+        final byte[] actual = dataBlockBytes[i];
+        for (int posInBlk = 0; posInBlk < actual.length; posInBlk++) {
+          final long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(
+              BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, posInBlk, i) + groupPosInFile;
+          Assert.assertTrue(posInFile < length);
+          final byte expected = getByte(posInFile);
+
+          if (killed) {
+            actual[posInBlk] = expected;
+          } else {
+            if(expected != actual[posInBlk]){
+              String s = "expected=" + expected + " but actual=" + actual[posInBlk]
+                  + ", posInFile=" + posInFile + ", posInBlk=" + posInBlk
+                  + ". group=" + group + ", i=" + i;
+              Assert.fail(s);
+            }
+          }
+        }
+      }
+
+      // check parity
+      verifyParityBlocks(dfs.getConf(), lbs.getLocatedBlocks().get(group)
+              .getBlockSize(),
+          BLOCK_STRIPED_CELL_SIZE, dataBlockBytes, parityBlockBytes, killedDnIndex);
+    }
+  }
+
+  static void verifyParityBlocks(Configuration conf, final long size, final int cellSize,
+      byte[][] dataBytes, byte[][] parityBytes, int[] killedDnIndex) {
+    Arrays.sort(killedDnIndex);
+    // verify the parity blocks
+    int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength(
+        size, cellSize, dataBytes.length, dataBytes.length);
+    final byte[][] expectedParityBytes = new byte[parityBytes.length][];
+    for (int i = 0; i < parityBytes.length; i++) {
+      expectedParityBytes[i] = new byte[parityBlkSize];
+    }
+    for (int i = 0; i < dataBytes.length; i++) {
+      if (dataBytes[i] == null) {
+        dataBytes[i] = new byte[dataBytes[0].length];
+      } else if (dataBytes[i].length < dataBytes[0].length) {
+        final byte[] tmp = dataBytes[i];
+        dataBytes[i] = new byte[dataBytes[0].length];
+        System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length);
+      }
+    }
+    final RawErasureEncoder encoder =
+        CodecUtil.createRSRawEncoder(conf, dataBytes.length, parityBytes.length);
+    encoder.encode(dataBytes, expectedParityBytes);
+    for (int i = 0; i < parityBytes.length; i++) {
+      if (Arrays.binarySearch(killedDnIndex, dataBytes.length + i) < 0){
+        Assert.assertArrayEquals("i=" + i + ", killedDnIndex=" + Arrays.toString(killedDnIndex),
+            expectedParityBytes[i], parityBytes[i]);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index 0641e8e..d78e88b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -18,26 +18,14 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
-import org.apache.hadoop.hdfs.util.StripedBlockUtil;
-import org.apache.hadoop.io.erasurecode.CodecUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -154,141 +142,15 @@ public class TestDFSStripedOutputStream {
         + cellSize + 123);
   }
 
-  private byte[] generateBytes(int cnt) {
-    byte[] bytes = new byte[cnt];
-    for (int i = 0; i < cnt; i++) {
-      bytes[i] = getByte(i);
-    }
-    return bytes;
-  }
-
-  private byte getByte(long pos) {
-    int mod = 29;
-    return (byte) (pos % mod + 1);
-  }
-
   private void testOneFile(String src, int writeBytes) throws Exception {
     src += "_" + writeBytes;
     Path testPath = new Path(src);
 
-    byte[] bytes = generateBytes(writeBytes);
+    byte[] bytes = StripedFileTestUtil.generateBytes(writeBytes);
     DFSTestUtil.writeFile(fs, testPath, new String(bytes));
     StripedFileTestUtil.waitBlockGroupsReported(fs, src);
 
-    // check file length
-    FileStatus status = fs.getFileStatus(testPath);
-    Assert.assertEquals(writeBytes, status.getLen());
-
-    checkData(src, writeBytes);
-  }
-
-  void checkData(String src, int writeBytes) throws IOException {
-    List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
-    LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L);
-
-    for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
-      Assert.assertTrue(firstBlock instanceof LocatedStripedBlock);
-      LocatedBlock[] blocks = StripedBlockUtil.
-          parseStripedBlockGroup((LocatedStripedBlock) firstBlock,
-              cellSize, dataBlocks, parityBlocks);
-      List<LocatedBlock> oneGroup = Arrays.asList(blocks);
-      blockGroupList.add(oneGroup);
-    }
-
-    // test each block group
-    for (int group = 0; group < blockGroupList.size(); group++) {
-      //get the data of this block
-      List<LocatedBlock> blockList = blockGroupList.get(group);
-      byte[][] dataBlockBytes = new byte[dataBlocks][];
-      byte[][] parityBlockBytes = new byte[parityBlocks][];
-
-      // for each block, use BlockReader to read data
-      for (int i = 0; i < blockList.size(); i++) {
-        LocatedBlock lblock = blockList.get(i);
-        if (lblock == null) {
-          continue;
-        }
-        ExtendedBlock block = lblock.getBlock();
-        byte[] blockBytes = new byte[(int)block.getNumBytes()];
-        if (i < dataBlocks) {
-          dataBlockBytes[i] = blockBytes;
-        } else {
-          parityBlockBytes[i - dataBlocks] = blockBytes;
-        }
-
-        if (block.getNumBytes() == 0) {
-          continue;
-        }
-
-        final BlockReader blockReader = BlockReaderTestUtil.getBlockReader(
-            fs, lblock, 0, block.getNumBytes());
-        blockReader.readAll(blockBytes, 0, (int) block.getNumBytes());
-        blockReader.close();
-      }
-
-      // check if we write the data correctly
-      for (int blkIdxInGroup = 0; blkIdxInGroup < dataBlockBytes.length;
-           blkIdxInGroup++) {
-        final byte[] actualBlkBytes = dataBlockBytes[blkIdxInGroup];
-        if (actualBlkBytes == null) {
-          continue;
-        }
-        for (int posInBlk = 0; posInBlk < actualBlkBytes.length; posInBlk++) {
-          // calculate the position of this byte in the file
-          long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(cellSize,
-              dataBlocks, posInBlk, blkIdxInGroup) +
-              group * blockSize * dataBlocks;
-          Assert.assertTrue(posInFile < writeBytes);
-          final byte expected = getByte(posInFile);
-
-          String s = "Unexpected byte " + actualBlkBytes[posInBlk]
-              + ", expect " + expected
-              + ". Block group index is " + group
-              + ", stripe index is " + posInBlk / cellSize
-              + ", cell index is " + blkIdxInGroup
-              + ", byte index is " + posInBlk % cellSize;
-          Assert.assertEquals(s, expected, actualBlkBytes[posInBlk]);
-        }
-      }
-
-      verifyParity(lbs.getLocatedBlocks().get(group).getBlockSize(),
-          cellSize, dataBlockBytes, parityBlockBytes);
-    }
-  }
-
-  void verifyParity(final long size, final int cellSize,
-      byte[][] dataBytes, byte[][] parityBytes) {
-    verifyParity(conf, size, cellSize, dataBytes, parityBytes, -1);
-  }
-
-  static void verifyParity(Configuration conf, final long size,
-                           final int cellSize, byte[][] dataBytes,
-                           byte[][] parityBytes, int killedDnIndex) {
-    // verify the parity blocks
-    int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength(
-        size, cellSize, dataBytes.length, dataBytes.length);
-    final byte[][] expectedParityBytes = new byte[parityBytes.length][];
-    for (int i = 0; i < parityBytes.length; i++) {
-      expectedParityBytes[i] = new byte[parityBlkSize];
-    }
-    for (int i = 0; i < dataBytes.length; i++) {
-      if (dataBytes[i] == null) {
-        dataBytes[i] = new byte[dataBytes[0].length];
-      } else if (dataBytes[i].length < dataBytes[0].length) {
-        final byte[] tmp = dataBytes[i];
-        dataBytes[i] = new byte[dataBytes[0].length];
-        System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length);
-      }
-    }
-    final RawErasureEncoder encoder =
-            CodecUtil.createRSRawEncoder(conf,
-                dataBytes.length, parityBytes.length);
-    encoder.encode(dataBytes, expectedParityBytes);
-    for (int i = 0; i < parityBytes.length; i++) {
-      if (i != killedDnIndex) {
-        Assert.assertArrayEquals("i=" + i + ", killedDnIndex=" + killedDnIndex,
-            expectedParityBytes[i], parityBytes[i]);
-      }
-    }
+    StripedFileTestUtil.checkData(fs, testPath, writeBytes,
+        new int[]{}, 0);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index 44a29e6..f6c2566 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -30,23 +31,18 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.SecurityTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -74,6 +70,7 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   private static final int FLUSH_POS
       = 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
+
   static {
     System.out.println("NUM_DATA_BLOCKS  = " + NUM_DATA_BLOCKS);
     System.out.println("NUM_PARITY_BLOCKS= " + NUM_PARITY_BLOCKS);
@@ -101,6 +98,32 @@ public class TestDFSStripedOutputStreamWithFailure {
     return lengths;
   }
 
+  private static final int[][] dnIndexSuite = {
+      {0, 1},
+      {0, 5},
+      {0, 6},
+      {0, 8},
+      {1, 5},
+      {1, 6},
+      {6, 8},
+      {0, 1, 2},
+      {3, 4, 5},
+      {0, 1, 6},
+      {0, 5, 6},
+      {0, 5, 8},
+      {0, 6, 7},
+      {5, 6, 7},
+      {6, 7, 8},
+  };
+
+  private int[] getKillPositions(int fileLen, int num) {
+    int[] positions = new int[num];
+    for (int i = 0; i < num; i++) {
+      positions[i] = fileLen * (i + 1) / (num + 1);
+    }
+    return positions;
+  }
+
   private static final List<Integer> LENGTHS = newLengths();
 
   static int getLength(int i) {
@@ -127,42 +150,26 @@ public class TestDFSStripedOutputStreamWithFailure {
     }
   }
 
-  private static byte getByte(long pos) {
-    return (byte)pos;
-  }
-
   private HdfsConfiguration newHdfsConfiguration() {
     final HdfsConfiguration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
+    conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     return conf;
   }
 
-  void runTest(final int length) {
-    final HdfsConfiguration conf = newHdfsConfiguration();
-    for (int dn = 0; dn < 9; dn++) {
-      try {
-        setup(conf);
-        runTest(length, dn, false, conf);
-      } catch (Exception e) {
-        final String err = "failed, dn=" + dn + ", length=" + length
-            + StringUtils.stringifyException(e);
-        LOG.error(err);
-        Assert.fail(err);
-      } finally {
-        tearDown();
-      }
-    }
-  }
-
   @Test(timeout=240000)
   public void testDatanodeFailure56() throws Exception {
     runTest(getLength(56));
   }
 
   @Test(timeout=240000)
+  public void testMultipleDatanodeFailure56() throws Exception {
+    runTestWithMultipleFailure(getLength(56));
+  }
+
+  @Test(timeout=240000)
   public void testBlockTokenExpired() throws Exception {
     final int length = NUM_DATA_BLOCKS * (BLOCK_SIZE - CELL_SIZE);
     final HdfsConfiguration conf = newHdfsConfiguration();
@@ -174,7 +181,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     for (int dn = 0; dn < 9; dn += 2) {
       try {
         setup(conf);
-        runTest(length, dn, true, conf);
+        runTest(length, new int[]{length/2}, new int[]{dn}, true);
       } catch (Exception e) {
         LOG.error("failed, dn=" + dn + ", length=" + length);
         throw e;
@@ -214,22 +221,8 @@ public class TestDFSStripedOutputStreamWithFailure {
         Assert.fail("Failed to validate available dns against blkGroupSize");
       } catch (IOException ioe) {
         // expected
-        GenericTestUtils.assertExceptionContains("Failed: the number of "
-            + "remaining blocks = 5 < the number of data blocks = 6", ioe);
-        DFSStripedOutputStream dfsout = (DFSStripedOutputStream) out
-            .getWrappedStream();
-
-        // get leading streamer and verify the last exception
-        StripedDataStreamer datastreamer = dfsout.getStripedDataStreamer(0);
-        try {
-          datastreamer.getLastException().check(true);
-          Assert.fail("Failed to validate available dns against blkGroupSize");
-        } catch (IOException le) {
-          GenericTestUtils.assertExceptionContains(
-              "Failed to get datablocks number of nodes from"
-                  + " namenode: blockGroupSize= 9, blocks.length= "
-                  + numDatanodes, le);
-        }
+        GenericTestUtils.assertExceptionContains("Failed to get 6 nodes from" +
+            " namenode: blockGroupSize= 9, blocks.length= 5", ioe);
       }
     } finally {
       tearDown();
@@ -258,42 +251,73 @@ public class TestDFSStripedOutputStreamWithFailure {
       int fileLength = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE - 1000;
       final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
       DFSTestUtil.writeFile(dfs, srcPath, new String(expected));
+      LOG.info("writing finished. Seek and read the file to verify.");
       StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength);
     } finally {
       tearDown();
     }
   }
 
-  private void runTest(final int length, final int dnIndex,
-      final boolean tokenExpire, final HdfsConfiguration conf) {
-    try {
-      runTest(length, length/2, dnIndex, tokenExpire, conf);
-    } catch(Exception e) {
-      LOG.info("FAILED", e);
-      Assert.fail(StringUtils.stringifyException(e));
+  void runTest(final int length) {
+    final HdfsConfiguration conf = newHdfsConfiguration();
+    for (int dn = 0; dn < 9; dn++) {
+      try {
+        setup(conf);
+        runTest(length, new int[]{length/2}, new int[]{dn}, false);
+      } catch (Throwable e) {
+        final String err = "failed, dn=" + dn + ", length=" + length
+            + StringUtils.stringifyException(e);
+        LOG.error(err);
+        Assert.fail(err);
+      } finally {
+        tearDown();
+      }
     }
   }
 
-  private void runTest(final int length, final int killPos,
-      final int dnIndex, final boolean tokenExpire,
-      final HdfsConfiguration conf) throws Exception {
-    if (killPos <= FLUSH_POS) {
-      LOG.warn("killPos=" + killPos + " <= FLUSH_POS=" + FLUSH_POS
-          + ", length=" + length + ", dnIndex=" + dnIndex);
-      return; //skip test
+  void runTestWithMultipleFailure(final int length) throws Exception {
+    final HdfsConfiguration conf = newHdfsConfiguration();
+    for(int i=0;i<dnIndexSuite.length;i++){
+      int[] dnIndex = dnIndexSuite[i];
+      int[] killPos = getKillPositions(length, dnIndex.length);
+      try {
+        setup(conf);
+        runTest(length, killPos, dnIndex, false);
+      } catch (Throwable e) {
+        final String err = "failed, killPos=" + Arrays.toString(killPos)
+            + ", dnIndex=" + Arrays.toString(dnIndex) + ", length=" + length;
+        LOG.error(err);
+        throw e;
+      } finally {
+        tearDown();
+      }
     }
-    Preconditions.checkArgument(length > killPos,
-        "length=%s <= killPos=%s", length, killPos);
+  }
 
-    // start a datanode now, will kill one later
-    cluster.startDataNodes(conf, 1, true, null, null);
-    cluster.waitActive();
+  /**
+   * runTest implementation
+   * @param length file length
+   * @param killPos killing positions in ascending order
+   * @param dnIndex DN index to kill when meets killing positions
+   * @param tokenExpire wait token to expire when kill a DN
+   * @throws Exception
+   */
+  private void runTest(final int length, final int[] killPos,
+      final int[] dnIndex, final boolean tokenExpire) throws Exception {
+    if (killPos[0] <= FLUSH_POS) {
+      LOG.warn("killPos=" + Arrays.toString(killPos) + " <= FLUSH_POS=" + FLUSH_POS
+          + ", length=" + length + ", dnIndex=" + Arrays.toString(dnIndex));
+      return; //skip test
+    }
+    Preconditions.checkArgument(length > killPos[0], "length=%s <= killPos=%s",
+        length, killPos);
+    Preconditions.checkArgument(killPos.length == dnIndex.length);
 
-    final Path p = new Path(dir, "dn" + dnIndex + "len" + length + "kill" +  killPos);
+    final Path p = new Path(dir, "dn" + Arrays.toString(dnIndex)
+        + "len" + length + "kill" +  Arrays.toString(killPos));
     final String fullPath = p.toString();
     LOG.info("fullPath=" + fullPath);
 
-
     if (tokenExpire) {
       final NameNode nn = cluster.getNameNode();
       final BlockManager bm = nn.getNamesystem().getBlockManager();
@@ -308,50 +332,56 @@ public class TestDFSStripedOutputStreamWithFailure {
     final DFSStripedOutputStream stripedOut
         = (DFSStripedOutputStream)out.getWrappedStream();
 
-    long oldGS = -1;
-    boolean killed = false;
+    long firstGS = -1;  // first GS of this block group which never proceeds blockRecovery
+    long oldGS = -1; // the old GS before bumping
+    int numKilled=0;
     for(; pos.get() < length; ) {
       final int i = pos.getAndIncrement();
-      if (i == killPos) {
+      if (numKilled < killPos.length &&  i == killPos[numKilled]) {
+        assertTrue(firstGS != -1);
         final long gs = getGenerationStamp(stripedOut);
-        Assert.assertTrue(oldGS != -1);
-        Assert.assertEquals(oldGS, gs);
+        if (numKilled == 0) {
+          assertEquals(firstGS, gs);
+        } else {
+          //TODO: implement hflush/hsync and verify gs strict greater than oldGS
+          assertTrue(gs >= oldGS);
+        }
+        oldGS = gs;
 
         if (tokenExpire) {
           DFSTestUtil.flushInternal(stripedOut);
           waitTokenExpires(out);
         }
 
-        killDatanode(cluster, stripedOut, dnIndex, pos);
-        killed = true;
+        killDatanode(cluster, stripedOut, dnIndex[numKilled], pos);
+        numKilled++;
       }
 
       write(out, i);
 
-      if (i == FLUSH_POS) {
-        oldGS = getGenerationStamp(stripedOut);
+      if (i % BLOCK_GROUP_SIZE == FLUSH_POS) {
+        firstGS = getGenerationStamp(stripedOut);
+        oldGS = firstGS;
       }
     }
     out.close();
+    assertEquals(dnIndex.length, numKilled);
 
     short expectedReported = StripedFileTestUtil.getRealTotalBlockNum(length);
-    if (length > dnIndex * CELL_SIZE || dnIndex >= NUM_DATA_BLOCKS) {
-      expectedReported--;
+    for(int idx :dnIndex) {
+      if (length > idx * CELL_SIZE || idx >= NUM_DATA_BLOCKS) {
+        expectedReported--;
+      }
     }
     DFSTestUtil.waitReplication(dfs, p, expectedReported);
 
-    Assert.assertTrue(killed);
-
-    // check file length
-    final FileStatus status = dfs.getFileStatus(p);
-    Assert.assertEquals(length, status.getLen());
-
-    checkData(dfs, fullPath, length, dnIndex, oldGS);
+    cluster.triggerBlockReports();
+    StripedFileTestUtil.checkData(dfs, p, length, dnIndex, oldGS);
   }
 
   static void write(FSDataOutputStream out, int i) throws IOException {
     try {
-      out.write(getByte(i));
+      out.write(StripedFileTestUtil.getByte(i));
     } catch(IOException ioe) {
       throw new IOException("Failed at i=" + i, ioe);
     }
@@ -359,10 +389,10 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   static long getGenerationStamp(DFSStripedOutputStream out)
       throws IOException {
+    DFSTestUtil.flushBuffer(out);
     final long gs = DFSTestUtil.flushInternal(out).getGenerationStamp();
     LOG.info("getGenerationStamp returns " + gs);
     return gs;
-
   }
 
   static DatanodeInfo getDatanodes(StripedDataStreamer streamer) {
@@ -399,106 +429,6 @@ public class TestDFSStripedOutputStreamWithFailure {
     cluster.stopDataNode(datanode.getXferAddr());
   }
 
-  static void checkData(DistributedFileSystem dfs, String src, int length,
-      int killedDnIndex, long oldGS) throws IOException {
-    List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
-    LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(src, 0L);
-    final int expectedNumGroup = (length - 1)/BLOCK_GROUP_SIZE + 1;
-    Assert.assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size());
-
-    for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
-      Assert.assertTrue(firstBlock instanceof LocatedStripedBlock);
-
-      final long gs = firstBlock.getBlock().getGenerationStamp();
-      final String s = "gs=" + gs + ", oldGS=" + oldGS;
-      LOG.info(s);
-      Assert.assertTrue(s, gs >= oldGS);
-
-      LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
-          (LocatedStripedBlock) firstBlock,
-          CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
-      blockGroupList.add(Arrays.asList(blocks));
-    }
-
-    // test each block group
-    for (int group = 0; group < blockGroupList.size(); group++) {
-      final boolean isLastGroup = group == blockGroupList.size() - 1;
-      final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE
-          : length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE;
-      final int numCellInGroup = (groupSize - 1)/CELL_SIZE + 1;
-      final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS;
-      final int lastCellSize = groupSize - (numCellInGroup - 1)*CELL_SIZE;
-
-      //get the data of this block
-      List<LocatedBlock> blockList = blockGroupList.get(group);
-      byte[][] dataBlockBytes = new byte[NUM_DATA_BLOCKS][];
-      byte[][] parityBlockBytes = new byte[NUM_PARITY_BLOCKS][];
-
-      // for each block, use BlockReader to read data
-      for (int i = 0; i < blockList.size(); i++) {
-        final int j = i >= NUM_DATA_BLOCKS? 0: i;
-        final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS
-            + (j <= lastCellIndex? 1: 0);
-        final int blockSize = numCellInBlock*CELL_SIZE
-            + (isLastGroup && j == lastCellIndex? lastCellSize - CELL_SIZE: 0);
-
-        final byte[] blockBytes = new byte[blockSize];
-        if (i < NUM_DATA_BLOCKS) {
-          dataBlockBytes[i] = blockBytes;
-        } else {
-          parityBlockBytes[i - NUM_DATA_BLOCKS] = blockBytes;
-        }
-
-        final LocatedBlock lb = blockList.get(i);
-        LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock
-            + ", blockSize=" + blockSize + ", lb=" + lb);
-        if (lb == null) {
-          continue;
-        }
-        final ExtendedBlock block = lb.getBlock();
-        Assert.assertEquals(blockSize, block.getNumBytes());
-
-
-        if (block.getNumBytes() == 0) {
-          continue;
-        }
-
-        if (i != killedDnIndex) {
-          final BlockReader blockReader = BlockReaderTestUtil.getBlockReader(
-              dfs, lb, 0, block.getNumBytes());
-          blockReader.readAll(blockBytes, 0, (int) block.getNumBytes());
-          blockReader.close();
-        }
-      }
-
-      // check data
-      final int groupPosInFile = group*BLOCK_GROUP_SIZE;
-      for (int i = 0; i < dataBlockBytes.length; i++) {
-        final byte[] actual = dataBlockBytes[i];
-        for (int posInBlk = 0; posInBlk < actual.length; posInBlk++) {
-          final long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(
-              CELL_SIZE, NUM_DATA_BLOCKS, posInBlk, i) + groupPosInFile;
-          Assert.assertTrue(posInFile < length);
-          final byte expected = getByte(posInFile);
-
-          if (i == killedDnIndex) {
-            actual[posInBlk] = expected;
-          } else {
-            String s = "expected=" + expected + " but actual=" + actual[posInBlk]
-                + ", posInFile=" + posInFile + ", posInBlk=" + posInBlk
-                + ". group=" + group + ", i=" + i;
-            Assert.assertEquals(s, expected, actual[posInBlk]);
-          }
-        }
-      }
-
-      // check parity
-      TestDFSStripedOutputStream.verifyParity(dfs.getConf(),
-          lbs.getLocatedBlocks().get(group).getBlockSize(),
-          CELL_SIZE, dataBlockBytes, parityBlockBytes,
-          killedDnIndex - dataBlockBytes.length);
-    }
-  }
 
   private void waitTokenExpires(FSDataOutputStream out) throws IOException {
     Token<BlockTokenIdentifier> token = DFSTestUtil.getBlockToken(out);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
index c0dca4e..764527d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -39,6 +41,12 @@ public class TestWriteStripedFileWithFailure {
   private static MiniDFSCluster cluster;
   private static FileSystem fs;
   private static Configuration conf = new HdfsConfiguration();
+
+  static {
+    GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
+  }
+
   private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
   private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
   private final int smallFileLength = blockSize * dataBlocks - 123;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 124bf80..ef31527 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -1745,7 +1745,7 @@ public class TestBalancer {
 
       // verify locations of striped blocks
       LocatedBlocks locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
-      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
 
       // add one datanode
       String newRack = "/rack" + (++numOfRacks);
@@ -1761,7 +1761,7 @@ public class TestBalancer {
 
       // verify locations of striped blocks
       locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
-      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
     } finally {
       cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index 3a9748f..7cf5656 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@ -488,7 +488,7 @@ public class TestMover {
           Assert.assertEquals(StorageType.DISK, type);
         }
       }
-      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
           dataBlocks + parityBlocks);
 
       // start 5 more datanodes
@@ -523,7 +523,7 @@ public class TestMover {
           Assert.assertEquals(StorageType.ARCHIVE, type);
         }
       }
-      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
           dataBlocks + parityBlocks);
 
     }finally{

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
index 64d33a4..abcdbc1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
@@ -42,7 +41,6 @@ import java.util.Arrays;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 public class TestAddOverReplicatedStripedBlocks {
 
@@ -64,6 +62,7 @@ public class TestAddOverReplicatedStripedBlocks {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     // disable block recovery
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
     SimulatedFSDataset.setFactory(conf);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.waitActive();
@@ -118,7 +117,7 @@ public class TestAddOverReplicatedStripedBlocks {
     // verify that all internal blocks exists
     lbs = cluster.getNameNodeRpc().getBlockLocations(
         filePath.toString(), 0, fileLen);
-    DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE);
+    StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE);
   }
 
   @Test
@@ -162,7 +161,7 @@ public class TestAddOverReplicatedStripedBlocks {
     // verify that all internal blocks exists
     lbs = cluster.getNameNodeRpc().getBlockLocations(
         filePath.toString(), 0, fileLen);
-    DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1);
+    StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1);
   }
 
   @Test
@@ -216,7 +215,7 @@ public class TestAddOverReplicatedStripedBlocks {
     // verify that all internal blocks exists
     lbs = cluster.getNameNodeRpc().getBlockLocations(
         filePath.toString(), 0, fileLen);
-    DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE);
+    StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE);
   }
 
   @Test
@@ -248,6 +247,7 @@ public class TestAddOverReplicatedStripedBlocks {
 
     // update blocksMap
     cluster.triggerBlockReports();
+    Thread.sleep(2000);
     // add to invalidates
     cluster.triggerHeartbeats();
     // datanode delete block
@@ -259,7 +259,7 @@ public class TestAddOverReplicatedStripedBlocks {
     // we are left GROUP_SIZE - 1 blocks.
     lbs = cluster.getNameNodeRpc().getBlockLocations(
         filePath.toString(), 0, fileLen);
-    DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1);
+    StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index c27ead5..735f84d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -73,6 +73,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -736,7 +737,13 @@ public class TestRetryCacheWithHA {
       DatanodeInfo[] newNodes = new DatanodeInfo[2];
       newNodes[0] = nodes[0];
       newNodes[1] = nodes[1];
-      String[] storageIDs = {"s0", "s1"};
+      final DatanodeManager dm = cluster.getNamesystem(0).getBlockManager()
+          .getDatanodeManager();
+      final String storageID1 = dm.getDatanode(newNodes[0]).getStorageInfos()[0]
+          .getStorageID();
+      final String storageID2 = dm.getDatanode(newNodes[1]).getStorageInfos()[0]
+          .getStorageID();
+      String[] storageIDs = {storageID1, storageID2};
       
       client.getNamenode().updatePipeline(client.getClientName(), oldBlock,
           newBlock, newNodes, storageIDs);


[36/50] [abbrv] hadoop git commit: HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too verbose and hurt performance. Contributed by Rui Li

Posted by wa...@apache.org.
HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too verbose and hurt performance. Contributed by Rui Li


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

Branch: refs/heads/trunk
Commit: 7bff8ca1c872ea534a96cbbc5f70134574e289ce
Parents: 1080c37
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Sep 23 14:13:26 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue Sep 22 13:27:24 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 ++
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 15 +++++++--
 .../hadoop/hdfs/DFSStripedInputStream.java      | 34 ++++++++++++++++++++
 3 files changed, 49 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7bff8ca1/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 0e21d22..b79ce64 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -444,3 +444,6 @@
 
     HDFS-9091. Erasure Coding: Provide DistributedFilesystem API to 
     getAllErasureCodingPolicies. (Rakesh R via zhz)
+
+    HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too
+    verbose and hurt performance. (Rui Li via Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7bff8ca1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index a5911cc..385ba4b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1057,9 +1057,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       }
     }
     if (chosenNode == null) {
-      DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() +
-          " after checking nodes = " + Arrays.toString(nodes) +
-          ", ignoredNodes = " + ignoredNodes);
+      reportLostBlock(block, ignoredNodes);
       return null;
     }
     final String dnAddr =
@@ -1071,6 +1069,17 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     return new DNAddrPair(chosenNode, targetAddr, storageType);
   }
 
+  /**
+   * Warn the user of a lost block
+   */
+  protected void reportLostBlock(LocatedBlock lostBlock,
+      Collection<DatanodeInfo> ignoredNodes) {
+    DatanodeInfo[] nodes = lostBlock.getLocations();
+    DFSClient.LOG.warn("No live nodes contain block " + lostBlock.getBlock() +
+        " after checking nodes = " + Arrays.toString(nodes) +
+        ", ignoredNodes = " + ignoredNodes);
+  }
+
   private static String getBestNodeDNAddrPairErrorString(
       DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
       DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7bff8ca1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
index b7c22c4..131a552 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@ -45,8 +45,11 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Set;
 import java.util.Collection;
 import java.util.Map;
@@ -154,6 +157,17 @@ public class DFSStripedInputStream extends DFSInputStream {
   private StripeRange curStripeRange;
   private final CompletionService<Void> readingService;
 
+  /**
+   * When warning the user of a lost block in striping mode, we remember the
+   * dead nodes we've logged. All other striping blocks on these nodes can be
+   * considered lost too, and we don't want to log a warning for each of them.
+   * This is to prevent the log from being too verbose. Refer to HDFS-8920.
+   *
+   * To minimize the overhead, we only store the datanodeUuid in this set
+   */
+  private final Set<String> warnedNodes = Collections.newSetFromMap(
+      new ConcurrentHashMap<String, Boolean>());
+
   DFSStripedInputStream(DFSClient dfsClient, String src,
       boolean verifyChecksum, ErasureCodingPolicy ecPolicy,
       LocatedBlocks locatedBlocks) throws IOException {
@@ -527,6 +541,26 @@ public class DFSStripedInputStream extends DFSInputStream {
     }
   }
 
+  @Override
+  protected void reportLostBlock(LocatedBlock lostBlock,
+      Collection<DatanodeInfo> ignoredNodes) {
+    DatanodeInfo[] nodes = lostBlock.getLocations();
+    if (nodes != null && nodes.length > 0) {
+      List<String> dnUUIDs = new ArrayList<>();
+      for (DatanodeInfo node : nodes) {
+        dnUUIDs.add(node.getDatanodeUuid());
+      }
+      if (!warnedNodes.containsAll(dnUUIDs)) {
+        DFSClient.LOG.warn(Arrays.toString(nodes) + " are unavailable and " +
+            "all striping blocks on them are lost. " +
+            "IgnoredNodes = " + ignoredNodes);
+        warnedNodes.addAll(dnUUIDs);
+      }
+    } else {
+      super.reportLostBlock(lostBlock, ignoredNodes);
+    }
+  }
+
   /**
    * The reader for reading a complete {@link AlignedStripe}. Note that an
    * {@link AlignedStripe} may cross multiple stripes with cellSize width.


[35/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

Change-Id: I8511c4d64b0959e79129febc179845a3892fedcc


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

Branch: refs/heads/trunk
Commit: 1080c3730068177ddd10dc313890ac1f5dc58f1a
Parents: a9e6681 dfd807a
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Mon Sep 21 13:51:36 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue Sep 22 13:27:09 2015 -0700

----------------------------------------------------------------------
 .../main/resources/assemblies/hadoop-tools.xml  |    7 +
 .../server/AuthenticationToken.java             |    3 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |  205 +-
 .../src/main/conf/log4j.properties              |   17 +-
 .../org/apache/hadoop/conf/Configuration.java   |   11 +-
 .../fs/CommonConfigurationKeysPublic.java       |    6 +
 .../java/org/apache/hadoop/fs/FileUtil.java     |   14 +-
 .../main/java/org/apache/hadoop/fs/Trash.java   |    3 +-
 .../hadoop/fs/shell/CommandWithDestination.java |   33 +-
 .../apache/hadoop/fs/shell/CopyCommands.java    |   19 +-
 .../java/org/apache/hadoop/fs/shell/Delete.java |   68 +-
 .../apache/hadoop/ha/ActiveStandbyElector.java  |    4 +
 .../apache/hadoop/io/retry/RetryPolicies.java   |   13 +-
 .../main/java/org/apache/hadoop/ipc/Client.java |    9 +-
 .../hadoop/metrics2/impl/MetricsConfig.java     |    3 +
 .../metrics2/impl/MetricsSourceAdapter.java     |   12 +-
 .../hadoop/metrics2/impl/MetricsSystemImpl.java |   20 +-
 .../apache/hadoop/metrics2/package-info.java    |    4 +-
 .../apache/hadoop/metrics2/sink/StatsDSink.java |  218 ++
 .../apache/hadoop/security/SaslPlainServer.java |    2 +-
 .../security/authorize/AccessControlList.java   |    2 +-
 .../java/org/apache/hadoop/util/LineReader.java |   17 +-
 .../org/apache/hadoop/util/VersionInfo.java     |    2 +-
 .../src/main/resources/core-default.xml         |   11 +
 .../src/site/markdown/FileSystemShell.md        |    2 +-
 .../hadoop-common/src/site/markdown/Metrics.md  |    3 +
 .../java/org/apache/hadoop/cli/TestCLI.java     |    2 +-
 .../org/apache/hadoop/cli/util/CLICommand.java  |    5 +-
 .../org/apache/hadoop/cli/util/CLITestCmd.java  |    6 +-
 .../org/apache/hadoop/fs/TestFsShellCopy.java   |   46 +
 .../apache/hadoop/io/retry/TestRetryProxy.java  |   13 +-
 .../java/org/apache/hadoop/ipc/TestIPC.java     |   38 +-
 .../java/org/apache/hadoop/ipc/TestSaslRPC.java |    9 +-
 .../metrics2/impl/TestMetricsSystemImpl.java    |   13 +
 .../hadoop/metrics2/impl/TestStatsDMetrics.java |  122 +
 .../authorize/TestAccessControlList.java        |    9 +
 .../delegation/web/TestWebDelegationToken.java  |    7 +-
 .../src/test/resources/testConf.xml             |   22 +-
 .../dev-support/findbugsExcludeFile.xml         |    1 +
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  105 +
 .../hdfs/client/HdfsClientConfigKeys.java       |   14 +
 .../apache/hadoop/hdfs/net/BasicInetPeer.java   |  133 +
 .../apache/hadoop/hdfs/net/EncryptedPeer.java   |  142 ++
 .../org/apache/hadoop/hdfs/net/NioInetPeer.java |  136 +
 .../java/org/apache/hadoop/hdfs/net/Peer.java   |    8 +-
 .../hdfs/protocol/ClientDatanodeProtocol.java   |    7 +
 .../hadoop/hdfs/protocol/DatanodeInfo.java      |   47 +-
 .../hadoop/hdfs/protocol/HdfsConstants.java     |    8 +
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |   84 +
 .../protocol/datatransfer/IOStreamPair.java     |   37 +
 .../hdfs/protocol/datatransfer/PipelineAck.java |  243 ++
 .../datatransfer/TrustedChannelResolver.java    |   81 +
 .../sasl/DataEncryptionKeyFactory.java          |   38 +
 .../datatransfer/sasl/DataTransferSaslUtil.java |  519 ++++
 .../sasl/SaslDataTransferClient.java            |  498 ++++
 .../datatransfer/sasl/SaslParticipant.java      |  210 ++
 .../SaslResponseWithNegotiatedCipherOption.java |   33 +
 .../ClientDatanodeProtocolTranslatorPB.java     |   17 +
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 2186 ++++++++++++++++-
 .../apache/hadoop/hdfs/util/LongBitFormat.java  |   71 +
 .../hadoop/hdfs/web/ByteRangeInputStream.java   |   11 +
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |    3 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |    4 +-
 .../src/main/proto/ClientDatanodeProtocol.proto |   16 +
 .../src/main/proto/hdfs.proto                   |    1 +
 .../services/org.apache.hadoop.fs.FileSystem    |   17 +
 .../hadoop/hdfs/nfs/mount/RpcProgramMountd.java |    4 +-
 .../hadoop/hdfs/nfs/nfs3/DFSClientCache.java    |    4 +-
 .../apache/hadoop/hdfs/nfs/nfs3/TestWrites.java |    7 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  409 ++-
 .../hdfs/server/namenode/FSEditLogTestUtil.java |    3 +-
 .../main/java/org/apache/hadoop/fs/Hdfs.java    |    7 +-
 .../java/org/apache/hadoop/fs/SWebHdfs.java     |   15 +-
 .../main/java/org/apache/hadoop/fs/WebHdfs.java |   14 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   10 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   50 +-
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   10 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |    8 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |    4 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |   15 +-
 .../apache/hadoop/hdfs/net/BasicInetPeer.java   |  133 -
 .../hadoop/hdfs/net/DomainPeerServer.java       |    5 +
 .../apache/hadoop/hdfs/net/EncryptedPeer.java   |  142 --
 .../org/apache/hadoop/hdfs/net/NioInetPeer.java |  136 -
 .../org/apache/hadoop/hdfs/net/PeerServer.java  |    9 +-
 .../apache/hadoop/hdfs/net/TcpPeerServer.java   |   70 +-
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |   84 -
 .../protocol/datatransfer/IOStreamPair.java     |   37 -
 .../hdfs/protocol/datatransfer/PipelineAck.java |  274 ---
 .../hdfs/protocol/datatransfer/Receiver.java    |   33 +-
 .../datatransfer/TrustedChannelResolver.java    |   81 -
 .../sasl/DataEncryptionKeyFactory.java          |   38 -
 .../datatransfer/sasl/DataTransferSaslUtil.java |  519 ----
 .../sasl/SaslDataTransferClient.java            |  498 ----
 .../sasl/SaslDataTransferServer.java            |    2 +-
 .../datatransfer/sasl/SaslParticipant.java      |  210 --
 .../SaslResponseWithNegotiatedCipherOption.java |   33 -
 ...tDatanodeProtocolServerSideTranslatorPB.java |   22 +-
 .../protocolPB/ClientNamenodeProtocolPB.java    |    4 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |  135 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  128 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |   10 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   12 +-
 ...rDatanodeProtocolServerSideTranslatorPB.java |    2 +-
 .../protocolPB/JournalProtocolTranslatorPB.java |    2 +-
 .../NamenodeProtocolServerSideTranslatorPB.java |    3 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 2321 +-----------------
 .../QJournalProtocolTranslatorPB.java           |    3 +-
 .../hadoop/hdfs/qjournal/server/Journal.java    |   40 +-
 .../hdfs/qjournal/server/JournalMetrics.java    |    7 +-
 .../hadoop/hdfs/server/balancer/Balancer.java   |  170 +-
 .../server/balancer/BalancerParameters.java     |  168 ++
 .../server/blockmanagement/BlockManager.java    |   97 +-
 .../BlockPlacementPolicyWithNodeGroup.java      |   14 +-
 .../BlockStoragePolicySuite.java                |   13 +-
 .../server/blockmanagement/DatanodeManager.java |    4 +-
 .../blockmanagement/DecommissionManager.java    |    2 +-
 .../hdfs/server/common/HdfsServerConstants.java |    6 -
 .../hdfs/server/common/MetricsLoggerTask.java   |  174 ++
 .../hdfs/server/datanode/BPOfferService.java    |    3 +-
 .../hdfs/server/datanode/BPServiceActor.java    |    7 +-
 .../hdfs/server/datanode/BlockReceiver.java     |    7 +-
 .../hadoop/hdfs/server/datanode/DNConf.java     |   22 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  130 +-
 .../hdfs/server/datanode/DataNodeMXBean.java    |    9 +-
 .../hdfs/server/datanode/DataXceiver.java       |   11 +-
 .../hdfs/server/datanode/DataXceiverServer.java |    7 +-
 .../hdfs/server/datanode/FinalizedReplica.java  |   15 +-
 .../hdfs/server/datanode/ReplicaInfo.java       |   82 -
 .../server/datanode/ReplicaUnderRecovery.java   |   10 -
 .../datanode/ReplicaWaitingToBeRecovered.java   |   15 +-
 .../erasurecode/ErasureCodingWorker.java        |    3 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  |    8 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   16 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |   72 +-
 .../hdfs/server/namenode/BackupImage.java       |   10 +-
 .../hdfs/server/namenode/CacheManager.java      |    4 +-
 .../hadoop/hdfs/server/namenode/DfsServlet.java |    3 +-
 .../server/namenode/EditLogFileInputStream.java |   60 +-
 .../server/namenode/EncryptionZoneManager.java  |    4 +-
 .../server/namenode/FSDirStatAndListingOp.java  |    3 +-
 .../hdfs/server/namenode/FSDirXAttrOp.java      |    9 +-
 .../hdfs/server/namenode/FSDirectory.java       |  138 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   10 +
 .../hdfs/server/namenode/FSEditLogLoader.java   |   57 +-
 .../hdfs/server/namenode/FSEditLogOp.java       |   20 +-
 .../hadoop/hdfs/server/namenode/FSImage.java    |  125 -
 .../server/namenode/FSImageFormatPBINode.java   |    9 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   88 +-
 .../server/namenode/FSPermissionChecker.java    |   41 +-
 .../server/namenode/FileJournalManager.java     |   47 +-
 .../hadoop/hdfs/server/namenode/INode.java      |   14 +-
 .../hdfs/server/namenode/INodeDirectory.java    |   14 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   19 +-
 .../hadoop/hdfs/server/namenode/INodeMap.java   |    2 +-
 .../hdfs/server/namenode/INodeReference.java    |   10 +-
 .../hdfs/server/namenode/INodeSymlink.java      |    2 +-
 .../hdfs/server/namenode/ImageServlet.java      |    5 +-
 .../hdfs/server/namenode/LeaseManager.java      |    6 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  192 +-
 .../hdfs/server/namenode/NameNodeMXBean.java    |    6 +
 .../hdfs/server/namenode/NamenodeFsck.java      |    6 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |    3 +
 .../hadoop/hdfs/server/namenode/SafeMode.java   |    3 -
 .../hdfs/server/namenode/SecondaryNameNode.java |    2 +-
 .../server/namenode/ha/BootstrapStandby.java    |    5 +-
 .../hdfs/server/namenode/ha/EditLogTailer.java  |    2 +-
 .../namenode/ha/IPFailoverProxyProvider.java    |    4 +-
 .../snapshot/DirectorySnapshottableFeature.java |   16 +-
 .../snapshot/DirectoryWithSnapshotFeature.java  |    5 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |    6 +-
 .../hdfs/server/namenode/snapshot/Snapshot.java |    3 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |   42 +-
 .../hdfs/tools/DFSZKFailoverController.java     |    3 +-
 .../org/apache/hadoop/hdfs/tools/GetGroups.java |    4 +-
 .../hadoop/hdfs/tools/NNHAServiceTarget.java    |    3 +-
 .../apache/hadoop/hdfs/util/LongBitFormat.java  |   71 -
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |    3 +
 .../services/org.apache.hadoop.fs.FileSystem    |    2 -
 .../src/main/resources/hdfs-default.xml         |   46 +
 .../src/main/webapps/hdfs/dfshealth.html        |    7 +-
 .../src/main/webapps/hdfs/dfshealth.js          |    9 +-
 .../src/main/webapps/hdfs/explorer.html         |   76 +-
 .../src/main/webapps/hdfs/explorer.js           |   89 +-
 .../src/main/webapps/static/dfs-dust.js         |   12 +-
 .../src/main/webapps/static/hadoop.css          |   22 +
 .../src/site/markdown/ArchivalStorage.md        |    4 +-
 .../src/site/markdown/HDFSCommands.md           |    6 +-
 .../hadoop-hdfs/src/site/markdown/HdfsDesign.md |    6 +-
 .../src/site/markdown/HdfsRollingUpgrade.md     |  293 +++
 .../src/site/markdown/HdfsSnapshots.md          |  301 +++
 .../site/resources/images/LazyPersistWrites.png |  Bin 107161 -> 90224 bytes
 .../src/site/xdoc/HdfsRollingUpgrade.xml        |  329 ---
 .../hadoop-hdfs/src/site/xdoc/HdfsSnapshots.xml |  303 ---
 .../org/apache/hadoop/cli/CLITestCmdDFS.java    |    8 +-
 .../hadoop/cli/CLITestCmdErasureCoding.java     |    5 +-
 .../java/org/apache/hadoop/cli/TestAclCLI.java  |    2 +-
 .../apache/hadoop/cli/TestCacheAdminCLI.java    |    7 +-
 .../apache/hadoop/cli/TestCryptoAdminCLI.java   |    6 +-
 .../org/apache/hadoop/cli/TestDeleteCLI.java    |   92 +
 .../apache/hadoop/cli/TestErasureCodingCLI.java |    2 +-
 .../java/org/apache/hadoop/cli/TestHDFSCLI.java |    4 +-
 .../org/apache/hadoop/cli/TestXAttrCLI.java     |    2 +-
 .../hadoop/fs/permission/TestStickyBit.java     |    3 +
 .../apache/hadoop/hdfs/BlockReaderTestUtil.java |    3 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   24 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |    2 +-
 .../hadoop/hdfs/TestAppendSnapshotTruncate.java |    3 +-
 .../hadoop/hdfs/TestBalancerBandwidth.java      |   55 +-
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |   24 +-
 .../hadoop/hdfs/TestDFSClientFailover.java      |    5 +-
 .../apache/hadoop/hdfs/TestDFSPermission.java   |   42 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |   56 +
 .../hadoop/hdfs/TestDFSShellGenericOptions.java |   13 +-
 .../hadoop/hdfs/TestDefaultNameNodePort.java    |   30 +-
 .../hadoop/hdfs/TestEncryptedTransfer.java      |    4 +-
 .../org/apache/hadoop/hdfs/TestFileAppend.java  |   72 -
 .../org/apache/hadoop/hdfs/TestFileStatus.java  |    5 +-
 .../org/apache/hadoop/hdfs/TestGetBlocks.java   |    7 +-
 .../apache/hadoop/hdfs/TestPersistBlocks.java   |    5 +-
 .../java/org/apache/hadoop/hdfs/TestQuota.java  |   32 +
 .../hdfs/TestWriteStripedFileWithFailure.java   |    3 +
 .../sasl/SaslDataTransferTestCase.java          |    2 +-
 .../datatransfer/sasl/TestSaslDataTransfer.java |    2 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |   32 +-
 .../hdfs/qjournal/TestSecureNNWithQJM.java      |    2 +-
 .../hdfs/qjournal/server/TestJournalNode.java   |    9 +
 .../hdfs/server/balancer/TestBalancer.java      |  199 +-
 .../balancer/TestBalancerWithHANameNodes.java   |    8 +-
 .../TestBalancerWithMultipleNameNodes.java      |  177 +-
 .../balancer/TestBalancerWithNodeGroup.java     |    4 +-
 .../blockmanagement/TestBlockTokenWithDFS.java  |    4 +-
 .../blockmanagement/TestReplicationPolicy.java  |    3 -
 .../TestReplicationPolicyWithNodeGroup.java     |   19 +
 .../hdfs/server/datanode/DataNodeTestUtils.java |   86 +-
 .../server/datanode/SimulatedFSDataset.java     |    2 +-
 .../server/datanode/TestDataNodeMXBean.java     |    6 +
 .../datanode/TestDataNodeMetricsLogger.java     |  224 ++
 .../datanode/TestDataNodeRollingUpgrade.java    |    8 +-
 .../TestDataNodeTransferSocketSize.java         |   71 +
 .../datanode/TestDataNodeVolumeFailure.java     |    4 +-
 .../server/datanode/TestDirectoryScanner.java   |    2 +-
 .../datanode/extdataset/ExternalVolumeImpl.java |    2 +-
 .../fsdataset/impl/FsDatasetTestUtil.java       |    6 -
 .../fsdataset/impl/LazyPersistTestCase.java     |   16 +-
 .../fsdataset/impl/TestDatanodeRestart.java     |   72 -
 .../datanode/fsdataset/impl/TestLazyWriter.java |    1 +
 .../fsdataset/impl/TestRbwSpaceReservation.java |  452 ----
 .../fsdataset/impl/TestSpaceReservation.java    |  576 +++++
 .../server/namenode/NNThroughputBenchmark.java  |    3 +-
 .../hdfs/server/namenode/NameNodeAdapter.java   |    2 +-
 .../TestCheckPointForSecurityTokens.java        |    4 +-
 .../namenode/TestDiskspaceQuotaUpdate.java      |    9 +-
 .../hdfs/server/namenode/TestEditLog.java       |  103 +-
 .../server/namenode/TestFSEditLogLoader.java    |   13 +-
 .../namenode/TestFSImageWithSnapshot.java       |    3 +-
 .../hdfs/server/namenode/TestFSNamesystem.java  |   25 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |    5 +-
 .../hdfs/server/namenode/TestINodeFile.java     |    4 +-
 .../namenode/TestListCorruptFileBlocks.java     |    3 +-
 .../hdfs/server/namenode/TestMetaSave.java      |   11 +
 .../server/namenode/TestNameNodeMXBean.java     |   20 +
 .../namenode/TestNameNodeMetricsLogger.java     |    5 +-
 .../server/namenode/TestStorageRestore.java     |    3 +-
 .../server/namenode/TestStripedINodeFile.java   |    2 +-
 .../namenode/ha/TestFailureToReadEdits.java     |    3 +-
 .../TestGetContentSummaryWithSnapshot.java      |  126 +
 .../hdfs/web/TestByteRangeInputStream.java      |   79 +
 .../org/apache/hadoop/tools/TestJMXGet.java     |    3 +
 .../test/resources/hadoop-metrics2.properties   |   85 +
 .../src/test/resources/log4j.properties         |   13 +
 .../src/test/resources/testDeleteConf.xml       |   83 +
 .../src/test/resources/testHDFSConf.xml         |  198 +-
 hadoop-mapreduce-project/CHANGES.txt            |  101 +-
 hadoop-mapreduce-project/bin/mapred             |    8 +
 .../v2/app/job/impl/TaskAttemptImpl.java        |   92 +-
 .../v2/app/rm/RMContainerAllocator.java         |   12 +-
 .../v2/app/rm/RMContainerRequestor.java         |    2 +-
 .../apache/hadoop/mapreduce/v2/app/MRApp.java   |   11 +-
 .../v2/app/job/impl/TestTaskAttempt.java        |  154 ++
 .../v2/app/rm/TestRMContainerAllocator.java     |  113 +-
 .../org/apache/hadoop/mapreduce/Cluster.java    |    2 +-
 .../lib/input/UncompressedSplitLineReader.java  |   31 +-
 .../lib/output/FileOutputCommitter.java         |   52 +-
 .../hadoop/mapred/TestLineRecordReader.java     |  138 ++
 .../lib/input/TestLineRecordReader.java         |  161 ++
 .../org/apache/hadoop/mapred/YARNRunner.java    |    4 +
 .../org/apache/hadoop/cli/CLITestCmdMR.java     |    3 +-
 .../org/apache/hadoop/fs/TestFileSystem.java    |   14 +-
 .../java/org/apache/hadoop/hdfs/NNBench.java    |   14 +-
 .../apache/hadoop/mapred/TestYARNRunner.java    |   10 +-
 .../mapreduce/security/TestMRCredentials.java   |    4 +-
 .../apache/hadoop/mapred/ShuffleHandler.java    |  177 +-
 .../hadoop/mapred/TestShuffleHandler.java       |  129 +
 hadoop-project/pom.xml                          |    7 +-
 hadoop-tools/hadoop-archive-logs/pom.xml        |  171 ++
 .../apache/hadoop/tools/HadoopArchiveLogs.java  |  403 +++
 .../hadoop/tools/HadoopArchiveLogsRunner.java   |  180 ++
 .../hadoop/tools/TestHadoopArchiveLogs.java     |  293 +++
 .../tools/TestHadoopArchiveLogsRunner.java      |  143 ++
 .../tools/mapred/UniformSizeInputFormat.java    |    3 +-
 .../hadoop/tools/util/ProducerConsumer.java     |   19 +-
 .../apache/hadoop/tools/util/WorkReport.java    |    2 +-
 .../apache/hadoop/tools/util/WorkRequest.java   |    2 +-
 .../hadoop/mapred/gridmix/SerialJobFactory.java |    2 +-
 .../hadoop/yarn/sls/RumenToSLSConverter.java    |    2 +-
 hadoop-tools/hadoop-tools-dist/pom.xml          |    5 +
 hadoop-tools/pom.xml                            |    1 +
 hadoop-yarn-project/CHANGES.txt                 |  504 ++--
 .../MoveApplicationAcrossQueuesResponse.java    |    5 +-
 .../yarn/api/records/ApplicationReport.java     |   31 +-
 .../hadoop/yarn/api/records/NodeLabel.java      |   16 +
 .../hadoop/yarn/api/records/ResourceOption.java |    3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   37 +-
 .../hadoop/yarn/server/api/ContainerType.java   |    1 -
 .../ResourceManagerAdministrationProtocol.java  |   16 +-
 .../RefreshNodesResourcesRequest.java           |   39 +
 .../RefreshNodesResourcesResponse.java          |   39 +
 ...esourcemanager_administration_protocol.proto |    3 +-
 ..._server_resourcemanager_service_protos.proto |    6 +
 .../src/main/proto/yarn_protos.proto            |    2 +
 .../yarn/conf/TestYarnConfigurationFields.java  |    2 +
 .../hadoop-yarn/hadoop-yarn-client/pom.xml      |   12 +
 .../yarn/client/api/impl/AMRMClientImpl.java    |    2 +-
 .../impl/ContainerManagementProtocolProxy.java  |   14 +-
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   10 +-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |  104 +-
 .../apache/hadoop/yarn/client/cli/QueueCLI.java |   11 +-
 .../hadoop/yarn/client/cli/RMAdminCLI.java      |   61 +-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |   53 +-
 .../hadoop/yarn/client/cli/TestRMAdminCLI.java  |   31 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |    8 +-
 .../application_1440536969523_0001.har/_SUCCESS |    0
 .../application_1440536969523_0001.har/_index   |    3 +
 .../_masterindex                                |    2 +
 .../application_1440536969523_0001.har/part-0   |  Bin 0 -> 795 bytes
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |    4 +
 .../impl/pb/ApplicationReportPBImpl.java        |   38 +
 .../org/apache/hadoop/yarn/client/RMProxy.java  |    3 +-
 .../apache/hadoop/yarn/client/ServerProxy.java  |    3 +-
 .../yarn/logaggregation/LogCLIHelpers.java      |   16 +-
 .../nodelabels/CommonNodeLabelsManager.java     |    2 +-
 .../yarn/security/ContainerTokenSelector.java   |    2 +-
 .../hadoop/yarn/security/NMTokenSelector.java   |    2 +-
 ...nagerAdministrationProtocolPBClientImpl.java |   19 +
 ...agerAdministrationProtocolPBServiceImpl.java |   22 +
 .../pb/RefreshNodesResourcesRequestPBImpl.java  |   72 +
 .../pb/RefreshNodesResourcesResponsePBImpl.java |   72 +
 .../yarn/webapp/log/AggregatedLogsBlock.java    |    7 +
 .../src/main/resources/yarn-default.xml         |   63 +-
 .../hadoop/yarn/api/TestApplicatonReport.java   |    2 +-
 .../hadoop/yarn/api/TestPBImplRecords.java      |   16 +
 .../hadoop/yarn/event/TestAsyncDispatcher.java  |    2 +
 .../logaggregation/TestAggregatedLogsBlock.java |   63 +-
 .../hadoop/yarn/webapp/JerseyTestBase.java      |   14 +-
 .../application_1440536969523_0001.har/_SUCCESS |    0
 .../application_1440536969523_0001.har/_index   |    3 +
 .../_masterindex                                |    2 +
 .../application_1440536969523_0001.har/part-0   |  Bin 0 -> 795 bytes
 .../hadoop-yarn/hadoop-yarn-registry/pom.xml    |    2 +-
 ...pplicationHistoryManagerOnTimelineStore.java |   29 +-
 .../yarn/server/api/records/NodeStatus.java     |    2 +-
 .../api/records/impl/pb/NodeStatusPBImpl.java   |    8 +-
 .../metrics/ApplicationMetricsConstants.java    |    6 +
 .../server/utils/YarnServerSecurityUtils.java   |  142 ++
 .../hadoop/yarn/server/webapp/AppBlock.java     |    6 +
 .../hadoop/yarn/server/webapp/WebPageUtils.java |    3 +-
 .../hadoop/yarn/server/webapp/WebServices.java  |    9 +
 .../hadoop/yarn/server/webapp/dao/AppInfo.java  |   12 +
 .../nodemanager/LinuxContainerExecutor.java     |    2 +-
 .../yarn/server/nodemanager/NodeManager.java    |    2 +-
 .../nodemanager/NodeStatusUpdaterImpl.java      |   25 +-
 .../amrmproxy/AMRMProxyApplicationContext.java  |   70 +
 .../AMRMProxyApplicationContextImpl.java        |  132 +
 .../nodemanager/amrmproxy/AMRMProxyService.java |  592 +++++
 .../amrmproxy/AMRMProxyTokenSecretManager.java  |  265 ++
 .../amrmproxy/AbstractRequestInterceptor.java   |  102 +
 .../amrmproxy/DefaultRequestInterceptor.java    |  138 ++
 .../amrmproxy/RequestInterceptor.java           |   71 +
 .../containermanager/ContainerManagerImpl.java  |   73 +-
 .../localizer/LocalResourcesTrackerImpl.java    |   64 +-
 .../localizer/ResourceLocalizationService.java  |    2 +-
 .../logaggregation/AppLogAggregator.java        |    2 +
 .../logaggregation/AppLogAggregatorImpl.java    |    6 +
 .../logaggregation/LogAggregationService.java   |   14 +-
 .../nodelabels/AbstractNodeLabelsProvider.java  |    7 +-
 .../ConfigurationNodeLabelsProvider.java        |   11 -
 .../nodemanager/webapp/NMWebServices.java       |   12 +-
 .../nodemanager/webapp/dao/ContainerInfo.java   |   46 +-
 .../nodemanager/DummyContainerManager.java      |    7 +-
 .../yarn/server/nodemanager/TestEventFlow.java  |    3 +-
 .../nodemanager/TestNodeManagerResync.java      |    2 +-
 .../nodemanager/TestNodeStatusUpdater.java      |    6 +-
 .../amrmproxy/BaseAMRMProxyTest.java            |  677 +++++
 .../amrmproxy/MockRequestInterceptor.java       |   65 +
 .../amrmproxy/MockResourceManagerFacade.java    |  469 ++++
 .../PassThroughRequestInterceptor.java          |   58 +
 .../amrmproxy/TestAMRMProxyService.java         |  484 ++++
 .../BaseContainerManagerTest.java               |    3 +-
 .../containermanager/TestContainerManager.java  |    5 +-
 .../TestContainerManagerRecovery.java           |    3 +-
 .../containermanager/TestNMProxy.java           |    3 +-
 .../TestLocalResourcesTrackerImpl.java          |   86 +-
 .../localizer/TestResourceRetention.java        |    2 +-
 .../TestLogAggregationService.java              |   17 +-
 .../TestConfigurationNodeLabelsProvider.java    |   69 +-
 .../webapp/TestNMWebServicesContainers.java     |   13 +-
 .../server/resourcemanager/AdminService.java    |   81 +-
 .../ApplicationMasterService.java               |   69 +-
 .../server/resourcemanager/ClientRMService.java |    2 +-
 .../resourcemanager/EmbeddedElectorService.java |    4 +
 .../resourcemanager/NodesListManager.java       |  144 +-
 .../resourcemanager/RMActiveServiceContext.java |   16 +-
 .../server/resourcemanager/RMAppManager.java    |    9 +
 .../yarn/server/resourcemanager/RMContext.java  |    5 +
 .../server/resourcemanager/RMContextImpl.java   |   12 +-
 .../resourcemanager/RMFatalEventType.java       |    5 +-
 .../server/resourcemanager/ResourceManager.java |    4 +-
 .../resourcemanager/ResourceTrackerService.java |   20 +-
 .../blacklist/BlacklistManager.java             |   47 +
 .../blacklist/BlacklistUpdates.java             |   47 +
 .../blacklist/DisabledBlacklistManager.java     |   45 +
 .../blacklist/SimpleBlacklistManager.java       |   84 +
 .../metrics/ApplicationCreatedEvent.java        |   16 +-
 .../metrics/SystemMetricsPublisher.java         |   14 +-
 .../placement/PlacementManager.java             |   95 +
 .../placement/PlacementRule.java                |   55 +
 .../UserGroupMappingPlacementRule.java          |  164 ++
 .../resourcemanager/recovery/RMStateStore.java  |   15 +-
 .../resource/DynamicResourceConfiguration.java  |  149 ++
 .../server/resourcemanager/rmapp/RMApp.java     |    6 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |   59 +-
 .../rmapp/attempt/RMAppAttempt.java             |    7 +
 .../rmapp/attempt/RMAppAttemptImpl.java         |   58 +-
 .../server/resourcemanager/rmnode/RMNode.java   |    3 -
 .../resourcemanager/rmnode/RMNodeEventType.java |    2 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |  456 ++--
 .../scheduler/AbstractYarnScheduler.java        |    2 +
 .../scheduler/AppSchedulingInfo.java            |   78 +-
 .../scheduler/SchedulerApplicationAttempt.java  |   37 +-
 .../scheduler/YarnScheduler.java                |   13 +
 .../scheduler/capacity/CapacityScheduler.java   |  153 +-
 .../CapacitySchedulerConfiguration.java         |   32 +-
 .../scheduler/capacity/LeafQueue.java           |    6 +-
 .../common/fica/FiCaSchedulerUtils.java         |   48 -
 .../scheduler/fair/FSAppAttempt.java            |   19 +-
 .../scheduler/fair/FairScheduler.java           |   37 +-
 .../fair/FairSchedulerConfiguration.java        |   22 +-
 .../scheduler/fifo/FifoScheduler.java           |   11 +-
 .../scheduler/policy/OrderingPolicy.java        |    2 +-
 .../resourcemanager/webapp/AboutBlock.java      |    2 +
 .../webapp/CapacitySchedulerPage.java           |   17 +-
 .../webapp/JAXBContextResolver.java             |    2 +-
 .../webapp/MetricsOverviewTable.java            |    3 +
 .../resourcemanager/webapp/NodeLabelsPage.java  |    6 +-
 .../resourcemanager/webapp/RMAppBlock.java      |    3 +-
 .../server/resourcemanager/webapp/RMWebApp.java |    5 +
 .../resourcemanager/webapp/RMWebAppFilter.java  |    5 +-
 .../resourcemanager/webapp/RMWebServices.java   |  123 +-
 .../webapp/dao/AppAttemptInfo.java              |   10 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |   29 +-
 .../resourcemanager/webapp/dao/AppPriority.java |   44 +
 .../dao/CapacitySchedulerLeafQueueInfo.java     |   14 +-
 .../resourcemanager/webapp/dao/ClusterInfo.java |    6 +
 .../webapp/dao/SchedulerInfo.java               |    6 +
 .../yarn/server/resourcemanager/MockRM.java     |   20 +-
 .../server/resourcemanager/RMHATestBase.java    |    2 +-
 .../server/resourcemanager/TestAppManager.java  |   54 +-
 .../resourcemanager/TestClientRMService.java    |  295 +--
 .../resourcemanager/TestRMAdminService.java     |   46 +
 .../yarn/server/resourcemanager/TestRMHA.java   |   71 +
 .../resourcemanager/TestRMNodeTransitions.java  |  202 +-
 .../TestTokenClientRMService.java               |  300 +++
 .../TestWorkPreservingRMRestart.java            |   21 +-
 .../applicationsmanager/MockAsm.java            |   10 +
 .../applicationsmanager/TestAMRestart.java      |  149 +-
 .../blacklist/TestBlacklistManager.java         |  118 +
 .../TestRMAppLogAggregationStatus.java          |    2 +-
 .../metrics/TestSystemMetricsPublisher.java     |   20 +-
 .../TestUserGroupMappingPlacementRule.java      |   89 +
 .../server/resourcemanager/rmapp/MockRMApp.java |   13 +
 .../rmapp/TestNodesListManager.java             |  102 +
 .../rmapp/TestRMAppTransitions.java             |    9 +-
 .../scheduler/TestAbstractYarnScheduler.java    |   14 +-
 .../capacity/TestCapacityScheduler.java         |  147 +-
 .../scheduler/capacity/TestQueueMappings.java   |  203 +-
 .../scheduler/fair/FairSchedulerTestBase.java   |    8 +-
 .../scheduler/fair/TestFairScheduler.java       |  120 +-
 .../scheduler/fifo/TestFifoScheduler.java       |    2 +-
 .../security/TestRMDelegationTokens.java        |    3 +
 .../resourcemanager/webapp/TestNodesPage.java   |    5 +-
 .../webapp/TestRMWebServices.java               |    8 +-
 .../webapp/TestRMWebServicesApps.java           |   34 +-
 .../TestRMWebServicesAppsModification.java      |  120 +
 .../webapp/TestRMWebServicesCapacitySched.java  |    2 +-
 .../TestRMWebServicesDelegationTokens.java      |    6 +-
 .../hadoop/yarn/server/MiniYARNCluster.java     |   42 +-
 .../src/site/markdown/CapacityScheduler.md      |    6 +-
 .../src/site/markdown/ResourceManagerRest.md    |  152 +-
 .../src/site/markdown/TimelineServer.md         |   38 +-
 500 files changed, 21586 insertions(+), 9191 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
index 0000000,23e8f57..2121dcf
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
@@@ -1,0 -1,83 +1,84 @@@
+ /**
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hdfs.protocol;
+ 
+ import java.net.URI;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.fs.FileEncryptionInfo;
+ import org.apache.hadoop.fs.LocatedFileStatus;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hdfs.DFSUtilClient;
+ 
+ /** 
+  * Interface that represents the over the wire information
+  * including block locations for a file.
+  */
+ @InterfaceAudience.Private
+ @InterfaceStability.Evolving
+ public class HdfsLocatedFileStatus extends HdfsFileStatus {
+   private final LocatedBlocks locations;
+ 
+   /**
+    * Constructor
+    * 
+    * @param length size
+    * @param isdir if this is directory
+    * @param block_replication the file's replication factor
+    * @param blocksize the file's block size
+    * @param modification_time most recent modification time
+    * @param access_time most recent access time
+    * @param permission permission
+    * @param owner owner
+    * @param group group
+    * @param symlink symbolic link
+    * @param path local path name in java UTF8 format 
+    * @param fileId the file id
+    * @param locations block locations
+    * @param feInfo file encryption info
+    */
+   public HdfsLocatedFileStatus(long length, boolean isdir,
+       int block_replication, long blocksize, long modification_time,
+       long access_time, FsPermission permission, String owner, String group,
+       byte[] symlink, byte[] path, long fileId, LocatedBlocks locations,
 -      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy) {
++      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy,
++      ErasureCodingPolicy ecPolicy) {
+     super(length, isdir, block_replication, blocksize, modification_time,
+         access_time, permission, owner, group, symlink, path, fileId,
 -        childrenNum, feInfo, storagePolicy);
++        childrenNum, feInfo, storagePolicy, ecPolicy);
+     this.locations = locations;
+   }
+ 
+   public LocatedBlocks getBlockLocations() {
+     return locations;
+   }
+ 
+   public final LocatedFileStatus makeQualifiedLocated(URI defaultUri,
+       Path path) {
+     return new LocatedFileStatus(getLen(), isDir(), getReplication(),
+         getBlockSize(), getModificationTime(),
+         getAccessTime(),
+         getPermission(), getOwner(), getGroup(),
+         isSymlink() ? new Path(getSymlink()) : null,
+         (getFullPath(path)).makeQualified(
+             defaultUri, null), // fully-qualify path
+         DFSUtilClient.locatedBlocks2Locations(getBlockLocations()));
+   }
+ }


[18/50] [abbrv] hadoop git commit: HDFS-8978. Erasure coding: fix 2 failed tests of DFSStripedOutputStream. Contributed by Walter Su.

Posted by wa...@apache.org.
HDFS-8978. Erasure coding: fix 2 failed tests of DFSStripedOutputStream. Contributed by Walter Su.


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

Branch: refs/heads/trunk
Commit: 60bd765ac1b7d21189efc47058c00ff956a2dc86
Parents: ddf4e78
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Sep 2 17:41:08 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Sep 2 17:41:08 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 ++
 .../apache/hadoop/hdfs/StripedDataStreamer.java |  3 +-
 .../apache/hadoop/hdfs/StripedFileTestUtil.java | 57 ++++++++++++++++++++
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |  1 +
 .../hadoop/hdfs/TestDFSStripedOutputStream.java | 32 +++++------
 .../TestDFSStripedOutputStreamWithFailure.java  |  9 ++++
 .../hdfs/TestReadStripedFileWithDecoding.java   |  9 +++-
 .../TestReadStripedFileWithMissingBlocks.java   | 18 ++++---
 .../hadoop/hdfs/TestRecoverStripedFile.java     | 26 +++------
 .../hadoop/hdfs/TestWriteReadStripedFile.java   | 35 ++++++------
 10 files changed, 130 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/60bd765a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index fb464bf..9fdf3ed 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -409,3 +409,6 @@
 
     HDFS-8937. Erasure coding: do not throw exception when setting replication on
     EC file. (Gao Rui via jing9)
+
+    HDFS-8978. Erasure coding: fix 2 failed tests of DFSStripedOutputStream.
+    (Walter Su via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60bd765a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
index a20caa5..a2b6a67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -67,7 +67,7 @@ public class StripedDataStreamer extends DataStreamer {
       this.queue = queue;
     }
 
-    T poll(final int i) throws InterruptedIOException {
+    T poll(final int i) throws IOException {
       for(;;) {
         synchronized(queue) {
           final T polled = queue.poll(i);
@@ -80,6 +80,7 @@ public class StripedDataStreamer extends DataStreamer {
               return queue.poll(i);
             } catch(IOException ioe) {
               LOG.warn("Failed to populate, " + this, ioe);
+              throw ioe;
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60bd765a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
index ca4b2aa..ded86e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import com.google.common.base.Joiner;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -25,6 +26,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.web.ByteRangeInputStream;
 import org.junit.Assert;
 
@@ -32,8 +35,11 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Random;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+
 public class StripedFileTestUtil {
   public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class);
 
@@ -225,6 +231,57 @@ public class StripedFileTestUtil {
   }
 
   /**
+   * If the length of blockGroup is less than a full stripe, it returns the the
+   * number of actual data internal blocks. Otherwise returns NUM_DATA_BLOCKS.
+   */
+  public static short getRealDataBlockNum(int numBytes) {
+    return (short) Math.min(dataBlocks,
+        (numBytes - 1) / BLOCK_STRIPED_CELL_SIZE + 1);
+  }
+
+  public static short getRealTotalBlockNum(int numBytes) {
+    return (short) (getRealDataBlockNum(numBytes) + parityBlocks);
+  }
+
+  /**
+   * Wait for all the internalBlocks of the blockGroups of the given file to be reported.
+   */
+  public static void waitBlockGroupsReported(DistributedFileSystem fs, String src)
+      throws IOException, InterruptedException, TimeoutException {
+    boolean success;
+    final int ATTEMPTS = 40;
+    int count = 0;
+
+    do {
+      success = true;
+      count++;
+      LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0);
+      for (LocatedBlock lb : lbs.getLocatedBlocks()) {
+        short expected = getRealTotalBlockNum((int) lb.getBlockSize());
+        int reported = lb.getLocations().length;
+        if (reported != expected){
+          success = false;
+          System.out.println("blockGroup " + lb.getBlock() + " of file " + src
+              + " has reported internalBlocks " + reported
+              + " (desired " + expected + "); locations "
+              + Joiner.on(' ').join(lb.getLocations()));
+          Thread.sleep(1000);
+          break;
+        }
+      }
+      if (success) {
+        System.out.println("All blockGroups of file " + src
+            + " verified to have all internalBlocks.");
+      }
+    } while (!success && count < ATTEMPTS);
+
+    if (count == ATTEMPTS) {
+      throw new TimeoutException("Timed out waiting for " + src +
+          " to have all the internalBlocks");
+    }
+  }
+
+  /**
    * Generate n random and different numbers within
    * specified non-negative integer range
    * @param min minimum of the range

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60bd765a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
index ed3c110..a09f0f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
@@ -70,6 +70,7 @@ public class TestDFSStripedInputStream {
   @Before
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     SimulatedFSDataset.setFactory(conf);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
         DATA_BLK_NUM + PARITY_BLK_NUM).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60bd765a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index 35e7e6d..a467f40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -66,6 +66,7 @@ public class TestDFSStripedOutputStream {
     int numDNs = dataBlocks + parityBlocks + 2;
     conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
     fs = cluster.getFileSystem();
@@ -79,76 +80,76 @@ public class TestDFSStripedOutputStream {
   }
 
   @Test
-  public void testFileEmpty() throws IOException {
+  public void testFileEmpty() throws Exception {
     testOneFile("/EmptyFile", 0);
   }
 
   @Test
-  public void testFileSmallerThanOneCell1() throws IOException {
+  public void testFileSmallerThanOneCell1() throws Exception {
     testOneFile("/SmallerThanOneCell", 1);
   }
 
   @Test
-  public void testFileSmallerThanOneCell2() throws IOException {
+  public void testFileSmallerThanOneCell2() throws Exception {
     testOneFile("/SmallerThanOneCell", cellSize - 1);
   }
 
   @Test
-  public void testFileEqualsWithOneCell() throws IOException {
+  public void testFileEqualsWithOneCell() throws Exception {
     testOneFile("/EqualsWithOneCell", cellSize);
   }
 
   @Test
-  public void testFileSmallerThanOneStripe1() throws IOException {
+  public void testFileSmallerThanOneStripe1() throws Exception {
     testOneFile("/SmallerThanOneStripe", cellSize * dataBlocks - 1);
   }
 
   @Test
-  public void testFileSmallerThanOneStripe2() throws IOException {
+  public void testFileSmallerThanOneStripe2() throws Exception {
     testOneFile("/SmallerThanOneStripe", cellSize + 123);
   }
 
   @Test
-  public void testFileEqualsWithOneStripe() throws IOException {
+  public void testFileEqualsWithOneStripe() throws Exception {
     testOneFile("/EqualsWithOneStripe", cellSize * dataBlocks);
   }
 
   @Test
-  public void testFileMoreThanOneStripe1() throws IOException {
+  public void testFileMoreThanOneStripe1() throws Exception {
     testOneFile("/MoreThanOneStripe1", cellSize * dataBlocks + 123);
   }
 
   @Test
-  public void testFileMoreThanOneStripe2() throws IOException {
+  public void testFileMoreThanOneStripe2() throws Exception {
     testOneFile("/MoreThanOneStripe2", cellSize * dataBlocks
             + cellSize * dataBlocks + 123);
   }
 
   @Test
-  public void testFileLessThanFullBlockGroup() throws IOException {
+  public void testFileLessThanFullBlockGroup() throws Exception {
     testOneFile("/LessThanFullBlockGroup",
         cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize);
   }
 
   @Test
-  public void testFileFullBlockGroup() throws IOException {
+  public void testFileFullBlockGroup() throws Exception {
     testOneFile("/FullBlockGroup", blockSize * dataBlocks);
   }
 
   @Test
-  public void testFileMoreThanABlockGroup1() throws IOException {
+  public void testFileMoreThanABlockGroup1() throws Exception {
     testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123);
   }
 
   @Test
-  public void testFileMoreThanABlockGroup2() throws IOException {
+  public void testFileMoreThanABlockGroup2() throws Exception {
     testOneFile("/MoreThanABlockGroup2",
         blockSize * dataBlocks + cellSize+ 123);
   }
 
 
   @Test
-  public void testFileMoreThanABlockGroup3() throws IOException {
+  public void testFileMoreThanABlockGroup3() throws Exception {
     testOneFile("/MoreThanABlockGroup3",
         blockSize * dataBlocks * 3 + cellSize * dataBlocks
         + cellSize + 123);
@@ -167,12 +168,13 @@ public class TestDFSStripedOutputStream {
     return (byte) (pos % mod + 1);
   }
 
-  private void testOneFile(String src, int writeBytes) throws IOException {
+  private void testOneFile(String src, int writeBytes) throws Exception {
     src += "_" + writeBytes;
     Path testPath = new Path(src);
 
     byte[] bytes = generateBytes(writeBytes);
     DFSTestUtil.writeFile(fs, testPath, new String(bytes));
+    StripedFileTestUtil.waitBlockGroupsReported(fs, src);
 
     // check file length
     FileStatus status = fs.getFileStatus(testPath);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60bd765a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index e8e562b..2aa8484 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -56,6 +56,7 @@ import org.junit.Test;
 
 import com.google.common.base.Preconditions;
 
+
 public class TestDFSStripedOutputStreamWithFailure {
   public static final Log LOG = LogFactory.getLog(
       TestDFSStripedOutputStreamWithFailure.class);
@@ -135,6 +136,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     return conf;
   }
 
@@ -331,6 +333,13 @@ public class TestDFSStripedOutputStreamWithFailure {
       }
     }
     out.close();
+
+    short expectedReported = StripedFileTestUtil.getRealTotalBlockNum(length);
+    if (length > dnIndex * CELL_SIZE || dnIndex >= NUM_DATA_BLOCKS) {
+      expectedReported--;
+    }
+    DFSTestUtil.waitReplication(dfs, p, expectedReported);
+
     Assert.assertTrue(killed);
 
     // check file length

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60bd765a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
index cb2ec11..9048fbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -62,6 +63,9 @@ public class TestReadStripedFileWithDecoding {
 
   @Before
   public void setup() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
         .numDataNodes(numDNs).build();
     cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
@@ -80,7 +84,7 @@ public class TestReadStripedFileWithDecoding {
    * Verify the decoding works correctly.
    */
   @Test(timeout=300000)
-  public void testReadWithDNFailure() throws IOException {
+  public void testReadWithDNFailure() throws Exception {
     for (int fileLength : fileLengths) {
       for (int dnFailureNum : dnFailureNums) {
         try {
@@ -161,7 +165,7 @@ public class TestReadStripedFileWithDecoding {
   }
 
   private void testReadWithDNFailure(int fileLength, int dnFailureNum)
-      throws IOException {
+      throws Exception {
     String fileType = fileLength < (blockSize * dataBlocks) ?
         "smallFile" : "largeFile";
     String src = "/dnFailure_" + dnFailureNum + "_" + fileType;
@@ -172,6 +176,7 @@ public class TestReadStripedFileWithDecoding {
     Path testPath = new Path(src);
     final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
     DFSTestUtil.writeFile(fs, testPath, bytes);
+    StripedFileTestUtil.waitBlockGroupsReported(fs, src);
 
     // shut down the DN that holds an internal data block
     BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60bd765a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
index 38256ab..08a5f58 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
@@ -44,13 +44,14 @@ public class TestReadStripedFileWithMissingBlocks {
   public static final Log LOG = LogFactory
       .getLog(TestReadStripedFileWithMissingBlocks.class);
   private static MiniDFSCluster cluster;
-  private static FileSystem fs;
+  private static DistributedFileSystem fs;
   private static Configuration conf = new HdfsConfiguration();
   private final int fileLength = blockSize * dataBlocks + 123;
 
   @Before
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
     fs = cluster.getFileSystem();
@@ -64,42 +65,43 @@ public class TestReadStripedFileWithMissingBlocks {
   }
 
   @Test
-  public void testReadFileWithMissingBlocks1() throws IOException {
+  public void testReadFileWithMissingBlocks1() throws Exception {
     readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 0);
   }
 
   @Test
-  public void testReadFileWithMissingBlocks2() throws IOException {
+  public void testReadFileWithMissingBlocks2() throws Exception {
     readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 1);
   }
 
   @Test
-  public void testReadFileWithMissingBlocks3() throws IOException {
+  public void testReadFileWithMissingBlocks3() throws Exception {
     readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 2);
   }
 
   @Test
-  public void testReadFileWithMissingBlocks4() throws IOException {
+  public void testReadFileWithMissingBlocks4() throws Exception {
     readFileWithMissingBlocks(new Path("/foo"), fileLength, 2, 0);
   }
 
   @Test
-  public void testReadFileWithMissingBlocks5() throws IOException {
+  public void testReadFileWithMissingBlocks5() throws Exception {
     readFileWithMissingBlocks(new Path("/foo"), fileLength, 2, 1);
   }
 
   @Test
-  public void testReadFileWithMissingBlocks6() throws IOException {
+  public void testReadFileWithMissingBlocks6() throws Exception {
     readFileWithMissingBlocks(new Path("/foo"), fileLength, 3, 0);
   }
 
   private void readFileWithMissingBlocks(Path srcPath, int fileLength,
       int missingDataNum, int missingParityNum)
-      throws IOException {
+      throws Exception {
     LOG.info("readFileWithMissingBlocks: (" + missingDataNum + ","
         + missingParityNum + ")");
     final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
     DFSTestUtil.writeFile(fs, srcPath, new String(expected));
+    StripedFileTestUtil.waitBlockGroupsReported(fs, srcPath.toUri().getPath());
     StripedFileTestUtil.verifyLength(fs, srcPath, fileLength);
     int dataBlocks = (fileLength - 1) / cellSize + 1;
     BlockLocation[] locs = fs.getFileBlockLocations(srcPath, 0, cellSize);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60bd765a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
index 5c17359..7a0851f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
@@ -182,9 +182,12 @@ public class TestRecoverStripedFile {
     }
     
     Path file = new Path(fileName);
-    
-    testCreateStripedFile(file, fileLen);
-    
+
+    final byte[] data = new byte[fileLen];
+    ThreadLocalRandom.current().nextBytes(data);
+    DFSTestUtil.writeFile(fs, file, data);
+    StripedFileTestUtil.waitBlockGroupsReported(fs, fileName);
+
     LocatedBlocks locatedBlocks = getLocatedBlocks(file);
     assertEquals(locatedBlocks.getFileLength(), fileLen);
     
@@ -380,21 +383,4 @@ public class TestRecoverStripedFile {
   private LocatedBlocks getLocatedBlocks(Path file) throws IOException {
     return fs.getClient().getLocatedBlocks(file.toString(), 0, Long.MAX_VALUE);
   }
-  
-  private void testCreateStripedFile(Path file, int dataLen)
-      throws IOException {
-    final byte[] data = new byte[dataLen];
-    ThreadLocalRandom.current().nextBytes(data);
-    writeContents(file, data);
-  }
-  
-  void writeContents(Path file, byte[] contents)
-      throws IOException {
-    FSDataOutputStream out = fs.create(file);
-    try {
-      out.write(contents, 0, contents.length);
-    } finally {
-      out.close();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60bd765a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
index 810edb2..b383c85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
@@ -45,7 +45,7 @@ import static org.apache.hadoop.hdfs.StripedFileTestUtil.stripesPerBlock;
 public class TestWriteReadStripedFile {
   public static final Log LOG = LogFactory.getLog(TestWriteReadStripedFile.class);
   private static MiniDFSCluster cluster;
-  private static FileSystem fs;
+  private static DistributedFileSystem fs;
   private static Configuration conf = new HdfsConfiguration();
 
   static {
@@ -69,32 +69,32 @@ public class TestWriteReadStripedFile {
   }
 
   @Test
-  public void testFileEmpty() throws IOException {
+  public void testFileEmpty() throws Exception {
     testOneFileUsingDFSStripedInputStream("/EmptyFile", 0);
     testOneFileUsingDFSStripedInputStream("/EmptyFile2", 0, true);
   }
 
   @Test
-  public void testFileSmallerThanOneCell1() throws IOException {
+  public void testFileSmallerThanOneCell1() throws Exception {
     testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1);
     testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell2", 1, true);
   }
 
   @Test
-  public void testFileSmallerThanOneCell2() throws IOException {
+  public void testFileSmallerThanOneCell2() throws Exception {
     testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1);
     testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell2", cellSize - 1,
         true);
   }
 
   @Test
-  public void testFileEqualsWithOneCell() throws IOException {
+  public void testFileEqualsWithOneCell() throws Exception {
     testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize);
     testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell2", cellSize, true);
   }
 
   @Test
-  public void testFileSmallerThanOneStripe1() throws IOException {
+  public void testFileSmallerThanOneStripe1() throws Exception {
     testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe",
         cellSize * dataBlocks - 1);
     testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe2",
@@ -102,7 +102,7 @@ public class TestWriteReadStripedFile {
   }
 
   @Test
-  public void testFileSmallerThanOneStripe2() throws IOException {
+  public void testFileSmallerThanOneStripe2() throws Exception {
     testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe",
         cellSize + 123);
     testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe2",
@@ -110,7 +110,7 @@ public class TestWriteReadStripedFile {
   }
 
   @Test
-  public void testFileEqualsWithOneStripe() throws IOException {
+  public void testFileEqualsWithOneStripe() throws Exception {
     testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe",
         cellSize * dataBlocks);
     testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe2",
@@ -118,7 +118,7 @@ public class TestWriteReadStripedFile {
   }
 
   @Test
-  public void testFileMoreThanOneStripe1() throws IOException {
+  public void testFileMoreThanOneStripe1() throws Exception {
     testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1",
         cellSize * dataBlocks + 123);
     testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe12",
@@ -126,7 +126,7 @@ public class TestWriteReadStripedFile {
   }
 
   @Test
-  public void testFileMoreThanOneStripe2() throws IOException {
+  public void testFileMoreThanOneStripe2() throws Exception {
     testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2",
         cellSize * dataBlocks + cellSize * dataBlocks + 123);
     testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe22",
@@ -134,7 +134,7 @@ public class TestWriteReadStripedFile {
   }
 
   @Test
-  public void testLessThanFullBlockGroup() throws IOException {
+  public void testLessThanFullBlockGroup() throws Exception {
     testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup",
         cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize);
     testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup2",
@@ -142,7 +142,7 @@ public class TestWriteReadStripedFile {
   }
 
   @Test
-  public void testFileFullBlockGroup() throws IOException {
+  public void testFileFullBlockGroup() throws Exception {
     testOneFileUsingDFSStripedInputStream("/FullBlockGroup",
         blockSize * dataBlocks);
     testOneFileUsingDFSStripedInputStream("/FullBlockGroup2",
@@ -150,7 +150,7 @@ public class TestWriteReadStripedFile {
   }
 
   @Test
-  public void testFileMoreThanABlockGroup1() throws IOException {
+  public void testFileMoreThanABlockGroup1() throws Exception {
     testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1",
         blockSize * dataBlocks + 123);
     testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup12",
@@ -158,7 +158,7 @@ public class TestWriteReadStripedFile {
   }
 
   @Test
-  public void testFileMoreThanABlockGroup2() throws IOException {
+  public void testFileMoreThanABlockGroup2() throws Exception {
     testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2",
         blockSize * dataBlocks + cellSize + 123);
     testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup22",
@@ -167,7 +167,7 @@ public class TestWriteReadStripedFile {
 
 
   @Test
-  public void testFileMoreThanABlockGroup3() throws IOException {
+  public void testFileMoreThanABlockGroup3() throws Exception {
     testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3",
         blockSize * dataBlocks * 3 + cellSize * dataBlocks
             + cellSize + 123);
@@ -177,15 +177,16 @@ public class TestWriteReadStripedFile {
   }
 
   private void testOneFileUsingDFSStripedInputStream(String src, int fileLength)
-      throws IOException {
+      throws Exception {
     testOneFileUsingDFSStripedInputStream(src, fileLength, false);
   }
 
   private void testOneFileUsingDFSStripedInputStream(String src, int fileLength,
-      boolean withDataNodeFailure) throws IOException {
+      boolean withDataNodeFailure) throws Exception {
     final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
     Path srcPath = new Path(src);
     DFSTestUtil.writeFile(fs, srcPath, new String(expected));
+    StripedFileTestUtil.waitBlockGroupsReported(fs, src);
 
     StripedFileTestUtil.verifyLength(fs, srcPath, fileLength);
 


[27/50] [abbrv] hadoop git commit: HDFS-9086. Rename dfs.datanode.stripedread.threshold.millis to dfs.datanode.stripedread.timeout.millis. Contributed by Andrew Wang.

Posted by wa...@apache.org.
HDFS-9086. Rename dfs.datanode.stripedread.threshold.millis to dfs.datanode.stripedread.timeout.millis. Contributed by Andrew Wang.

Change-Id: Ice86c5d46d29e94087c0f186b941d5394e7ac0e6


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

Branch: refs/heads/trunk
Commit: 82a88b92b46911c1b33d11898f8f678a134a9b69
Parents: 607bda2
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Fri Sep 18 10:45:29 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Fri Sep 18 10:45:29 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt  |  3 +++
 .../main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java   |  4 ++--
 .../server/datanode/erasurecode/ErasureCodingWorker.java  | 10 +++++-----
 .../org/apache/hadoop/hdfs/util/StripedBlockUtil.java     |  6 +++---
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml       |  8 ++++----
 .../hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md    |  2 +-
 6 files changed, 18 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/82a88b92/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 3a29db1..8ff696b 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -432,3 +432,6 @@
 
     HDFS-9097. Erasure coding: update EC command "-s" flag to "-p" when 
     specifying policy. (zhz)
+
+    HDFS-9086. Rename dfs.datanode.stripedread.threshold.millis to 
+    dfs.datanode.stripedread.timeout.millis. (wang via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82a88b92/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index f7cda18..53c6cdb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -399,8 +399,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT = 20;
   public static final String  DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.stripedread.buffer.size";
   public static final int     DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 64 * 1024;
-  public static final String  DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_KEY = "dfs.datanode.stripedread.threshold.millis";
-  public static final int     DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT = 5000; //5s
+  public static final String  DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY = "dfs.datanode.stripedread.timeout.millis";
+  public static final int     DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT = 5000; //5s
   public static final String  DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY = "dfs.datanode.striped.blockrecovery.threads.size";
   public static final int     DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT = 8;
   public static final String  DFS_DATANODE_DNS_INTERFACE_KEY = "dfs.datanode.dns.interface";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82a88b92/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
index 56b54f1..d1fa478 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@ -95,16 +95,16 @@ public final class ErasureCodingWorker {
 
   private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL;
   private ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
-  private final int STRIPED_READ_THRESHOLD_MILLIS;
+  private final int STRIPED_READ_TIMEOUT_MILLIS;
   private final int STRIPED_READ_BUFFER_SIZE;
 
   public ErasureCodingWorker(Configuration conf, DataNode datanode) {
     this.datanode = datanode;
     this.conf = conf;
 
-    STRIPED_READ_THRESHOLD_MILLIS = conf.getInt(
-        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_KEY,
-        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT);
+    STRIPED_READ_TIMEOUT_MILLIS = conf.getInt(
+        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY,
+        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
     initializeStripedReadThreadPool(conf.getInt(
         DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_KEY, 
         DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT));
@@ -556,7 +556,7 @@ public final class ErasureCodingWorker {
         try {
           StripingChunkReadResult result =
               StripedBlockUtil.getNextCompletedStripedRead(
-                  readService, futures, STRIPED_READ_THRESHOLD_MILLIS);
+                  readService, futures, STRIPED_READ_TIMEOUT_MILLIS);
           int resultIndex = -1;
           if (result.state == StripingChunkReadResult.SUCCESSFUL) {
             resultIndex = result.index;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82a88b92/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
index 4fded73..897b092 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@ -203,12 +203,12 @@ public class StripedBlockUtil {
    */
   public static StripingChunkReadResult getNextCompletedStripedRead(
       CompletionService<Void> readService, Map<Future<Void>, Integer> futures,
-      final long threshold) throws InterruptedException {
+      final long timeoutMillis) throws InterruptedException {
     Preconditions.checkArgument(!futures.isEmpty());
     Future<Void> future = null;
     try {
-      if (threshold > 0) {
-        future = readService.poll(threshold, TimeUnit.MILLISECONDS);
+      if (timeoutMillis > 0) {
+        future = readService.poll(timeoutMillis, TimeUnit.MILLISECONDS);
       } else {
         future = readService.take();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82a88b92/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 0db56dd..acbad07 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2403,23 +2403,23 @@
 </property>
 
 <property>
-  <name>dfs.datanode.stripedread.threshold.millis</name>
+  <name>dfs.datanode.stripedread.timeout.millis</name>
   <value>5000</value>
-  <description>datanode striped read threshold in millisecond.
+  <description>Datanode striped read timeout in milliseconds.
   </description>
 </property>
 
 <property>
   <name>dfs.datanode.stripedread.threads</name>
   <value>20</value>
-  <description>datanode striped read thread pool size.
+  <description>Number of threads used by the Datanode for background recovery work.
   </description>
 </property>
 
 <property>
   <name>dfs.datanode.stripedread.buffer.size</name>
   <value>262144</value>
-  <description>datanode striped read buffer size.
+  <description>Datanode striped read buffer size.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82a88b92/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 2323a6d..5b3aa34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -101,7 +101,7 @@ Deployment
 
   Erasure coding background recovery work on the DataNodes can also be tuned via the following configuration parameters:
 
-  1. `dfs.datanode.stripedread.threshold.millis` - Timeout for striped reads. Default value is 5000 ms.
+  1. `dfs.datanode.stripedread.timeout.millis` - Timeout for striped reads. Default value is 5000 ms.
   1. `dfs.datanode.stripedread.threads` - Number of concurrent reader threads. Default value is 20 threads.
   1. `dfs.datanode.stripedread.buffer.size` - Buffer size for reader service. Default value is 256KB.
 


[34/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index d921507,ae0a3f6..a7cacc7
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@@ -17,25 -17,150 +17,157 @@@
   */
  package org.apache.hadoop.hdfs.protocolPB;
  
+ import java.io.EOFException;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.EnumSet;
++import java.util.HashMap;
+ import java.util.List;
++import java.util.Map;
++import java.util.Set;
+ 
+ import com.google.common.base.Preconditions;
  import com.google.common.collect.Lists;
+ import com.google.common.primitives.Shorts;
  import com.google.protobuf.ByteString;
  import com.google.protobuf.CodedInputStream;
+ 
 -import static com.google.common.base.Preconditions.checkNotNull;
 -
+ import org.apache.hadoop.crypto.CipherOption;
+ import org.apache.hadoop.crypto.CipherSuite;
+ import org.apache.hadoop.crypto.CryptoProtocolVersion;
+ import org.apache.hadoop.fs.CacheFlag;
+ import org.apache.hadoop.fs.ContentSummary;
+ import org.apache.hadoop.fs.CreateFlag;
+ import org.apache.hadoop.fs.FileEncryptionInfo;
+ import org.apache.hadoop.fs.FsServerDefaults;
+ import org.apache.hadoop.fs.Path;
  import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.fs.XAttr;
+ import org.apache.hadoop.fs.XAttrSetFlag;
+ import org.apache.hadoop.fs.permission.AclEntry;
+ import org.apache.hadoop.fs.permission.AclEntryScope;
+ import org.apache.hadoop.fs.permission.AclEntryType;
+ import org.apache.hadoop.fs.permission.AclStatus;
+ import org.apache.hadoop.fs.permission.FsAction;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hdfs.DFSUtilClient;
+ import org.apache.hadoop.hdfs.inotify.Event;
+ import org.apache.hadoop.hdfs.inotify.EventBatch;
+ import org.apache.hadoop.hdfs.inotify.EventBatchList;
+ import org.apache.hadoop.hdfs.protocol.Block;
+ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
+ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+ import org.apache.hadoop.hdfs.protocol.CachePoolStats;
+ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
  import org.apache.hadoop.hdfs.protocol.DatanodeID;
  import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
  import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
+ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
  import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
++import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+ import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryScopeProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTypeProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.FsActionProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
+ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheFlagProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolStatsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeStorageReportProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
  import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
  import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
+ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos;
++import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ContentSummaryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DataEncryptionKeyProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeLocalInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
++import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.FileType;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageReportProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
+ import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto.XAttrNamespaceProto;
+ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrSetFlagProto;
+ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
  import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
  import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
  import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
+ import org.apache.hadoop.io.EnumSetWritable;
+ import org.apache.hadoop.io.Text;
++import org.apache.hadoop.io.erasurecode.ECSchema;
  import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
  import org.apache.hadoop.security.token.Token;
  import org.apache.hadoop.util.DataChecksum;
@@@ -264,4 -396,1934 +403,2033 @@@ public class PBHelperClient 
      assert size >= 0;
      return new ExactSizeInputStream(input, size);
    }
+ 
+   public static CipherOption convert(HdfsProtos.CipherOptionProto proto) {
+     if (proto != null) {
+       CipherSuite suite = null;
+       if (proto.getSuite() != null) {
+         suite = convert(proto.getSuite());
+       }
+       byte[] inKey = null;
+       if (proto.getInKey() != null) {
+         inKey = proto.getInKey().toByteArray();
+       }
+       byte[] inIv = null;
+       if (proto.getInIv() != null) {
+         inIv = proto.getInIv().toByteArray();
+       }
+       byte[] outKey = null;
+       if (proto.getOutKey() != null) {
+         outKey = proto.getOutKey().toByteArray();
+       }
+       byte[] outIv = null;
+       if (proto.getOutIv() != null) {
+         outIv = proto.getOutIv().toByteArray();
+       }
+       return new CipherOption(suite, inKey, inIv, outKey, outIv);
+     }
+     return null;
+   }
+ 
+   public static CipherSuite convert(HdfsProtos.CipherSuiteProto proto) {
+     switch (proto) {
+     case AES_CTR_NOPADDING:
+       return CipherSuite.AES_CTR_NOPADDING;
+     default:
+       // Set to UNKNOWN and stash the unknown enum value
+       CipherSuite suite = CipherSuite.UNKNOWN;
+       suite.setUnknownValue(proto.getNumber());
+       return suite;
+     }
+   }
+ 
+   public static HdfsProtos.CipherOptionProto convert(CipherOption option) {
+     if (option != null) {
+       HdfsProtos.CipherOptionProto.Builder builder = HdfsProtos.CipherOptionProto.
+           newBuilder();
+       if (option.getCipherSuite() != null) {
+         builder.setSuite(convert(option.getCipherSuite()));
+       }
+       if (option.getInKey() != null) {
+         builder.setInKey(ByteString.copyFrom(option.getInKey()));
+       }
+       if (option.getInIv() != null) {
+         builder.setInIv(ByteString.copyFrom(option.getInIv()));
+       }
+       if (option.getOutKey() != null) {
+         builder.setOutKey(ByteString.copyFrom(option.getOutKey()));
+       }
+       if (option.getOutIv() != null) {
+         builder.setOutIv(ByteString.copyFrom(option.getOutIv()));
+       }
+       return builder.build();
+     }
+     return null;
+   }
+ 
+   public static HdfsProtos.CipherSuiteProto convert(CipherSuite suite) {
+     switch (suite) {
+     case UNKNOWN:
+       return HdfsProtos.CipherSuiteProto.UNKNOWN;
+     case AES_CTR_NOPADDING:
+       return HdfsProtos.CipherSuiteProto.AES_CTR_NOPADDING;
+     default:
+       return null;
+     }
+   }
+ 
+   public static List<HdfsProtos.CipherOptionProto> convertCipherOptions(
+       List<CipherOption> options) {
+     if (options != null) {
+       List<HdfsProtos.CipherOptionProto> protos =
+           Lists.newArrayListWithCapacity(options.size());
+       for (CipherOption option : options) {
+         protos.add(convert(option));
+       }
+       return protos;
+     }
+     return null;
+   }
+ 
+   public static List<CipherOption> convertCipherOptionProtos(
+       List<HdfsProtos.CipherOptionProto> protos) {
+     if (protos != null) {
+       List<CipherOption> options =
+           Lists.newArrayListWithCapacity(protos.size());
+       for (HdfsProtos.CipherOptionProto proto : protos) {
+         options.add(convert(proto));
+       }
+       return options;
+     }
+     return null;
+   }
+ 
 -  public static LocatedBlock convert(LocatedBlockProto proto) {
++  public static LocatedBlock convertLocatedBlockProto(LocatedBlockProto proto) {
+     if (proto == null) return null;
+     List<DatanodeInfoProto> locs = proto.getLocsList();
+     DatanodeInfo[] targets = new DatanodeInfo[locs.size()];
+     for (int i = 0; i < locs.size(); i++) {
+       targets[i] = convert(locs.get(i));
+     }
+ 
+     final StorageType[] storageTypes = convertStorageTypes(
+         proto.getStorageTypesList(), locs.size());
+ 
+     final int storageIDsCount = proto.getStorageIDsCount();
+     final String[] storageIDs;
+     if (storageIDsCount == 0) {
+       storageIDs = null;
+     } else {
+       Preconditions.checkState(storageIDsCount == locs.size());
+       storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
+     }
+ 
++    int[] indices = null;
++    final int indexCount = proto.getBlockIndexCount();
++    if (indexCount > 0) {
++      indices = new int[indexCount];
++      for (int i = 0; i < indexCount; i++) {
++        indices[i] = proto.getBlockIndex(i);
++      }
++    }
++
+     // Set values from the isCached list, re-using references from loc
 -    List<DatanodeInfo> cachedLocs = new ArrayList<>(locs.size());
++    List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
+     List<Boolean> isCachedList = proto.getIsCachedList();
+     for (int i=0; i<isCachedList.size(); i++) {
+       if (isCachedList.get(i)) {
+         cachedLocs.add(targets[i]);
+       }
+     }
+ 
 -    LocatedBlock lb = new LocatedBlock(convert(proto.getB()), targets,
 -        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
 -        cachedLocs.toArray(new DatanodeInfo[0]));
++    final LocatedBlock lb;
++    if (indices == null) {
++      lb = new LocatedBlock(PBHelperClient.convert(proto.getB()), targets,
++          storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
++          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
++    } else {
++      lb = new LocatedStripedBlock(PBHelperClient.convert(proto.getB()), targets,
++          storageIDs, storageTypes, indices, proto.getOffset(),
++          proto.getCorrupt(),
++          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
++      List<TokenProto> tokenProtos = proto.getBlockTokensList();
++      Token<BlockTokenIdentifier>[] blockTokens = new Token[indices.length];
++      for (int i = 0; i < indices.length; i++) {
++        blockTokens[i] = convert(tokenProtos.get(i));
++      }
++      ((LocatedStripedBlock) lb).setBlockTokens(blockTokens);
++    }
+     lb.setBlockToken(convert(proto.getBlockToken()));
+ 
+     return lb;
+   }
+ 
+   static public DatanodeInfo convert(DatanodeInfoProto di) {
+     if (di == null) return null;
+     return new DatanodeInfo(
+         convert(di.getId()),
+         di.hasLocation() ? di.getLocation() : null,
+         di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
+         di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
+         di.getLastUpdate(), di.getLastUpdateMonotonic(),
+         di.getXceiverCount(), convert(di.getAdminState()),
+         di.hasUpgradeDomain() ? di.getUpgradeDomain() : null);
+   }
+ 
+   public static StorageType[] convertStorageTypes(
+       List<StorageTypeProto> storageTypesList, int expectedSize) {
+     final StorageType[] storageTypes = new StorageType[expectedSize];
+     if (storageTypesList.size() != expectedSize) {
+      // missing storage types
+       Preconditions.checkState(storageTypesList.isEmpty());
+       Arrays.fill(storageTypes, StorageType.DEFAULT);
+     } else {
+       for (int i = 0; i < storageTypes.length; ++i) {
+         storageTypes[i] = convertStorageType(storageTypesList.get(i));
+       }
+     }
+     return storageTypes;
+   }
+ 
+   public static Token<BlockTokenIdentifier> convert(
+       TokenProto blockToken) {
+     return new Token<>(blockToken.getIdentifier()
+         .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+         blockToken.getKind()), new Text(blockToken.getService()));
+   }
+ 
+   // DatanodeId
+   public static DatanodeID convert(DatanodeIDProto dn) {
+     return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
+         dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
+         .getInfoSecurePort() : 0, dn.getIpcPort());
+   }
+ 
+   public static AdminStates convert(AdminState adminState) {
+     switch(adminState) {
+     case DECOMMISSION_INPROGRESS:
+       return AdminStates.DECOMMISSION_INPROGRESS;
+     case DECOMMISSIONED:
+       return AdminStates.DECOMMISSIONED;
+     case NORMAL:
+     default:
+       return AdminStates.NORMAL;
+     }
+   }
+ 
+   // LocatedBlocks
+   public static LocatedBlocks convert(LocatedBlocksProto lb) {
+     return new LocatedBlocks(
+         lb.getFileLength(), lb.getUnderConstruction(),
 -        convertLocatedBlock(lb.getBlocksList()),
 -        lb.hasLastBlock() ? convert(lb.getLastBlock()) : null,
++        convertLocatedBlocks(lb.getBlocksList()),
++        lb.hasLastBlock() ?
++            convertLocatedBlockProto(lb.getLastBlock()) : null,
+         lb.getIsLastBlockComplete(),
 -        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) :
 -            null);
++        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null,
++        lb.hasEcPolicy() ? convertErasureCodingPolicy(lb.getEcPolicy()) : null);
+   }
+ 
+   public static BlockStoragePolicy[] convertStoragePolicies(
+       List<BlockStoragePolicyProto> policyProtos) {
+     if (policyProtos == null || policyProtos.size() == 0) {
+       return new BlockStoragePolicy[0];
+     }
+     BlockStoragePolicy[] policies = new BlockStoragePolicy[policyProtos.size()];
+     int i = 0;
+     for (BlockStoragePolicyProto proto : policyProtos) {
+       policies[i++] = convert(proto);
+     }
+     return policies;
+   }
+ 
+   public static EventBatchList convert(GetEditsFromTxidResponseProto resp) throws
+     IOException {
+     final InotifyProtos.EventsListProto list = resp.getEventsList();
+     final long firstTxid = list.getFirstTxid();
+     final long lastTxid = list.getLastTxid();
+ 
+     List<EventBatch> batches = Lists.newArrayList();
+     if (list.getEventsList().size() > 0) {
+       throw new IOException("Can't handle old inotify server response.");
+     }
+     for (InotifyProtos.EventBatchProto bp : list.getBatchList()) {
+       long txid = bp.getTxid();
+       if ((txid != -1) && ((txid < firstTxid) || (txid > lastTxid))) {
+         throw new IOException("Error converting TxidResponseProto: got a " +
+             "transaction id " + txid + " that was outside the range of [" +
+             firstTxid + ", " + lastTxid + "].");
+       }
+       List<Event> events = Lists.newArrayList();
+       for (InotifyProtos.EventProto p : bp.getEventsList()) {
+         switch (p.getType()) {
+           case EVENT_CLOSE:
+             InotifyProtos.CloseEventProto close =
+                 InotifyProtos.CloseEventProto.parseFrom(p.getContents());
+             events.add(new Event.CloseEvent(close.getPath(),
+                 close.getFileSize(), close.getTimestamp()));
+             break;
+           case EVENT_CREATE:
+             InotifyProtos.CreateEventProto create =
+                 InotifyProtos.CreateEventProto.parseFrom(p.getContents());
+             events.add(new Event.CreateEvent.Builder()
+                 .iNodeType(createTypeConvert(create.getType()))
+                 .path(create.getPath())
+                 .ctime(create.getCtime())
+                 .ownerName(create.getOwnerName())
+                 .groupName(create.getGroupName())
+                 .perms(convert(create.getPerms()))
+                 .replication(create.getReplication())
+                 .symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
+                     create.getSymlinkTarget())
+                 .defaultBlockSize(create.getDefaultBlockSize())
+                 .overwrite(create.getOverwrite()).build());
+             break;
+           case EVENT_METADATA:
+             InotifyProtos.MetadataUpdateEventProto meta =
+                 InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
+             events.add(new Event.MetadataUpdateEvent.Builder()
+                 .path(meta.getPath())
+                 .metadataType(metadataUpdateTypeConvert(meta.getType()))
+                 .mtime(meta.getMtime())
+                 .atime(meta.getAtime())
+                 .replication(meta.getReplication())
+                 .ownerName(
+                     meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
+                 .groupName(
+                     meta.getGroupName().isEmpty() ? null : meta.getGroupName())
+                 .perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
+                 .acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
+                     meta.getAclsList()))
+                 .xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
+                     meta.getXAttrsList()))
+                 .xAttrsRemoved(meta.getXAttrsRemoved())
+                 .build());
+             break;
+           case EVENT_RENAME:
+             InotifyProtos.RenameEventProto rename =
+                 InotifyProtos.RenameEventProto.parseFrom(p.getContents());
+             events.add(new Event.RenameEvent.Builder()
+                   .srcPath(rename.getSrcPath())
+                   .dstPath(rename.getDestPath())
+                   .timestamp(rename.getTimestamp())
+                   .build());
+             break;
+           case EVENT_APPEND:
+             InotifyProtos.AppendEventProto append =
+                 InotifyProtos.AppendEventProto.parseFrom(p.getContents());
+             events.add(new Event.AppendEvent.Builder().path(append.getPath())
+                 .newBlock(append.hasNewBlock() && append.getNewBlock())
+                 .build());
+             break;
+           case EVENT_UNLINK:
+             InotifyProtos.UnlinkEventProto unlink =
+                 InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
+             events.add(new Event.UnlinkEvent.Builder()
+                   .path(unlink.getPath())
+                   .timestamp(unlink.getTimestamp())
+                   .build());
+             break;
+           case EVENT_TRUNCATE:
+             InotifyProtos.TruncateEventProto truncate =
+                 InotifyProtos.TruncateEventProto.parseFrom(p.getContents());
+             events.add(new Event.TruncateEvent(truncate.getPath(),
+                 truncate.getFileSize(), truncate.getTimestamp()));
+             break;
+           default:
+             throw new RuntimeException("Unexpected inotify event type: " +
+                 p.getType());
+         }
+       }
+       batches.add(new EventBatch(txid, events.toArray(new Event[0])));
+     }
+     return new EventBatchList(batches, resp.getEventsList().getFirstTxid(),
+         resp.getEventsList().getLastTxid(), resp.getEventsList().getSyncTxid());
+   }
+ 
+   // Located Block Arrays and Lists
 -  public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) {
++  public static LocatedBlockProto[] convertLocatedBlocks(LocatedBlock[] lb) {
++    if (lb == null) return null;
++    return convertLocatedBlocks2(Arrays.asList(lb))
++        .toArray(new LocatedBlockProto[lb.length]);
++  }
++
++  public static LocatedBlock[] convertLocatedBlocks(LocatedBlockProto[] lb) {
++    if (lb == null) return null;
++    return convertLocatedBlocks(Arrays.asList(lb))
++        .toArray(new LocatedBlock[lb.length]);
++  }
++
++  public static List<LocatedBlock> convertLocatedBlocks(
++      List<LocatedBlockProto> lb) {
+     if (lb == null) return null;
 -    return convertLocatedBlock2(Arrays.asList(lb)).toArray(
 -        new LocatedBlockProto[lb.length]);
++    final int len = lb.size();
++    List<LocatedBlock> result = new ArrayList<>(len);
++    for (LocatedBlockProto aLb : lb) {
++      result.add(convertLocatedBlockProto(aLb));
++    }
++    return result;
+   }
+ 
 -  public static List<LocatedBlockProto> convertLocatedBlock2(List<LocatedBlock> lb) {
++  public static List<LocatedBlockProto> convertLocatedBlocks2(
++      List<LocatedBlock> lb) {
+     if (lb == null) return null;
+     final int len = lb.size();
+     List<LocatedBlockProto> result = new ArrayList<>(len);
 -    for (int i = 0; i < len; ++i) {
 -      result.add(convert(lb.get(i)));
++    for (LocatedBlock aLb : lb) {
++      result.add(convertLocatedBlock(aLb));
+     }
+     return result;
+   }
+ 
 -  public static LocatedBlockProto convert(LocatedBlock b) {
++  public static LocatedBlockProto convertLocatedBlock(LocatedBlock b) {
+     if (b == null) return null;
+     Builder builder = LocatedBlockProto.newBuilder();
+     DatanodeInfo[] locs = b.getLocations();
+     List<DatanodeInfo> cachedLocs =
+         Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
+     for (int i = 0; i < locs.length; i++) {
+       DatanodeInfo loc = locs[i];
 -      builder.addLocs(i, convert(loc));
++      builder.addLocs(i, PBHelperClient.convert(loc));
+       boolean locIsCached = cachedLocs.contains(loc);
+       builder.addIsCached(locIsCached);
+       if (locIsCached) {
+         cachedLocs.remove(loc);
+       }
+     }
+     Preconditions.checkArgument(cachedLocs.size() == 0,
+         "Found additional cached replica locations that are not in the set of"
 -        + " storage-backed locations!");
++            + " storage-backed locations!");
+ 
+     StorageType[] storageTypes = b.getStorageTypes();
+     if (storageTypes != null) {
 -      for (int i = 0; i < storageTypes.length; ++i) {
 -        builder.addStorageTypes(convertStorageType(storageTypes[i]));
++      for (StorageType storageType : storageTypes) {
++        builder.addStorageTypes(convertStorageType(storageType));
+       }
+     }
+     final String[] storageIDs = b.getStorageIDs();
+     if (storageIDs != null) {
+       builder.addAllStorageIDs(Arrays.asList(storageIDs));
+     }
++    if (b instanceof LocatedStripedBlock) {
++      LocatedStripedBlock sb = (LocatedStripedBlock) b;
++      int[] indices = sb.getBlockIndices();
++      Token<BlockTokenIdentifier>[] blockTokens = sb.getBlockTokens();
++      for (int i = 0; i < indices.length; i++) {
++        builder.addBlockIndex(indices[i]);
++        builder.addBlockTokens(PBHelperClient.convert(blockTokens[i]));
++      }
++    }
+ 
 -    return builder.setB(convert(b.getBlock()))
 -        .setBlockToken(convert(b.getBlockToken()))
++    return builder.setB(PBHelperClient.convert(b.getBlock()))
++        .setBlockToken(PBHelperClient.convert(b.getBlockToken()))
+         .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
+   }
+ 
+   public static BlockStoragePolicy convert(BlockStoragePolicyProto proto) {
+     List<StorageTypeProto> cList = proto.getCreationPolicy()
+         .getStorageTypesList();
+     StorageType[] creationTypes = convertStorageTypes(cList, cList.size());
+     List<StorageTypeProto> cfList = proto.hasCreationFallbackPolicy() ? proto
+         .getCreationFallbackPolicy().getStorageTypesList() : null;
+     StorageType[] creationFallbackTypes = cfList == null ? StorageType
+         .EMPTY_ARRAY : convertStorageTypes(cfList, cfList.size());
+     List<StorageTypeProto> rfList = proto.hasReplicationFallbackPolicy() ?
+         proto.getReplicationFallbackPolicy().getStorageTypesList() : null;
+     StorageType[] replicationFallbackTypes = rfList == null ? StorageType
+         .EMPTY_ARRAY : convertStorageTypes(rfList, rfList.size());
+     return new BlockStoragePolicy((byte) proto.getPolicyId(), proto.getName(),
+         creationTypes, creationFallbackTypes, replicationFallbackTypes);
+   }
+ 
+   public static FsActionProto convert(FsAction v) {
+     return FsActionProto.valueOf(v != null ? v.ordinal() : 0);
+   }
+ 
+   public static XAttrProto convertXAttrProto(XAttr a) {
+     XAttrProto.Builder builder = XAttrProto.newBuilder();
+     builder.setNamespace(convert(a.getNameSpace()));
+     if (a.getName() != null) {
+       builder.setName(a.getName());
+     }
+     if (a.getValue() != null) {
+       builder.setValue(getByteString(a.getValue()));
+     }
+     return builder.build();
+   }
+ 
+   public static List<XAttr> convert(ListXAttrsResponseProto a) {
+     final List<XAttrProto> xAttrs = a.getXAttrsList();
+     return convertXAttrs(xAttrs);
+   }
+ 
+   public static List<XAttr> convert(GetXAttrsResponseProto a) {
+     List<XAttrProto> xAttrs = a.getXAttrsList();
+     return convertXAttrs(xAttrs);
+   }
+ 
+   public static List<XAttr> convertXAttrs(List<XAttrProto> xAttrSpec) {
+     ArrayList<XAttr> xAttrs = Lists.newArrayListWithCapacity(xAttrSpec.size());
+     for (XAttrProto a : xAttrSpec) {
+       XAttr.Builder builder = new XAttr.Builder();
+       builder.setNameSpace(convert(a.getNamespace()));
+       if (a.hasName()) {
+         builder.setName(a.getName());
+       }
+       if (a.hasValue()) {
+         builder.setValue(a.getValue().toByteArray());
+       }
+       xAttrs.add(builder.build());
+     }
+     return xAttrs;
+   }
+ 
+   static XAttrNamespaceProto convert(XAttr.NameSpace v) {
+     return XAttrNamespaceProto.valueOf(v.ordinal());
+   }
+ 
+   static XAttr.NameSpace convert(XAttrNamespaceProto v) {
+     return castEnum(v, XATTR_NAMESPACE_VALUES);
+   }
+ 
+   static <T extends Enum<T>, U extends Enum<U>> U castEnum(T from, U[] to) {
+     return to[from.ordinal()];
+   }
+ 
+   static InotifyProtos.MetadataUpdateType metadataUpdateTypeConvert(
+       Event.MetadataUpdateEvent.MetadataType type) {
+     switch (type) {
+     case TIMES:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_TIMES;
+     case REPLICATION:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_REPLICATION;
+     case OWNER:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_OWNER;
+     case PERMS:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_PERMS;
+     case ACLS:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_ACLS;
+     case XATTRS:
+       return InotifyProtos.MetadataUpdateType.META_TYPE_XATTRS;
+     default:
+       return null;
+     }
+   }
+ 
+   private static Event.MetadataUpdateEvent.MetadataType metadataUpdateTypeConvert(
+       InotifyProtos.MetadataUpdateType type) {
+     switch (type) {
+     case META_TYPE_TIMES:
+       return Event.MetadataUpdateEvent.MetadataType.TIMES;
+     case META_TYPE_REPLICATION:
+       return Event.MetadataUpdateEvent.MetadataType.REPLICATION;
+     case META_TYPE_OWNER:
+       return Event.MetadataUpdateEvent.MetadataType.OWNER;
+     case META_TYPE_PERMS:
+       return Event.MetadataUpdateEvent.MetadataType.PERMS;
+     case META_TYPE_ACLS:
+       return Event.MetadataUpdateEvent.MetadataType.ACLS;
+     case META_TYPE_XATTRS:
+       return Event.MetadataUpdateEvent.MetadataType.XATTRS;
+     default:
+       return null;
+     }
+   }
+ 
+   static InotifyProtos.INodeType createTypeConvert(Event.CreateEvent.INodeType
+                                                        type) {
+     switch (type) {
+     case DIRECTORY:
+       return InotifyProtos.INodeType.I_TYPE_DIRECTORY;
+     case FILE:
+       return InotifyProtos.INodeType.I_TYPE_FILE;
+     case SYMLINK:
+       return InotifyProtos.INodeType.I_TYPE_SYMLINK;
+     default:
+       return null;
+     }
+   }
+ 
+   public static List<LocatedBlock> convertLocatedBlock(
+       List<LocatedBlockProto> lb) {
+     if (lb == null) return null;
+     final int len = lb.size();
+     List<LocatedBlock> result = new ArrayList<>(len);
+     for (int i = 0; i < len; ++i) {
 -      result.add(convert(lb.get(i)));
++      result.add(convertLocatedBlockProto(lb.get(i)));
+     }
+     return result;
+   }
+ 
+   public static List<AclEntry> convertAclEntry(List<AclEntryProto> aclSpec) {
+     ArrayList<AclEntry> r = Lists.newArrayListWithCapacity(aclSpec.size());
+     for (AclEntryProto e : aclSpec) {
+       AclEntry.Builder builder = new AclEntry.Builder();
+       builder.setType(convert(e.getType()));
+       builder.setScope(convert(e.getScope()));
+       builder.setPermission(convert(e.getPermissions()));
+       if (e.hasName()) {
+         builder.setName(e.getName());
+       }
+       r.add(builder.build());
+     }
+     return r;
+   }
+ 
+   static AclEntryScopeProto convert(AclEntryScope v) {
+     return AclEntryScopeProto.valueOf(v.ordinal());
+   }
+ 
+   private static AclEntryScope convert(AclEntryScopeProto v) {
+     return castEnum(v, ACL_ENTRY_SCOPE_VALUES);
+   }
+ 
+   static AclEntryTypeProto convert(AclEntryType e) {
+     return AclEntryTypeProto.valueOf(e.ordinal());
+   }
+ 
+   private static AclEntryType convert(AclEntryTypeProto v) {
+     return castEnum(v, ACL_ENTRY_TYPE_VALUES);
+   }
+ 
+   public static FsAction convert(FsActionProto v) {
+     return castEnum(v, FSACTION_VALUES);
+   }
+ 
+   public static FsPermission convert(FsPermissionProto p) {
+     return new FsPermissionExtension((short)p.getPerm());
+   }
+ 
+   private static Event.CreateEvent.INodeType createTypeConvert(
+       InotifyProtos.INodeType type) {
+     switch (type) {
+     case I_TYPE_DIRECTORY:
+       return Event.CreateEvent.INodeType.DIRECTORY;
+     case I_TYPE_FILE:
+       return Event.CreateEvent.INodeType.FILE;
+     case I_TYPE_SYMLINK:
+       return Event.CreateEvent.INodeType.SYMLINK;
+     default:
+       return null;
+     }
+   }
+ 
+   public static HdfsProtos.FileEncryptionInfoProto convert(
+       FileEncryptionInfo info) {
+     if (info == null) {
+       return null;
+     }
+     return HdfsProtos.FileEncryptionInfoProto.newBuilder()
+         .setSuite(convert(info.getCipherSuite()))
+         .setCryptoProtocolVersion(convert(info.getCryptoProtocolVersion()))
+         .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
+         .setIv(getByteString(info.getIV()))
+         .setEzKeyVersionName(info.getEzKeyVersionName())
+         .setKeyName(info.getKeyName())
+         .build();
+   }
+ 
+   public static CryptoProtocolVersionProto convert(CryptoProtocolVersion
+       version) {
+     switch(version) {
+     case UNKNOWN:
+       return CryptoProtocolVersionProto.UNKNOWN_PROTOCOL_VERSION;
+     case ENCRYPTION_ZONES:
+       return CryptoProtocolVersionProto.ENCRYPTION_ZONES;
+     default:
+       return null;
+     }
+   }
+ 
+   public static FileEncryptionInfo convert(
+       HdfsProtos.FileEncryptionInfoProto proto) {
+     if (proto == null) {
+       return null;
+     }
+     CipherSuite suite = convert(proto.getSuite());
+     CryptoProtocolVersion version = convert(proto.getCryptoProtocolVersion());
+     byte[] key = proto.getKey().toByteArray();
+     byte[] iv = proto.getIv().toByteArray();
+     String ezKeyVersionName = proto.getEzKeyVersionName();
+     String keyName = proto.getKeyName();
+     return new FileEncryptionInfo(suite, version, key, iv, keyName,
+         ezKeyVersionName);
+   }
+ 
+   public static CryptoProtocolVersion convert(CryptoProtocolVersionProto
+       proto) {
+     switch(proto) {
+     case ENCRYPTION_ZONES:
+       return CryptoProtocolVersion.ENCRYPTION_ZONES;
+     default:
+       // Set to UNKNOWN and stash the unknown enum value
+       CryptoProtocolVersion version = CryptoProtocolVersion.UNKNOWN;
+       version.setUnknownValue(proto.getNumber());
+       return version;
+     }
+   }
+ 
+   public static List<XAttrProto> convertXAttrProto(
+       List<XAttr> xAttrSpec) {
+     if (xAttrSpec == null) {
+       return Lists.newArrayListWithCapacity(0);
+     }
+     ArrayList<XAttrProto> xAttrs = Lists.newArrayListWithCapacity(
+         xAttrSpec.size());
+     for (XAttr a : xAttrSpec) {
+       XAttrProto.Builder builder = XAttrProto.newBuilder();
+       builder.setNamespace(convert(a.getNameSpace()));
+       if (a.getName() != null) {
+         builder.setName(a.getName());
+       }
+       if (a.getValue() != null) {
+         builder.setValue(getByteString(a.getValue()));
+       }
+       xAttrs.add(builder.build());
+     }
+     return xAttrs;
+   }
+ 
+   /**
+    * The flag field in PB is a bitmask whose values are the same a the
+    * emum values of XAttrSetFlag
+    */
+   public static int convert(EnumSet<XAttrSetFlag> flag) {
+     int value = 0;
+     if (flag.contains(XAttrSetFlag.CREATE)) {
+       value |= XAttrSetFlagProto.XATTR_CREATE.getNumber();
+     }
+     if (flag.contains(XAttrSetFlag.REPLACE)) {
+       value |= XAttrSetFlagProto.XATTR_REPLACE.getNumber();
+     }
+     return value;
+   }
+ 
+   public static EncryptionZone convert(EncryptionZoneProto proto) {
+     return new EncryptionZone(proto.getId(), proto.getPath(),
+         convert(proto.getSuite()), convert(proto.getCryptoProtocolVersion()),
+         proto.getKeyName());
+   }
+ 
+   public static AclStatus convert(GetAclStatusResponseProto e) {
+     AclStatusProto r = e.getResult();
+     AclStatus.Builder builder = new AclStatus.Builder();
+     builder.owner(r.getOwner()).group(r.getGroup()).stickyBit(r.getSticky())
+         .addEntries(convertAclEntry(r.getEntriesList()));
+     if (r.hasPermission()) {
+       builder.setPermission(convert(r.getPermission()));
+     }
+     return builder.build();
+   }
+ 
+   public static List<AclEntryProto> convertAclEntryProto(
+       List<AclEntry> aclSpec) {
+     ArrayList<AclEntryProto> r = Lists.newArrayListWithCapacity(aclSpec.size());
+     for (AclEntry e : aclSpec) {
+       AclEntryProto.Builder builder = AclEntryProto.newBuilder();
+       builder.setType(convert(e.getType()));
+       builder.setScope(convert(e.getScope()));
+       builder.setPermissions(convert(e.getPermission()));
+       if (e.getName() != null) {
+         builder.setName(e.getName());
+       }
+       r.add(builder.build());
+     }
+     return r;
+   }
+ 
+   public static CachePoolEntry convert(CachePoolEntryProto proto) {
+     CachePoolInfo info = convert(proto.getInfo());
+     CachePoolStats stats = convert(proto.getStats());
+     return new CachePoolEntry(info, stats);
+   }
+ 
+   public static CachePoolInfo convert (CachePoolInfoProto proto) {
+     // Pool name is a required field, the rest are optional
 -    String poolName = checkNotNull(proto.getPoolName());
++    String poolName = Preconditions.checkNotNull(proto.getPoolName());
+     CachePoolInfo info = new CachePoolInfo(poolName);
+     if (proto.hasOwnerName()) {
+         info.setOwnerName(proto.getOwnerName());
+     }
+     if (proto.hasGroupName()) {
+       info.setGroupName(proto.getGroupName());
+     }
+     if (proto.hasMode()) {
+       info.setMode(new FsPermission((short)proto.getMode()));
+     }
+     if (proto.hasLimit())  {
+       info.setLimit(proto.getLimit());
+     }
+     if (proto.hasMaxRelativeExpiry()) {
+       info.setMaxRelativeExpiryMs(proto.getMaxRelativeExpiry());
+     }
+     return info;
+   }
+ 
+   public static CachePoolStats convert (CachePoolStatsProto proto) {
+     CachePoolStats.Builder builder = new CachePoolStats.Builder();
+     builder.setBytesNeeded(proto.getBytesNeeded());
+     builder.setBytesCached(proto.getBytesCached());
+     builder.setBytesOverlimit(proto.getBytesOverlimit());
+     builder.setFilesNeeded(proto.getFilesNeeded());
+     builder.setFilesCached(proto.getFilesCached());
+     return builder.build();
+   }
+ 
+   public static CachePoolInfoProto convert(CachePoolInfo info) {
+     CachePoolInfoProto.Builder builder = CachePoolInfoProto.newBuilder();
+     builder.setPoolName(info.getPoolName());
+     if (info.getOwnerName() != null) {
+       builder.setOwnerName(info.getOwnerName());
+     }
+     if (info.getGroupName() != null) {
+       builder.setGroupName(info.getGroupName());
+     }
+     if (info.getMode() != null) {
+       builder.setMode(info.getMode().toShort());
+     }
+     if (info.getLimit() != null) {
+       builder.setLimit(info.getLimit());
+     }
+     if (info.getMaxRelativeExpiryMs() != null) {
+       builder.setMaxRelativeExpiry(info.getMaxRelativeExpiryMs());
+     }
+     return builder.build();
+   }
+ 
+   public static CacheDirectiveInfoProto convert
+       (CacheDirectiveInfo info) {
+     CacheDirectiveInfoProto.Builder builder =
+         CacheDirectiveInfoProto.newBuilder();
+     if (info.getId() != null) {
+       builder.setId(info.getId());
+     }
+     if (info.getPath() != null) {
+       builder.setPath(info.getPath().toUri().getPath());
+     }
+     if (info.getReplication() != null) {
+       builder.setReplication(info.getReplication());
+     }
+     if (info.getPool() != null) {
+       builder.setPool(info.getPool());
+     }
+     if (info.getExpiration() != null) {
+       builder.setExpiration(convert(info.getExpiration()));
+     }
+     return builder.build();
+   }
+ 
+   public static CacheDirectiveInfoExpirationProto convert(
+       CacheDirectiveInfo.Expiration expiration) {
+     return CacheDirectiveInfoExpirationProto.newBuilder()
+         .setIsRelative(expiration.isRelative())
+         .setMillis(expiration.getMillis())
+         .build();
+   }
+ 
+   public static CacheDirectiveEntry convert(CacheDirectiveEntryProto proto) {
+     CacheDirectiveInfo info = convert(proto.getInfo());
+     CacheDirectiveStats stats = convert(proto.getStats());
+     return new CacheDirectiveEntry(info, stats);
+   }
+ 
+   public static CacheDirectiveStats convert(CacheDirectiveStatsProto proto) {
+     CacheDirectiveStats.Builder builder = new CacheDirectiveStats.Builder();
+     builder.setBytesNeeded(proto.getBytesNeeded());
+     builder.setBytesCached(proto.getBytesCached());
+     builder.setFilesNeeded(proto.getFilesNeeded());
+     builder.setFilesCached(proto.getFilesCached());
+     builder.setHasExpired(proto.getHasExpired());
+     return builder.build();
+   }
+ 
+   public static CacheDirectiveInfo convert
+       (CacheDirectiveInfoProto proto) {
+     CacheDirectiveInfo.Builder builder =
+         new CacheDirectiveInfo.Builder();
+     if (proto.hasId()) {
+       builder.setId(proto.getId());
+     }
+     if (proto.hasPath()) {
+       builder.setPath(new Path(proto.getPath()));
+     }
+     if (proto.hasReplication()) {
+       builder.setReplication(Shorts.checkedCast(
+           proto.getReplication()));
+     }
+     if (proto.hasPool()) {
+       builder.setPool(proto.getPool());
+     }
+     if (proto.hasExpiration()) {
+       builder.setExpiration(convert(proto.getExpiration()));
+     }
+     return builder.build();
+   }
+ 
+   public static CacheDirectiveInfo.Expiration convert(
+       CacheDirectiveInfoExpirationProto proto) {
+     if (proto.getIsRelative()) {
+       return CacheDirectiveInfo.Expiration.newRelative(proto.getMillis());
+     }
+     return CacheDirectiveInfo.Expiration.newAbsolute(proto.getMillis());
+   }
+ 
+   public static int convertCacheFlags(EnumSet<CacheFlag> flags) {
+     int value = 0;
+     if (flags.contains(CacheFlag.FORCE)) {
+       value |= CacheFlagProto.FORCE.getNumber();
+     }
+     return value;
+   }
+ 
+   public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) {
+     if (reportProto == null) {
+       return null;
+     }
+     String snapshotDir = reportProto.getSnapshotRoot();
+     String fromSnapshot = reportProto.getFromSnapshot();
+     String toSnapshot = reportProto.getToSnapshot();
+     List<SnapshotDiffReportEntryProto> list = reportProto
+         .getDiffReportEntriesList();
+     List<DiffReportEntry> entries = new ArrayList<>();
+     for (SnapshotDiffReportEntryProto entryProto : list) {
+       DiffReportEntry entry = convert(entryProto);
+       if (entry != null)
+         entries.add(entry);
+     }
+     return new SnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot,
+         entries);
+   }
+ 
+   public static DiffReportEntry convert(SnapshotDiffReportEntryProto entry) {
+     if (entry == null) {
+       return null;
+     }
+     DiffType type = DiffType.getTypeFromLabel(entry
+         .getModificationLabel());
+     return type == null ? null : new DiffReportEntry(type, entry.getFullpath()
+         .toByteArray(), entry.hasTargetPath() ? entry.getTargetPath()
+         .toByteArray() : null);
+   }
+ 
+   public static SnapshottableDirectoryStatus[] convert(
+       SnapshottableDirectoryListingProto sdlp) {
+     if (sdlp == null)
+       return null;
+     List<SnapshottableDirectoryStatusProto> list = sdlp
+         .getSnapshottableDirListingList();
+     if (list.isEmpty()) {
+       return new SnapshottableDirectoryStatus[0];
+     } else {
+       SnapshottableDirectoryStatus[] result =
+           new SnapshottableDirectoryStatus[list.size()];
+       for (int i = 0; i < list.size(); i++) {
+         result[i] = convert(list.get(i));
+       }
+       return result;
+     }
+   }
+ 
+   public static SnapshottableDirectoryStatus convert(
+       SnapshottableDirectoryStatusProto sdirStatusProto) {
+     if (sdirStatusProto == null) {
+       return null;
+     }
+     final HdfsFileStatusProto status = sdirStatusProto.getDirStatus();
+     return new SnapshottableDirectoryStatus(
+         status.getModificationTime(),
+         status.getAccessTime(),
+         convert(status.getPermission()),
+         status.getOwner(),
+         status.getGroup(),
+         status.getPath().toByteArray(),
+         status.getFileId(),
+         status.getChildrenNum(),
+         sdirStatusProto.getSnapshotNumber(),
+         sdirStatusProto.getSnapshotQuota(),
+         sdirStatusProto.getParentFullpath().toByteArray());
+   }
+ 
+   // DataEncryptionKey
+   public static DataEncryptionKey convert(DataEncryptionKeyProto bet) {
+     String encryptionAlgorithm = bet.getEncryptionAlgorithm();
+     return new DataEncryptionKey(bet.getKeyId(),
+         bet.getBlockPoolId(),
+         bet.getNonce().toByteArray(),
+         bet.getEncryptionKey().toByteArray(),
+         bet.getExpiryDate(),
+         encryptionAlgorithm.isEmpty() ? null : encryptionAlgorithm);
+   }
+ 
+   public static Token<DelegationTokenIdentifier> convertDelegationToken(
+       TokenProto blockToken) {
+     return new Token<>(blockToken.getIdentifier()
+         .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+         blockToken.getKind()), new Text(blockToken.getService()));
+   }
+ 
+   // Arrays of DatanodeId
+   public static DatanodeIDProto[] convert(DatanodeID[] did) {
+     if (did == null)
+       return null;
+     final int len = did.length;
+     DatanodeIDProto[] result = new DatanodeIDProto[len];
+     for (int i = 0; i < len; ++i) {
+       result[i] = convert(did[i]);
+     }
+     return result;
+   }
+ 
+   public static FsPermissionProto convert(FsPermission p) {
+     return FsPermissionProto.newBuilder().setPerm(p.toExtendedShort()).build();
+   }
+ 
+   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
+     if (fs == null)
+       return null;
+     return new HdfsLocatedFileStatus(
+         fs.getLength(), fs.getFileType().equals(FileType.IS_DIR),
+         fs.getBlockReplication(), fs.getBlocksize(),
+         fs.getModificationTime(), fs.getAccessTime(),
+         convert(fs.getPermission()), fs.getOwner(), fs.getGroup(),
+         fs.getFileType().equals(FileType.IS_SYMLINK) ?
+             fs.getSymlink().toByteArray() : null,
+         fs.getPath().toByteArray(),
+         fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
+         fs.hasLocations() ? convert(fs.getLocations()) : null,
+         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
+         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
+         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
 -            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
++            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
++    fs.hasEcPolicy() ? convertErasureCodingPolicy(fs.getEcPolicy()) : null);
+   }
+ 
+   public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
+     if (c == null)
+       return null;
+     List<String> fileList = c.getFilesList();
+     return new CorruptFileBlocks(fileList.toArray(new String[fileList.size()]),
+         c.getCookie());
+   }
+ 
+   public static ContentSummary convert(ContentSummaryProto cs) {
+     if (cs == null) return null;
+     ContentSummary.Builder builder = new ContentSummary.Builder();
+     builder.length(cs.getLength()).
+         fileCount(cs.getFileCount()).
+         directoryCount(cs.getDirectoryCount()).
+         quota(cs.getQuota()).
+         spaceConsumed(cs.getSpaceConsumed()).
+         spaceQuota(cs.getSpaceQuota());
+     if (cs.hasTypeQuotaInfos()) {
+       for (HdfsProtos.StorageTypeQuotaInfoProto info :
+           cs.getTypeQuotaInfos().getTypeQuotaInfoList()) {
+         StorageType type = convertStorageType(info.getType());
+         builder.typeConsumed(type, info.getConsumed());
+         builder.typeQuota(type, info.getQuota());
+       }
+     }
+     return builder.build();
+   }
+ 
+   public static RollingUpgradeActionProto convert(RollingUpgradeAction a) {
+     switch (a) {
+     case QUERY:
+       return RollingUpgradeActionProto.QUERY;
+     case PREPARE:
+       return RollingUpgradeActionProto.START;
+     case FINALIZE:
+       return RollingUpgradeActionProto.FINALIZE;
+     default:
+       throw new IllegalArgumentException("Unexpected value: " + a);
+     }
+   }
+ 
+   public static RollingUpgradeInfo convert(RollingUpgradeInfoProto proto) {
+     RollingUpgradeStatusProto status = proto.getStatus();
+     return new RollingUpgradeInfo(status.getBlockPoolId(),
+         proto.getCreatedRollbackImages(),
+         proto.getStartTime(), proto.getFinalizeTime());
+   }
+ 
+   public static DatanodeStorageReport[] convertDatanodeStorageReports(
+       List<DatanodeStorageReportProto> protos) {
+     final DatanodeStorageReport[] reports
+         = new DatanodeStorageReport[protos.size()];
+     for(int i = 0; i < reports.length; i++) {
+       reports[i] = convertDatanodeStorageReport(protos.get(i));
+     }
+     return reports;
+   }
+ 
+   public static DatanodeStorageReport convertDatanodeStorageReport(
+       DatanodeStorageReportProto proto) {
+     return new DatanodeStorageReport(
+         convert(proto.getDatanodeInfo()),
+         convertStorageReports(proto.getStorageReportsList()));
+   }
+ 
+   public static StorageReport[] convertStorageReports(
+       List<StorageReportProto> list) {
+     final StorageReport[] report = new StorageReport[list.size()];
+     for (int i = 0; i < report.length; i++) {
+       report[i] = convert(list.get(i));
+     }
+     return report;
+   }
+ 
+   public static StorageReport convert(StorageReportProto p) {
+     return new StorageReport(
+         p.hasStorage() ?
+             convert(p.getStorage()) :
+             new DatanodeStorage(p.getStorageUuid()),
+         p.getFailed(), p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
+         p.getBlockPoolUsed());
+   }
+ 
+   public static DatanodeStorage convert(DatanodeStorageProto s) {
+     return new DatanodeStorage(s.getStorageUuid(),
+         convertState(s.getState()), convertStorageType(s.getStorageType()));
+   }
+ 
+   private static State convertState(StorageState state) {
+     switch(state) {
+     case READ_ONLY_SHARED:
+       return State.READ_ONLY_SHARED;
+     case NORMAL:
+     default:
+       return State.NORMAL;
+     }
+   }
+ 
+   public static SafeModeActionProto convert(
+       SafeModeAction a) {
+     switch (a) {
+     case SAFEMODE_LEAVE:
+       return SafeModeActionProto.SAFEMODE_LEAVE;
+     case SAFEMODE_ENTER:
+       return SafeModeActionProto.SAFEMODE_ENTER;
+     case SAFEMODE_GET:
+       return SafeModeActionProto.SAFEMODE_GET;
+     default:
+       throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
+     }
+   }
+ 
+   public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
+     DatanodeInfo[] info = new DatanodeInfo[list.size()];
+     for (int i = 0; i < info.length; i++) {
+       info[i] = convert(list.get(i));
+     }
+     return info;
+   }
+ 
+   public static long[] convert(GetFsStatsResponseProto res) {
+     long[] result = new long[7];
+     result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
+     result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
+     result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
+     result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
+     result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
+     result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
+     result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
+         res.getMissingReplOneBlocks();
+     return result;
+   }
+ 
+   public static DatanodeReportTypeProto
+     convert(DatanodeReportType t) {
+     switch (t) {
+     case ALL: return DatanodeReportTypeProto.ALL;
+     case LIVE: return DatanodeReportTypeProto.LIVE;
+     case DEAD: return DatanodeReportTypeProto.DEAD;
+     case DECOMMISSIONING: return DatanodeReportTypeProto.DECOMMISSIONING;
+     default:
+       throw new IllegalArgumentException("Unexpected data type report:" + t);
+     }
+   }
+ 
+   public static DirectoryListing convert(DirectoryListingProto dl) {
+     if (dl == null)
+       return null;
+     List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
+     return new DirectoryListing(partList.isEmpty() ?
+         new HdfsLocatedFileStatus[0] :
+         convert(partList.toArray(new HdfsFileStatusProto[partList.size()])),
+         dl.getRemainingEntries());
+   }
+ 
+   public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
+     if (fs == null) return null;
+     final int len = fs.length;
+     HdfsFileStatus[] result = new HdfsFileStatus[len];
+     for (int i = 0; i < len; ++i) {
+       result[i] = convert(fs[i]);
+     }
+     return result;
+   }
+ 
+   // The creatFlag field in PB is a bitmask whose values are the same a the
+   // emum values of CreateFlag
+   public static int convertCreateFlag(EnumSetWritable<CreateFlag> flag) {
+     int value = 0;
+     if (flag.contains(CreateFlag.APPEND)) {
+       value |= CreateFlagProto.APPEND.getNumber();
+     }
+     if (flag.contains(CreateFlag.CREATE)) {
+       value |= CreateFlagProto.CREATE.getNumber();
+     }
+     if (flag.contains(CreateFlag.OVERWRITE)) {
+       value |= CreateFlagProto.OVERWRITE.getNumber();
+     }
+     if (flag.contains(CreateFlag.LAZY_PERSIST)) {
+       value |= CreateFlagProto.LAZY_PERSIST.getNumber();
+     }
+     if (flag.contains(CreateFlag.NEW_BLOCK)) {
+       value |= CreateFlagProto.NEW_BLOCK.getNumber();
+     }
+     return value;
+   }
+ 
+   public static FsServerDefaults convert(FsServerDefaultsProto fs) {
+     if (fs == null) return null;
+     return new FsServerDefaults(
+         fs.getBlockSize(), fs.getBytesPerChecksum(),
+         fs.getWritePacketSize(), (short) fs.getReplication(),
+         fs.getFileBufferSize(),
+         fs.getEncryptDataTransfer(),
+         fs.getTrashInterval(),
+         convert(fs.getChecksumType()));
+   }
+ 
+   public static List<CryptoProtocolVersionProto> convert(
+       CryptoProtocolVersion[] versions) {
+     List<CryptoProtocolVersionProto> protos =
+         Lists.newArrayListWithCapacity(versions.length);
+     for (CryptoProtocolVersion v: versions) {
+       protos.add(convert(v));
+     }
+     return protos;
+   }
+ 
+   static List<StorageTypesProto> convert(StorageType[][] types) {
+     List<StorageTypesProto> list = Lists.newArrayList();
+     if (types != null) {
+       for (StorageType[] ts : types) {
+         StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
+         builder.addAllStorageTypes(convertStorageTypes(ts));
+         list.add(builder.build());
+       }
+     }
+     return list;
+   }
+ 
+   public static BlockStoragePolicyProto convert(BlockStoragePolicy policy) {
+     BlockStoragePolicyProto.Builder builder = BlockStoragePolicyProto
+         .newBuilder().setPolicyId(policy.getId()).setName(policy.getName());
+     // creation storage types
+     StorageTypesProto creationProto = convert(policy.getStorageTypes());
+     Preconditions.checkArgument(creationProto != null);
+     builder.setCreationPolicy(creationProto);
+     // creation fallback
+     StorageTypesProto creationFallbackProto = convert(
+         policy.getCreationFallbacks());
+     if (creationFallbackProto != null) {
+       builder.setCreationFallbackPolicy(creationFallbackProto);
+     }
+     // replication fallback
+     StorageTypesProto replicationFallbackProto = convert(
+         policy.getReplicationFallbacks());
+     if (replicationFallbackProto != null) {
+       builder.setReplicationFallbackPolicy(replicationFallbackProto);
+     }
+     return builder.build();
+   }
+ 
+   public static StorageTypesProto convert(StorageType[] types) {
+     if (types == null || types.length == 0) {
+       return null;
+     }
+     List<StorageTypeProto> list = convertStorageTypes(types);
+     return StorageTypesProto.newBuilder().addAllStorageTypes(list).build();
+   }
+ 
+   public static DatanodeID[] convert(DatanodeIDProto[] did) {
+     if (did == null) return null;
+     final int len = did.length;
+     DatanodeID[] result = new DatanodeID[len];
+     for (int i = 0; i < len; ++i) {
+       result[i] = convert(did[i]);
+     }
+     return result;
+   }
+ 
+   // Block
+   public static BlockProto convert(Block b) {
+     return BlockProto.newBuilder().setBlockId(b.getBlockId())
+         .setGenStamp(b.getGenerationStamp()).setNumBytes(b.getNumBytes())
+         .build();
+   }
+ 
+   public static Block convert(BlockProto b) {
+     return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
+   }
+ 
+   static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
+     if (di == null) return null;
+     DatanodeInfo[] result = new DatanodeInfo[di.length];
+     for (int i = 0; i < di.length; i++) {
+       result[i] = convert(di[i]);
+     }
+     return result;
+   }
+ 
+   public static DatanodeStorageReportProto convertDatanodeStorageReport(
+       DatanodeStorageReport report) {
+     return DatanodeStorageReportProto.newBuilder()
+         .setDatanodeInfo(convert(report.getDatanodeInfo()))
+         .addAllStorageReports(convertStorageReports(report.getStorageReports()))
+         .build();
+   }
+ 
+   public static List<DatanodeStorageReportProto> convertDatanodeStorageReports(
+       DatanodeStorageReport[] reports) {
+     final List<DatanodeStorageReportProto> protos
+         = new ArrayList<>(reports.length);
+     for(int i = 0; i < reports.length; i++) {
+       protos.add(convertDatanodeStorageReport(reports[i]));
+     }
+     return protos;
+   }
+ 
+   public static LocatedBlock[] convertLocatedBlock(LocatedBlockProto[] lb) {
+     if (lb == null) return null;
+     return convertLocatedBlock(Arrays.asList(lb)).toArray(
+         new LocatedBlock[lb.length]);
+   }
+ 
+   public static LocatedBlocksProto convert(LocatedBlocks lb) {
+     if (lb == null) {
+       return null;
+     }
+     LocatedBlocksProto.Builder builder =
+         LocatedBlocksProto.newBuilder();
+     if (lb.getLastLocatedBlock() != null) {
 -      builder.setLastBlock(convert(lb.getLastLocatedBlock()));
++      builder.setLastBlock(
++          convertLocatedBlock(lb.getLastLocatedBlock()));
+     }
+     if (lb.getFileEncryptionInfo() != null) {
+       builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
+     }
++    if (lb.getErasureCodingPolicy() != null) {
++      builder.setEcPolicy(convertErasureCodingPolicy(lb.getErasureCodingPolicy()));
++    }
+     return builder.setFileLength(lb.getFileLength())
+         .setUnderConstruction(lb.isUnderConstruction())
 -        .addAllBlocks(convertLocatedBlock2(lb.getLocatedBlocks()))
++        .addAllBlocks(convertLocatedBlocks2(lb.getLocatedBlocks()))
+         .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
+   }
+ 
+   public static DataEncryptionKeyProto convert(DataEncryptionKey bet) {
+     DataEncryptionKeyProto.Builder b = DataEncryptionKeyProto.newBuilder()
+         .setKeyId(bet.keyId)
+         .setBlockPoolId(bet.blockPoolId)
+         .setNonce(ByteString.copyFrom(bet.nonce))
+         .setEncryptionKey(ByteString.copyFrom(bet.encryptionKey))
+         .setExpiryDate(bet.expiryDate);
+     if (bet.encryptionAlgorithm != null) {
+       b.setEncryptionAlgorithm(bet.encryptionAlgorithm);
+     }
+     return b.build();
+   }
+ 
+   public static FsServerDefaultsProto convert(FsServerDefaults fs) {
+     if (fs == null) return null;
+     return FsServerDefaultsProto.newBuilder().
+       setBlockSize(fs.getBlockSize()).
+       setBytesPerChecksum(fs.getBytesPerChecksum()).
+       setWritePacketSize(fs.getWritePacketSize())
+       .setReplication(fs.getReplication())
+       .setFileBufferSize(fs.getFileBufferSize())
+       .setEncryptDataTransfer(fs.getEncryptDataTransfer())
+       .setTrashInterval(fs.getTrashInterval())
+       .setChecksumType(convert(fs.getChecksumType()))
+       .build();
+   }
+ 
+   public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
+     EnumSet<CreateFlag> result =
+        EnumSet.noneOf(CreateFlag.class);
+     if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
+       result.add(CreateFlag.APPEND);
+     }
+     if ((flag & CreateFlagProto.CREATE_VALUE) == CreateFlagProto.CREATE_VALUE) {
+       result.add(CreateFlag.CREATE);
+     }
+     if ((flag & CreateFlagProto.OVERWRITE_VALUE)
+         == CreateFlagProto.OVERWRITE_VALUE) {
+       result.add(CreateFlag.OVERWRITE);
+     }
+     if ((flag & CreateFlagProto.LAZY_PERSIST_VALUE)
+         == CreateFlagProto.LAZY_PERSIST_VALUE) {
+       result.add(CreateFlag.LAZY_PERSIST);
+     }
+     if ((flag & CreateFlagProto.NEW_BLOCK_VALUE)
+         == CreateFlagProto.NEW_BLOCK_VALUE) {
+       result.add(CreateFlag.NEW_BLOCK);
+     }
+     return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
+   }
+ 
+   public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
+     EnumSet<CacheFlag> result = EnumSet.noneOf(CacheFlag.class);
+     if ((flags & CacheFlagProto.FORCE_VALUE) == CacheFlagProto.FORCE_VALUE) {
+       result.add(CacheFlag.FORCE);
+     }
+     return result;
+   }
+ 
+   public static HdfsFileStatusProto convert(HdfsFileStatus fs) {
+     if (fs == null)
+       return null;
+     FileType fType = FileType.IS_FILE;
+     if (fs.isDir()) {
+       fType = FileType.IS_DIR;
+     } else if (fs.isSymlink()) {
+       fType = FileType.IS_SYMLINK;
+     }
+ 
+     HdfsFileStatusProto.Builder builder =
+      HdfsFileStatusProto.newBuilder().
+       setLength(fs.getLen()).
+       setFileType(fType).
+       setBlockReplication(fs.getReplication()).
+       setBlocksize(fs.getBlockSize()).
+       setModificationTime(fs.getModificationTime()).
+       setAccessTime(fs.getAccessTime()).
+       setPermission(convert(fs.getPermission())).
+       setOwner(fs.getOwner()).
+       setGroup(fs.getGroup()).
+       setFileId(fs.getFileId()).
+       setChildrenNum(fs.getChildrenNum()).
+       setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
+       setStoragePolicy(fs.getStoragePolicy());
+     if (fs.isSymlink())  {
+       builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
+     }
+     if (fs.getFileEncryptionInfo() != null) {
+       builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo()));
+     }
+     if (fs instanceof HdfsLocatedFileStatus) {
+       final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
+       LocatedBlocks locations = lfs.getBlockLocations();
+       if (locations != null) {
+         builder.setLocations(convert(locations));
+       }
+     }
++    if(fs.getErasureCodingPolicy() != null) {
++      builder.setEcPolicy(convertErasureCodingPolicy(fs.getErasureCodingPolicy()));
++    }
+     return builder.build();
+   }
+ 
+   public static SnapshottableDirectoryStatusProto convert(
+       SnapshottableDirectoryStatus status) {
+     if (status == null) {
+       return null;
+     }
+     int snapshotNumber = status.getSnapshotNumber();
+     int snapshotQuota = status.getSnapshotQuota();
+     byte[] parentFullPath = status.getParentFullPath();
+     ByteString parentFullPathBytes = ByteString.copyFrom(
+         parentFullPath == null ? DFSUtilClient.EMPTY_BYTES : parentFullPath);
+     HdfsFileStatusProto fs = convert(status.getDirStatus());
+     SnapshottableDirectoryStatusProto.Builder builder =
+         SnapshottableDirectoryStatusProto
+         .newBuilder().setSnapshotNumber(snapshotNumber)
+         .setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
+         .setDirStatus(fs);
+     return builder.build();
+   }
+ 
+   public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
+     if (fs == null) return null;
+     final int len = fs.length;
+     HdfsFileStatusProto[] result = new HdfsFileStatusProto[len];
+     for (int i = 0; i < len; ++i) {
+       result[i] = convert(fs[i]);
+     }
+     return result;
+   }
+ 
+   public static DirectoryListingProto convert(DirectoryListing d) {
+     if (d == null)
+       return null;
+     return DirectoryListingProto.newBuilder().
+         addAllPartialListing(Arrays.asList(
+             convert(d.getPartialListing()))).
+         setRemainingEntries(d.getRemainingEntries()).
+         build();
+   }
+ 
+   public static GetFsStatsResponseProto convert(long[] fsStats) {
+     GetFsStatsResponseProto.Builder result = GetFsStatsResponseProto
+         .newBuilder();
+     if (fsStats.length >= ClientProtocol.GET_STATS_CAPACITY_IDX + 1)
+       result.setCapacity(fsStats[ClientProtocol.GET_STATS_CAPACITY_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_USED_IDX + 1)
+       result.setUsed(fsStats[ClientProtocol.GET_STATS_USED_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_REMAINING_IDX + 1)
+       result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
+       result.setUnderReplicated(
+               fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
+       result.setCorruptBlocks(
+           fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
+       result.setMissingBlocks(
+           fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
+     if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
+       result.setMissingReplOneBlocks(
+           fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
+     return result.build();
+   }
+ 
+   public static DatanodeReportType convert(DatanodeReportTypeProto t) {
+     switch (t) {
+     case ALL: return DatanodeReportType.ALL;
+     case LIVE: return DatanodeReportType.LIVE;
+     case DEAD: return DatanodeReportType.DEAD;
+     case DECOMMISSIONING: return DatanodeReportType.DECOMMISSIONING;
+     default:
+       throw new IllegalArgumentException("Unexpected data type report:" + t);
+     }
+   }
+ 
+   public static SafeModeAction convert(
+       SafeModeActionProto a) {
+     switch (a) {
+     case SAFEMODE_LEAVE:
+       return SafeModeAction.SAFEMODE_LEAVE;
+     case SAFEMODE_ENTER:
+       return SafeModeAction.SAFEMODE_ENTER;
+     case SAFEMODE_GET:
+       return SafeModeAction.SAFEMODE_GET;
+     default:
+       throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
+     }
+   }
+ 
+   public static RollingUpgradeAction convert(RollingUpgradeActionProto a) {
+     switch (a) {
+     case QUERY:
+       return RollingUpgradeAction.QUERY;
+     case START:
+       return RollingUpgradeAction.PREPARE;
+     case FINALIZE:
+       return RollingUpgradeAction.FINALIZE;
+     default:
+       throw new IllegalArgumentException("Unexpected value: " + a);
+     }
+   }
+ 
+   public static RollingUpgradeStatusProto convertRollingUpgradeStatus(
+       RollingUpgradeStatus status) {
+     return RollingUpgradeStatusProto.newBuilder()
+         .setBlockPoolId(status.getBlockPoolId())
+         .setFinalized(status.isFinalized())
+         .build();
+   }
+ 
+   public static RollingUpgradeStatus convert(RollingUpgradeStatusProto proto) {
+     return new RollingUpgradeStatus(proto.getBlockPoolId(),
+         proto.getFinalized());
+   }
+ 
+   public static RollingUpgradeInfoProto convert(RollingUpgradeInfo info) {
+     return RollingUpgradeInfoProto.newBuilder()
+         .setStatus(convertRollingUpgradeStatus(info))
+         .setCreatedRollbackImages(info.createdRollbackImages())
+         .setStartTime(info.getStartTime())
+         .setFinalizeTime(info.getFinalizeTime())
+         .build();
+   }
+ 
+   public static CorruptFileBlocksProto convert(CorruptFileBlocks c) {
+     if (c == null)
+       return null;
+     return CorruptFileBlocksProto.newBuilder().
+         addAllFiles(Arrays.asList(c.getFiles())).
+         setCookie(c.getCookie()).
+         build();
+   }
+ 
+   public static ContentSummaryProto convert(ContentSummary cs) {
+     if (cs == null) return null;
+     ContentSummaryProto.Builder builder = ContentSummaryProto.newBuilder();
+         builder.setLength(cs.getLength()).
+         setFileCount(cs.getFileCount()).
+         setDirectoryCount(cs.getDirectoryCount()).
+         setQuota(cs.getQuota()).
+         setSpaceConsumed(cs.getSpaceConsumed()).
+         setSpaceQuota(cs.getSpaceQuota());
+ 
+     if (cs.isTypeQuotaSet() || cs.isTypeConsumedAvailable()) {
+       HdfsProtos.StorageTypeQuotaInfosProto.Builder isb =
+           HdfsProtos.StorageTypeQuotaInfosProto.newBuilder();
+       for (StorageType t: StorageType.getTypesSupportingQuota()) {
+         HdfsProtos.StorageTypeQuotaInfoProto info =
+             HdfsProtos.StorageTypeQuotaInfoProto.newBuilder().
+                 setType(convertStorageType(t)).
+                 setConsumed(cs.getTypeConsumed(t)).
+                 setQuota(cs.getTypeQuota(t)).
+                 build();
+         isb.addTypeQuotaInfo(info);
+       }
+       builder.setTypeQuotaInfos(isb);
+     }
+     return builder.build();
+   }
+ 
+   public static DatanodeStorageProto convert(DatanodeStorage s) {
+     return DatanodeStorageProto.newBuilder()
+         .setState(convertState(s.getState()))
+         .setStorageType(convertStorageType(s.getStorageType()))
+         .setStorageUuid(s.getStorageID()).build();
+   }
+ 
+   private static StorageState convertState(State state) {
+     switch(state) {
+     case READ_ONLY_SHARED:
+       return StorageState.READ_ONLY_SHARED;
+     case NORMAL:
+     default:
+       return StorageState.NORMAL;
+     }
+   }
+ 
+   public static StorageReportProto convert(StorageReport r) {
+     StorageReportProto.Builder builder = StorageReportProto.newBuilder()
+         .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
+         .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
+         .setStorageUuid(r.getStorage().getStorageID())
+         .setStorage(convert(r.getStorage()));
+     return builder.build();
+   }
+ 
+   public static List<StorageReportProto> convertStorageReports(StorageReport[] storages) {
+     final List<StorageReportProto> protos = new ArrayList<StorageReportProto>(
+         storages.length);
+     for(int i = 0; i < storages.length; i++) {
+       protos.add(convert(storages[i]));
+     }
+     return protos;
+   }
+ 
+   public static SnapshottableDirectoryListingProto convert(
+       SnapshottableDirectoryStatus[] status) {
+     if (status == null)
+       return null;
+     SnapshottableDirectoryStatusProto[] protos =
+         new SnapshottableDirectoryStatusProto[status.length];
+     for (int i = 0; i < status.length; i++) {
+       protos[i] = convert(status[i]);
+     }
+     List<SnapshottableDirectoryStatusProto> protoList = Arrays.asList(protos);
+     return SnapshottableDirectoryListingProto.newBuilder()
+         .addAllSnapshottableDirListing(protoList).build();
+   }
+ 
+   public static SnapshotDiffReportEntryProto convert(DiffReportEntry entry) {
+     if (entry == null) {
+       return null;
+     }
+     ByteString sourcePath = ByteString
+         .copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry
+             .getSourcePath());
+     String modification = entry.getType().getLabel();
+     SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
+         .newBuilder().setFullpath(sourcePath)
+         .setModificationLabel(modification);
+     if (entry.getType() == DiffType.RENAME) {
+       ByteString targetPath = ByteString
+           .copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry
+               .getTargetPath());
+       builder.setTargetPath(targetPath);
+     }
+     return builder.build();
+   }
+ 
+   public static SnapshotDiffReportProto convert(SnapshotDiffReport report) {
+     if (report == null) {
+       return null;
+     }
+     List<DiffReportEntry> entries = report.getDiffList();
+     List<SnapshotDiffReportEntryProto> entryProtos = new ArrayList<>();
+     for (DiffReportEntry entry : entries) {
+       SnapshotDiffReportEntryProto entryProto = convert(entry);
+       if (entryProto != null)
+         entryProtos.add(entryProto);
+     }
+ 
+     SnapshotDiffReportProto reportProto = SnapshotDiffReportProto.newBuilder()
+         .setSnapshotRoot(report.getSnapshotRoot())
+         .setFromSnapshot(report.getFromSnapshot())
+         .setToSnapshot(report.getLaterSnapshotName())
+         .addAllDiffReportEntries(entryProtos).build();
+     return reportProto;
+   }
+ 
+   public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
+     CacheDirectiveStatsProto.Builder builder =
+         CacheDirectiveStatsProto.newBuilder();
+     builder.setBytesNeeded(stats.getBytesNeeded());
+     builder.setBytesCached(stats.getBytesCached());
+     builder.setFilesNeeded(stats.getFilesNeeded());
+     builder.setFilesCached(stats.getFilesCached());
+     builder.setHasExpired(stats.hasExpired());
+     return builder.build();
+   }
+ 
+   public static CacheDirectiveEntryProto convert(CacheDirectiveEntry entry) {
+     CacheDirectiveEntryProto.Builder builder =
+         CacheDirectiveEntryProto.newBuilder();
+     builder.setInfo(convert(entry.getInfo()));
+     builder.setStats(convert(entry.getStats()));
+     return builder.build();
+   }
+ 
+   public static boolean[] convertBooleanList(
+     List<Boolean> targetPinningsList) {
+     final boolean[] targetPinnings = new boolean[targetPinningsList.size()];
+     for (int i = 0; i < targetPinningsList.size(); i++) {
+       targetPinnings[i] = targetPinningsList.get(i);
+     }
+     return targetPinnings;
+   }
+ 
+   public static CachePoolStatsProto convert(CachePoolStats stats) {
+     CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
+     builder.setBytesNeeded(stats.getBytesNeeded());
+     builder.setBytesCached(stats.getBytesCached());
+     builder.setBytesOverlimit(stats.getBytesOverlimit());
+     builder.setFilesNeeded(stats.getFilesNeeded());
+     builder.setFilesCached(stats.getFilesCached());
+     return builder.build();
+   }
+ 
+   public static CachePoolEntryProto convert(CachePoolEntry entry) {
+     CachePoolEntryProto.Builder builder = CachePoolEntryProto.newBuilder();
+     builder.setInfo(convert(entry.getInfo()));
+     builder.setStats(convert(entry.getStats()));
+     return builder.build();
+   }
+ 
+   public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
+     DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder();
+     builder.setSoftwareVersion(info.getSoftwareVersion());
+     builder.setConfigVersion(info.getConfigVersion());
+     builder.setUptime(info.getUptime());
+     return builder.build();
+   }
+ 
+   public static GetAclStatusResponseProto convert(AclStatus e) {
+     AclStatusProto.Builder builder = AclStatusProto.newBuilder();
+     builder.setOwner(e.getOwner())
+         .setGroup(e.getGroup()).setSticky(e.isStickyBit())
+         .addAllEntries(convertAclEntryProto(e.getEntries()));
+     if (e.getPermission() != null) {
+       builder.setPermission(convert(e.getPermission()));
+     }
+     AclStatusProto r = builder.build();
+     return GetAclStatusResponseProto.newBuilder().setResult(r).build();
+   }
+ 
+   public static EnumSet<XAttrSetFlag> convert(int flag) {
+     EnumSet<XAttrSetFlag> result =
+         EnumSet.noneOf(XAttrSetFlag.class);
+     if ((flag & XAttrSetFlagProto.XATTR_CREATE_VALUE) ==
+         XAttrSetFlagProto.XATTR_CREATE_VALUE) {
+       result.add(XAttrSetFlag.CREATE);
+     }
+     if ((flag & XAttrSetFlagProto.XATTR_REPLACE_VALUE) ==
+         XAttrSetFlagProto.XATTR_REPLACE_VALUE) {
+       result.add(XAttrSetFlag.REPLACE);
+     }
+     return result;
+   }
+ 
+   public static XAttr convertXAttr(XAttrProto a) {
+     XAttr.Builder builder = new XAttr.Builder();
+     builder.setNameSpace(convert(a.getNamespace()));
+     if (a.hasName()) {
+       builder.setName(a.getName());
+     }
+     if (a.hasValue()) {
+       builder.setValue(a.getValue().toByteArray());
+     }
+     return builder.build();
+   }
+ 
+   public static GetXAttrsResponseProto convertXAttrsResponse(
+       List<XAttr> xAttrs) {
+     GetXAttrsResponseProto.Builder builder = GetXAttrsResponseProto
+         .newBuilder();
+     if (xAttrs != null) {
+       builder.addAllXAttrs(convertXAttrProto(xAttrs));
+     }
+     return builder.build();
+   }
+ 
+   public static ListXAttrsResponseProto convertListXAttrsResponse(
+     List<XAttr> names) {
+     ListXAttrsResponseProto.Builder builder =
+       ListXAttrsResponseProto.newBuilder();
+     if (names != null) {
+       builder.addAllXAttrs(convertXAttrProto(names));
+     }
+     return builder.build();
+   }
+ 
+   public static EncryptionZoneProto convert(EncryptionZone zone) {
+     return EncryptionZoneProto.newBuilder()
+         .setId(zone.getId())
+         .setPath(zone.getPath())
+         .setSuite(convert(zone.getSuite()))
+         .setCryptoProtocolVersion(convert(zone.getVersion()))
+         .setKeyName(zone.getKeyName())
+         .build();
+   }
+ 
+   public static SlotId convert(ShortCircuitShmSlotProto slotId) {
+     return new SlotId(convert(slotId.getShmId()),
+         slotId.getSlotIdx());
+   }
+ 
+   public static GetEditsFromTxidResponseProto convertEditsResponse(EventBatchList el) {
+     InotifyProtos.EventsListProto.Builder builder =
+         InotifyProtos.EventsListProto.newBuilder();
+     for (EventBatch b : el.getBatches()) {
+       List<InotifyProtos.EventProto> events = Lists.newArrayList();
+       for (Event e : b.getEvents()) {
+         switch (e.getEventType()) {
+           case CLOSE:
+             Event.CloseEvent ce = (Event.CloseEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_CLOSE)
+                 .setContents(
+                     InotifyProtos.CloseEventProto.newBuilder()
+                         .setPath(ce.getPath())
+                         .setFileSize(ce.getFileSize())
+                         .setTimestamp(ce.getTimestamp()).build().toByteString()
+                 ).build());
+             break;
+           case CREATE:
+             Event.CreateEvent ce2 = (Event.CreateEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_CREATE)
+                 .setContents(
+                     InotifyProtos.CreateEventProto.newBuilder()
+                         .setType(createTypeConvert(ce2.getiNodeType()))
+                         .setPath(ce2.getPath())
+                         .setCtime(ce2.getCtime())
+                         .setOwnerName(ce2.getOwnerName())
+                         .setGroupName(ce2.getGroupName())
+                         .setPerms(convert(ce2.getPerms()))
+                         .setReplication(ce2.getReplication())
+                         .setSymlinkTarget(ce2.getSymlinkTarget() == null ?
+                             "" : ce2.getSymlinkTarget())
+                         .setDefaultBlockSize(ce2.getDefaultBlockSize())
+                         .setOverwrite(ce2.getOverwrite()).build().toByteString()
+                 ).build());
+             break;
+           case METADATA:
+             Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
+             InotifyProtos.MetadataUpdateEventProto.Builder metaB =
+                 InotifyProtos.MetadataUpdateEventProto.newBuilder()
+                     .setPath(me.getPath())
+                     .setType(metadataUpdateTypeConvert(me.getMetadataType()))
+                     .setMtime(me.getMtime())
+                     .setAtime(me.getAtime())
+                     .setReplication(me.getReplication())
+                     .setOwnerName(me.getOwnerName() == null ? "" :
+                         me.getOwnerName())
+                     .setGroupName(me.getGroupName() == null ? "" :
+                         me.getGroupName())
+                     .addAllAcls(me.getAcls() == null ?
+                         Lists.<AclEntryProto>newArrayList() :
+                         convertAclEntryProto(me.getAcls()))
+                     .addAllXAttrs(me.getxAttrs() == null ?
+                         Lists.<XAttrProto>newArrayList() :
+                         convertXAttrProto(me.getxAttrs()))
+                     .setXAttrsRemoved(me.isxAttrsRemoved());
+             if (me.getPerms() != null) {
+               metaB.setPerms(convert(me.getPerms()));
+             }
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_METADATA)
+                 .setContents(metaB.build().toByteString())
+                 .build());
+             break;
+           case RENAME:
+             Event.RenameEvent re = (Event.RenameEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_RENAME)
+                 .setContents(
+                     InotifyProtos.RenameEventProto.newBuilder()
+                         .setSrcPath(re.getSrcPath())
+                         .setDestPath(re.getDstPath())
+                         .setTimestamp(re.getTimestamp()).build().toByteString()
+                 ).build());
+             break;
+           case APPEND:
+             Event.AppendEvent re2 = (Event.AppendEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_APPEND)
+                 .setContents(InotifyProtos.AppendEventProto.newBuilder()
+                     .setPath(re2.getPath())
+                     .setNewBlock(re2.toNewBlock()).build().toByteString())
+                 .build());
+             break;
+           case UNLINK:
+             Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_UNLINK)
+                 .setContents(
+                     InotifyProtos.UnlinkEventProto.newBuilder()
+                         .setPath(ue.getPath())
+                         .setTimestamp(ue.getTimestamp()).build().toByteString()
+                 ).build());
+             break;
+           case TRUNCATE:
+             Event.TruncateEvent te = (Event.TruncateEvent) e;
+             events.add(InotifyProtos.EventProto.newBuilder()
+                 .setType(InotifyProtos.EventType.EVENT_TRUNCATE)
+                 .setContents(
+                     InotifyProtos.TruncateEventProto.newBuilder()
+                         .setPath(te.getPath())
+                         .setFileSize(te.getFileSize())
+                         .setTimestamp(te.getTimestamp()).build().toByteString()
+                 ).build());
+             break;
+           default:
+             throw new RuntimeException("Unexpected inotify event: " + e);
+         }
+       }
+       builder.addBatch(InotifyProtos.EventBatchProto.newBuilder().
+           setTxid(b.getTxid()).
+           addAllEvents(events));
+     }
+     builder.setFirstTxid(el.getFirstTxid());
+     builder.setLastTxid(el.getLastTxid());
+     builder.setSyncTxid(el.getSyncTxid());
+     return GetEditsFromTxidResponseProto.newBuilder().setEventsList(
+         builder.build()).build();
+   }
+ 
+   public static CryptoProtocolVersion[] convertCryptoProtocolVersions(
+       List<CryptoProtocolVersionProto> protos) {
+     List<CryptoProtocolVersion> versions =
+         Lists.newArrayListWithCapacity(protos.size());
+     for (CryptoProtocolVersionProto p: protos) {
+       versions.add(convert(p));
+     }
+     return versions.toArray(new CryptoProtocolVersion[]{});
+   }
+ 
+   public static HdfsProtos.PerFileEncryptionInfoProto convertPerFileEncInfo(
+       FileEncryptionInfo info) {
+     if (info == null) {
+       return null;
+     }
+     return HdfsProtos.PerFileEncryptionInfoProto.newBuilder()
+         .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
+         .setIv(getByteString(info.getIV()))
+         .setEzKeyVersionName(info.getEzKeyVersionName())
+         .build();
+   }
+ 
+   public static HdfsProtos.ZoneEncryptionInfoProto convert(
+       CipherSuite suite, CryptoProtocolVersion version, String keyName) {
+     if (suite == null || version == null || keyName == null) {
+       return null;
+     }
+     return HdfsProtos.ZoneEncryptionInfoProto.newBuilder()
+         .setSuite(convert(suite))
+         .setCryptoProtocolVersion(convert(version))
+         .setKeyName(keyName)
+         .build();
+   }
+ 
+   public static FileEncryptionInfo convert(
+       HdfsProtos.PerFileEncryptionInfoProto fileProto,
+       CipherSuite suite, CryptoProtocolVersion version, String keyName) {
+     if (fileProto == null || suite == null || version == null ||
+         keyName == null) {
+       return null;
+     }
+     byte[] key = fileProto.getKey().toByteArray();
+     byte[] iv = fileProto.getIv().toByteArray();
+     String ezKeyVersionName = fileProto.getEzKeyVersionName();
+     return new FileEncryptionInf

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------


[40/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
index dabae2c,0000000..7c64b37
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@@ -1,1014 -1,0 +1,1016 @@@
 +/**
 + * 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.hdfs.server.datanode.erasurecode;
 +
 +import java.io.BufferedOutputStream;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.OutputStream;
 +import java.net.InetSocketAddress;
 +import java.net.Socket;
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.BitSet;
 +import java.util.Collection;
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.concurrent.Callable;
 +import java.util.concurrent.CompletionService;
 +import java.util.concurrent.ExecutorCompletionService;
 +import java.util.concurrent.Future;
 +import java.util.concurrent.LinkedBlockingQueue;
 +import java.util.concurrent.SynchronousQueue;
 +import java.util.concurrent.ThreadPoolExecutor;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.StorageType;
 +import org.apache.hadoop.hdfs.BlockReader;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.DFSPacket;
 +import org.apache.hadoop.hdfs.DFSUtil;
 +import org.apache.hadoop.hdfs.DFSUtilClient;
 +import org.apache.hadoop.hdfs.RemoteBlockReader2;
 +import org.apache.hadoop.hdfs.net.Peer;
 +import org.apache.hadoop.hdfs.net.TcpPeerServer;
 +import org.apache.hadoop.hdfs.protocol.DatanodeID;
 +import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 +import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 +import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 +import org.apache.hadoop.hdfs.server.datanode.DataNode;
 +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
 +import org.apache.hadoop.io.IOUtils;
 +import org.apache.hadoop.io.erasurecode.CodecUtil;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 +import org.apache.hadoop.net.NetUtils;
 +import org.apache.hadoop.security.token.Token;
 +import org.apache.hadoop.util.Daemon;
 +import org.apache.hadoop.util.DataChecksum;
 +
 +import com.google.common.base.Preconditions;
 +
 +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.convertIndex4Decode;
 +
 +/**
 + * ErasureCodingWorker handles the erasure coding recovery work commands. These
 + * commands would be issued from Namenode as part of Datanode's heart beat
 + * response. BPOfferService delegates the work to this class for handling EC
 + * commands.
 + */
 +public final class ErasureCodingWorker {
 +  private static final Log LOG = DataNode.LOG;
 +  
 +  private final DataNode datanode; 
 +  private final Configuration conf;
 +
 +  private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL;
 +  private ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
 +  private final int STRIPED_READ_TIMEOUT_MILLIS;
 +  private final int STRIPED_READ_BUFFER_SIZE;
 +
 +  public ErasureCodingWorker(Configuration conf, DataNode datanode) {
 +    this.datanode = datanode;
 +    this.conf = conf;
 +
 +    STRIPED_READ_TIMEOUT_MILLIS = conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
 +    initializeStripedReadThreadPool(conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_KEY, 
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT));
 +    STRIPED_READ_BUFFER_SIZE = conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT);
 +
 +    initializeStripedBlkRecoveryThreadPool(conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT));
 +  }
 +  
 +  private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) {
 +    return CodecUtil.createRSRawDecoder(conf, numDataUnits, numParityUnits);
 +  }
 +
 +  private void initializeStripedReadThreadPool(int num) {
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Using striped reads; pool threads=" + num);
 +    }
 +    STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
 +        TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
 +        new Daemon.DaemonFactory() {
 +      private final AtomicInteger threadIndex = new AtomicInteger(0);
 +
 +      @Override
 +      public Thread newThread(Runnable r) {
 +        Thread t = super.newThread(r);
 +        t.setName("stripedRead-" + threadIndex.getAndIncrement());
 +        return t;
 +      }
 +    }, new ThreadPoolExecutor.CallerRunsPolicy() {
 +      @Override
 +      public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {
 +        LOG.info("Execution for striped reading rejected, "
 +            + "Executing in current thread");
 +        // will run in the current thread
 +        super.rejectedExecution(runnable, e);
 +      }
 +    });
 +    STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
 +  }
 +
 +  private void initializeStripedBlkRecoveryThreadPool(int num) {
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Using striped block recovery; pool threads=" + num);
 +    }
 +    STRIPED_BLK_RECOVERY_THREAD_POOL = new ThreadPoolExecutor(2, num, 60,
 +        TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
 +        new Daemon.DaemonFactory() {
 +          private final AtomicInteger threadIdx = new AtomicInteger(0);
 +
 +          @Override
 +          public Thread newThread(Runnable r) {
 +            Thread t = super.newThread(r);
 +            t.setName("stripedBlockRecovery-" + threadIdx.getAndIncrement());
 +            return t;
 +          }
 +        });
 +    STRIPED_BLK_RECOVERY_THREAD_POOL.allowCoreThreadTimeOut(true);
 +  }
 +
 +  /**
 +   * Handles the Erasure Coding recovery work commands.
 +   * 
 +   * @param ecTasks
 +   *          BlockECRecoveryInfo
 +   */
 +  public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
 +    for (BlockECRecoveryInfo recoveryInfo : ecTasks) {
 +      try {
 +        STRIPED_BLK_RECOVERY_THREAD_POOL
 +            .submit(new ReconstructAndTransferBlock(recoveryInfo));
 +      } catch (Throwable e) {
 +        LOG.warn("Failed to recover striped block "
 +            + recoveryInfo.getExtendedBlock().getLocalBlock(), e);
 +      }
 +    }
 +  }
 +
 +  /**
 +   * ReconstructAndTransferBlock recover one or more missed striped block in the
 +   * striped block group, the minimum number of live striped blocks should be
 +   * no less than data block number.
 +   * 
 +   * | <- Striped Block Group -> |
 +   *  blk_0      blk_1       blk_2(*)   blk_3   ...   <- A striped block group
 +   *    |          |           |          |  
 +   *    v          v           v          v 
 +   * +------+   +------+   +------+   +------+
 +   * |cell_0|   |cell_1|   |cell_2|   |cell_3|  ...    
 +   * +------+   +------+   +------+   +------+     
 +   * |cell_4|   |cell_5|   |cell_6|   |cell_7|  ...
 +   * +------+   +------+   +------+   +------+
 +   * |cell_8|   |cell_9|   |cell10|   |cell11|  ...
 +   * +------+   +------+   +------+   +------+
 +   *  ...         ...       ...         ...
 +   *  
 +   * 
 +   * We use following steps to recover striped block group, in each round, we
 +   * recover <code>bufferSize</code> data until finish, the 
 +   * <code>bufferSize</code> is configurable and may be less or larger than 
 +   * cell size:
 +   * step1: read <code>bufferSize</code> data from minimum number of sources 
 +   *        required by recovery.
 +   * step2: decode data for targets.
 +   * step3: transfer data to targets.
 +   * 
 +   * In step1, try to read <code>bufferSize</code> data from minimum number
 +   * of sources , if there is corrupt or stale sources, read from new source
 +   * will be scheduled. The best sources are remembered for next round and 
 +   * may be updated in each round.
 +   * 
 +   * In step2, typically if source blocks we read are all data blocks, we 
 +   * need to call encode, and if there is one parity block, we need to call
 +   * decode. Notice we only read once and recover all missed striped block 
 +   * if they are more than one.
 +   * 
 +   * In step3, send the recovered data to targets by constructing packet 
 +   * and send them directly. Same as continuous block replication, we 
 +   * don't check the packet ack. Since the datanode doing the recovery work
 +   * are one of the source datanodes, so the recovered data are sent 
 +   * remotely.
 +   * 
 +   * There are some points we can do further improvements in next phase:
 +   * 1. we can read the block file directly on the local datanode, 
 +   *    currently we use remote block reader. (Notice short-circuit is not
 +   *    a good choice, see inline comments).
 +   * 2. We need to check the packet ack for EC recovery? Since EC recovery
 +   *    is more expensive than continuous block replication, it needs to 
 +   *    read from several other datanodes, should we make sure the 
 +   *    recovered result received by targets? 
 +   */
 +  private class ReconstructAndTransferBlock implements Runnable {
 +    private final int dataBlkNum;
 +    private final int parityBlkNum;
 +    private final int cellSize;
 +    
 +    private RawErasureDecoder decoder;
 +
 +    // Striped read buffer size
 +    private int bufferSize;
 +
 +    private final ExtendedBlock blockGroup;
 +    private final int minRequiredSources;
 +    // position in striped internal block
 +    private long positionInBlock;
 +
 +    // sources
 +    private final short[] liveIndices;
 +    private final DatanodeInfo[] sources;
 +
 +    private final List<StripedReader> stripedReaders;
 +
 +    // The buffers and indices for striped blocks whose length is 0
 +    private ByteBuffer[] zeroStripeBuffers;
 +    private short[] zeroStripeIndices;
 +
 +    // targets
 +    private final DatanodeInfo[] targets;
 +    private final StorageType[] targetStorageTypes;
 +
 +    private final short[] targetIndices;
 +    private final ByteBuffer[] targetBuffers;
 +
 +    private final Socket[] targetSockets;
 +    private final DataOutputStream[] targetOutputStreams;
 +    private final DataInputStream[] targetInputStreams;
 +
 +    private final long[] blockOffset4Targets;
 +    private final long[] seqNo4Targets;
 +
 +    private final static int WRITE_PACKET_SIZE = 64 * 1024;
 +    private DataChecksum checksum;
 +    private int maxChunksPerPacket;
 +    private byte[] packetBuf;
 +    private byte[] checksumBuf;
 +    private int bytesPerChecksum;
 +    private int checksumSize;
 +
 +    private final CachingStrategy cachingStrategy;
 +
 +    private final Map<Future<Void>, Integer> futures = new HashMap<>();
 +    private final CompletionService<Void> readService =
 +        new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL);
 +
 +    ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) {
 +      ErasureCodingPolicy ecPolicy = recoveryInfo.getErasureCodingPolicy();
 +      dataBlkNum = ecPolicy.getNumDataUnits();
 +      parityBlkNum = ecPolicy.getNumParityUnits();
 +      cellSize = ecPolicy.getCellSize();
 +
 +      blockGroup = recoveryInfo.getExtendedBlock();
 +      final int cellsNum = (int)((blockGroup.getNumBytes() - 1) / cellSize + 1);
 +      minRequiredSources = Math.min(cellsNum, dataBlkNum);
 +
 +      liveIndices = recoveryInfo.getLiveBlockIndices();
 +      sources = recoveryInfo.getSourceDnInfos();
 +      stripedReaders = new ArrayList<>(sources.length);
 +
 +      Preconditions.checkArgument(liveIndices.length >= minRequiredSources,
 +          "No enough live striped blocks.");
 +      Preconditions.checkArgument(liveIndices.length == sources.length,
 +          "liveBlockIndices and source dns should match");
 +
 +      if (minRequiredSources < dataBlkNum) {
 +        zeroStripeBuffers = 
 +            new ByteBuffer[dataBlkNum - minRequiredSources];
 +        zeroStripeIndices = new short[dataBlkNum - minRequiredSources];
 +      }
 +
 +      targets = recoveryInfo.getTargetDnInfos();
 +      targetStorageTypes = recoveryInfo.getTargetStorageTypes();
 +      targetIndices = new short[targets.length];
 +      targetBuffers = new ByteBuffer[targets.length];
 +
 +      Preconditions.checkArgument(targetIndices.length <= parityBlkNum,
 +          "Too much missed striped blocks.");
 +
 +      targetSockets = new Socket[targets.length];
 +      targetOutputStreams = new DataOutputStream[targets.length];
 +      targetInputStreams = new DataInputStream[targets.length];
 +
 +      blockOffset4Targets = new long[targets.length];
 +      seqNo4Targets = new long[targets.length];
 +
 +      for (int i = 0; i < targets.length; i++) {
 +        blockOffset4Targets[i] = 0;
 +        seqNo4Targets[i] = 0;
 +      }
 +
 +      getTargetIndices();
 +      cachingStrategy = CachingStrategy.newDefaultStrategy();
 +    }
 +
 +    private ByteBuffer allocateBuffer(int length) {
 +      return ByteBuffer.allocate(length);
 +    }
 +
 +    private ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) {
 +      return StripedBlockUtil.constructInternalBlock(blockGroup, cellSize,
 +          dataBlkNum, i);
 +    }
 +
 +    private long getBlockLen(ExtendedBlock blockGroup, int i) { 
 +      return StripedBlockUtil.getInternalBlockLength(blockGroup.getNumBytes(),
 +          cellSize, dataBlkNum, i);
 +    }
 +
 +    /**
 +     * StripedReader is used to read from one source DN, it contains a block
 +     * reader, buffer and striped block index.
 +     * Only allocate StripedReader once for one source, and the StripedReader
 +     * has the same array order with sources. Typically we only need to allocate
 +     * minimum number (minRequiredSources) of StripedReader, and allocate
 +     * new for new source DN if some existing DN invalid or slow.
 +     * If some source DN is corrupt, set the corresponding blockReader to 
 +     * null and will never read from it again.
 +     *  
 +     * @param i the array index of sources
 +     * @param offsetInBlock offset for the internal block
 +     * @return StripedReader
 +     */
 +    private StripedReader addStripedReader(int i, long offsetInBlock) {
 +      StripedReader reader = new StripedReader(liveIndices[i]);
 +      stripedReaders.add(reader);
 +
 +      BlockReader blockReader = newBlockReader(
 +          getBlock(blockGroup, liveIndices[i]), offsetInBlock, sources[i]);
 +      if (blockReader != null) {
 +        initChecksumAndBufferSizeIfNeeded(blockReader);
 +        reader.blockReader = blockReader;
 +      }
 +      reader.buffer = allocateBuffer(bufferSize);
 +      return reader;
 +    }
 +
 +    @Override
 +    public void run() {
 +      datanode.incrementXmitsInProgress();
 +      try {
 +        // Store the array indices of source DNs we have read successfully.
 +        // In each iteration of read, the success list may be updated if
 +        // some source DN is corrupted or slow. And use the updated success
 +        // list of DNs for next iteration read.
 +        int[] success = new int[minRequiredSources];
 +
 +        int nsuccess = 0;
 +        for (int i = 0; 
 +            i < sources.length && nsuccess < minRequiredSources; i++) {
 +          StripedReader reader = addStripedReader(i, 0);
 +          if (reader.blockReader != null) {
 +            success[nsuccess++] = i;
 +          }
 +        }
 +
 +        if (nsuccess < minRequiredSources) {
 +          String error = "Can't find minimum sources required by "
 +              + "recovery, block id: " + blockGroup.getBlockId();
 +          throw new IOException(error);
 +        }
 +
 +        if (zeroStripeBuffers != null) {
 +          for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +            zeroStripeBuffers[i] = allocateBuffer(bufferSize);
 +          }
 +        }
 +
 +        for (int i = 0; i < targets.length; i++) {
 +          targetBuffers[i] = allocateBuffer(bufferSize);
 +        }
 +
 +        checksumSize = checksum.getChecksumSize();
 +        int chunkSize = bytesPerChecksum + checksumSize;
 +        maxChunksPerPacket = Math.max(
 +            (WRITE_PACKET_SIZE - PacketHeader.PKT_MAX_HEADER_LEN)/chunkSize, 1);
 +        int maxPacketSize = chunkSize * maxChunksPerPacket 
 +            + PacketHeader.PKT_MAX_HEADER_LEN;
 +
 +        packetBuf = new byte[maxPacketSize];
 +        checksumBuf = new byte[checksumSize * (bufferSize / bytesPerChecksum)];
 +
 +        // targetsStatus store whether some target is success, it will record
 +        // any failed target once, if some target failed (invalid DN or transfer
 +        // failed), will not transfer data to it any more.
 +        boolean[] targetsStatus = new boolean[targets.length];
 +        if (initTargetStreams(targetsStatus) == 0) {
 +          String error = "All targets are failed.";
 +          throw new IOException(error);
 +        }
 +
 +        long firstStripedBlockLength = getBlockLen(blockGroup, 0);
 +        while (positionInBlock < firstStripedBlockLength) {
 +          int toRead = Math.min(
 +              bufferSize, (int)(firstStripedBlockLength - positionInBlock));
 +          // step1: read from minimum source DNs required for reconstruction.
 +          //   The returned success list is the source DNs we do real read from
 +          success = readMinimumStripedData4Recovery(success);
 +
 +          // step2: decode to reconstruct targets
 +          long remaining = firstStripedBlockLength - positionInBlock;
 +          int toRecoverLen = remaining < bufferSize ? 
 +              (int)remaining : bufferSize;
 +          recoverTargets(success, targetsStatus, toRecoverLen);
 +
 +          // step3: transfer data
 +          if (transferData2Targets(targetsStatus) == 0) {
 +            String error = "Transfer failed for all targets.";
 +            throw new IOException(error);
 +          }
 +
 +          clearBuffers();
 +          positionInBlock += toRead;
 +        }
 +
 +        endTargetBlocks(targetsStatus);
 +
 +        // Currently we don't check the acks for packets, this is similar as
 +        // block replication.
 +      } catch (Throwable e) {
 +        LOG.warn("Failed to recover striped block: " + blockGroup, e);
 +      } finally {
 +        datanode.decrementXmitsInProgress();
 +        // close block readers
 +        for (StripedReader stripedReader : stripedReaders) {
 +          closeBlockReader(stripedReader.blockReader);
 +        }
 +        for (int i = 0; i < targets.length; i++) {
 +          IOUtils.closeStream(targetOutputStreams[i]);
 +          IOUtils.closeStream(targetInputStreams[i]);
 +          IOUtils.closeStream(targetSockets[i]);
 +        }
 +      }
 +    }
 +
 +    // init checksum from block reader
 +    private void initChecksumAndBufferSizeIfNeeded(BlockReader blockReader) {
 +      if (checksum == null) {
 +        checksum = blockReader.getDataChecksum();
 +        bytesPerChecksum = checksum.getBytesPerChecksum();
 +        // The bufferSize is flat to divide bytesPerChecksum
 +        int readBufferSize = STRIPED_READ_BUFFER_SIZE;
 +        bufferSize = readBufferSize < bytesPerChecksum ? bytesPerChecksum :
 +          readBufferSize - readBufferSize % bytesPerChecksum;
 +      } else {
 +        assert blockReader.getDataChecksum().equals(checksum);
 +      }
 +    }
 +
 +    private void getTargetIndices() {
 +      BitSet bitset = new BitSet(dataBlkNum + parityBlkNum);
 +      for (int i = 0; i < sources.length; i++) {
 +        bitset.set(liveIndices[i]);
 +      }
 +      int m = 0;
 +      int k = 0;
 +      for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {
 +        if (!bitset.get(i)) {
 +          if (getBlockLen(blockGroup, i) > 0) {
 +            if (m < targets.length) {
 +              targetIndices[m++] = (short)i;
 +            }
 +          } else {
 +            zeroStripeIndices[k++] = (short)i;
 +          }
 +        }
 +      }
 +    }
 +
 +    private long getReadLength(int index) {
 +      long blockLen = getBlockLen(blockGroup, index);
 +      long remaining = blockLen - positionInBlock;
 +      return remaining > bufferSize ? bufferSize : remaining;
 +    }
 +
 +    /**
 +     * Read from minimum source DNs required for reconstruction in the iteration.
 +     * First try the success list which we think they are the best DNs
 +     * If source DN is corrupt or slow, try to read some other source DN, 
 +     * and will update the success list. 
 +     * 
 +     * Remember the updated success list and return it for following 
 +     * operations and next iteration read.
 +     * 
 +     * @param success the initial success list of source DNs we think best
 +     * @return updated success list of source DNs we do real read
 +     * @throws IOException
 +     */
 +    private int[] readMinimumStripedData4Recovery(final int[] success)
 +        throws IOException {
 +      int nsuccess = 0;
 +      int[] newSuccess = new int[minRequiredSources];
 +      BitSet used = new BitSet(sources.length);
 +      /*
 +       * Read from minimum source DNs required, the success list contains
 +       * source DNs which we think best.
 +       */
 +      for (int i = 0; i < minRequiredSources; i++) {
 +        StripedReader reader = stripedReaders.get(success[i]);
 +        if (getReadLength(liveIndices[success[i]]) > 0) {
 +          Callable<Void> readCallable = readFromBlock(
 +              reader.blockReader, reader.buffer);
 +          Future<Void> f = readService.submit(readCallable);
 +          futures.put(f, success[i]);
 +        } else {
 +          // If the read length is 0, we don't need to do real read
 +          reader.buffer.position(0);
 +          newSuccess[nsuccess++] = success[i];
 +        }
 +        used.set(success[i]);
 +      }
 +
 +      while (!futures.isEmpty()) {
 +        try {
 +          StripingChunkReadResult result =
 +              StripedBlockUtil.getNextCompletedStripedRead(
 +                  readService, futures, STRIPED_READ_TIMEOUT_MILLIS);
 +          int resultIndex = -1;
 +          if (result.state == StripingChunkReadResult.SUCCESSFUL) {
 +            resultIndex = result.index;
 +          } else if (result.state == StripingChunkReadResult.FAILED) {
 +            // If read failed for some source DN, we should not use it anymore 
 +            // and schedule read from another source DN.
 +            StripedReader failedReader = stripedReaders.get(result.index);
 +            closeBlockReader(failedReader.blockReader);
 +            failedReader.blockReader = null;
 +            resultIndex = scheduleNewRead(used);
 +          } else if (result.state == StripingChunkReadResult.TIMEOUT) {
 +            // If timeout, we also schedule a new read.
 +            resultIndex = scheduleNewRead(used);
 +          }
 +          if (resultIndex >= 0) {
 +            newSuccess[nsuccess++] = resultIndex;
 +            if (nsuccess >= minRequiredSources) {
 +              // cancel remaining reads if we read successfully from minimum
 +              // number of source DNs required by reconstruction.
 +              cancelReads(futures.keySet());
 +              futures.clear();
 +              break;
 +            }
 +          }
 +        } catch (InterruptedException e) {
 +          LOG.info("Read data interrupted.", e);
 +          break;
 +        }
 +      }
 +
 +      if (nsuccess < minRequiredSources) {
 +        String error = "Can't read data from minimum number of sources "
 +            + "required by reconstruction, block id: " + blockGroup.getBlockId();
 +        throw new IOException(error);
 +      }
 +
 +      return newSuccess;
 +    }
 +    
 +    private void paddingBufferToLen(ByteBuffer buffer, int len) {
 +      int toPadding = len - buffer.position();
 +      for (int i = 0; i < toPadding; i++) {
 +        buffer.put((byte) 0);
 +      }
 +    }
 +    
 +    // Initialize decoder
 +    private void initDecoderIfNecessary() {
 +      if (decoder == null) {
 +        decoder = newDecoder(dataBlkNum, parityBlkNum);
 +      }
 +    }
 +
 +    private int[] getErasedIndices(boolean[] targetsStatus) {
 +      int[] result = new int[targets.length];
 +      int m = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          result[m++] = convertIndex4Decode(targetIndices[i], 
 +              dataBlkNum, parityBlkNum);
 +        }
 +      }
 +      return Arrays.copyOf(result, m);
 +    }
 +
 +    private void recoverTargets(int[] success, boolean[] targetsStatus,
 +        int toRecoverLen) {
 +      initDecoderIfNecessary();
 +      ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum];
 +      for (int i = 0; i < success.length; i++) {
 +        StripedReader reader = stripedReaders.get(success[i]);
 +        ByteBuffer buffer = reader.buffer;
 +        paddingBufferToLen(buffer, toRecoverLen);
 +        inputs[convertIndex4Decode(reader.index, dataBlkNum, parityBlkNum)] = 
 +            (ByteBuffer)buffer.flip();
 +      }
 +      if (success.length < dataBlkNum) {
 +        for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +          ByteBuffer buffer = zeroStripeBuffers[i];
 +          paddingBufferToLen(buffer, toRecoverLen);
 +          int index = convertIndex4Decode(zeroStripeIndices[i], dataBlkNum,
 +              parityBlkNum);
 +          inputs[index] = (ByteBuffer)buffer.flip();
 +        }
 +      }
 +      int[] erasedIndices = getErasedIndices(targetsStatus);
 +      ByteBuffer[] outputs = new ByteBuffer[erasedIndices.length];
 +      int m = 0;
 +      for (int i = 0; i < targetBuffers.length; i++) {
 +        if (targetsStatus[i]) {
 +          outputs[m++] = targetBuffers[i];
 +          outputs[i].limit(toRecoverLen);
 +        }
 +      }
 +      decoder.decode(inputs, erasedIndices, outputs);
 +
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          long blockLen = getBlockLen(blockGroup, targetIndices[i]);
 +          long remaining = blockLen - positionInBlock;
 +          if (remaining < 0) {
 +            targetBuffers[i].limit(0);
 +          } else if (remaining < toRecoverLen) {
 +            targetBuffers[i].limit((int)remaining);
 +          }
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Schedule a read from some new source DN if some DN is corrupted
 +     * or slow, this is called from the read iteration.
 +     * Initially we may only have <code>minRequiredSources</code> number of 
 +     * StripedReader.
 +     * If the position is at the end of target block, don't need to do 
 +     * real read, and return the array index of source DN, otherwise -1.
 +     * 
 +     * @param used the used source DNs in this iteration.
 +     * @return the array index of source DN if don't need to do real read.
 +     */
 +    private int scheduleNewRead(BitSet used) {
 +      StripedReader reader = null;
 +      // step1: initially we may only have <code>minRequiredSources</code>
 +      // number of StripedReader, and there may be some source DNs we never 
 +      // read before, so will try to create StripedReader for one new source DN
 +      // and try to read from it. If found, go to step 3.
 +      int m = stripedReaders.size();
 +      while (reader == null && m < sources.length) {
 +        reader = addStripedReader(m, positionInBlock);
 +        if (getReadLength(liveIndices[m]) > 0) {
 +          if (reader.blockReader == null) {
 +            reader = null;
 +            m++;
 +          }
 +        } else {
 +          used.set(m);
 +          return m;
 +        }
 +      }
 +
 +      // step2: if there is no new source DN we can use, try to find a source 
 +      // DN we ever read from but because some reason, e.g., slow, it
 +      // is not in the success DN list at the begin of this iteration, so 
 +      // we have not tried it in this iteration. Now we have a chance to 
 +      // revisit it again.
 +      for (int i = 0; reader == null && i < stripedReaders.size(); i++) {
 +        if (!used.get(i)) {
 +          StripedReader r = stripedReaders.get(i);
 +          if (getReadLength(liveIndices[i]) > 0) {
 +            closeBlockReader(r.blockReader);
 +            r.blockReader = newBlockReader(
 +                getBlock(blockGroup, liveIndices[i]), positionInBlock,
 +                sources[i]);
 +            if (r.blockReader != null) {
 +              m = i;
 +              reader = r;
 +            }
 +          } else {
 +            used.set(i);
 +            r.buffer.position(0);
 +            return i;
 +          }
 +        }
 +      }
 +
 +      // step3: schedule if find a correct source DN and need to do real read.
 +      if (reader != null) {
 +        Callable<Void> readCallable = readFromBlock(
 +            reader.blockReader, reader.buffer);
 +        Future<Void> f = readService.submit(readCallable);
 +        futures.put(f, m);
 +        used.set(m);
 +      }
 +
 +      return -1;
 +    }
 +
 +    // cancel all reads.
 +    private void cancelReads(Collection<Future<Void>> futures) {
 +      for (Future<Void> future : futures) {
 +        future.cancel(true);
 +      }
 +    }
 +
 +    private Callable<Void> readFromBlock(final BlockReader reader,
 +        final ByteBuffer buf) {
 +      return new Callable<Void>() {
 +
 +        @Override
 +        public Void call() throws Exception {
 +          try {
 +            actualReadFromBlock(reader, buf);
 +            return null;
 +          } catch (IOException e) {
 +            LOG.info(e.getMessage());
 +            throw e;
 +          }
 +        }
 +
 +      };
 +    }
 +
 +    /**
 +     * Read bytes from block
 +     */
 +    private void actualReadFromBlock(BlockReader reader, ByteBuffer buf)
 +        throws IOException {
 +      int len = buf.remaining();
 +      int n = 0;
 +      while (n < len) {
 +        int nread = reader.read(buf);
 +        if (nread <= 0) {
 +          break;
 +        }
 +        n += nread;
 +      }
 +    }
 +
 +    // close block reader
 +    private void closeBlockReader(BlockReader blockReader) {
 +      try {
 +        if (blockReader != null) {
 +          blockReader.close();
 +        }
 +      } catch (IOException e) {
 +        // ignore
 +      }
 +    }
 +
 +    private InetSocketAddress getSocketAddress4Transfer(DatanodeInfo dnInfo) {
 +      return NetUtils.createSocketAddr(dnInfo.getXferAddr(
 +          datanode.getDnConf().getConnectToDnViaHostname()));
 +    }
 +
 +    private BlockReader newBlockReader(final ExtendedBlock block, 
 +        long offsetInBlock, DatanodeInfo dnInfo) {
 +      if (offsetInBlock >= block.getNumBytes()) {
 +        return null;
 +      }
 +      try {
 +        InetSocketAddress dnAddr = getSocketAddress4Transfer(dnInfo);
 +        Token<BlockTokenIdentifier> blockToken = datanode.getBlockAccessToken(
 +            block, EnumSet.of(BlockTokenIdentifier.AccessMode.READ));
 +        /*
 +         * This can be further improved if the replica is local, then we can
 +         * read directly from DN and need to check the replica is FINALIZED
 +         * state, notice we should not use short-circuit local read which
 +         * requires config for domain-socket in UNIX or legacy config in Windows.
++         *
++         * TODO: add proper tracer
 +         */
 +        return RemoteBlockReader2.newBlockReader(
 +            "dummy", block, blockToken, offsetInBlock, 
 +            block.getNumBytes() - offsetInBlock, true,
 +            "", newConnectedPeer(block, dnAddr, blockToken, dnInfo), dnInfo,
-             null, cachingStrategy);
++            null, cachingStrategy, null);
 +      } catch (IOException e) {
 +        return null;
 +      }
 +    }
 +
 +    private Peer newConnectedPeer(ExtendedBlock b, InetSocketAddress addr,
 +        Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
 +        throws IOException {
 +      Peer peer = null;
 +      boolean success = false;
 +      Socket sock = null;
 +      final int socketTimeout = datanode.getDnConf().getSocketTimeout(); 
 +      try {
 +        sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
 +        NetUtils.connect(sock, addr, socketTimeout);
 +        peer = DFSUtilClient.peerFromSocketAndKey(datanode.getSaslClient(),
 +            sock, datanode.getDataEncryptionKeyFactoryForBlock(b),
 +            blockToken, datanodeId);
 +        peer.setReadTimeout(socketTimeout);
 +        success = true;
 +        return peer;
 +      } finally {
 +        if (!success) {
 +          IOUtils.cleanup(LOG, peer);
 +          IOUtils.closeSocket(sock);
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Send data to targets
 +     */
 +    private int transferData2Targets(boolean[] targetsStatus) {
 +      int nsuccess = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          boolean success = false;
 +          try {
 +            ByteBuffer buffer = targetBuffers[i];
 +            
 +            if (buffer.remaining() == 0) {
 +              continue;
 +            }
 +
 +            checksum.calculateChunkedSums(
 +                buffer.array(), 0, buffer.remaining(), checksumBuf, 0);
 +
 +            int ckOff = 0;
 +            while (buffer.remaining() > 0) {
 +              DFSPacket packet = new DFSPacket(packetBuf, maxChunksPerPacket,
 +                  blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, false);
 +              int maxBytesToPacket = maxChunksPerPacket * bytesPerChecksum;
 +              int toWrite = buffer.remaining() > maxBytesToPacket ?
 +                  maxBytesToPacket : buffer.remaining();
 +              int ckLen = ((toWrite - 1) / bytesPerChecksum + 1) * checksumSize;
 +              packet.writeChecksum(checksumBuf, ckOff, ckLen);
 +              ckOff += ckLen;
 +              packet.writeData(buffer, toWrite);
 +
 +              // Send packet
 +              packet.writeTo(targetOutputStreams[i]);
 +
 +              blockOffset4Targets[i] += toWrite;
 +              nsuccess++;
 +              success = true;
 +            }
 +          } catch (IOException e) {
 +            LOG.warn(e.getMessage());
 +          }
 +          targetsStatus[i] = success;
 +        }
 +      }
 +      return nsuccess;
 +    }
 +
 +    /**
 +     * clear all buffers
 +     */
 +    private void clearBuffers() {
 +      for (StripedReader stripedReader : stripedReaders) {
 +        if (stripedReader.buffer != null) {
 +          stripedReader.buffer.clear();
 +        }
 +      }
 +
 +      if (zeroStripeBuffers != null) {
 +        for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +          zeroStripeBuffers[i].clear();
 +        }
 +      }
 +
 +      for (int i = 0; i < targetBuffers.length; i++) {
 +        if (targetBuffers[i] != null) {
 +          cleanBuffer(targetBuffers[i]);
 +        }
 +      }
 +    }
 +    
 +    private ByteBuffer cleanBuffer(ByteBuffer buffer) {
 +      Arrays.fill(buffer.array(), (byte) 0);
 +      return (ByteBuffer)buffer.clear();
 +    }
 +
 +    // send an empty packet to mark the end of the block
 +    private void endTargetBlocks(boolean[] targetsStatus) {
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          try {
 +            DFSPacket packet = new DFSPacket(packetBuf, 0, 
 +                blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, true);
 +            packet.writeTo(targetOutputStreams[i]);
 +            targetOutputStreams[i].flush();
 +          } catch (IOException e) {
 +            LOG.warn(e.getMessage());
 +          }
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Initialize  output/input streams for transferring data to target
 +     * and send create block request. 
 +     */
 +    private int initTargetStreams(boolean[] targetsStatus) {
 +      int nsuccess = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        Socket socket = null;
 +        DataOutputStream out = null;
 +        DataInputStream in = null;
 +        boolean success = false;
 +        try {
 +          InetSocketAddress targetAddr = 
 +              getSocketAddress4Transfer(targets[i]);
 +          socket = datanode.newSocket();
 +          NetUtils.connect(socket, targetAddr, 
 +              datanode.getDnConf().getSocketTimeout());
 +          socket.setSoTimeout(datanode.getDnConf().getSocketTimeout());
 +
 +          ExtendedBlock block = getBlock(blockGroup, targetIndices[i]);
 +          Token<BlockTokenIdentifier> blockToken = 
 +              datanode.getBlockAccessToken(block,
 +                  EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
 +
 +          long writeTimeout = datanode.getDnConf().getSocketWriteTimeout();
 +          OutputStream unbufOut = NetUtils.getOutputStream(socket, writeTimeout);
 +          InputStream unbufIn = NetUtils.getInputStream(socket);
 +          DataEncryptionKeyFactory keyFactory =
 +            datanode.getDataEncryptionKeyFactoryForBlock(block);
 +          IOStreamPair saslStreams = datanode.getSaslClient().socketSend(
 +              socket, unbufOut, unbufIn, keyFactory, blockToken, targets[i]);
 +
 +          unbufOut = saslStreams.out;
 +          unbufIn = saslStreams.in;
 +
 +          out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-               DFSUtil.getSmallBufferSize(conf)));
++              DFSUtilClient.getSmallBufferSize(conf)));
 +          in = new DataInputStream(unbufIn);
 +
 +          DatanodeInfo source = new DatanodeInfo(datanode.getDatanodeId());
 +          new Sender(out).writeBlock(block, targetStorageTypes[i], 
 +              blockToken, "", new DatanodeInfo[]{targets[i]}, 
 +              new StorageType[]{targetStorageTypes[i]}, source, 
 +              BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, 0, 
 +              checksum, cachingStrategy, false, false, null);
 +
 +          targetSockets[i] = socket;
 +          targetOutputStreams[i] = out;
 +          targetInputStreams[i] = in;
 +          nsuccess++;
 +          success = true;
 +        } catch (Throwable e) {
 +          LOG.warn(e.getMessage());
 +        } finally {
 +          if (!success) {
 +            IOUtils.closeStream(out);
 +            IOUtils.closeStream(in);
 +            IOUtils.closeStream(socket);
 +          }
 +        }
 +        targetsStatus[i] = success;
 +      }
 +      return nsuccess;
 +    }
 +  }
 +
 +  private static class StripedReader {
 +    private final short index; // internal block index
 +    private BlockReader blockReader;
 +    private ByteBuffer buffer;
 +
 +    private StripedReader(short index) {
 +      this.index = index;
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 3e001d3,98af592..67c6fc1
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@@ -180,9 -178,8 +180,10 @@@ class FSDirStatAndListingOp 
        }
  
        final FileEncryptionInfo feInfo = isReservedName ? null
-           : fsd.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
+           : FSDirEncryptionZoneOp.getFileEncryptionInfo(fsd, inode,
 -              iip.getPathSnapshotId(), iip);
++          iip.getPathSnapshotId(), iip);
 +      final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.
 +          getErasureCodingPolicy(fsd.getFSNamesystem(), iip);
  
        final LocatedBlocks blocks = bm.createLocatedBlocks(
            inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
@@@ -443,12 -440,9 +444,12 @@@
      long blocksize = 0;
      final boolean isEncrypted;
  
-     final FileEncryptionInfo feInfo = isRawPath ? null :
-         fsd.getFileEncryptionInfo(node, snapshot, iip);
+     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
+         .getFileEncryptionInfo(fsd, node, snapshot, iip);
  
 +    final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
 +        fsd.getFSNamesystem(), iip);
 +
      if (node.isFile()) {
        final INodeFile fileNode = node.asFile();
        size = fileNode.computeFileSize(snapshot);
@@@ -500,10 -495,8 +503,10 @@@
      long blocksize = 0;
      LocatedBlocks loc = null;
      final boolean isEncrypted;
-     final FileEncryptionInfo feInfo = isRawPath ? null :
-         fsd.getFileEncryptionInfo(node, snapshot, iip);
+     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
+         .getFileEncryptionInfo(fsd, node, snapshot, iip);
 +    final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
 +        fsd.getFSNamesystem(), iip);
      if (node.isFile()) {
        final INodeFile fileNode = node.asFile();
        size = fileNode.computeFileSize(snapshot);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index b6b151c,4dda27d..a94b61c
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@@ -131,11 -131,9 +131,10 @@@ import org.apache.commons.logging.impl.
  import org.apache.hadoop.HadoopIllegalArgumentException;
  import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.conf.Configuration;
- import org.apache.hadoop.crypto.CipherSuite;
  import org.apache.hadoop.crypto.CryptoProtocolVersion;
- import org.apache.hadoop.crypto.key.KeyProvider;
+ import org.apache.hadoop.crypto.key.KeyProvider.Metadata;
  import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 +import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
  import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
  import org.apache.hadoop.fs.CacheFlag;
  import org.apache.hadoop.fs.ContentSummary;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 4143964,79a3773..e9363b4
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@@ -2057,18 -2043,6 +2061,18 @@@ class NameNodeRpcServer implements Name
    public void removeSpanReceiver(long id) throws IOException {
      checkNNStartup();
      namesystem.checkSuperuserPrivilege();
-     nn.spanReceiverHost.removeSpanReceiver(id);
+     nn.tracerConfigurationManager.removeSpanReceiver(id);
    }
 +
 +  @Override // ClientProtocol
 +  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
 +    checkNNStartup();
 +    return namesystem.getErasureCodingPolicies();
 +  }
 +
 +  @Override // ClientProtocol
 +  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
 +    checkNNStartup();
 +    return namesystem.getErasureCodingPolicy(src);
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 9d24b91,727259f..6dd7b89
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@@ -33,7 -33,7 +33,8 @@@ package hadoop.hdfs.datanode
  
  import "HAServiceProtocol.proto";
  import "hdfs.proto";
 +import "erasurecoding.proto";
+ import "HdfsServer.proto";
  
  /**
   * Information to identify a datanode to a namenode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
index 0000000,3b60e51..66b2a33
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
@@@ -1,0 -1,201 +1,198 @@@
+ /**
+  * 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.
+  */
+ 
+ /**
+  * These .proto interfaces are private and stable.
+  * Please see http://wiki.apache.org/hadoop/Compatibility
+  * for what changes are allowed for a *stable* .proto interface.
+  */
+ 
+ // This file contains protocol buffers that are used throughout HDFS -- i.e.
+ // by the client, server, and data transfer protocols.
+ 
+ 
+ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+ option java_outer_classname = "HdfsServerProtos";
+ option java_generate_equals_and_hash = true;
+ package hadoop.hdfs;
+ 
+ import "hdfs.proto";
+ 
+ /**
 - * A list of storage IDs.
 - */
 -message StorageUuidsProto {
 -  repeated string storageUuids = 1;
 -}
 -
 -/**
+  * Block access token information
+  */
+ message BlockKeyProto {
+   required uint32 keyId = 1;      // Key identifier
+   required uint64 expiryDate = 2; // Expiry time in milliseconds
+   optional bytes keyBytes = 3;    // Key secret
+ }
+ 
+ /**
+  * Current key and set of block keys at the namenode.
+  */
+ message ExportedBlockKeysProto {
+   required bool isBlockTokenEnabled = 1;
+   required uint64 keyUpdateInterval = 2;
+   required uint64 tokenLifeTime = 3;
+   required BlockKeyProto currentKey = 4;
+   repeated BlockKeyProto allKeys = 5;
+ }
+ 
+ /**
+  * Block and datanodes where is it located
+  */
+ message BlockWithLocationsProto {
+   required BlockProto block = 1;   // Block
+   repeated string datanodeUuids = 2; // Datanodes with replicas of the block
+   repeated string storageUuids = 3;  // Storages with replicas of the block
+   repeated StorageTypeProto storageTypes = 4;
++
++  optional bytes indices = 5;
++  optional uint32 dataBlockNum = 6;
++  optional uint32 cellSize = 7;
+ }
+ 
+ /**
+  * List of block with locations
+  */
+ message BlocksWithLocationsProto {
+   repeated BlockWithLocationsProto blocks = 1;
+ }
+ 
+ /**
+  * Editlog information with available transactions
+  */
+ message RemoteEditLogProto {
+   required uint64 startTxId = 1;  // Starting available edit log transaction
+   required uint64 endTxId = 2;    // Ending available edit log transaction
+   optional bool isInProgress = 3 [default = false];
+ }
+ 
+ /**
+  * Enumeration of editlogs available on a remote namenode
+  */
+ message RemoteEditLogManifestProto {
+   repeated RemoteEditLogProto logs = 1;
+ }
+ 
+ /**
+  * Namespace information that describes namespace on a namenode
+  */
+ message NamespaceInfoProto {
+   required string buildVersion = 1;         // Software revision version (e.g. an svn or git revision)
+   required uint32 unused = 2;               // Retained for backward compatibility
+   required string blockPoolID = 3;          // block pool used by the namespace
+   required StorageInfoProto storageInfo = 4;// Node information
+   required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
+   optional uint64 capabilities = 6 [default = 0]; // feature flags
+ }
+ 
+ /**
+  * State of a block replica at a datanode
+  */
+ enum ReplicaStateProto {
+   FINALIZED = 0;  // State of a replica when it is not modified
+   RBW = 1;        // State of replica that is being written to
+   RWR = 2;        // State of replica that is waiting to be recovered
+   RUR = 3;        // State of replica that is under recovery
+   TEMPORARY = 4;  // State of replica that is created for replication
+ }
+ 
+ /**
+  * Block that needs to be recovered with at a given location
+  */
+ message RecoveringBlockProto {
+   required uint64 newGenStamp = 1;        // New genstamp post recovery
+   required LocatedBlockProto block = 2;   // Block to be recovered
+   optional BlockProto truncateBlock = 3;  // New block for recovery (truncate)
+ }
+ 
+ /**
+  * Unique signature to identify checkpoint transactions.
+  */
+ message CheckpointSignatureProto {
+   required string blockPoolId = 1;
+   required uint64 mostRecentCheckpointTxId = 2;
+   required uint64 curSegmentTxId = 3;
+   required StorageInfoProto storageInfo = 4;
+ }
+ 
+ /**
+  * Command returned from primary to checkpointing namenode.
+  * This command has checkpoint signature that identifies
+  * checkpoint transaction and is needed for further
+  * communication related to checkpointing.
+  */
+ message CheckpointCommandProto {
+   // Unique signature to identify checkpoint transation
+   required CheckpointSignatureProto signature = 1;
+ 
+   // If true, return transfer image to primary upon the completion of checkpoint
+   required bool needToReturnImage = 2;
+ }
+ 
+ /**
+  * Command sent from one namenode to another namenode.
+  */
+ message NamenodeCommandProto {
+   enum Type {
+     NamenodeCommand = 0;      // Base command
+     CheckPointCommand = 1;    // Check point command
+   }
+   required uint32 action = 1;
+   required Type type = 2;
+   optional CheckpointCommandProto checkpointCmd = 3;
+ }
+ 
+ /**
+  * void request
+  */
+ message VersionRequestProto {
+ }
+ 
+ /**
+  * Version response from namenode.
+  */
+ message VersionResponseProto {
+   required NamespaceInfoProto info = 1;
+ }
+ 
+ /**
+  * Common node information shared by all the nodes in the cluster
+  */
+ message StorageInfoProto {
+   required uint32 layoutVersion = 1; // Layout version of the file system
+   required uint32 namespceID = 2;    // File system namespace ID
+   required string clusterID = 3;     // ID of the cluster
+   required uint64 cTime = 4;         // File system creation time
+ }
+ 
+ /**
+  * Information sent by a namenode to identify itself to the primary namenode.
+  */
+ message NamenodeRegistrationProto {
+   required string rpcAddress = 1;    // host:port of the namenode RPC address
+   required string httpAddress = 2;   // host:port of the namenode http server
+   enum NamenodeRoleProto {
+     NAMENODE = 1;
+     BACKUP = 2;
+     CHECKPOINT = 3;
+   }
+   required StorageInfoProto storageInfo = 3;  // Node information
+   optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
index d1b16b1,50d548a..ce7aee3
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
@@@ -73,21 -74,17 +73,21 @@@ public class TestBlockTokenWithDFS 
    private static final String FILE_TO_READ = "/fileToRead.dat";
    private static final String FILE_TO_WRITE = "/fileToWrite.dat";
    private static final String FILE_TO_APPEND = "/fileToAppend.dat";
 -  private final byte[] rawData = new byte[FILE_SIZE];
  
    {
-     ((Log4JLogger) DFSClient.LOG).getLogger().setLevel(Level.ALL);
+     GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
 +  }
 +
 +  public static byte[] generateBytes(int fileSize){
      Random r = new Random();
 +    byte[] rawData = new byte[fileSize];
      r.nextBytes(rawData);
 +    return rawData;
    }
  
 -  private void createFile(FileSystem fs, Path filename) throws IOException {
 +  private void createFile(FileSystem fs, Path filename, byte[] expected) throws IOException {
      FSDataOutputStream out = fs.create(filename);
 -    out.write(rawData);
 +    out.write(expected);
      out.close();
    }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index 917b0f2,df07a62..2bb3d5f
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@@ -55,9 -46,8 +54,10 @@@ import org.apache.hadoop.hdfs.server.bl
  import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
  import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
  import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 +import org.apache.hadoop.hdfs.protocol.Block;
  import org.apache.hadoop.io.IOUtils;
+ import org.apache.hadoop.test.GenericTestUtils;
  import org.apache.hadoop.test.PathUtils;
  import org.apache.log4j.Level;
  import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 5efc94d,a84ddd0..6df88fd
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@@ -1660,60 -1657,4 +1662,60 @@@ public class TestFsck 
        }
      }
    }
 +
 +  @Test
 +  public void testECFsck() throws Exception {
 +    MiniDFSCluster cluster = null;
 +    FileSystem fs = null;
 +    try {
 +      Configuration conf = new HdfsConfiguration();
 +      final long precision = 1L;
 +      conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
 +      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
 +      int totalSize = ErasureCodingPolicyManager.getSystemDefaultPolicy().getNumDataUnits()
 +                      + ErasureCodingPolicyManager.getSystemDefaultPolicy().getNumParityUnits();
 +      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(totalSize).build();
 +      fs = cluster.getFileSystem();
 +
 +      // create a contiguous file
 +      Path replDirPath = new Path("/replicated");
 +      Path replFilePath = new Path(replDirPath, "replfile");
 +      final short factor = 3;
 +      DFSTestUtil.createFile(fs, replFilePath, 1024, factor, 0);
 +      DFSTestUtil.waitReplication(fs, replFilePath, factor);
 +
 +      // create a large striped file
 +      Path ecDirPath = new Path("/striped");
 +      Path largeFilePath = new Path(ecDirPath, "largeFile");
 +      DFSTestUtil.createStripedFile(cluster, largeFilePath, ecDirPath, 1, 2, true);
 +
 +      // create a small striped file
 +      Path smallFilePath = new Path(ecDirPath, "smallFile");
 +      DFSTestUtil.writeFile(fs, smallFilePath, "hello world!");
 +
 +      long replTime = fs.getFileStatus(replFilePath).getAccessTime();
 +      long ecTime = fs.getFileStatus(largeFilePath).getAccessTime();
 +      Thread.sleep(precision);
 +      setupAuditLogs();
 +      String outStr = runFsck(conf, 0, true, "/");
 +      verifyAuditLogs();
 +      assertEquals(replTime, fs.getFileStatus(replFilePath).getAccessTime());
 +      assertEquals(ecTime, fs.getFileStatus(largeFilePath).getAccessTime());
 +      System.out.println(outStr);
 +      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
 +      if (fs != null) {try{fs.close();} catch(Exception e){}}
 +      cluster.shutdown();
 +
 +      // restart the cluster; bring up namenode but not the data nodes
 +      cluster = new MiniDFSCluster.Builder(conf)
 +          .numDataNodes(0).format(false).build();
 +      outStr = runFsck(conf, 1, true, "/");
 +      // expect the result is corrupt
 +      assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
 +      System.out.println(outStr);
 +    } finally {
 +      if (fs != null) {try{fs.close();} catch(Exception e){}}
 +      if (cluster != null) { cluster.shutdown(); }
 +    }
 +  }
- }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------


[03/50] [abbrv] hadoop git commit: Merge commit '456e901a4c5c639267ee87b8e5f1319f256d20c2' (HDFS-6407. Add sorting and pagination in the datanode tab of the NN Web UI. Contributed by Haohui Mai.) into HDFS-7285-merge

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 36ce133,508da85..dfea5f3
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@@ -42,7 -44,8 +44,9 @@@ import javax.management.ObjectName
  import org.apache.hadoop.HadoopIllegalArgumentException;
  import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.conf.Configuration;
++import org.apache.hadoop.fs.FileEncryptionInfo;
  import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.hdfs.DFSUtilClient;
  import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
  import org.apache.hadoop.hdfs.DFSConfigKeys;
  import org.apache.hadoop.hdfs.DFSUtil;
@@@ -52,12 -55,10 +56,11 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
  import org.apache.hadoop.hdfs.protocol.DatanodeID;
  import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
  import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
--import org.apache.hadoop.fs.FileEncryptionInfo;
  import org.apache.hadoop.hdfs.protocol.LocatedBlock;
  import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
  import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
  import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
  import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
@@@ -77,22 -79,17 +81,24 @@@ import org.apache.hadoop.hdfs.server.pr
  import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
  import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
  import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
  import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
  import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
  import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
  import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
  import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
  import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
  import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
 +
+ import org.apache.hadoop.metrics2.util.MBeans;
  import org.apache.hadoop.net.Node;
  import org.apache.hadoop.security.UserGroupInformation;
 +import org.apache.hadoop.security.token.Token;
  import org.apache.hadoop.util.Daemon;
  import org.apache.hadoop.util.LightWeightGSet;
  import org.apache.hadoop.util.Time;
@@@ -818,11 -786,12 +835,11 @@@ public class BlockManager implements Bl
      }
      return locations;
    }
--  
 -  private List<LocatedBlock> createLocatedBlockList(
 -      final BlockInfo[] blocks,
++
 +  private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
        final long offset, final long length, final int nrBlocksToReturn,
        final AccessMode mode) throws IOException {
 -    int curBlk = 0;
 +    int curBlk;
      long curPos = 0, blkSize = 0;
      int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
      for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
@@@ -875,25 -844,19 +892,26 @@@
    }
  
    /** @return a LocatedBlock for the given block */
-   private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) {
 -  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
 -      ) throws IOException {
 -    if (blk instanceof BlockInfoContiguousUnderConstruction) {
 -      if (blk.isComplete()) {
 -        throw new IOException(
 -            "blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
 -            + ", blk=" + blk);
++  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos)
++      throws IOException {
 +    if (!blk.isComplete()) {
 +      if (blk.isStriped()) {
-         final BlockInfoUnderConstructionStriped uc =
-             (BlockInfoUnderConstructionStriped) blk;
++        final BlockInfoStripedUnderConstruction uc =
++            (BlockInfoStripedUnderConstruction) blk;
 +        final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
 +        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
 +            blk);
 +        return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
 +            false);
 +      } else {
-         assert blk instanceof BlockInfoUnderConstructionContiguous;
-         final BlockInfoUnderConstructionContiguous uc =
-             (BlockInfoUnderConstructionContiguous) blk;
++        assert blk instanceof BlockInfoContiguousUnderConstruction;
++        final BlockInfoContiguousUnderConstruction uc =
++            (BlockInfoContiguousUnderConstruction) blk;
 +        final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
 +        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
 +            blk);
 +        return newLocatedBlock(eb, storages, pos, false);
        }
 -      final BlockInfoContiguousUnderConstruction uc =
 -          (BlockInfoContiguousUnderConstruction) blk;
 -      final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
 -      final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
 -      return newLocatedBlock(eb, storages, pos, false);
      }
  
      // get block locations
@@@ -1188,17 -1121,13 +1206,17 @@@
        return;
      }
      StringBuilder datanodes = new StringBuilder();
 -    for(DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
 +    for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock,
 +        State.NORMAL)) {
        final DatanodeDescriptor node = storage.getDatanodeDescriptor();
 -      invalidateBlocks.add(b, node, false);
 -      datanodes.append(node).append(" ");
 +      final Block b = getBlockOnStorage(storedBlock, storage);
 +      if (b != null) {
 +        invalidateBlocks.add(b, node, false);
 +        datanodes.append(node).append(" ");
 +      }
      }
      if (datanodes.length() != 0) {
-       blockLog.info("BLOCK* addToInvalidates: {} {}", storedBlock,
 -      blockLog.debug("BLOCK* addToInvalidates: {} {}", b,
++      blockLog.debug("BLOCK* addToInvalidates: {} {}", storedBlock,
            datanodes.toString());
      }
    }
@@@ -1267,8 -1188,8 +1285,8 @@@
        DatanodeStorageInfo storageInfo,
        DatanodeDescriptor node) throws IOException {
  
 -    if (b.corrupted.isDeleted()) {
 +    if (b.stored.isDeleted()) {
-       blockLog.info("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
+       blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
            " corrupt as it does not belong to any file", b);
        addToInvalidates(b.corrupted, node);
        return;
@@@ -1323,9 -1237,9 +1341,9 @@@
     * @return true if the block was successfully invalidated and no longer
     * present in the BlocksMap
     */
 -  private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn
 -      ) throws IOException {
 +  private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn,
 +      NumberReplicas nr) throws IOException {
-     blockLog.info("BLOCK* invalidateBlock: {} on {}", b, dn);
+     blockLog.debug("BLOCK* invalidateBlock: {} on {}", b, dn);
      DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
      if (node == null) {
        throw new IOException("Cannot invalidate " + b
@@@ -1333,8 -1247,9 +1351,8 @@@
      }
  
      // Check how many copies we have of the block
 -    NumberReplicas nr = countNodes(b.stored);
      if (nr.replicasOnStaleNodes() > 0) {
-       blockLog.info("BLOCK* invalidateBlocks: postponing " +
+       blockLog.debug("BLOCK* invalidateBlocks: postponing " +
            "invalidation of {} on {} because {} replica(s) are located on " +
            "nodes with potentially out-of-date block reports", b, dn,
            nr.replicasOnStaleNodes());
@@@ -1478,12 -1391,12 +1496,12 @@@
              // do not schedule more if enough replicas is already pending
              numEffectiveReplicas = numReplicas.liveReplicas() +
                                      pendingReplications.getNumReplicas(block);
 -      
 +
              if (numEffectiveReplicas >= requiredReplication) {
                if ( (pendingReplications.getNumReplicas(block) > 0) ||
 -                   (blockHasEnoughRacks(block)) ) {
 +                   (blockHasEnoughRacks(block, requiredReplication)) ) {
                  neededReplications.remove(block, priority); // remove from neededReplications
-                 blockLog.info("BLOCK* Removing {} from neededReplications as" +
+                 blockLog.debug("BLOCK* Removing {} from neededReplications as" +
                          " it has enough replicas", block);
                  continue;
                }
@@@ -1565,10 -1463,10 +1583,10 @@@
  
            if (numEffectiveReplicas >= requiredReplication) {
              if ( (pendingReplications.getNumReplicas(block) > 0) ||
 -                 (blockHasEnoughRacks(block)) ) {
 +                 (blockHasEnoughRacks(block, requiredReplication)) ) {
                neededReplications.remove(block, priority); // remove from neededReplications
                rw.targets = null;
-               blockLog.info("BLOCK* Removing {} from neededReplications as" +
+               blockLog.debug("BLOCK* Removing {} from neededReplications as" +
                        " it has enough replicas", block);
                continue;
              }
@@@ -1637,11 -1510,11 +1655,11 @@@
          DatanodeStorageInfo[] targets = rw.targets;
          if (targets != null && targets.length != 0) {
            StringBuilder targetList = new StringBuilder("datanode(s)");
 -          for (int k = 0; k < targets.length; k++) {
 +          for (DatanodeStorageInfo target : targets) {
              targetList.append(' ');
 -            targetList.append(targets[k].getDatanodeDescriptor());
 +            targetList.append(target.getDatanodeDescriptor());
            }
-           blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNodes,
 -          blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.srcNode,
++          blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.srcNodes,
                rw.block, targetList);
          }
        }
@@@ -1882,11 -1765,8 +1921,11 @@@
      final Block reportedBlock;
      final ReplicaState reportedState;
      
 -    StatefulBlockInfo(BlockInfoContiguousUnderConstruction storedBlock,
 +    StatefulBlockInfo(BlockInfo storedBlock,
          Block reportedBlock, ReplicaState reportedState) {
 +      Preconditions.checkArgument(
-           storedBlock instanceof BlockInfoUnderConstructionContiguous ||
-           storedBlock instanceof BlockInfoUnderConstructionStriped);
++          storedBlock instanceof BlockInfoContiguousUnderConstruction ||
++          storedBlock instanceof BlockInfoStripedUnderConstruction);
        this.storedBlock = storedBlock;
        this.reportedBlock = reportedBlock;
        this.reportedState = reportedState;
@@@ -2263,8 -2141,8 +2311,8 @@@
              QUEUE_REASON_FUTURE_GENSTAMP);
          continue;
        }
--      
 -      BlockInfo storedBlock = blocksMap.getStoredBlock(iblk);
++
 +      BlockInfo storedBlock = getStoredBlock(iblk);
        // If block does not belong to any file, we are done.
        if (storedBlock == null) continue;
        
@@@ -2306,9 -2186,9 +2354,9 @@@
    }
  
    private void reportDiff(DatanodeStorageInfo storageInfo, 
--      BlockListAsLongs newReport, 
 -      Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
 -      Collection<Block> toRemove,           // remove from DatanodeDescriptor
++      BlockListAsLongs newReport,
 +      Collection<BlockInfoToAdd> toAdd,     // add to DatanodeDescriptor
 +      Collection<BlockInfo> toRemove,       // remove from DatanodeDescriptor
        Collection<Block> toInvalidate,       // should be removed from DN
        Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
        Collection<StatefulBlockInfo> toUC) { // add to under-construction list
@@@ -2342,10 -2220,10 +2390,11 @@@
  
      // collect blocks that have not been reported
      // all of them are next to the delimiter
-     Iterator<BlockInfo> it = storageInfo.new BlockIterator(delimiter.getNext(0));
+     Iterator<BlockInfo> it =
+         storageInfo.new BlockIterator(delimiter.getNext(0));
 -    while(it.hasNext())
 +    while (it.hasNext()) {
        toRemove.add(it.next());
 +    }
      storageInfo.removeBlock(delimiter);
    }
  
@@@ -2382,8 -2260,8 +2431,8 @@@
     */
    private BlockInfo processReportedBlock(
        final DatanodeStorageInfo storageInfo,
--      final Block block, final ReplicaState reportedState, 
 -      final Collection<BlockInfo> toAdd,
++      final Block block, final ReplicaState reportedState,
 +      final Collection<BlockInfoToAdd> toAdd,
        final Collection<Block> toInvalidate, 
        final Collection<BlockToMarkCorrupt> toCorrupt,
        final Collection<StatefulBlockInfo> toUC) {
@@@ -2717,10 -2578,9 +2766,10 @@@
      assert block != null && namesystem.hasWriteLock();
      BlockInfo storedBlock;
      DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
-     if (block instanceof BlockInfoUnderConstructionContiguous ||
-         block instanceof BlockInfoUnderConstructionStriped) {
 -    if (block instanceof BlockInfoContiguousUnderConstruction) {
++    if (block instanceof BlockInfoContiguousUnderConstruction ||
++        block instanceof BlockInfoStripedUnderConstruction) {
        //refresh our copy in case the block got completed in another thread
 -      storedBlock = blocksMap.getStoredBlock(block);
 +      storedBlock = getStoredBlock(block);
      } else {
        storedBlock = block;
      }
@@@ -3275,26 -3055,6 +3325,26 @@@
      }
    }
  
 +  private void processChosenExcessReplica(
 +      final Collection<DatanodeStorageInfo> nonExcess,
 +      final DatanodeStorageInfo chosen, BlockInfo storedBlock) {
 +    nonExcess.remove(chosen);
 +    addToExcessReplicate(chosen.getDatanodeDescriptor(), storedBlock);
 +    //
 +    // The 'excessblocks' tracks blocks until we get confirmation
 +    // that the datanode has deleted them; the only way we remove them
 +    // is when we get a "removeBlock" message.
 +    //
 +    // The 'invalidate' list is used to inform the datanode the block
 +    // should be deleted.  Items are removed from the invalidate list
 +    // upon giving instructions to the datanodes.
 +    //
 +    final Block blockToInvalidate = getBlockOnStorage(storedBlock, chosen);
 +    addToInvalidates(blockToInvalidate, chosen.getDatanodeDescriptor());
-     blockLog.info("BLOCK* chooseExcessReplicates: "
-         +"({}, {}) is added to invalidated blocks set", chosen, storedBlock);
++    blockLog.debug("BLOCK* chooseExcessReplicates: "
++        + "({}, {}) is added to invalidated blocks set", chosen, storedBlock);
 +  }
 +
    /** Check if we can use delHint */
    static boolean useDelHint(boolean isFirst, DatanodeStorageInfo delHint,
        DatanodeStorageInfo added, List<DatanodeStorageInfo> moreThan1Racks,
@@@ -3356,6 -3116,19 +3406,19 @@@
          return;
        }
  
+       CachedBlock cblock = namesystem.getCacheManager().getCachedBlocks()
 -          .get(new CachedBlock(block.getBlockId(), (short) 0, false));
++          .get(new CachedBlock(storedBlock.getBlockId(), (short) 0, false));
+       if (cblock != null) {
+         boolean removed = false;
+         removed |= node.getPendingCached().remove(cblock);
+         removed |= node.getCached().remove(cblock);
+         removed |= node.getPendingUncached().remove(cblock);
+         if (removed) {
+           blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
 -              + "related lists on node {}", block, node);
++              + "related lists on node {}", storedBlock, node);
+         }
+       }
+ 
        //
        // It's possible that the block was removed because of a datanode
        // failure. If the block is still valid, check if replication is
@@@ -3454,7 -3212,10 +3517,10 @@@
      //
      // Modify the blocks->datanode map and node's map.
      //
-     pendingReplications.decrement(getStoredBlock(block), node);
+     BlockInfo storedBlock = getStoredBlock(block);
+     if (storedBlock != null) {
 -      pendingReplications.decrement(getStoredBlock(block), node);
++      pendingReplications.decrement(storedBlock, node);
+     }
      processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
          delHintNode);
    }
@@@ -4138,57 -3819,22 +4204,57 @@@
          null);
    }
  
 -  private static class ReplicationWork {
 -
 -    private final BlockInfo block;
 -    private final BlockCollection bc;
 -
 -    private final DatanodeDescriptor srcNode;
 -    private final List<DatanodeDescriptor> containingNodes;
 -    private final List<DatanodeStorageInfo> liveReplicaStorages;
 -    private final int additionalReplRequired;
 +  public static LocatedStripedBlock newLocatedStripedBlock(
 +      ExtendedBlock b, DatanodeStorageInfo[] storages,
 +      int[] indices, long startOffset, boolean corrupt) {
 +    // startOffset is unknown
 +    return new LocatedStripedBlock(
 +        b, DatanodeStorageInfo.toDatanodeInfos(storages),
 +        DatanodeStorageInfo.toStorageIDs(storages),
 +        DatanodeStorageInfo.toStorageTypes(storages),
 +        indices, startOffset, corrupt,
 +        null);
 +  }
  
 -    private DatanodeStorageInfo targets[];
 -    private final int priority;
 +  public static LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info,
 +      DatanodeStorageInfo[] locs, long offset) throws IOException {
 +    final LocatedBlock lb;
 +    if (info.isStriped()) {
 +      lb = newLocatedStripedBlock(eb, locs,
-           ((BlockInfoUnderConstructionStriped)info).getBlockIndices(),
++          ((BlockInfoStripedUnderConstruction)info).getBlockIndices(),
 +          offset, false);
 +    } else {
 +      lb = newLocatedBlock(eb, locs, offset, false);
 +    }
 +    return lb;
 +  }
  
 -    public ReplicationWork(BlockInfo block,
 +  /**
 +   * This class is used internally by {@link this#computeRecoveryWorkForBlocks}
 +   * to represent a task to recover a block through replication or erasure
 +   * coding. Recovery is done by transferring data from srcNodes to targets
 +   */
 +  private abstract static class BlockRecoveryWork {
 +    final BlockInfo block;
 +    final BlockCollection bc;
 +
 +    /**
 +     * An erasure coding recovery task has multiple source nodes.
 +     * A replication task only has 1 source node, stored on top of the array
 +     */
 +    final DatanodeDescriptor[] srcNodes;
 +    /** Nodes containing the block; avoid them in choosing new targets */
 +    final List<DatanodeDescriptor> containingNodes;
 +    /** Required by {@link BlockPlacementPolicy#chooseTarget} */
 +    final List<DatanodeStorageInfo> liveReplicaStorages;
 +    final int additionalReplRequired;
 +
 +    DatanodeStorageInfo[] targets;
 +    final int priority;
 +
 +    BlockRecoveryWork(BlockInfo block,
          BlockCollection bc,
 -        DatanodeDescriptor srcNode,
 +        DatanodeDescriptor[] srcNodes,
          List<DatanodeDescriptor> containingNodes,
          List<DatanodeStorageInfo> liveReplicaStorages,
          int additionalReplRequired,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index 9173920,0dbf485..5bfae42
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@@ -130,17 -123,13 +130,17 @@@ class BlocksMap 
        return;
  
      blockInfo.setBlockCollection(null);
 -    for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
 +    final int size = blockInfo instanceof BlockInfoContiguous ?
 +        blockInfo.numNodes() : blockInfo.getCapacity();
 +    for(int idx = size - 1; idx >= 0; idx--) {
        DatanodeDescriptor dn = blockInfo.getDatanode(idx);
 -      dn.removeBlock(blockInfo); // remove from the list and wipe the location
 +      if (dn != null) {
 +        dn.removeBlock(blockInfo); // remove from the list and wipe the location
 +      }
      }
    }
--  
 -  /** Returns the block object it it exists in the map. */
++
 +  /** Returns the block object if it exists in the map. */
    BlockInfo getStoredBlock(Block b) {
      return blocks.get(b);
    }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 108ce2f,87ce753..87394f6
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@@ -31,10 -31,8 +31,7 @@@ import java.util.Queue
  import java.util.Set;
  
  import com.google.common.annotations.VisibleForTesting;
 -
  import com.google.common.collect.ImmutableList;
- 
- import org.apache.commons.logging.Log;
- import org.apache.commons.logging.LogFactory;
  import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.classification.InterfaceStability;
  import org.apache.hadoop.fs.StorageType;
@@@ -50,9 -46,10 +47,11 @@@ import org.apache.hadoop.hdfs.server.pr
  import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
  import org.apache.hadoop.hdfs.util.EnumCounters;
  import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
  import org.apache.hadoop.util.IntrusiveCollection;
  import org.apache.hadoop.util.Time;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  
  /**
   * This class extends the DatanodeInfo class with ephemeral information (eg

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index 2275d91,216d6d2..bb9a706
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@@ -253,18 -252,10 +254,18 @@@ public class DatanodeStorageInfo 
      }
  
      // add to the head of the data-node list
 -    b.addStorage(this);
 +    b.addStorage(this, reportedBlock);
 +    insertToList(b);
 +    return result;
 +  }
 +
-   AddBlockResult addBlock(BlockInfoContiguous b) {
++  AddBlockResult addBlock(BlockInfo b) {
 +    return addBlock(b, b);
 +  }
 +
 +  public void insertToList(BlockInfo b) {
      blockList = b.listInsert(blockList, this);
      numBlocks++;
 -    return result;
    }
  
    public boolean removeBlock(BlockInfo b) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
index 47afb05,ebc15b8..7e8f479
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
@@@ -240,10 -206,10 +241,10 @@@ class UnderReplicatedBlocks implements 
  
    /** remove a block from a under replication queue */
    synchronized boolean remove(BlockInfo block,
-                               int oldReplicas, 
+                               int oldReplicas,
                                int decommissionedReplicas,
                                int oldExpectedReplicas) {
 -    int priLevel = getPriority(oldReplicas,
 +    int priLevel = getPriority(block, oldReplicas,
                                 decommissionedReplicas,
                                 oldExpectedReplicas);
      boolean removedBlock = remove(block, priLevel);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index dc6acd5,afacebb..34d92d0
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@@ -368,17 -356,15 +369,16 @@@ public class Mover 
        final BlockStoragePolicy policy = blockStoragePolicies[policyId];
        if (policy == null) {
          LOG.warn("Failed to get the storage policy of file " + fullPath);
-         return false;
+         return;
        }
 -      final List<StorageType> types = policy.chooseStorageTypes(
 +      List<StorageType> types = policy.chooseStorageTypes(
            status.getReplication());
  
 +      final ErasureCodingPolicy ecPolicy = status.getErasureCodingPolicy();
        final LocatedBlocks locatedBlocks = status.getBlockLocations();
-       boolean hasRemaining = false;
        final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
        List<LocatedBlock> lbs = locatedBlocks.getLocatedBlocks();
-       for(int i = 0; i < lbs.size(); i++) {
+       for (int i = 0; i < lbs.size(); i++) {
          if (i == lbs.size() - 1 && !lastBlkComplete) {
            // last block is incomplete, skip it
            continue;
@@@ -390,22 -373,22 +390,25 @@@
          final StorageTypeDiff diff = new StorageTypeDiff(types,
              lb.getStorageTypes());
          if (!diff.removeOverlap(true)) {
 -          if (scheduleMoves4Block(diff, lb)) {
 +          if (scheduleMoves4Block(diff, lb, ecPolicy)) {
-             hasRemaining |= (diff.existing.size() > 1 &&
-                 diff.expected.size() > 1);
+             result.updateHasRemaining(diff.existing.size() > 1
+                 && diff.expected.size() > 1);
+             // One block scheduled successfully, set noBlockMoved to false
+             result.setNoBlockMoved(false);
+           } else {
+             result.updateHasRemaining(true);
            }
          }
        }
-       return hasRemaining;
      }
  
 -    boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb) {
 +    boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb,
 +                                ErasureCodingPolicy ecPolicy) {
        final List<MLocation> locations = MLocation.toLocations(lb);
 -      Collections.shuffle(locations);
 -      final DBlock db = newDBlock(lb.getBlock().getLocalBlock(), locations);
 +      if (!(lb instanceof LocatedStripedBlock)) {
 +        Collections.shuffle(locations);
 +      }
 +      final DBlock db = newDBlock(lb, locations, ecPolicy);
  
        for (final StorageType t : diff.existing) {
          for (final MLocation ml : locations) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
index 0000000,3d79d09..ef67c1a
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
@@@ -1,0 -1,261 +1,268 @@@
+ /**
+  * 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.hdfs.server.namenode;
+ 
+ import java.io.FileNotFoundException;
+ import java.io.IOException;
+ import java.util.List;
+ 
+ import org.apache.hadoop.fs.FileAlreadyExistsException;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.fs.permission.FsAction;
+ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp;
+ import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion.Feature;
+ 
+ import com.google.common.base.Preconditions;
+ 
+ /**
+  * Helper class to perform append operation.
+  */
+ final class FSDirAppendOp {
+ 
+   /**
+    * Private constructor for preventing FSDirAppendOp object creation.
+    * Static-only class.
+    */
+   private FSDirAppendOp() {}
+ 
+   /**
+    * Append to an existing file.
+    * <p>
+    *
+    * The method returns the last block of the file if this is a partial block,
+    * which can still be used for writing more data. The client uses the
+    * returned block locations to form the data pipeline for this block.<br>
+    * The {@link LocatedBlock} will be null if the last block is full.
+    * The client then allocates a new block with the next call using
+    * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#addBlock}.
+    * <p>
+    *
+    * For description of parameters and exceptions thrown see
+    * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#append}
+    *
+    * @param fsn namespace
+    * @param srcArg path name
+    * @param pc permission checker to check fs permission
+    * @param holder client name
+    * @param clientMachine client machine info
+    * @param newBlock if the data is appended to a new block
+    * @param logRetryCache whether to record RPC ids in editlog for retry cache
+    *                      rebuilding
+    * @return the last block with status
+    */
+   static LastBlockWithStatus appendFile(final FSNamesystem fsn,
+       final String srcArg, final FSPermissionChecker pc, final String holder,
+       final String clientMachine, final boolean newBlock,
+       final boolean logRetryCache) throws IOException {
+     assert fsn.hasWriteLock();
+ 
+     final byte[][] pathComponents = FSDirectory
+         .getPathComponentsForReservedPath(srcArg);
+     final LocatedBlock lb;
+     final FSDirectory fsd = fsn.getFSDirectory();
+     final String src;
+     fsd.writeLock();
+     try {
+       src = fsd.resolvePath(pc, srcArg, pathComponents);
+       final INodesInPath iip = fsd.getINodesInPath4Write(src);
+       // Verify that the destination does not exist as a directory already
+       final INode inode = iip.getLastINode();
+       final String path = iip.getPath();
+       if (inode != null && inode.isDirectory()) {
+         throw new FileAlreadyExistsException("Cannot append to directory "
+             + path + "; already exists as a directory.");
+       }
+       if (fsd.isPermissionEnabled()) {
+         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
+       }
+ 
+       if (inode == null) {
+         throw new FileNotFoundException(
+             "Failed to append to non-existent file " + path + " for client "
+                 + clientMachine);
+       }
+       final INodeFile file = INodeFile.valueOf(inode, path, true);
++
++      // not support appending file with striped blocks
++      if (file.isStriped()) {
++        throw new UnsupportedOperationException(
++            "Cannot append to files with striped block " + src);
++      }
++
+       BlockManager blockManager = fsd.getBlockManager();
+       final BlockStoragePolicy lpPolicy = blockManager
+           .getStoragePolicy("LAZY_PERSIST");
+       if (lpPolicy != null && lpPolicy.getId() == file.getStoragePolicyID()) {
+         throw new UnsupportedOperationException(
+             "Cannot append to lazy persist file " + path);
+       }
+       // Opening an existing file for append - may need to recover lease.
+       fsn.recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE, iip, path, holder,
+           clientMachine, false);
+ 
+       final BlockInfo lastBlock = file.getLastBlock();
+       // Check that the block has at least minimum replication.
+       if (lastBlock != null && lastBlock.isComplete()
+           && !blockManager.isSufficientlyReplicated(lastBlock)) {
+         throw new IOException("append: lastBlock=" + lastBlock + " of src="
+             + path + " is not sufficiently replicated yet.");
+       }
+       lb = prepareFileForAppend(fsn, iip, holder, clientMachine, newBlock,
+           true, logRetryCache);
+     } catch (IOException ie) {
+       NameNode.stateChangeLog
+           .warn("DIR* NameSystem.append: " + ie.getMessage());
+       throw ie;
+     } finally {
+       fsd.writeUnlock();
+     }
+ 
+     HdfsFileStatus stat = FSDirStatAndListingOp.getFileInfo(fsd, src, false,
+         FSDirectory.isReservedRawName(srcArg));
+     if (lb != null) {
+       NameNode.stateChangeLog.debug(
+           "DIR* NameSystem.appendFile: file {} for {} at {} block {} block"
+               + " size {}", srcArg, holder, clientMachine, lb.getBlock(), lb
+               .getBlock().getNumBytes());
+     }
+     return new LastBlockWithStatus(lb, stat);
+   }
+ 
+   /**
+    * Convert current node to under construction.
+    * Recreate in-memory lease record.
+    *
+    * @param fsn namespace
+    * @param iip inodes in the path containing the file
+    * @param leaseHolder identifier of the lease holder on this file
+    * @param clientMachine identifier of the client machine
+    * @param newBlock if the data is appended to a new block
+    * @param writeToEditLog whether to persist this change to the edit log
+    * @param logRetryCache whether to record RPC ids in editlog for retry cache
+    *                      rebuilding
+    * @return the last block locations if the block is partial or null otherwise
+    * @throws IOException
+    */
+   static LocatedBlock prepareFileForAppend(final FSNamesystem fsn,
+       final INodesInPath iip, final String leaseHolder,
+       final String clientMachine, final boolean newBlock,
+       final boolean writeToEditLog, final boolean logRetryCache)
+       throws IOException {
+     assert fsn.hasWriteLock();
+ 
+     final INodeFile file = iip.getLastINode().asFile();
+     final QuotaCounts delta = verifyQuotaForUCBlock(fsn, file, iip);
+ 
+     file.recordModification(iip.getLatestSnapshotId());
+     file.toUnderConstruction(leaseHolder, clientMachine);
+ 
+     fsn.getLeaseManager().addLease(
+         file.getFileUnderConstructionFeature().getClientName(), file.getId());
+ 
+     LocatedBlock ret = null;
+     if (!newBlock) {
+       FSDirectory fsd = fsn.getFSDirectory();
+       ret = fsd.getBlockManager().convertLastBlockToUnderConstruction(file, 0);
+       if (ret != null && delta != null) {
+         Preconditions.checkState(delta.getStorageSpace() >= 0, "appending to"
+             + " a block with size larger than the preferred block size");
+         fsd.writeLock();
+         try {
+           fsd.updateCountNoQuotaCheck(iip, iip.length() - 1, delta);
+         } finally {
+           fsd.writeUnlock();
+         }
+       }
+     } else {
+       BlockInfo lastBlock = file.getLastBlock();
+       if (lastBlock != null) {
+         ExtendedBlock blk = new ExtendedBlock(fsn.getBlockPoolId(), lastBlock);
+         ret = new LocatedBlock(blk, new DatanodeInfo[0]);
+       }
+     }
+ 
+     if (writeToEditLog) {
+       final String path = iip.getPath();
+       if (NameNodeLayoutVersion.supports(Feature.APPEND_NEW_BLOCK,
+           fsn.getEffectiveLayoutVersion())) {
+         fsn.getEditLog().logAppendFile(path, file, newBlock, logRetryCache);
+       } else {
+         fsn.getEditLog().logOpenFile(path, file, false, logRetryCache);
+       }
+     }
+     return ret;
+   }
+ 
+   /**
+    * Verify quota when using the preferred block size for UC block. This is
+    * usually used by append and truncate.
+    *
+    * @throws QuotaExceededException when violating the storage quota
+    * @return expected quota usage update. null means no change or no need to
+    *         update quota usage later
+    */
+   private static QuotaCounts verifyQuotaForUCBlock(FSNamesystem fsn,
+       INodeFile file, INodesInPath iip) throws QuotaExceededException {
+     FSDirectory fsd = fsn.getFSDirectory();
+     if (!fsn.isImageLoaded() || fsd.shouldSkipQuotaChecks()) {
+       // Do not check quota if editlog is still being processed
+       return null;
+     }
+     if (file.getLastBlock() != null) {
+       final QuotaCounts delta = computeQuotaDeltaForUCBlock(fsn, file);
+       fsd.readLock();
+       try {
+         FSDirectory.verifyQuota(iip, iip.length() - 1, delta, null);
+         return delta;
+       } finally {
+         fsd.readUnlock();
+       }
+     }
+     return null;
+   }
+ 
+   /** Compute quota change for converting a complete block to a UC block. */
+   private static QuotaCounts computeQuotaDeltaForUCBlock(FSNamesystem fsn,
+       INodeFile file) {
+     final QuotaCounts delta = new QuotaCounts.Builder().build();
+     final BlockInfo lastBlock = file.getLastBlock();
+     if (lastBlock != null) {
+       final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes();
+       final short repl = file.getPreferredBlockReplication();
+       delta.addStorageSpace(diff * repl);
+       final BlockStoragePolicy policy = fsn.getFSDirectory()
+           .getBlockStoragePolicySuite().getPolicy(file.getStoragePolicyID());
+       List<StorageType> types = policy.chooseStorageTypes(repl);
+       for (StorageType t : types) {
+         if (t.supportTypeQuota()) {
+           delta.addTypeSpace(t, diff);
+         }
+       }
+     }
+     return delta;
+   }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index bad7c42,4a45074..6ec97c9
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@@ -28,9 -27,8 +28,10 @@@ import org.apache.hadoop.fs.InvalidPath
  import org.apache.hadoop.fs.permission.FsAction;
  import org.apache.hadoop.fs.permission.FsPermission;
  import org.apache.hadoop.hdfs.DFSUtil;
+ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
  import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
  import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@@ -140,9 -139,69 +142,71 @@@ class FSDirStatAndListingOp 
      return getContentSummaryInt(fsd, iip);
    }
  
+   /**
+    * Get block locations within the specified range.
+    * @see ClientProtocol#getBlockLocations(String, long, long)
+    * @throws IOException
+    */
+   static GetBlockLocationsResult getBlockLocations(
+       FSDirectory fsd, FSPermissionChecker pc, String src, long offset,
+       long length, boolean needBlockToken) throws IOException {
+     Preconditions.checkArgument(offset >= 0,
+         "Negative offset is not supported. File: " + src);
+     Preconditions.checkArgument(length >= 0,
+         "Negative length is not supported. File: " + src);
+     CacheManager cm = fsd.getFSNamesystem().getCacheManager();
+     BlockManager bm = fsd.getBlockManager();
+     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+     boolean isReservedName = FSDirectory.isReservedRawName(src);
+     fsd.readLock();
+     try {
+       src = fsd.resolvePath(pc, src, pathComponents);
+       final INodesInPath iip = fsd.getINodesInPath(src, true);
+       final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
+       if (fsd.isPermissionEnabled()) {
+         fsd.checkPathAccess(pc, iip, FsAction.READ);
+         fsd.checkUnreadableBySuperuser(pc, inode, iip.getPathSnapshotId());
+       }
+ 
+       final long fileSize = iip.isSnapshot()
+           ? inode.computeFileSize(iip.getPathSnapshotId())
+           : inode.computeFileSizeNotIncludingLastUcBlock();
+ 
+       boolean isUc = inode.isUnderConstruction();
+       if (iip.isSnapshot()) {
+         // if src indicates a snapshot file, we need to make sure the returned
+         // blocks do not exceed the size of the snapshot file.
+         length = Math.min(length, fileSize - offset);
+         isUc = false;
+       }
+ 
+       final FileEncryptionInfo feInfo = isReservedName ? null
+           : fsd.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
++      final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
++          fsd.getFSNamesystem(), iip);
+ 
+       final LocatedBlocks blocks = bm.createLocatedBlocks(
+           inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
 -          length, needBlockToken, iip.isSnapshot(), feInfo);
++          length, needBlockToken, iip.isSnapshot(), feInfo, ecZone);
+ 
+       // Set caching information for the located blocks.
+       for (LocatedBlock lb : blocks.getLocatedBlocks()) {
+         cm.setCachedLocations(lb);
+       }
+ 
+       final long now = now();
+       boolean updateAccessTime = fsd.isAccessTimeSupported()
+           && !iip.isSnapshot()
+           && now > inode.getAccessTime() + fsd.getAccessTimePrecision();
+       return new GetBlockLocationsResult(updateAccessTime, blocks);
+     } finally {
+       fsd.readUnlock();
+     }
+   }
+ 
    private static byte getStoragePolicyID(byte inodePolicy, byte parentPolicy) {
-     return inodePolicy != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ? inodePolicy :
-         parentPolicy;
+     return inodePolicy != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED
+         ? inodePolicy : parentPolicy;
    }
  
    /**
@@@ -457,9 -505,9 +518,9 @@@
        final long fileSize = !inSnapshot && isUc ?
            fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
  
-       loc = fsd.getFSNamesystem().getBlockManager().createLocatedBlocks(
+       loc = fsd.getBlockManager().createLocatedBlocks(
            fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
 -          inSnapshot, feInfo);
 +          inSnapshot, feInfo, ecZone);
        if (loc == null) {
          loc = new LocatedBlocks();
        }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
index 0000000,474c257..215a761
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
@@@ -1,0 -1,360 +1,370 @@@
+ /**
+  * 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.hdfs.server.namenode;
+ 
+ import java.io.IOException;
+ 
+ import org.apache.hadoop.HadoopIllegalArgumentException;
+ import org.apache.hadoop.fs.UnresolvedLinkException;
+ import org.apache.hadoop.fs.permission.FsAction;
+ import org.apache.hadoop.hdfs.protocol.Block;
+ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp;
+ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ 
+ /**
+  * Helper class to perform truncate operation.
+  */
+ final class FSDirTruncateOp {
+ 
+   /**
+    * Private constructor for preventing FSDirTruncateOp object creation.
+    * Static-only class.
+    */
+   private FSDirTruncateOp() {}
+ 
+   /**
+    * Truncate a file to a given size.
+    *
+    * @param fsn namespace
+    * @param srcArg path name
+    * @param newLength the target file size
+    * @param clientName client name
+    * @param clientMachine client machine info
+    * @param mtime modified time
+    * @param toRemoveBlocks to be removed blocks
+    * @param pc permission checker to check fs permission
+    * @return tuncate result
+    * @throws IOException
+    */
+   static TruncateResult truncate(final FSNamesystem fsn, final String srcArg,
+       final long newLength, final String clientName,
+       final String clientMachine, final long mtime,
+       final BlocksMapUpdateInfo toRemoveBlocks, final FSPermissionChecker pc)
+       throws IOException, UnresolvedLinkException {
+     assert fsn.hasWriteLock();
+ 
+     FSDirectory fsd = fsn.getFSDirectory();
+     byte[][] pathComponents = FSDirectory
+         .getPathComponentsForReservedPath(srcArg);
+     final String src;
+     final INodesInPath iip;
+     final boolean onBlockBoundary;
+     Block truncateBlock = null;
+     fsd.writeLock();
+     try {
+       src = fsd.resolvePath(pc, srcArg, pathComponents);
+       iip = fsd.getINodesInPath4Write(src, true);
+       if (fsd.isPermissionEnabled()) {
+         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
+       }
+       INodeFile file = INodeFile.valueOf(iip.getLastINode(), src);
++
++      // not support truncating file with striped blocks
++      if (file.isStriped()) {
++        throw new UnsupportedOperationException(
++            "Cannot truncate file with striped block " + src);
++      }
++
+       final BlockStoragePolicy lpPolicy = fsd.getBlockManager()
+           .getStoragePolicy("LAZY_PERSIST");
+ 
+       if (lpPolicy != null && lpPolicy.getId() == file.getStoragePolicyID()) {
+         throw new UnsupportedOperationException(
+             "Cannot truncate lazy persist file " + src);
+       }
+ 
+       // Check if the file is already being truncated with the same length
+       final BlockInfo last = file.getLastBlock();
+       if (last != null && last.getBlockUCState()
+           == BlockUCState.UNDER_RECOVERY) {
+         final Block truncatedBlock = ((BlockInfoContiguousUnderConstruction) last)
+             .getTruncateBlock();
+         if (truncatedBlock != null) {
+           final long truncateLength = file.computeFileSize(false, false)
+               + truncatedBlock.getNumBytes();
+           if (newLength == truncateLength) {
+             return new TruncateResult(false, fsd.getAuditFileInfo(iip));
+           }
+         }
+       }
+ 
+       // Opening an existing file for truncate. May need lease recovery.
+       fsn.recoverLeaseInternal(RecoverLeaseOp.TRUNCATE_FILE, iip, src,
+           clientName, clientMachine, false);
+       // Truncate length check.
+       long oldLength = file.computeFileSize();
+       if (oldLength == newLength) {
+         return new TruncateResult(true, fsd.getAuditFileInfo(iip));
+       }
+       if (oldLength < newLength) {
+         throw new HadoopIllegalArgumentException(
+             "Cannot truncate to a larger file size. Current size: " + oldLength
+                 + ", truncate size: " + newLength + ".");
+       }
+       // Perform INodeFile truncation.
+       final QuotaCounts delta = new QuotaCounts.Builder().build();
+       onBlockBoundary = unprotectedTruncate(fsn, iip, newLength,
+           toRemoveBlocks, mtime, delta);
+       if (!onBlockBoundary) {
+         // Open file for write, but don't log into edits
+         long lastBlockDelta = file.computeFileSize() - newLength;
+         assert lastBlockDelta > 0 : "delta is 0 only if on block bounday";
+         truncateBlock = prepareFileForTruncate(fsn, iip, clientName,
+             clientMachine, lastBlockDelta, null);
+       }
+ 
+       // update the quota: use the preferred block size for UC block
+       fsd.updateCountNoQuotaCheck(iip, iip.length() - 1, delta);
+     } finally {
+       fsd.writeUnlock();
+     }
+ 
+     fsn.getEditLog().logTruncate(src, clientName, clientMachine, newLength,
+         mtime, truncateBlock);
+     return new TruncateResult(onBlockBoundary, fsd.getAuditFileInfo(iip));
+   }
+ 
+   /**
+    * Unprotected truncate implementation. Unlike
+    * {@link FSDirTruncateOp#truncate}, this will not schedule block recovery.
+    *
+    * @param fsn namespace
+    * @param src path name
+    * @param clientName client name
+    * @param clientMachine client machine info
+    * @param newLength the target file size
+    * @param mtime modified time
+    * @param truncateBlock truncate block
+    * @throws IOException
+    */
+   static void unprotectedTruncate(final FSNamesystem fsn, final String src,
+       final String clientName, final String clientMachine,
+       final long newLength, final long mtime, final Block truncateBlock)
+       throws UnresolvedLinkException, QuotaExceededException,
+       SnapshotAccessControlException, IOException {
+     assert fsn.hasWriteLock();
+ 
+     FSDirectory fsd = fsn.getFSDirectory();
+     INodesInPath iip = fsd.getINodesInPath(src, true);
+     INodeFile file = iip.getLastINode().asFile();
+     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
+     boolean onBlockBoundary = unprotectedTruncate(fsn, iip, newLength,
+         collectedBlocks, mtime, null);
+ 
+     if (!onBlockBoundary) {
+       BlockInfo oldBlock = file.getLastBlock();
+       Block tBlk = prepareFileForTruncate(fsn, iip, clientName, clientMachine,
+           file.computeFileSize() - newLength, truncateBlock);
+       assert Block.matchingIdAndGenStamp(tBlk, truncateBlock) &&
+           tBlk.getNumBytes() == truncateBlock.getNumBytes() :
+           "Should be the same block.";
+       if (oldBlock.getBlockId() != tBlk.getBlockId()
+           && !file.isBlockInLatestSnapshot(oldBlock)) {
+         fsd.getBlockManager().removeBlockFromMap(oldBlock);
+       }
+     }
+     assert onBlockBoundary == (truncateBlock == null) :
+       "truncateBlock is null iff on block boundary: " + truncateBlock;
+     fsn.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
+   }
+ 
+   /**
+    * Convert current INode to UnderConstruction. Recreate lease. Create new
+    * block for the truncated copy. Schedule truncation of the replicas.
+    *
+    * @param fsn namespace
+    * @param iip inodes in the path containing the file
+    * @param leaseHolder lease holder
+    * @param clientMachine client machine info
+    * @param lastBlockDelta last block delta size
+    * @param newBlock new block
+    * @return the returned block will be written to editLog and passed back
+    *         into this method upon loading.
+    * @throws IOException
+    */
+   @VisibleForTesting
+   static Block prepareFileForTruncate(FSNamesystem fsn, INodesInPath iip,
+       String leaseHolder, String clientMachine, long lastBlockDelta,
+       Block newBlock) throws IOException {
+     assert fsn.hasWriteLock();
+ 
+     INodeFile file = iip.getLastINode().asFile();
++    assert !file.isStriped();
+     file.recordModification(iip.getLatestSnapshotId());
+     file.toUnderConstruction(leaseHolder, clientMachine);
+     assert file.isUnderConstruction() : "inode should be under construction.";
+     fsn.getLeaseManager().addLease(
+         file.getFileUnderConstructionFeature().getClientName(), file.getId());
+     boolean shouldRecoverNow = (newBlock == null);
+     BlockInfo oldBlock = file.getLastBlock();
++
+     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(fsn, file, oldBlock);
+     if (newBlock == null) {
 -      newBlock = (shouldCopyOnTruncate) ? fsn.createNewBlock() : new Block(
 -          oldBlock.getBlockId(), oldBlock.getNumBytes(),
++      newBlock = (shouldCopyOnTruncate) ? fsn.createNewBlock(false)
++          : new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
+           fsn.nextGenerationStamp(fsn.getBlockIdManager().isLegacyBlock(
+               oldBlock)));
+     }
+ 
+     BlockInfoContiguousUnderConstruction truncatedBlockUC;
+     BlockManager blockManager = fsn.getFSDirectory().getBlockManager();
+     if (shouldCopyOnTruncate) {
+       // Add new truncateBlock into blocksMap and
+       // use oldBlock as a source for copy-on-truncate recovery
+       truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock,
+           file.getPreferredBlockReplication());
+       truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
+       truncatedBlockUC.setTruncateBlock(oldBlock);
 -      file.setLastBlock(truncatedBlockUC, blockManager.getStorages(oldBlock));
++      file.convertLastBlockToUC(truncatedBlockUC,
++          blockManager.getStorages(oldBlock));
+       blockManager.addBlockCollection(truncatedBlockUC, file);
+ 
+       NameNode.stateChangeLog.debug(
+           "BLOCK* prepareFileForTruncate: Scheduling copy-on-truncate to new"
+               + " size {}  new block {} old block {}",
+           truncatedBlockUC.getNumBytes(), newBlock,
+           truncatedBlockUC.getTruncateBlock());
+     } else {
+       // Use new generation stamp for in-place truncate recovery
+       blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
+       oldBlock = file.getLastBlock();
+       assert !oldBlock.isComplete() : "oldBlock should be under construction";
+       truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock;
+       truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
+       truncatedBlockUC.getTruncateBlock().setNumBytes(
+           oldBlock.getNumBytes() - lastBlockDelta);
+       truncatedBlockUC.getTruncateBlock().setGenerationStamp(
+           newBlock.getGenerationStamp());
+ 
+       NameNode.stateChangeLog.debug(
+           "BLOCK* prepareFileForTruncate: {} Scheduling in-place block "
+               + "truncate to new size {}", truncatedBlockUC.getTruncateBlock()
+               .getNumBytes(), truncatedBlockUC);
+     }
+     if (shouldRecoverNow) {
+       truncatedBlockUC.initializeBlockRecovery(newBlock.getGenerationStamp());
+     }
+ 
+     return newBlock;
+   }
+ 
+   /**
+    * Truncate has the following properties:
+    * 1.) Any block deletions occur now.
+    * 2.) INode length is truncated now - new clients can only read up to
+    *     the truncated length.
+    * 3.) INode will be set to UC and lastBlock set to UNDER_RECOVERY.
+    * 4.) NN will trigger DN truncation recovery and waits for DNs to report.
+    * 5.) File is considered UNDER_RECOVERY until truncation recovery
+    *     completes.
+    * 6.) Soft and hard Lease expiration require truncation recovery to
+    *     complete.
+    *
+    * @return true if on the block boundary or false if recovery is need
+    */
+   private static boolean unprotectedTruncate(FSNamesystem fsn,
+       INodesInPath iip, long newLength, BlocksMapUpdateInfo collectedBlocks,
+       long mtime, QuotaCounts delta) throws IOException {
+     assert fsn.hasWriteLock();
+ 
+     INodeFile file = iip.getLastINode().asFile();
+     int latestSnapshot = iip.getLatestSnapshotId();
+     file.recordModification(latestSnapshot, true);
+ 
+     verifyQuotaForTruncate(fsn, iip, file, newLength, delta);
+ 
+     long remainingLength =
+         file.collectBlocksBeyondMax(newLength, collectedBlocks);
+     file.excludeSnapshotBlocks(latestSnapshot, collectedBlocks);
+     file.setModificationTime(mtime);
+     // return whether on a block boundary
+     return (remainingLength - newLength) == 0;
+   }
+ 
+   private static void verifyQuotaForTruncate(FSNamesystem fsn,
+       INodesInPath iip, INodeFile file, long newLength, QuotaCounts delta)
+       throws QuotaExceededException {
+     FSDirectory fsd = fsn.getFSDirectory();
+     if (!fsn.isImageLoaded() || fsd.shouldSkipQuotaChecks()) {
+       // Do not check quota if edit log is still being processed
+       return;
+     }
+     final BlockStoragePolicy policy = fsd.getBlockStoragePolicySuite()
+         .getPolicy(file.getStoragePolicyID());
+     file.computeQuotaDeltaForTruncate(newLength, policy, delta);
+     fsd.readLock();
+     try {
+       FSDirectory.verifyQuota(iip, iip.length() - 1, delta, null);
+     } finally {
+       fsd.readUnlock();
+     }
+   }
+ 
+   /**
+    * Defines if a replica needs to be copied on truncate or
+    * can be truncated in place.
+    */
+   private static boolean shouldCopyOnTruncate(FSNamesystem fsn, INodeFile file,
+       BlockInfo blk) {
+     if (!fsn.isUpgradeFinalized()) {
+       return true;
+     }
+     if (fsn.isRollingUpgrade()) {
+       return true;
+     }
+     return file.isBlockInLatestSnapshot(blk);
+   }
+ 
+   /**
+    * Result of truncate operation.
+    */
+   static class TruncateResult {
+     private final boolean result;
+     private final HdfsFileStatus stat;
+ 
+     public TruncateResult(boolean result, HdfsFileStatus stat) {
+       this.result = result;
+       this.stat = stat;
+     }
+ 
+     /**
+      * @return true if client does not need to wait for block recovery,
+      *          false if client needs to wait for block recovery.
+      */
+     boolean getResult() {
+       return result;
+     }
+ 
+     /**
+      * @return file information.
+      */
+     HdfsFileStatus getFileStatus() {
+       return stat;
+     }
+   }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 8f4f51a,3d30a19..b9466f6
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@@ -45,9 -43,7 +45,9 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
  import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
++import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
  import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@@ -527,31 -515,17 +526,31 @@@ class FSDirWriteFileOp 
        final INodeFile fileINode = inodesInPath.getLastINode().asFile();
        Preconditions.checkState(fileINode.isUnderConstruction());
  
 -      // check quota limits and updated space consumed
 -      fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
 -          fileINode.getPreferredBlockReplication(), true);
 -
        // associate new last block for the file
 -      BlockInfoContiguousUnderConstruction blockInfo =
 -        new BlockInfoContiguousUnderConstruction(
 -            block,
 -            fileINode.getFileReplication(),
 -            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
 +      final BlockInfo blockInfo;
 +      if (isStriped) {
 +        ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
 +            fsd.getFSNamesystem(), inodesInPath);
 +        ErasureCodingPolicy ecPolicy = ecZone.getErasureCodingPolicy();
 +        short numDataUnits = (short) ecPolicy.getNumDataUnits();
 +        short numParityUnits = (short) ecPolicy.getNumParityUnits();
 +        short numLocations = (short) (numDataUnits + numParityUnits);
 +
 +        // check quota limits and updated space consumed
 +        fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
 +            numLocations, true);
-         blockInfo = new BlockInfoUnderConstructionStriped(block, ecPolicy,
++        blockInfo = new BlockInfoStripedUnderConstruction(block, ecPolicy,
 +            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
 +      } else {
 +        // check quota limits and updated space consumed
 +        fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
 +            fileINode.getPreferredBlockReplication(), true);
 +
 +        short numLocations = fileINode.getFileReplication();
-         blockInfo = new BlockInfoUnderConstructionContiguous(block,
++        blockInfo = new BlockInfoContiguousUnderConstruction(block,
 +            numLocations, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
              targets);
 +      }
        fsd.getBlockManager().addBlockCollection(blockInfo, fileINode);
        fileINode.addBlock(blockInfo);
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index b5b7178,3dd076d..008a327
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@@ -36,20 -36,16 +36,20 @@@ import org.apache.hadoop.classification
  import org.apache.hadoop.classification.InterfaceStability;
  import org.apache.hadoop.fs.FileSystem;
  import org.apache.hadoop.fs.XAttrSetFlag;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
++import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
  import org.apache.hadoop.hdfs.protocol.Block;
  import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
  import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
  import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
  import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@@ -988,17 -969,10 +988,17 @@@ public class FSEditLogLoader 
        Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
      }
      // add the new block
 -    BlockInfo newBI = new BlockInfoContiguousUnderConstruction(
 -          newBlock, file.getPreferredBlockReplication());
 -    fsNamesys.getBlockManager().addBlockCollection(newBI, file);
 -    file.addBlock(newBI);
 +    final BlockInfo newBlockInfo;
 +    boolean isStriped = ecZone != null;
 +    if (isStriped) {
-       newBlockInfo = new BlockInfoUnderConstructionStriped(newBlock,
++      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
 +          ecZone.getErasureCodingPolicy());
 +    } else {
-       newBlockInfo = new BlockInfoUnderConstructionContiguous(newBlock,
++      newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
 +          file.getPreferredBlockReplication());
 +    }
 +    fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file);
 +    file.addBlock(newBlockInfo);
      fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
    }
    
@@@ -1076,13 -1049,8 +1076,13 @@@
            // TODO: shouldn't this only be true for the last block?
            // what about an old-version fsync() where fsync isn't called
            // until several blocks in?
 -          newBI = new BlockInfoContiguousUnderConstruction(
 -              newBlock, file.getPreferredBlockReplication());
 +          if (isStriped) {
-             newBI = new BlockInfoUnderConstructionStriped(newBlock,
++            newBI = new BlockInfoStripedUnderConstruction(newBlock,
 +                ecZone.getErasureCodingPolicy());
 +          } else {
-             newBI = new BlockInfoUnderConstructionContiguous(newBlock,
++            newBI = new BlockInfoContiguousUnderConstruction(newBlock,
 +                file.getPreferredBlockReplication());
 +          }
          } else {
            // OP_CLOSE should add finalized blocks. This code path
            // is only executed when loading edits written by prior

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index 2e490e7,30517d0..e7c87d6
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@@ -778,9 -776,9 +778,9 @@@ public class FSImageFormat 
              clientMachine = FSImageSerialization.readString(in);
              // convert the last block to BlockUC
              if (blocks.length > 0) {
 -              BlockInfo lastBlk = blocks[blocks.length - 1];
 -              blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(
 -                  lastBlk, replication);
 +              Block lastBlk = blocks[blocks.length - 1];
 +              blocks[blocks.length - 1] =
-                   new BlockInfoUnderConstructionContiguous(lastBlk, replication);
++                  new BlockInfoContiguousUnderConstruction(lastBlk, replication);
              }
            }
          }
@@@ -1144,7 -1141,7 +1144,7 @@@
        + " option to automatically rename these paths during upgrade.";
  
    /**
--   * Same as {@link #renameReservedPathsOnUpgrade(String)}, but for a single
++   * Same as {@link #renameReservedPathsOnUpgrade}, but for a single
     * byte array path component.
     */
    private static byte[] renameReservedComponentOnUpgrade(byte[] component,
@@@ -1164,7 -1161,7 +1164,7 @@@
    }
  
    /**
--   * Same as {@link #renameReservedPathsOnUpgrade(String)}, but for a single
++   * Same as {@link #renameReservedPathsOnUpgrade}, but for a single
     * byte array path component.
     */
    private static byte[] renameReservedRootComponentOnUpgrade(byte[] component,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index 653bd4a,e8378e5..51b04d0
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@@ -45,9 -44,7 +45,9 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocolPB.PBHelper;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
++import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
  import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
  import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@@@ -366,15 -364,8 +375,15 @@@ public final class FSImageFormatPBINod
          if (blocks.length > 0) {
            BlockInfo lastBlk = file.getLastBlock();
            // replace the last block of file
 -          file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction(
 -              lastBlk, replication));
 +          final BlockInfo ucBlk;
 +          if (isStriped) {
 +            BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
-             ucBlk = new BlockInfoUnderConstructionStriped(striped, ecPolicy);
++            ucBlk = new BlockInfoStripedUnderConstruction(striped, ecPolicy);
 +          } else {
-             ucBlk = new BlockInfoUnderConstructionContiguous(lastBlk,
++            ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
 +                replication);
 +          }
 +          file.setBlock(file.numBlocks() - 1, ucBlk);
          }
        }
        return file;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index d87378c,f71cf0b..af3f813
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@@ -33,8 -33,9 +33,8 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
  import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
  import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
  import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
  import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@@ -138,10 -137,9 +138,10 @@@ public class FSImageSerialization 
      // last block is UNDER_CONSTRUCTION
      if(numBlocks > 0) {
        blk.readFields(in);
-       blocksContiguous[i] = new BlockInfoUnderConstructionContiguous(
 -      blocks[i] = new BlockInfoContiguousUnderConstruction(
 -        blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
++      blocksContiguous[i] = new BlockInfoContiguousUnderConstruction(
 +          blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
      }
 +
      PermissionStatus perm = PermissionStatus.read(in);
      String clientName = readString(in);
      String clientMachine = readString(in);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 657f29d,d34242c..a7107d7
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@@ -168,10 -166,8 +165,9 @@@ import org.apache.hadoop.hdfs.DFSUtil
  import org.apache.hadoop.hdfs.HAUtil;
  import org.apache.hadoop.hdfs.HdfsConfiguration;
  import org.apache.hadoop.hdfs.UnknownCryptoProtocolVersionException;
- import org.apache.hadoop.hdfs.XAttrHelper;
  import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
  import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
  import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
  import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
  import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
@@@ -207,10 -199,7 +203,10 @@@ import org.apache.hadoop.hdfs.security.
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
  import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
  import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@@ -4240,8 -3714,9 +3775,9 @@@ public class FSNamesystem implements Na
  
          while (it.hasNext()) {
            Block b = it.next();
 -          BlockInfo blockInfo = blockManager.getStoredBlock(b);
 +          BlockInfo blockInfo = getStoredBlock(b);
-           if (blockInfo.getBlockCollection().getStoragePolicyID() == lpPolicy.getId()) {
+           if (blockInfo.getBlockCollection().getStoragePolicyID()
+               == lpPolicy.getId()) {
              filesToDelete.add(blockInfo.getBlockCollection());
            }
          }
@@@ -6670,8 -6142,8 +6221,9 @@@
    public void setFSDirectory(FSDirectory dir) {
      this.dir = dir;
    }
 +
    /** @return the cache manager. */
+   @Override
    public CacheManager getCacheManager() {
      return cacheManager;
    }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 25415ef,3f242e0..13f180a
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@@ -39,11 -37,10 +39,12 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
  import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
  import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
  import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
  import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
@@@ -445,16 -413,6 +446,15 @@@ public class INodeFile extends INodeWit
      setStoragePolicyID(storagePolicyId);
    }
  
- 
 +  /**
 +   * @return true if the file is in the striping layout.
 +   */
 +  @VisibleForTesting
 +  @Override
 +  public boolean isStriped() {
 +    return HeaderFormat.isStriped(header);
 +  }
 +
    @Override // INodeFileAttributes
    public long getHeaderLong() {
      return header;
@@@ -483,9 -439,9 +483,8 @@@
      snapshotBlocks = getDiffs().findLaterSnapshotBlocks(snapshot);
      return (snapshotBlocks == null) ? getBlocks() : snapshotBlocks;
    }
--
-   /** Used during concat to update the BlockCollection for each block */
-   void updateBlockCollection() {
+   /** Used during concat to update the BlockCollection for each block. */
+   private void updateBlockCollection() {
      if (blocks != null) {
        for(BlockInfo b : blocks) {
          b.setBlockCollection(this);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
index f93218f,3a5dc12..2943fc2
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
@@@ -63,17 -78,16 +78,17 @@@ public class NameNodeLayoutVersion 
     * </ul>
     */
    public static enum Feature implements LayoutFeature {
-     ROLLING_UPGRADE(-55, -53, "Support rolling upgrade", false),
-     EDITLOG_LENGTH(-56, "Add length field to every edit log op"),
-     XATTRS(-57, "Extended attributes"),
-     CREATE_OVERWRITE(-58, "Use single editlog record for " +
+     ROLLING_UPGRADE(-55, -53, -55, "Support rolling upgrade", false),
+     EDITLOG_LENGTH(-56, -56, "Add length field to every edit log op"),
+     XATTRS(-57, -57, "Extended attributes"),
+     CREATE_OVERWRITE(-58, -58, "Use single editlog record for " +
        "creating file with overwrite"),
-     XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"),
-     BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
-     TRUNCATE(-61, "Truncate"),
-     APPEND_NEW_BLOCK(-62, "Support appending to new block"),
-     QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types"),
-     ERASURE_CODING(-64, "Support erasure coding");
+     XATTRS_NAMESPACE_EXT(-59, -59, "Increase number of xattr namespaces"),
+     BLOCK_STORAGE_POLICY(-60, -60, "Block Storage policy"),
+     TRUNCATE(-61, -61, "Truncate"),
+     APPEND_NEW_BLOCK(-62, -61, "Support appending to new block"),
 -    QUOTA_BY_STORAGE_TYPE(-63, -61, "Support quota for specific storage types");
++    QUOTA_BY_STORAGE_TYPE(-63, -61, "Support quota for specific storage types"),
++    ERASURE_CODING(-64, -61, "Support erasure coding");
  
      private final FeatureInfo info;
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 3a2c3d4,7d4cd7e..9d43c15
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@@ -455,17 -442,11 +455,17 @@@ public class NamenodeFsck implements Da
      if (blocks == null) { // the file is deleted
        return;
      }
 -    collectFileSummary(path, file, res, blocks);
 -    collectBlocksSummary(parent, file, res, blocks);
 +
-     final Result r = file.getReplication() == 0? ecRes: replRes;
++    final Result r = file.getErasureCodingPolicy() != null ? ecRes: replRes;
 +    collectFileSummary(path, file, r, blocks);
 +    if (showprogress && (replRes.totalFiles + ecRes.totalFiles) % 100 == 0) {
 +      out.println();
 +      out.flush();
 +    }
 +    collectBlocksSummary(parent, file, r, blocks);
    }
  
 -  private void checkDir(String path, Result res) throws IOException {
 +  private void checkDir(String path, Result replRes, Result ecRes) throws IOException {
      if (snapshottableDirs != null && snapshottableDirs.contains(path)) {
        String snapshotPath = (path.endsWith(Path.SEPARATOR) ? path : path
            + Path.SEPARATOR)


[26/50] [abbrv] hadoop git commit: HDFS-9097. Erasure coding: update EC command "-s" flag to "-p" when specifying policy. Contributed by Zhe Zhang.

Posted by wa...@apache.org.
HDFS-9097. Erasure coding: update EC command "-s" flag to "-p" when specifying policy. Contributed by Zhe Zhang.

Change-Id: I82919e55c4a0d8646d25a53aac757098a56b3029


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

Branch: refs/heads/trunk
Commit: 607bda2a21aebca7b363d57ddece8ce8b5db6c63
Parents: e36129b
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Thu Sep 17 14:52:37 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Thu Sep 17 14:52:37 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt              |  3 +++
 .../hadoop/hdfs/tools/erasurecode/ECCommand.java      |  6 +++---
 .../src/site/markdown/HDFSErasureCoding.md            |  4 ++--
 .../src/test/resources/testErasureCodingConf.xml      | 14 +++++++-------
 4 files changed, 15 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/607bda2a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 0345a54..3a29db1 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -429,3 +429,6 @@
     (Rakesh R via zhz)
 
     HDFS-9088. Cleanup erasure coding documentation. (wang via zhz)
+
+    HDFS-9097. Erasure coding: update EC command "-s" flag to "-p" when 
+    specifying policy. (zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/607bda2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
index b857d3e..978fe47 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
@@ -80,11 +80,11 @@ public abstract class ECCommand extends Command {
    */
   static class SetECPolicyCommand extends ECCommand {
     public static final String NAME = "setPolicy";
-    public static final String USAGE = "[-s <policyName>] <path>";
+    public static final String USAGE = "[-p <policyName>] <path>";
     public static final String DESCRIPTION = 
         "Set a specified erasure coding policy to a directory\n"
         + "Options :\n"
-        + "  -s <policyName> : erasure coding policy name to encode files. "
+        + "  -p <policyName> : erasure coding policy name to encode files. "
         + "If not passed the default policy will be used\n"
         + "  <path>  : Path to a directory. Under this directory "
         + "files will be encoded using specified erasure coding policy";
@@ -93,7 +93,7 @@ public abstract class ECCommand extends Command {
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
-      ecPolicyName = StringUtils.popOptionWithArgument("-s", args);
+      ecPolicyName = StringUtils.popOptionWithArgument("-p", args);
       if (args.isEmpty()) {
         throw new HadoopIllegalArgumentException("<path> is missing");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/607bda2a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 3040bf5..2323a6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -110,7 +110,7 @@ Deployment
   HDFS provides an `erasurecode` subcommand to perform administrative commands related to erasure coding.
 
        hdfs erasurecode [generic options]
-         [-setPolicy [-s <policyName>] <path>]
+         [-setPolicy [-p <policyName>] <path>]
          [-getPolicy <path>]
          [-listPolicies]
          [-usage [cmd ...]]
@@ -118,7 +118,7 @@ Deployment
 
 Below are the details about each command.
 
- *  `[-setPolicy [-s <policyName>] <path>]`
+ *  `[-setPolicy [-p <policyName>] <path>]`
 
     Sets an ErasureCoding policy on a directory at the specified path.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/607bda2a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index 3a10a50..e72d862 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -61,7 +61,7 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-setPolicy \[-s &lt;policyName&gt;\] &lt;path&gt;(.)*</expected-output>
+          <expected-output>^-setPolicy \[-p &lt;policyName&gt;\] &lt;path&gt;(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -109,7 +109,7 @@
       <description>setPolicy : set erasure coding policy on a directory to encode files</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>
@@ -179,7 +179,7 @@
       <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir</ec-admin-command>
         <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
@@ -197,7 +197,7 @@
       <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -s RS-6-3-64k /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir</ec-admin-command>
         <command>-fs NAMENODE -touchz /ecdir/ecfile</command>
         <ec-admin-command>-fs NAMENODE -getPolicy /ecdir/ecfile</ec-admin-command>
       </test-commands>
@@ -250,7 +250,7 @@
       <description>setPolicy : illegal parameters - policy name is missing</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -s</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -p</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>
@@ -258,7 +258,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-setPolicy: option -s requires 1 argument(.)*</expected-output>
+          <expected-output>^-setPolicy: option -p requires 1 argument(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -284,7 +284,7 @@
       <description>setPolicy : illegal parameters - invalidpolicy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
-        <ec-admin-command>-fs NAMENODE -setPolicy -s invalidpolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -p invalidpolicy /ecdir</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /ecdir</command>


[25/50] [abbrv] hadoop git commit: HDFS-9088. Cleanup erasure coding documentation. Contributed by Andrew Wang.

Posted by wa...@apache.org.
HDFS-9088. Cleanup erasure coding documentation. Contributed by Andrew Wang.

Change-Id: Ic3ec1f29fef0e27c46fff66fd28a51f8c4c61e71


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

Branch: refs/heads/trunk
Commit: e36129b61abd9edbdd77e053a5e2bfdad434d164
Parents: ced438a
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Thu Sep 17 09:56:32 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Thu Sep 17 09:56:32 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   2 +
 .../src/site/markdown/HDFSErasureCoding.md      | 123 +++++++++----------
 2 files changed, 57 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e36129b6/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index acf62cb..0345a54 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -427,3 +427,5 @@
 
     HDFS-8899. Erasure Coding: use threadpool for EC recovery tasks on DataNode.
     (Rakesh R via zhz)
+
+    HDFS-9088. Cleanup erasure coding documentation. (wang via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e36129b6/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 44c209e..3040bf5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -19,108 +19,95 @@ HDFS Erasure Coding
     * [Purpose](#Purpose)
     * [Background](#Background)
     * [Architecture](#Architecture)
-    * [Hardware resources](#Hardware_resources)
     * [Deployment](#Deployment)
-        * [Configuration details](#Configuration_details)
-        * [Deployment details](#Deployment_details)
+        * [Cluster and hardware configuration](#Cluster_and_hardware_configuration)
+        * [Configuration keys](#Configuration_keys)
         * [Administrative commands](#Administrative_commands)
 
 Purpose
 -------
-  Replication is expensive -- the default 3x replication scheme has 200% overhead in storage space and other resources (e.g., network bandwidth).
-  However, for “warm” and “cold” datasets with relatively low I/O activities, secondary block replicas are rarely accessed during normal operations, but still consume the same amount of resources as the primary ones.
+  Replication is expensive -- the default 3x replication scheme in HDFS has 200% overhead in storage space and other resources (e.g., network bandwidth).
+  However, for warm and cold datasets with relatively low I/O activities, additional block replicas are rarely accessed during normal operations, but still consume the same amount of resources as the first replica.
 
-  Therefore, a natural improvement is to use Erasure Coding (EC) in place of replication, which provides the same level of fault tolerance with much less storage space. In typical Erasure Coding(EC) setups, the storage overhead is ≤ 50%.
+  Therefore, a natural improvement is to use Erasure Coding (EC) in place of replication, which provides the same level of fault-tolerance with much less storage space. In typical Erasure Coding (EC) setups, the storage overhead is no more than 50%.
 
 Background
 ----------
 
   In storage systems, the most notable usage of EC is Redundant Array of Inexpensive Disks (RAID). RAID implements EC through striping, which divides logically sequential data (such as a file) into smaller units (such as bit, byte, or block) and stores consecutive units on different disks. In the rest of this guide this unit of striping distribution is termed a striping cell (or cell). For each stripe of original data cells, a certain number of parity cells are calculated and stored -- the process of which is called encoding. The error on any striping cell can be recovered through decoding calculation based on surviving data and parity cells.
 
-  Integrating the EC function with HDFS could get storage efficient deployments. It can provide similar data tolerance as traditional HDFS replication based deployments but it stores only one original replica data and parity cells.
-  In a typical case, A file with 6 blocks will actually be consume space of 6*3 = 18 blocks with replication factor 3. But with EC (6 data,3 parity) deployment, it will only consume space of 9 blocks.
+  Integrating EC with HDFS can improve storage efficiency while still providing similar data durability as traditional replication-based HDFS deployments.
+  As an example, a 3x replicated file with 6 blocks will consume 6*3 = 18 blocks of disk space. But with EC (6 data, 3 parity) deployment, it will only consume 9 blocks of disk space.
 
 Architecture
 ------------
-  In the context of EC, striping has several critical advantages. First, it enables online EC which bypasses the conversion phase and immediately saves storage space. Online EC also enhances sequential I/O performance by leveraging multiple disk spindles in parallel; this is especially desirable in clusters with high end networking  . Second, it naturally distributes a small file to multiple DataNodes and eliminates the need to bundle multiple files into a single coding group. This greatly simplifies file operations such as deletion, quota reporting, and migration between federated namespaces.
+  In the context of EC, striping has several critical advantages. First, it enables online EC (writing data immediately in EC format), avoiding a conversion phase and immediately saving storage space. Online EC also enhances sequential I/O performance by leveraging multiple disk spindles in parallel; this is especially desirable in clusters with high end networking. Second, it naturally distributes a small file to multiple DataNodes and eliminates the need to bundle multiple files into a single coding group. This greatly simplifies file operations such as deletion, quota reporting, and migration between federated namespaces.
 
-  As in general HDFS clusters, small files could account for over 3/4 of total storage consumption. So, In this first phase of erasure coding work, HDFS supports striping model. In the near future, HDFS will supports contiguous layout as second second phase work. So this guide focuses more on striping model EC.
+  In typical HDFS clusters, small files can account for over 3/4 of total storage consumption. To better support small files, in this first phase of work HDFS supports EC with striping. In the future, HDFS will also support a contiguous EC layout. See the design doc and discussion on [HDFS-7285](https://issues.apache.org/jira/browse/HDFS-7285) for more information.
 
- *  **NameNode Extensions** - Under the striping layout, a HDFS file is logically composed of block groups, each of which contains a certain number of   internal blocks.
-   To eliminate the need for NameNode to monitor all internal blocks, a new hierarchical block naming protocol is introduced, where the ID of a block group can be inferred from any of its internal blocks. This allows each block group to be managed as a new type of BlockInfo named BlockInfoStriped, which tracks its own internal blocks by attaching an index to each replica location.
+ *  **NameNode Extensions** - Striped HDFS files are logically composed of block groups, each of which contains a certain number of internal blocks.
+    To reduce NameNode memory consumption from these additional blocks, a new hierarchical block naming protocol was introduced. The ID of a block group can be inferred from the ID of any of its internal blocks. This allows management at the level of the block group rather than the block.
 
- *  **Client Extensions** - The basic principle behind the extensions is to allow the client node to work on multiple internal blocks in a block group in
-    parallel.
+ *  **Client Extensions** - The client read and write paths were enhanced to work on multiple internal blocks in a block group in parallel.
     On the output / write path, DFSStripedOutputStream manages a set of data streamers, one for each DataNode storing an internal block in the current block group. The streamers mostly
     work asynchronously. A coordinator takes charge of operations on the entire block group, including ending the current block group, allocating a new block group, and so forth.
     On the input / read path, DFSStripedInputStream translates a requested logical byte range of data as ranges into internal blocks stored on DataNodes. It then issues read requests in
     parallel. Upon failures, it issues additional read requests for decoding.
 
- *  **DataNode Extensions** - ErasureCodingWorker(ECWorker) is for reconstructing erased erasure coding blocks and runs along with the Datanode process. Erased block details would have been found out by Namenode ReplicationMonitor thread and sent to Datanode via its heartbeat responses as discussed in the previous sections. For each reconstruction task,
-   i.e. ReconstructAndTransferBlock, it will start an internal daemon thread that performs 3 key tasks:
+ *  **DataNode Extensions** - The DataNode runs an additional ErasureCodingWorker (ECWorker) task for background recovery of failed erasure coded blocks. Failed EC blocks are detected by the NameNode, which then chooses a DataNode to do the recovery work. The recovery task is passed as a heartbeat response. This process is similar to how replicated blocks are re-replicated on failure. Reconstruction performs three key tasks:
 
-      _1.Read the data from source nodes:_ For reading the data blocks from different source nodes, it uses a dedicated thread pool.
-         The thread pool is initialized when ErasureCodingWorker initializes. Based on the EC policy, it schedules the read requests to all source targets and ensures only to read
-         minimum required input blocks for reconstruction.
+      1. _Read the data from source nodes:_ Input data is read in parallel from source nodes using a dedicated thread pool.
+        Based on the EC policy, it schedules the read requests to all source targets and reads only the minimum number of input blocks for reconstruction.
 
-      _2.Decode the data and generate the output data:_ Actual decoding/encoding is done by using RawErasureEncoder API currently.
-        All the erased data and/or parity blocks will be recovered together.
+      1. _Decode the data and generate the output data:_ New data and parity blocks are decoded from the input data. All missing data and parity blocks are decoded together.
 
-     _3.Transfer the generated data blocks to target nodes:_ Once decoding is finished, it will encapsulate the output data to packets and send them to
-        target Datanodes.
-   To accommodate heterogeneous workloads, we allow files and directories in an HDFS cluster to have different replication and EC policies.
-*   **ErasureCodingPolicy**
-    Information on how to encode/decode a file is encapsulated in an ErasureCodingPolicy class. Each policy is defined by the following 2 pieces of information:
-    _1.The ECScema: This includes the numbers of data and parity blocks in an EC group (e.g., 6+3), as well as the codec algorithm (e.g., Reed-Solomon).
+      1. _Transfer the generated data blocks to target nodes:_ Once decoding is finished, the recovered blocks are transferred to target DataNodes.
 
-    _2.The size of a striping cell.
+ *  **ErasureCoding policy**
+    To accommodate heterogeneous workloads, we allow files and directories in an HDFS cluster to have different replication and EC policies.
+    Information on how to encode/decode a file is encapsulated in an ErasureCodingPolicy class. Each policy is defined by the following 2 pieces of information:
 
-   Client and Datanode uses EC codec framework directly for doing the endoing/decoding work.
+      1. _The ECSchema:_ This includes the numbers of data and parity blocks in an EC group (e.g., 6+3), as well as the codec algorithm (e.g., Reed-Solomon).
 
- *  **Erasure Codec Framework**
-     We support a generic EC framework which allows system users to define, configure, and deploy multiple coding schemas such as conventional Reed-Solomon, HitchHicker and
-     so forth.
-     ErasureCoder is provided to encode or decode for a block group in the middle level, and RawErasureCoder is provided to perform the concrete algorithm calculation in the low level. ErasureCoder can
-     combine and make use of different RawErasureCoders for tradeoff. We abstracted coder type, data blocks size, parity blocks size into ECSchema. A default system schema using RS (6, 3) is built-in.
-     For the system default codec Reed-Solomon we implemented both RSRawErasureCoder in pure Java and NativeRawErasureCoder based on Intel ISA-L. Below is the performance
-     comparing for different coding chunk size. We can see that the native coder can outperform the Java coder by up to 35X.
+      1. _The size of a striping cell._ This determines the granularity of striped reads and writes, including buffer sizes and encoding work.
 
-     _Intel® Storage Acceleration-Library(Intel® ISA-L)_ ISA-L is an Open Source Version and is a collection of low-level functions used in storage applications.
-     The open source version contains fast erasure codes that implement a general Reed-Solomon type encoding for blocks of data that helps protect against
-     erasure of whole blocks. The general ISA-L library contains an expanded set of functions used for data protection, hashing, encryption, etc. By
-     leveraging instruction sets like SSE, AVX and AVX2, the erasure coding functions are much optimized and outperform greatly on IA platforms. ISA-L
-     supports Linux, Windows and other platforms as well. Additionally, it also supports incremental coding so applications don’t have to wait all source
-     blocks to be available before to perform the coding, which can be used in HDFS.
+    Currently, HDFS supports the Reed-Solomon and XOR erasure coding algorithms. Additional algorithms are planned as future work.
+    The system default scheme is Reed-Solomon (6, 3) with a cell size of 64KB.
 
-Hardware resources
-------------------
-  For using EC feature, you need to prepare for the following.
-    Depending on the ECSchemas used, we need to have minimum number of Datanodes available in the cluster. Example if we use ReedSolomon(6, 3) ECSchema,
-    then minimum nodes required is 9 to succeed the write. It can tolerate up to 3 failures.
 
 Deployment
 ----------
 
-### Configuration details
+### Cluster and hardware configuration
+
+  Erasure coding places additional demands on the cluster in terms of CPU and network.
 
-  In the EC feature, raw coders are configurable. So, users need to decide the RawCoder algorithms.
-  Configure the customized algorithms with configuration key "*io.erasurecode.codecs*".
+  Encoding and decoding work consumes additional CPU on both HDFS clients and DataNodes.
 
-  Default Reed-Solomon based raw coders available in built, which can be configured by using the configuration key "*io.erasurecode.codec.rs.rawcoder*".
-  And also another default raw coder available if XOR based raw coder. Which could be configured by using "*io.erasurecode.codec.xor.rawcoder*"
+  Erasure coded files are also spread across racks for rack fault-tolerance.
+  This means that when reading and writing striped files, most operations are off-rack.
+  Network bisection bandwidth is thus very important.
 
-  _EarasureCodingWorker Confugurations:_
-    dfs.datanode.stripedread.threshold.millis - Threshold time for polling timeout for read service. Default value is 5000
-    dfs.datanode.stripedread.threads – Number striped read thread pool threads. Default value is 20
-    dfs.datanode.stripedread.buffer.size - Buffer size for reader service. Default value is 256 * 1024
+  For rack fault-tolerance, it is also important to have at least as many racks as the configured EC stripe width.
+  For the default EC policy of RS (6,3), this means minimally 9 racks, and ideally 10 or 11 to handle planned and unplanned outages.
+  For clusters with fewer racks than the stripe width, HDFS cannot maintain rack fault-tolerance, but will still attempt
+  to spread a striped file across multiple nodes to preserve node-level fault-tolerance.
 
-### Deployment details
+### Configuration keys
 
-  With the striping model, client machine is responsible for do the EC endoing and tranferring data to the datanodes.
-  So, EC with striping model expects client machines with hghg end configurations especially of CPU and network.
+  The codec implementation for Reed-Solomon and XOR can be configured with the following client and DataNode configuration keys:
+  `io.erasurecode.codec.rs.rawcoder` and `io.erasurecode.codec.xor.rawcoder`.
+  The default implementations for both of these codecs are pure Java.
+
+  Erasure coding background recovery work on the DataNodes can also be tuned via the following configuration parameters:
+
+  1. `dfs.datanode.stripedread.threshold.millis` - Timeout for striped reads. Default value is 5000 ms.
+  1. `dfs.datanode.stripedread.threads` - Number of concurrent reader threads. Default value is 20 threads.
+  1. `dfs.datanode.stripedread.buffer.size` - Buffer size for reader service. Default value is 256KB.
 
 ### Administrative commands
- ErasureCoding command-line is provided to perform administrative commands related to ErasureCoding. This can be accessed by executing the following command.
+
+  HDFS provides an `erasurecode` subcommand to perform administrative commands related to erasure coding.
 
        hdfs erasurecode [generic options]
          [-setPolicy [-s <policyName>] <path>]
@@ -131,18 +118,18 @@ Deployment
 
 Below are the details about each command.
 
-*  **SetPolicy command**: `[-setPolicy [-s <policyName>] <path>]`
+ *  `[-setPolicy [-s <policyName>] <path>]`
 
-    SetPolicy command is used to set an ErasureCoding policy on a directory at the specified path.
+    Sets an ErasureCoding policy on a directory at the specified path.
 
-      `path`: Refer to a pre-created directory in HDFS. This is a mandatory parameter.
+      `path`: An directory in HDFS. This is a mandatory parameter. Setting a policy only affects newly created files, and does not affect existing files.
 
-      `policyName`: This is an optional parameter, specified using ‘-s’ flag. Refer to the name of ErasureCodingPolicy to be used for encoding files under this directory. If not specified the system default ErasureCodingPolicy will be used.
+      `policyName`: The ErasureCoding policy to be used for files under this directory. This is an optional parameter, specified using ‘-s’ flag. If no policy is specified, the system default ErasureCodingPolicy will be used.
 
-*  **GetPolicy command**: `[-getPolicy <path>]`
+ *  `[-getPolicy <path>]`
 
-     GetPolicy command is used to get details of the ErasureCoding policy of a file or directory at the specified path.
+     Get details of the ErasureCoding policy of a file or directory at the specified path.
 
-*  **ListPolicies command**:  `[-listPolicies]`
+ *  `[-listPolicies]`
 
-     Lists all supported ErasureCoding policies. For setPolicy command, one of these policies' name should be provided.
\ No newline at end of file
+     Lists all supported ErasureCoding policies. These names are suitable for use with the `setPolicy` command.


[20/50] [abbrv] hadoop git commit: HDFS-8833. Erasure coding: store EC schema and cell size in INodeFile and eliminate notion of EC zones.

Posted by wa...@apache.org.
HDFS-8833. Erasure coding: store EC schema and cell size in INodeFile and eliminate notion of EC zones.


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

Branch: refs/heads/trunk
Commit: f62237bc2f02afe11ce185e13aa51a60b5960037
Parents: 60bd765
Author: Zhe Zhang <zh...@apache.org>
Authored: Wed Sep 9 23:07:20 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Wed Sep 9 23:07:55 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  14 +-
 .../hadoop/hdfs/protocol/ErasureCodingZone.java |  54 -----
 .../src/main/proto/ClientNamenodeProtocol.proto |   8 +-
 .../src/main/proto/erasurecoding.proto          |  20 +-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  22 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |  40 ++--
 .../apache/hadoop/hdfs/client/HdfsAdmin.java    |  23 +-
 .../hadoop/hdfs/protocol/ErasureCodingZone.java |  54 -----
 ...tNamenodeProtocolServerSideTranslatorPB.java |  29 ++-
 .../ClientNamenodeProtocolTranslatorPB.java     |  27 ++-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  15 --
 .../server/blockmanagement/BlockManager.java    |  16 +-
 .../hdfs/server/common/HdfsServerConstants.java |   4 +-
 .../namenode/ErasureCodingPolicyManager.java    |   2 +-
 .../namenode/ErasureCodingZoneManager.java      | 163 --------------
 .../server/namenode/FSDirErasureCodingOp.java   | 159 +++++++++-----
 .../hdfs/server/namenode/FSDirRenameOp.java     |   2 -
 .../server/namenode/FSDirStatAndListingOp.java  |  17 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  10 +-
 .../hdfs/server/namenode/FSDirectory.java       |   4 -
 .../hdfs/server/namenode/FSEditLogLoader.java   |  32 ++-
 .../hdfs/server/namenode/FSNamesystem.java      |  39 ++--
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  14 ++
 .../server/namenode/INodeFileAttributes.java    |  10 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  15 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |  10 +-
 .../hdfs/tools/erasurecode/ECCommand.java       |  42 ++--
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   4 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |   2 +-
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |   2 +-
 .../TestDFSStripedOutputStreamWithFailure.java  |   2 +-
 .../hadoop/hdfs/TestErasureCodingPolicies.java  | 211 ++++++++++++++++++
 .../hadoop/hdfs/TestErasureCodingZones.java     | 217 -------------------
 .../hadoop/hdfs/TestFileStatusWithECPolicy.java |  10 +-
 .../hdfs/TestReadStripedFileWithDecoding.java   |   2 +-
 .../TestReadStripedFileWithMissingBlocks.java   |   2 +-
 .../hadoop/hdfs/TestRecoverStripedFile.java     |   2 +-
 .../hdfs/TestSafeModeWithStripedFile.java       |   2 +-
 .../hadoop/hdfs/TestWriteReadStripedFile.java   |   2 +-
 .../hdfs/TestWriteStripedFileWithFailure.java   |   5 +-
 .../hdfs/server/balancer/TestBalancer.java      |   3 +-
 .../TestBlockTokenWithDFSStriped.java           |   2 +-
 .../TestSequentialBlockGroupId.java             |  14 +-
 .../hadoop/hdfs/server/mover/TestMover.java     |   4 +-
 .../TestAddOverReplicatedStripedBlocks.java     |   4 +-
 .../server/namenode/TestAddStripedBlocks.java   |   2 +-
 .../server/namenode/TestFSEditLogLoader.java    |   4 +-
 .../hdfs/server/namenode/TestFSImage.java       |   4 +-
 .../namenode/TestQuotaWithStripedBlocks.java    |   3 +-
 .../server/namenode/TestStripedINodeFile.java   |  20 +-
 ...TestOfflineImageViewerWithStripedBlocks.java |   2 +-
 .../test/resources/testErasureCodingConf.xml    | 150 ++++++-------
 53 files changed, 635 insertions(+), 888 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 7756bb9..a0504a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1485,11 +1485,13 @@ public interface ClientProtocol {
   EventBatchList getEditsFromTxid(long txid) throws IOException;
 
   /**
-   * Create an erasure coding zone with specified policy, if any, otherwise
-   * default
+   * Set an erasure coding policy on a specified path.
+   * @param src The path to set policy on.
+   * @param ecPolicy The erasure coding policy. If null, default policy will
+   *                 be used
    */
   @AtMostOnce
-  void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy)
+  void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
       throws IOException;
 
   /**
@@ -1501,11 +1503,11 @@ public interface ClientProtocol {
   ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException;
 
   /**
-   * Get the information about the EC zone for the path
-   * 
+   * Get the information about the EC policy for the path
+   *
    * @param src path to get the info for
    * @throws IOException
    */
   @Idempotent
-  ErasureCodingZone getErasureCodingZone(String src) throws IOException;
+  ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
deleted file mode 100644
index 533b630..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-/**
- * Information about the EC Zone at the specified path.
- */
-public class ErasureCodingZone {
-
-  private String dir;
-  private ErasureCodingPolicy ecPolicy;
-
-  public ErasureCodingZone(String dir, ErasureCodingPolicy ecPolicy) {
-    this.dir = dir;
-    this.ecPolicy = ecPolicy;
-  }
-
-  /**
-   * Get directory of the EC zone.
-   * 
-   * @return
-   */
-  public String getDir() {
-    return dir;
-  }
-
-  /**
-   * Get the erasure coding policy for the EC Zone
-   * 
-   * @return
-   */
-  public ErasureCodingPolicy getErasureCodingPolicy() {
-    return ecPolicy;
-  }
-
-  @Override
-  public String toString() {
-    return "Dir: " + getDir() + ", Policy: " + ecPolicy;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index fb10e9c..6a140eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -867,14 +867,14 @@ service ClientNamenodeProtocol {
       returns(ListEncryptionZonesResponseProto);
   rpc getEZForPath(GetEZForPathRequestProto)
       returns(GetEZForPathResponseProto);
-  rpc createErasureCodingZone(CreateErasureCodingZoneRequestProto)
-      returns(CreateErasureCodingZoneResponseProto);
+  rpc setErasureCodingPolicy(SetErasureCodingPolicyRequestProto)
+      returns(SetErasureCodingPolicyResponseProto);
   rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto)
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
       returns(GetEditsFromTxidResponseProto);
   rpc getErasureCodingPolicies(GetErasureCodingPoliciesRequestProto)
       returns(GetErasureCodingPoliciesResponseProto);
-  rpc getErasureCodingZone(GetErasureCodingZoneRequestProto)
-      returns(GetErasureCodingZoneResponseProto);
+  rpc getErasureCodingPolicy(GetErasureCodingPolicyRequestProto)
+      returns(GetErasureCodingPolicyResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
index d27f782..fa24aef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
@@ -23,20 +23,12 @@ package hadoop.hdfs;
 
 import "hdfs.proto";
 
-/**
- * ErasureCodingZone
- */
-message ErasureCodingZoneProto {
-  required string dir = 1;
-  required ErasureCodingPolicyProto ecPolicy = 2;
-}
-
-message CreateErasureCodingZoneRequestProto {
+message SetErasureCodingPolicyRequestProto {
   required string src = 1;
   optional ErasureCodingPolicyProto ecPolicy = 2;
 }
 
-message CreateErasureCodingZoneResponseProto {
+message SetErasureCodingPolicyResponseProto {
 }
 
 message GetErasureCodingPoliciesRequestProto { // void request
@@ -46,12 +38,12 @@ message GetErasureCodingPoliciesResponseProto {
   repeated ErasureCodingPolicyProto ecPolicies = 1;
 }
 
-message GetErasureCodingZoneRequestProto {
-  required string src = 1; // path to get the zone info
+message GetErasureCodingPolicyRequestProto {
+  required string src = 1; // path to get the policy info
 }
 
-message GetErasureCodingZoneResponseProto {
-  optional ErasureCodingZoneProto ECZone = 1;
+message GetErasureCodingPolicyResponseProto {
+  optional ErasureCodingPolicyProto ecPolicy = 1;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 9fdf3ed..2f13310 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -412,3 +412,6 @@
 
     HDFS-8978. Erasure coding: fix 2 failed tests of DFSStripedOutputStream.
     (Walter Su via jing9)
+
+    HDFS-8833. Erasure coding: store EC schema and cell size in INodeFile and 
+    eliminate notion of EC zones. (zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 12120f4..4d28dca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -114,7 +114,6 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -2901,12 +2900,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return new EncryptionZoneIterator(namenode, traceSampler);
   }
 
-  public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy)
+
+  public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
       throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("createErasureCodingZone", src);
+    TraceScope scope = getPathTraceScope("setErasureCodingPolicy", src);
     try {
-      namenode.createErasureCodingZone(src, ecPolicy);
+      namenode.setErasureCodingPolicy(src, ecPolicy);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
           SafeModeException.class,
@@ -3223,17 +3223,19 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   /**
-   * Get the erasure coding zone information for the specified path
-   * 
+   * Get the erasure coding policy information for the specified path
+   *
    * @param src path to get the information for
-   * @return Returns the zone information if path is in EC Zone, null otherwise
+   * @return Returns the policy information if file or directory on the path is
+   * erasure coded, null otherwise
    * @throws IOException
    */
-  public ErasureCodingZone getErasureCodingZone(String src) throws IOException {
+
+  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getErasureCodingZone", src);
+    TraceScope scope = getPathTraceScope("getErasureCodingPolicy", src);
     try {
-      return namenode.getErasureCodingZone(src);
+      return namenode.getErasureCodingPolicy(src);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(FileNotFoundException.class,
           AccessControlException.class, UnresolvedPathException.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 1f8ab44..903f763 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -73,7 +73,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -2263,20 +2262,20 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   /**
-   * Create the erasurecoding zone
-   * 
-   * @param path Directory to create the ec zone
-   * @param ecPolicy erasure coding policy for the zone. If not specified default will be used.
+   * Set the source path to the specified erasure coding policy.
+   *
+   * @param path     The directory to set the policy
+   * @param ecPolicy The erasure coding policy. If not specified default will be used.
    * @throws IOException
    */
-  public void createErasureCodingZone(final Path path, final ErasureCodingPolicy ecPolicy)
+  public void setErasureCodingPolicy(final Path path, final ErasureCodingPolicy ecPolicy)
       throws IOException {
     Path absF = fixRelativePart(path);
     new FileSystemLinkResolver<Void>() {
       @Override
       public Void doCall(final Path p) throws IOException,
           UnresolvedLinkException {
-        dfs.createErasureCodingZone(getPathName(p), ecPolicy);
+        dfs.setErasureCodingPolicy(getPathName(p), ecPolicy);
         return null;
       }
 
@@ -2284,42 +2283,43 @@ public class DistributedFileSystem extends FileSystem {
       public Void next(final FileSystem fs, final Path p) throws IOException {
         if (fs instanceof DistributedFileSystem) {
           DistributedFileSystem myDfs = (DistributedFileSystem) fs;
-          myDfs.createErasureCodingZone(p, ecPolicy);
+          myDfs.setErasureCodingPolicy(p, ecPolicy);
           return null;
         }
         throw new UnsupportedOperationException(
-            "Cannot createErasureCodingZone through a symlink to a "
+            "Cannot setErasureCodingPolicy through a symlink to a "
                 + "non-DistributedFileSystem: " + path + " -> " + p);
       }
     }.resolve(this, absF);
   }
 
   /**
-   * Get ErasureCoding zone information for the specified path
-   * 
-   * @param path
-   * @return Returns the zone information if path is in EC zone, null otherwise
+   * Get erasure coding policy information for the specified path
+   *
+   * @param path The path of the file or directory
+   * @return Returns the policy information if file or directory on the path
+   * is erasure coded, null otherwise
    * @throws IOException
    */
-  public ErasureCodingZone getErasureCodingZone(final Path path)
+  public ErasureCodingPolicy getErasureCodingPolicy(final Path path)
       throws IOException {
     Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<ErasureCodingZone>() {
+    return new FileSystemLinkResolver<ErasureCodingPolicy>() {
       @Override
-      public ErasureCodingZone doCall(final Path p) throws IOException,
+      public ErasureCodingPolicy doCall(final Path p) throws IOException,
           UnresolvedLinkException {
-        return dfs.getErasureCodingZone(getPathName(p));
+        return dfs.getErasureCodingPolicy(getPathName(p));
       }
 
       @Override
-      public ErasureCodingZone next(final FileSystem fs, final Path p)
+      public ErasureCodingPolicy next(final FileSystem fs, final Path p)
           throws IOException {
         if (fs instanceof DistributedFileSystem) {
           DistributedFileSystem myDfs = (DistributedFileSystem) fs;
-          return myDfs.getErasureCodingZone(p);
+          return myDfs.getErasureCodingPolicy(p);
         }
         throw new UnsupportedOperationException(
-            "Cannot getErasureCodingZone through a symlink to a "
+            "Cannot getErasureCodingPolicy through a symlink to a "
                 + "non-DistributedFileSystem: " + path + " -> " + p);
       }
     }.resolve(this, absF);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
index e6e67cb..7b055fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
@@ -367,31 +366,33 @@ public class HdfsAdmin {
   }
 
   /**
-   * Create the ErasureCoding zone
+   * Set the source path to the specified erasure coding policy.
    *
-   * @param path Directory to create the ErasureCoding zone
-   * @param ecPolicy erasure coding policy for the zone. If null, the default will be used.
+   * @param path The source path referring to a directory.
+   * @param ecPolicy The erasure coding policy for the directory.
+   *                 If null, the default will be used.
    * @throws IOException
    */
-  public void createErasureCodingZone(final Path path,
+  public void setErasureCodingPolicy(final Path path,
       final ErasureCodingPolicy ecPolicy) throws IOException {
-    dfs.createErasureCodingZone(path, ecPolicy);
+    dfs.setErasureCodingPolicy(path, ecPolicy);
   }
 
   /**
-   * Get the ErasureCoding zone information for the specified path
+   * Get the erasure coding policy information for the specified path
    *
    * @param path
-   * @return Returns the zone information if path is in EC zone, null otherwise
+   * @return Returns the policy information if file or directory on the path is
+   *          erasure coded. Null otherwise.
    * @throws IOException
    */
-  public ErasureCodingZone getErasureCodingZone(final Path path)
+  public ErasureCodingPolicy getErasureCodingPolicy(final Path path)
       throws IOException {
-    return dfs.getErasureCodingZone(path);
+    return dfs.getErasureCodingPolicy(path);
   }
 
   /**
-   * Get the ErasureCoding policies supported.
+   * Get the Erasure coding policies supported.
    *
    * @throws IOException
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
deleted file mode 100644
index 533b630..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-/**
- * Information about the EC Zone at the specified path.
- */
-public class ErasureCodingZone {
-
-  private String dir;
-  private ErasureCodingPolicy ecPolicy;
-
-  public ErasureCodingZone(String dir, ErasureCodingPolicy ecPolicy) {
-    this.dir = dir;
-    this.ecPolicy = ecPolicy;
-  }
-
-  /**
-   * Get directory of the EC zone.
-   * 
-   * @return
-   */
-  public String getDir() {
-    return dir;
-  }
-
-  /**
-   * Get the erasure coding policy for the EC Zone
-   * 
-   * @return
-   */
-  public ErasureCodingPolicy getErasureCodingPolicy() {
-    return ecPolicy;
-  }
-
-  @Override
-  public String toString() {
-    return "Dir: " + getDir() + ", Policy: " + ecPolicy;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 97445a6..3217484 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -203,10 +202,10 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptio
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
@@ -1402,14 +1401,14 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public CreateErasureCodingZoneResponseProto createErasureCodingZone(
-      RpcController controller, CreateErasureCodingZoneRequestProto req)
+  public SetErasureCodingPolicyResponseProto setErasureCodingPolicy(
+      RpcController controller, SetErasureCodingPolicyRequestProto req)
       throws ServiceException {
     try {
       ErasureCodingPolicy ecPolicy = req.hasEcPolicy() ? PBHelper.convertErasureCodingPolicy(req
           .getEcPolicy()) : null;
-      server.createErasureCodingZone(req.getSrc(), ecPolicy);
-      return CreateErasureCodingZoneResponseProto.newBuilder().build();
+      server.setErasureCodingPolicy(req.getSrc(), ecPolicy);
+      return SetErasureCodingPolicyResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1554,13 +1553,13 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public GetErasureCodingZoneResponseProto getErasureCodingZone(RpcController controller,
-      GetErasureCodingZoneRequestProto request) throws ServiceException {
+  public GetErasureCodingPolicyResponseProto getErasureCodingPolicy(RpcController controller,
+      GetErasureCodingPolicyRequestProto request) throws ServiceException {
     try {
-      ErasureCodingZone ecZone = server.getErasureCodingZone(request.getSrc());
-      GetErasureCodingZoneResponseProto.Builder builder = GetErasureCodingZoneResponseProto.newBuilder();
-      if (ecZone != null) {
-        builder.setECZone(PBHelper.convertErasureCodingZone(ecZone));
+      ErasureCodingPolicy ecPolicy = server.getErasureCodingPolicy(request.getSrc());
+      GetErasureCodingPolicyResponseProto.Builder builder = GetErasureCodingPolicyResponseProto.newBuilder();
+      if (ecPolicy != null) {
+        builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(ecPolicy));
       }
       return builder.build();
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index c8c468e..8419244 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -58,7 +58,6 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -168,9 +167,9 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathR
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
@@ -1420,17 +1419,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy)
+  public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
       throws IOException {
-    final CreateErasureCodingZoneRequestProto.Builder builder =
-        CreateErasureCodingZoneRequestProto.newBuilder();
+    final SetErasureCodingPolicyRequestProto.Builder builder =
+        SetErasureCodingPolicyRequestProto.newBuilder();
     builder.setSrc(src);
     if (ecPolicy != null) {
       builder.setEcPolicy(PBHelper.convertErasureCodingPolicy(ecPolicy));
     }
-    CreateErasureCodingZoneRequestProto req = builder.build();
+    SetErasureCodingPolicyRequestProto req = builder.build();
     try {
-      rpcProxy.createErasureCodingZone(null, req);
+      rpcProxy.setErasureCodingPolicy(null, req);
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1577,14 +1576,14 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ErasureCodingZone getErasureCodingZone(String src) throws IOException {
-    GetErasureCodingZoneRequestProto req = GetErasureCodingZoneRequestProto.newBuilder()
+  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
+    GetErasureCodingPolicyRequestProto req = GetErasureCodingPolicyRequestProto.newBuilder()
         .setSrc(src).build();
     try {
-      GetErasureCodingZoneResponseProto response = rpcProxy.getErasureCodingZone(
+      GetErasureCodingPolicyResponseProto response = rpcProxy.getErasureCodingPolicy(
           null, req);
-      if (response.hasECZone()) {
-        return PBHelper.convertErasureCodingZone(response.getECZone());
+      if (response.hasEcPolicy()) {
+        return PBHelper.convertErasureCodingPolicy(response.getEcPolicy());
       }
       return null;
     } catch (ServiceException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index f419c46..ce39e15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -75,7 +75,6 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -132,7 +131,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterComm
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
@@ -2975,19 +2973,6 @@ public class PBHelper {
         .setCellSize(policy.getCellSize());
     return builder.build();
   }
-
-  public static ErasureCodingZoneProto convertErasureCodingZone(
-      ErasureCodingZone ecZone) {
-    return ErasureCodingZoneProto.newBuilder().setDir(ecZone.getDir())
-        .setEcPolicy(convertErasureCodingPolicy(ecZone.getErasureCodingPolicy()))
-        .build();
-  }
-
-  public static ErasureCodingZone convertErasureCodingZone(
-      ErasureCodingZoneProto ecZoneProto) {
-    return new ErasureCodingZone(ecZoneProto.getDir(),
-        convertErasureCodingPolicy(ecZoneProto.getEcPolicy()));
-  }
   
   public static BlockECRecoveryInfo convertBlockECRecoveryInfo(
       BlockECRecoveryInfoProto blockEcRecoveryInfoProto) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 8232ab9..3c1c461 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -924,11 +923,9 @@ public class BlockManager implements BlockStatsMXBean {
       final boolean isFileUnderConstruction, final long offset,
       final long length, final boolean needBlockToken,
       final boolean inSnapshot, FileEncryptionInfo feInfo,
-      ErasureCodingZone ecZone)
+      ErasureCodingPolicy ecPolicy)
       throws IOException {
     assert namesystem.hasReadLock();
-    final ErasureCodingPolicy ecPolicy = ecZone != null ? ecZone
-        .getErasureCodingPolicy() : null;
     if (blocks == null) {
       return null;
     } else if (blocks.length == 0) {
@@ -1607,14 +1604,14 @@ public class BlockManager implements BlockStatsMXBean {
       assert rw instanceof ErasureCodingWork;
       assert rw.getTargets().length > 0;
       String src = getBlockCollection(block).getName();
-      ErasureCodingZone ecZone = null;
+      ErasureCodingPolicy ecPolicy = null;
       try {
-        ecZone = namesystem.getErasureCodingZoneForPath(src);
+        ecPolicy = namesystem.getErasureCodingPolicyForPath(src);
       } catch (IOException e) {
         blockLog
-            .warn("Failed to get the EC zone for the file {} ", src);
+            .warn("Failed to get EC policy for the file {} ", src);
       }
-      if (ecZone == null) {
+      if (ecPolicy == null) {
         blockLog.warn("No erasure coding policy found for the file {}. "
             + "So cannot proceed for recovery", src);
         // TODO: we may have to revisit later for what we can do better to
@@ -1624,8 +1621,7 @@ public class BlockManager implements BlockStatsMXBean {
       rw.getTargets()[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
           new ExtendedBlock(namesystem.getBlockPoolId(), block),
           rw.getSrcNodes(), rw.getTargets(),
-          ((ErasureCodingWork) rw).getLiveBlockIndicies(),
-          ecZone.getErasureCodingPolicy());
+          ((ErasureCodingWork) rw).getLiveBlockIndicies(), ecPolicy);
     } else {
       rw.getSrcNodes()[0].addBlockToBeReplicated(block, targets);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index da52717..eb8ea8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -387,8 +387,8 @@ public interface HdfsServerConstants {
       "raw.hdfs.crypto.file.encryption.info";
   String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER =
       "security.hdfs.unreadable.by.superuser";
-  String XATTR_ERASURECODING_ZONE =
-      "raw.hdfs.erasurecoding.zone";
+  String XATTR_ERASURECODING_POLICY =
+      "raw.hdfs.erasurecoding.policy";
 
   long BLOCK_GROUP_INDEX_MASK = 15;
   byte MAX_BLOCKS_IN_GROUP = 16;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 71ac36a..b77279b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -83,7 +83,7 @@ public final class ErasureCodingPolicyManager {
 
   /**
    * Get system-wide default policy, which can be used by default
-   * when no policy is specified for an EC zone.
+   * when no policy is specified for a path.
    * @return ecPolicy
    */
   public static ErasureCodingPolicy getSystemDefaultPolicy() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
deleted file mode 100644
index a0ac033..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-import org.apache.hadoop.fs.XAttr;
-import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.hdfs.XAttrHelper;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.WritableUtils;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.List;
-
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_ZONE;
-
-/**
- * Manages the list of erasure coding zones in the filesystem.
- * <p/>
- * The ErasureCodingZoneManager has its own lock, but relies on the FSDirectory
- * lock being held for many operations. The FSDirectory lock should not be
- * taken if the manager lock is already held.
- * TODO: consolidate zone logic w/ encrypt. zones {@link EncryptionZoneManager}
- */
-public class ErasureCodingZoneManager {
-  private final FSDirectory dir;
-
-  /**
-   * Construct a new ErasureCodingZoneManager.
-   *
-   * @param dir Enclosing FSDirectory
-   */
-  public ErasureCodingZoneManager(FSDirectory dir) {
-    this.dir = dir;
-  }
-
-  ErasureCodingPolicy getErasureCodingPolicy(INodesInPath iip) throws IOException {
-    ErasureCodingZone ecZone = getErasureCodingZone(iip);
-    return ecZone == null ? null : ecZone.getErasureCodingPolicy();
-  }
-
-  ErasureCodingZone getErasureCodingZone(INodesInPath iip) throws IOException {
-    assert dir.hasReadLock();
-    Preconditions.checkNotNull(iip, "INodes cannot be null");
-    List<INode> inodes = iip.getReadOnlyINodes();
-    for (int i = inodes.size() - 1; i >= 0; i--) {
-      final INode inode = inodes.get(i);
-      if (inode == null) {
-        continue;
-      }
-      // We don't allow symlinks in an EC zone, or pointing to a file/dir in
-      // an EC. Therefore if a symlink is encountered, the dir shouldn't have
-      // EC
-      // TODO: properly support symlinks in EC zones
-      if (inode.isSymlink()) {
-        return null;
-      }
-      final List<XAttr> xAttrs = inode.getXAttrFeature() == null ?
-          new ArrayList<XAttr>(0)
-          : inode.getXAttrFeature().getXAttrs();
-      for (XAttr xAttr : xAttrs) {
-        if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixedName(xAttr))) {
-          ByteArrayInputStream bIn=new ByteArrayInputStream(xAttr.getValue());
-          DataInputStream dIn=new DataInputStream(bIn);
-          String ecPolicyName = WritableUtils.readString(dIn);
-          ErasureCodingPolicy ecPolicy = dir.getFSNamesystem()
-              .getErasureCodingPolicyManager().getPolicy(ecPolicyName);
-          return new ErasureCodingZone(dir.getInode(inode.getId())
-              .getFullPathName(), ecPolicy);
-        }
-      }
-    }
-    return null;
-  }
-
-  List<XAttr> createErasureCodingZone(final INodesInPath srcIIP,
-      ErasureCodingPolicy ecPolicy) throws IOException {
-    assert dir.hasWriteLock();
-    Preconditions.checkNotNull(srcIIP, "INodes cannot be null");
-    String src = srcIIP.getPath();
-    if (dir.isNonEmptyDirectory(srcIIP)) {
-      throw new IOException(
-          "Attempt to create an erasure coding zone for a " +
-              "non-empty directory " + src);
-    }
-    if (srcIIP.getLastINode() != null &&
-        !srcIIP.getLastINode().isDirectory()) {
-      throw new IOException("Attempt to create an erasure coding zone " +
-          "for a file " + src);
-    }
-    if (getErasureCodingPolicy(srcIIP) != null) {
-      throw new IOException("Directory " + src + " is already in an " +
-          "erasure coding zone.");
-    }
-
-    // System default erasure coding policy will be used since no specified.
-    if (ecPolicy == null) {
-      ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
-    }
-
-    final XAttr ecXAttr;
-    DataOutputStream dOut = null;
-    try {
-      ByteArrayOutputStream bOut = new ByteArrayOutputStream();
-      dOut = new DataOutputStream(bOut);
-      WritableUtils.writeString(dOut, ecPolicy.getName());
-      ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE,
-          bOut.toByteArray());
-    } finally {
-      IOUtils.closeStream(dOut);
-    }
-    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
-    xattrs.add(ecXAttr);
-    FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs,
-        EnumSet.of(XAttrSetFlag.CREATE));
-    return xattrs;
-  }
-
-  void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src)
-      throws IOException {
-    assert dir.hasReadLock();
-    final ErasureCodingZone srcZone = getErasureCodingZone(srcIIP);
-    final ErasureCodingZone dstZone = getErasureCodingZone(dstIIP);
-    if (srcZone != null && srcZone.getDir().equals(src) && dstZone == null) {
-      return;
-    }
-    final ErasureCodingPolicy srcECPolicy =
-        srcZone != null ? srcZone.getErasureCodingPolicy() : null;
-    final ErasureCodingPolicy dstECPolicy =
-        dstZone != null ? dstZone.getErasureCodingPolicy() : null;
-    if (srcECPolicy != null && !srcECPolicy.equals(dstECPolicy) ||
-        dstECPolicy != null && !dstECPolicy.equals(srcECPolicy)) {
-      throw new IOException(
-          src + " can't be moved because the source and destination have " +
-              "different erasure coding policies.");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index 8c515d1..4162760 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -17,14 +17,27 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.WritableUtils;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_POLICY;
 
 /**
  * Helper class to perform erasure coding related operations.
@@ -38,18 +51,17 @@ final class FSDirErasureCodingOp {
   private FSDirErasureCodingOp() {}
 
   /**
-   * Create an erasure coding zone on directory src.
+   * Set an erasure coding policy on the given path.
    *
-   * @param fsn namespace
-   * @param srcArg the path of a directory which will be the root of the
-   *          erasure coding zone. The directory must be empty.
-   * @param ecPolicy erasure coding policy for the erasure coding zone
+   * @param fsn The namespace
+   * @param srcArg The path of the target directory.
+   * @param ecPolicy The erasure coding policy to set on the target directory.
    * @param logRetryCache whether to record RPC ids in editlog for retry
    *          cache rebuilding
    * @return {@link HdfsFileStatus}
    * @throws IOException
    */
-  static HdfsFileStatus createErasureCodingZone(final FSNamesystem fsn,
+  static HdfsFileStatus setErasureCodingPolicy(final FSNamesystem fsn,
       final String srcArg, final ErasureCodingPolicy ecPolicy,
       final boolean logRetryCache) throws IOException {
     assert fsn.hasWriteLock();
@@ -66,8 +78,7 @@ final class FSDirErasureCodingOp {
     fsd.writeLock();
     try {
       iip = fsd.getINodesInPath4Write(src, false);
-      xAttrs = fsn.getErasureCodingZoneManager().createErasureCodingZone(
-          iip, ecPolicy);
+      xAttrs = createErasureCodingPolicyXAttr(fsn, iip, ecPolicy);
     } finally {
       fsd.writeUnlock();
     }
@@ -75,62 +86,83 @@ final class FSDirErasureCodingOp {
     return fsd.getAuditFileInfo(iip);
   }
 
-  /**
-   * Get the erasure coding zone information for specified path.
-   *
-   * @param fsn namespace
-   * @param src path
-   * @return {@link ErasureCodingZone}
-   * @throws IOException
-   */
-  static ErasureCodingZone getErasureCodingZone(final FSNamesystem fsn,
-      final String src) throws IOException {
-    assert fsn.hasReadLock();
+  static List<XAttr> createErasureCodingPolicyXAttr(final FSNamesystem fsn,
+      final INodesInPath srcIIP, ErasureCodingPolicy ecPolicy) throws IOException {
+    FSDirectory fsd = fsn.getFSDirectory();
+    assert fsd.hasWriteLock();
+    Preconditions.checkNotNull(srcIIP, "INodes cannot be null");
+    String src = srcIIP.getPath();
+    if (srcIIP.getLastINode() != null &&
+        !srcIIP.getLastINode().isDirectory()) {
+      throw new IOException("Attempt to set an erasure coding policy " +
+          "for a file " + src);
+    }
+    if (getErasureCodingPolicyForPath(fsn, srcIIP) != null) {
+      throw new IOException("Directory " + src + " already has an " +
+          "erasure coding policy.");
+    }
 
-    final INodesInPath iip = getINodesInPath(fsn, src);
-    return getErasureCodingZoneForPath(fsn, iip);
+    // System default erasure coding policy will be used since no specified.
+    if (ecPolicy == null) {
+      ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    }
+
+    final XAttr ecXAttr;
+    DataOutputStream dOut = null;
+    try {
+      ByteArrayOutputStream bOut = new ByteArrayOutputStream();
+      dOut = new DataOutputStream(bOut);
+      WritableUtils.writeString(dOut, ecPolicy.getName());
+      ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_POLICY,
+          bOut.toByteArray());
+    } finally {
+      IOUtils.closeStream(dOut);
+    }
+    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
+    xattrs.add(ecXAttr);
+    FSDirXAttrOp.unprotectedSetXAttrs(fsd, src, xattrs,
+        EnumSet.of(XAttrSetFlag.CREATE));
+    return xattrs;
   }
 
   /**
-   * Get erasure coding zone information for specified path.
+   * Get the erasure coding policy information for specified path.
    *
    * @param fsn namespace
-   * @param iip inodes in the path containing the file
-   * @return {@link ErasureCodingZone}
+   * @param src path
+   * @return {@link ErasureCodingPolicy}
    * @throws IOException
    */
-  static ErasureCodingZone getErasureCodingZone(final FSNamesystem fsn,
-      final INodesInPath iip) throws IOException {
+  static ErasureCodingPolicy getErasureCodingPolicy(final FSNamesystem fsn,
+      final String src) throws IOException {
     assert fsn.hasReadLock();
 
-    return getErasureCodingZoneForPath(fsn, iip);
+    final INodesInPath iip = getINodesInPath(fsn, src);
+    return getErasureCodingPolicyForPath(fsn, iip);
   }
 
   /**
-   * Check if the file is in erasure coding zone.
+   * Check if the file or directory has an erasure coding policy.
    *
    * @param fsn namespace
    * @param srcArg path
-   * @return true represents the file is in erasure coding zone, false otw
+   * @return Whether the file or directory has an erasure coding policy.
    * @throws IOException
    */
-  static boolean isInErasureCodingZone(final FSNamesystem fsn,
+  static boolean hasErasureCodingPolicy(final FSNamesystem fsn,
       final String srcArg) throws IOException {
-    assert fsn.hasReadLock();
-
-    final INodesInPath iip = getINodesInPath(fsn, srcArg);
-    return getErasureCodingPolicyForPath(fsn, iip) != null;
+    return hasErasureCodingPolicy(fsn, getINodesInPath(fsn, srcArg));
   }
 
   /**
-   * Check if the file is in erasure coding zone.
+   * Check if the file or directory has an erasure coding policy.
    *
    * @param fsn namespace
    * @param iip inodes in the path containing the file
-   * @return true represents the file is in erasure coding zone, false otw
+   * @return Whether the file or directory has an erasure coding policy.
    * @throws IOException
    */
-  static boolean isInErasureCodingZone(final FSNamesystem fsn,
+  static boolean hasErasureCodingPolicy(final FSNamesystem fsn,
       final INodesInPath iip) throws IOException {
     return getErasureCodingPolicy(fsn, iip) != null;
   }
@@ -178,25 +210,46 @@ final class FSDirErasureCodingOp {
     return iip;
   }
 
-  private static ErasureCodingZone getErasureCodingZoneForPath(
-      final FSNamesystem fsn, final INodesInPath iip) throws IOException {
-    final FSDirectory fsd = fsn.getFSDirectory();
-    fsd.readLock();
-    try {
-      return fsn.getErasureCodingZoneManager().getErasureCodingZone(iip);
-    } finally {
-      fsd.readUnlock();
-    }
-  }
-
-  private static ErasureCodingPolicy getErasureCodingPolicyForPath(final FSNamesystem fsn,
-      final INodesInPath iip) throws IOException {
-    final FSDirectory fsd = fsn.getFSDirectory();
+  private static ErasureCodingPolicy getErasureCodingPolicyForPath(FSNamesystem fsn,
+      INodesInPath iip) throws IOException {
+    Preconditions.checkNotNull(iip, "INodes cannot be null");
+    FSDirectory fsd = fsn.getFSDirectory();
     fsd.readLock();
     try {
-      return fsn.getErasureCodingZoneManager().getErasureCodingPolicy(iip);
+      List<INode> inodes = iip.getReadOnlyINodes();
+      for (int i = inodes.size() - 1; i >= 0; i--) {
+        final INode inode = inodes.get(i);
+        if (inode == null) {
+          continue;
+        }
+        /**
+         * TODO: lookup {@link ErasureCodingPolicyManager#getSystemPolices()}
+         */
+        if (inode.isFile()) {
+          return inode.asFile().getErasureCodingPolicyID() == 0 ?
+              null : ErasureCodingPolicyManager.getSystemDefaultPolicy();
+        }
+        // We don't allow setting EC policies on paths with a symlink. Thus
+        // if a symlink is encountered, the dir shouldn't have EC policy.
+        // TODO: properly support symlinks
+        if (inode.isSymlink()) {
+          return null;
+        }
+        final XAttrFeature xaf = inode.getXAttrFeature();
+        if (xaf != null) {
+          XAttr xattr = xaf.getXAttr(XATTR_ERASURECODING_POLICY);
+          if (xattr != null) {
+            ByteArrayInputStream bIn = new ByteArrayInputStream(xattr.getValue());
+            DataInputStream dIn = new DataInputStream(bIn);
+            String ecPolicyName = WritableUtils.readString(dIn);
+            return fsd.getFSNamesystem().getErasureCodingPolicyManager().
+                getPolicy(ecPolicyName);
+          }
+        }
+      }
     } finally {
       fsd.readUnlock();
     }
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
index ccdef1f..b015708 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
@@ -185,7 +185,6 @@ class FSDirRenameOp {
     }
 
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
-    fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src);
     // Ensure dst has quota to accommodate rename
     verifyFsLimitsForRename(fsd, srcIIP, dstIIP);
     verifyQuotaForRename(fsd, srcIIP, dstIIP);
@@ -358,7 +357,6 @@ class FSDirRenameOp {
 
     BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite();
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
-    fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src);
     final INode dstInode = dstIIP.getLastINode();
     List<INodeDirectory> snapshottableDirs = new ArrayList<>();
     if (dstInode != null) { // Destination exists

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 6ec97c9..e178c68 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -182,12 +181,12 @@ class FSDirStatAndListingOp {
 
       final FileEncryptionInfo feInfo = isReservedName ? null
           : fsd.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
-      final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
-          fsd.getFSNamesystem(), iip);
+      final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.
+          getErasureCodingPolicy(fsd.getFSNamesystem(), iip);
 
       final LocatedBlocks blocks = bm.createLocatedBlocks(
           inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
-          length, needBlockToken, iip.isSnapshot(), feInfo, ecZone);
+          length, needBlockToken, iip.isSnapshot(), feInfo, ecPolicy);
 
       // Set caching information for the located blocks.
       for (LocatedBlock lb : blocks.getLocatedBlocks()) {
@@ -447,10 +446,8 @@ class FSDirStatAndListingOp {
     final FileEncryptionInfo feInfo = isRawPath ? null :
         fsd.getFileEncryptionInfo(node, snapshot, iip);
 
-    final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
+    final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
         fsd.getFSNamesystem(), iip);
-    final ErasureCodingPolicy ecPolicy =
-        ecZone != null ? ecZone.getErasureCodingPolicy() : null;
 
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
@@ -505,7 +502,7 @@ class FSDirStatAndListingOp {
     final boolean isEncrypted;
     final FileEncryptionInfo feInfo = isRawPath ? null :
         fsd.getFileEncryptionInfo(node, snapshot, iip);
-    final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
+    final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
         fsd.getFSNamesystem(), iip);
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
@@ -520,7 +517,7 @@ class FSDirStatAndListingOp {
 
       loc = fsd.getBlockManager().createLocatedBlocks(
           fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
-          inSnapshot, feInfo, ecZone);
+          inSnapshot, feInfo, ecPolicy);
       if (loc == null) {
         loc = new LocatedBlocks();
       }
@@ -531,8 +528,6 @@ class FSDirStatAndListingOp {
     }
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
-    final ErasureCodingPolicy ecPolicy =
-        ecZone != null ? ecZone.getErasureCodingPolicy() : null;
 
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index e480959..867b451 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -486,8 +485,8 @@ class FSDirWriteFileOp {
     Preconditions.checkNotNull(existing);
     assert fsd.hasWriteLock();
     try {
-      // check if the file is in an EC zone
-      final boolean isStriped = FSDirErasureCodingOp.isInErasureCodingZone(
+      // check if the file has an EC policy
+      final boolean isStriped = FSDirErasureCodingOp.hasErasureCodingPolicy(
           fsd.getFSNamesystem(), existing);
       if (underConstruction) {
         newNode = newINodeFile(id, permissions, modificationTime,
@@ -533,9 +532,8 @@ class FSDirWriteFileOp {
       // associate new last block for the file
       final BlockInfo blockInfo;
       if (isStriped) {
-        ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
+        ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
             fsd.getFSNamesystem(), inodesInPath);
-        ErasureCodingPolicy ecPolicy = ecZone.getErasureCodingPolicy();
         short numDataUnits = (short) ecPolicy.getNumDataUnits();
         short numParityUnits = (short) ecPolicy.getNumParityUnits();
         short numLocations = (short) (numDataUnits + numParityUnits);
@@ -586,7 +584,7 @@ class FSDirWriteFileOp {
     INodesInPath newiip;
     fsd.writeLock();
     try {
-      final boolean isStriped = FSDirErasureCodingOp.isInErasureCodingZone(
+      final boolean isStriped = FSDirErasureCodingOp.hasErasureCodingPolicy(
           fsd.getFSNamesystem(), existing);
       INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
           modTime, modTime, replication, preferredBlockSize, isStriped);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index c9fb6cd..6538aba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -215,9 +215,6 @@ public class FSDirectory implements Closeable {
   @VisibleForTesting
   public final EncryptionZoneManager ezManager;
 
-  @VisibleForTesting
-  public final ErasureCodingZoneManager ecZoneManager;
-
   /**
    * Caches frequently used file names used in {@link INode} to reuse 
    * byte[] objects and reduce heap usage.
@@ -314,7 +311,6 @@ public class FSDirectory implements Closeable {
     namesystem = ns;
     this.editLog = ns.getEditLog();
     ezManager = new EncryptionZoneManager(this, conf);
-    ecZoneManager = new ErasureCodingZoneManager(this);
   }
     
   FSNamesystem getFSNamesystem() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 68324f3..1691548 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
@@ -416,9 +416,9 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
+      ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
           fsDir.getFSNamesystem(), iip);
-      updateBlocks(fsDir, addCloseOp, iip, newFile, ecZone);
+      updateBlocks(fsDir, addCloseOp, iip, newFile, ecPolicy);
       break;
     }
     case OP_CLOSE: {
@@ -438,9 +438,9 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
+      ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
           fsDir.getFSNamesystem(), iip);
-      updateBlocks(fsDir, addCloseOp, iip, file, ecZone);
+      updateBlocks(fsDir, addCloseOp, iip, file, ecPolicy);
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -498,9 +498,9 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
-      ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
+      ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
           fsDir.getFSNamesystem(), iip);
-      updateBlocks(fsDir, updateOp, iip, oldFile, ecZone);
+      updateBlocks(fsDir, updateOp, iip, oldFile, ecPolicy);
 
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
@@ -517,9 +517,9 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // add the new block to the INodeFile
-      ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
+      ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
           fsDir.getFSNamesystem(), iip);
-      addNewBlock(addBlockOp, oldFile, ecZone);
+      addNewBlock(addBlockOp, oldFile, ecPolicy);
       break;
     }
     case OP_SET_REPLICATION: {
@@ -961,7 +961,7 @@ public class FSEditLogLoader {
    * Add a new block into the given INodeFile
    */
   private void addNewBlock(AddBlockOp op, INodeFile file,
-      ErasureCodingZone ecZone) throws IOException {
+      ErasureCodingPolicy ecPolicy) throws IOException {
     BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
     Block newBlock= op.getLastBlock();
@@ -988,10 +988,9 @@ public class FSEditLogLoader {
     }
     // add the new block
     final BlockInfo newBlockInfo;
-    boolean isStriped = ecZone != null;
+    boolean isStriped = ecPolicy != null;
     if (isStriped) {
-      newBlockInfo = new BlockInfoStriped(newBlock,
-          ecZone.getErasureCodingPolicy());
+      newBlockInfo = new BlockInfoStriped(newBlock, ecPolicy);
     } else {
       newBlockInfo = new BlockInfoContiguous(newBlock,
           file.getPreferredBlockReplication());
@@ -1008,7 +1007,7 @@ public class FSEditLogLoader {
    * @throws IOException
    */
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
-      INodesInPath iip, INodeFile file, ErasureCodingZone ecZone)
+      INodesInPath iip, INodeFile file, ErasureCodingPolicy ecPolicy)
       throws IOException {
     // Update its block list
     BlockInfo[] oldBlocks = file.getBlocks();
@@ -1068,7 +1067,7 @@ public class FSEditLogLoader {
         throw new IOException("Trying to delete non-existant block " + oldBlock);
       }
     } else if (newBlocks.length > oldBlocks.length) {
-      final boolean isStriped = ecZone != null;
+      final boolean isStriped = ecPolicy != null;
       // We're adding blocks
       for (int i = oldBlocks.length; i < newBlocks.length; i++) {
         Block newBlock = newBlocks[i];
@@ -1078,8 +1077,7 @@ public class FSEditLogLoader {
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
           if (isStriped) {
-            newBI = new BlockInfoStriped(newBlock,
-                ecZone.getErasureCodingPolicy());
+            newBI = new BlockInfoStriped(newBlock, ecPolicy);
           } else {
             newBI = new BlockInfoContiguous(newBlock,
                 file.getPreferredBlockReplication());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 5f39446..ed52ca4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -177,7 +177,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -187,7 +186,6 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -2133,7 +2131,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      if (!FSDirErasureCodingOp.isInErasureCodingZone(this, src)) {
+      if (!FSDirErasureCodingOp.hasErasureCodingPolicy(this, src)) {
         blockManager.verifyReplication(src, replication, clientMachine);
       }
     } finally {
@@ -3206,9 +3204,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final long diff;
     final short replicationFactor;
     if (fileINode.isStriped()) {
-      final ErasureCodingZone ecZone = FSDirErasureCodingOp
-          .getErasureCodingZone(this, iip);
-      final ErasureCodingPolicy ecPolicy = ecZone.getErasureCodingPolicy();
+      final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp
+          .getErasureCodingPolicy(this, iip);
       final short numDataUnits = (short) ecPolicy.getNumDataUnits();
       final short numParityUnits = (short) ecPolicy.getNumParityUnits();
 
@@ -6241,11 +6238,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return ecPolicyManager;
   }
 
-  /** @return the ErasureCodingZoneManager. */
-  public ErasureCodingZoneManager getErasureCodingZoneManager() {
-    return dir.ecZoneManager;
-  }
-
   @Override  // NameNodeMXBean
   public String getCorruptFiles() {
     List<String> list = new ArrayList<String>();
@@ -7192,15 +7184,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
-   * Create an erasure coding zone on directory src.
-   * @param srcArg  the path of a directory which will be the root of the
-   *                erasure coding zone. The directory must be empty.
-   * @param ecPolicy  erasure coding policy for the erasure coding zone
+   * Set an erasure coding policy on the given path.
+   * @param srcArg  The path of the target directory.
+   * @param ecPolicy The erasure coding policy to set on the target directory.
    * @throws AccessControlException  if the caller is not the superuser.
    * @throws UnresolvedLinkException if the path can't be resolved.
    * @throws SafeModeException       if the Namenode is in safe mode.
    */
-  void createErasureCodingZone(final String srcArg, final ErasureCodingPolicy
+  void setErasureCodingPolicy(final String srcArg, final ErasureCodingPolicy
       ecPolicy, final boolean logRetryCache) throws IOException,
       UnresolvedLinkException, SafeModeException, AccessControlException {
     checkSuperuserPrivilege();
@@ -7210,8 +7201,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot create erasure coding zone on " + srcArg);
-      resultingStat = FSDirErasureCodingOp.createErasureCodingZone(this,
+      checkNameNodeSafeMode("Cannot set erasure coding policy on " + srcArg);
+      resultingStat = FSDirErasureCodingOp.setErasureCodingPolicy(this,
           srcArg, ecPolicy, logRetryCache);
       success = true;
     } finally {
@@ -7219,21 +7210,21 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (success) {
         getEditLog().logSync();
       }
-      logAuditEvent(success, "createErasureCodingZone", srcArg, null,
+      logAuditEvent(success, "setErasureCodingPolicy", srcArg, null,
           resultingStat);
     }
   }
 
   /**
-   * Get the erasure coding zone information for specified path
+   * Get the erasure coding policy information for specified path
    */
-  ErasureCodingZone getErasureCodingZone(String src)
+  ErasureCodingPolicy getErasureCodingPolicy(String src)
       throws AccessControlException, UnresolvedLinkException, IOException {
     checkOperation(OperationCategory.READ);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return getErasureCodingZoneForPath(src);
+      return getErasureCodingPolicyForPath(src);
     } finally {
       readUnlock();
     }
@@ -7461,9 +7452,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override
-  public ErasureCodingZone getErasureCodingZoneForPath(String src)
+  public ErasureCodingPolicy getErasureCodingPolicyForPath(String src)
       throws IOException {
-    return FSDirErasureCodingOp.getErasureCodingZone(this, src);
+    return FSDirErasureCodingOp.getErasureCodingPolicy(this, src);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index ae9b0d2..6f7b702 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -437,6 +437,20 @@ public class INodeFile extends INodeWithAdditionalFields
     setStoragePolicyID(storagePolicyId);
   }
 
+
+  /**
+   * @return The ID of the erasure coding policy on the file. 0 represents no
+   *          EC policy (file is in contiguous format). 1 represents the system
+   *          default EC policy:
+   *          {@link ErasureCodingPolicyManager#SYS_DEFAULT_POLICY}.
+   * TODO: support more policies by reusing {@link HeaderFormat#REPLICATION}.
+   */
+  @VisibleForTesting
+  @Override
+  public byte getErasureCodingPolicyID() {
+    return isStriped() ? (byte)1 : (byte)0;
+  }
+
   /**
    * @return true if the file is in the striping layout.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
index 13bd9e9..8cd481a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile.HeaderFormat;
-import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
+
 /**
  * The attributes of a file.
  */
@@ -32,6 +32,9 @@ public interface INodeFileAttributes extends INodeAttributes {
   /** @return whether the file is striped (instead of contiguous) */
   public boolean isStriped();
 
+  /** @return whether the file is striped (instead of contiguous) */
+  public byte getErasureCodingPolicyID();
+
   /** @return preferred block size in bytes */
   public long getPreferredBlockSize();
 
@@ -78,6 +81,11 @@ public interface INodeFileAttributes extends INodeAttributes {
     }
 
     @Override
+    public byte getErasureCodingPolicyID() {
+      return isStriped() ? (byte)1 : (byte)0;
+    }
+
+    @Override
     public long getPreferredBlockSize() {
       return HeaderFormat.getPreferredBlockSize(header);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index ab29e4d..4143964 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -85,7 +85,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
@@ -1423,8 +1422,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // RefreshAuthorizationPolicyProtocol
   public void refreshUserToGroupsMappings() throws IOException {
-    LOG.info("Refreshing all user-to-groups mappings. Requested by user: " + 
-             getRemoteUser().getShortUserName());
+    LOG.info("Refreshing all user-to-groups mappings. Requested by user: " +
+        getRemoteUser().getShortUserName());
     Groups.getUserToGroupsMappingService().refresh();
   }
 
@@ -1557,7 +1556,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
     namesystem.checkOperation(OperationCategory.WRITE);
     CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache,
-      null);
+        null);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return (String) cacheEntry.getPayload();
     }
@@ -1849,7 +1848,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public void createErasureCodingZone(String src, ErasureCodingPolicy ecPolicy)
+  public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
       throws IOException {
     checkNNStartup();
     final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
@@ -1858,7 +1857,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
     boolean success = false;
     try {
-      namesystem.createErasureCodingZone(src, ecPolicy, cacheEntry != null);
+      namesystem.setErasureCodingPolicy(src, ecPolicy, cacheEntry != null);
       success = true;
     } finally {
       RetryCache.setState(cacheEntry, success);
@@ -2068,8 +2067,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public ErasureCodingZone getErasureCodingZone(String src) throws IOException {
+  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
     checkNNStartup();
-    return namesystem.getErasureCodingZone(src);
+    return namesystem.getErasureCodingPolicy(src);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f62237bc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 923a335..e1702d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
@@ -52,16 +52,16 @@ public interface Namesystem extends RwLock, SafeMode {
   void checkOperation(OperationCategory read) throws StandbyException;
 
   /**
-   * Gets the ECZone for path
+   * Gets the erasure coding policy for the path
    * @param src
    *          - path
-   * @return {@link ErasureCodingZone}
+   * @return {@link ErasureCodingPolicy}
    * @throws IOException
    */
-  ErasureCodingZone getErasureCodingZoneForPath(String src)
+  ErasureCodingPolicy getErasureCodingPolicyForPath(String src)
       throws IOException;
 
   boolean isInSnapshot(BlockInfo blockUC);
 
   CacheManager getCacheManager();
-}
\ No newline at end of file
+}


[29/50] [abbrv] hadoop git commit: HDFS-9113. ErasureCodingWorker#processErasureCodingTasks should not fail to process remaining tasks due to one invalid ECTask. Contributed by Uma Maheswara Rao G.

Posted by wa...@apache.org.
HDFS-9113. ErasureCodingWorker#processErasureCodingTasks should not fail to process remaining tasks due to one invalid ECTask. Contributed by Uma Maheswara Rao G.


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

Branch: refs/heads/trunk
Commit: b762199adbd10173c588df67bd227393c5bbcce9
Parents: c457095
Author: Uma Mahesh <um...@apache.org>
Authored: Sat Sep 19 21:18:42 2015 -0700
Committer: Uma Mahesh <um...@apache.org>
Committed: Sat Sep 19 21:18:42 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +
 .../erasurecode/ErasureCodingWorker.java        |  9 ++-
 .../hadoop/hdfs/TestRecoverStripedFile.java     | 62 +++++++++++++++-----
 3 files changed, 57 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b762199a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 468cc56..db63d53 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -438,3 +438,6 @@
 
     HDFS-8550. Erasure Coding: Fix FindBugs Multithreaded correctness Warning.
     (Rakesh R via zhz)
+
+    HDFS-9113. ErasureCodingWorker#processErasureCodingTasks should not fail to process
+    remaining tasks due to one invalid ECTask (umamahesh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b762199a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
index d1fa478..7a7cd24 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@ -175,8 +175,13 @@ public final class ErasureCodingWorker {
    */
   public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
     for (BlockECRecoveryInfo recoveryInfo : ecTasks) {
-      STRIPED_BLK_RECOVERY_THREAD_POOL.submit(new ReconstructAndTransferBlock(
-          recoveryInfo));
+      try {
+        STRIPED_BLK_RECOVERY_THREAD_POOL
+            .submit(new ReconstructAndTransferBlock(recoveryInfo));
+      } catch (Throwable e) {
+        LOG.warn("Failed to recover striped block "
+            + recoveryInfo.getExtendedBlock().getLocalBlock(), e);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b762199a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
index b581845..ed2ca7b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
@@ -17,10 +17,23 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -28,30 +41,20 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.BitSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-
 public class TestRecoverStripedFile {
   public static final Log LOG = LogFactory.getLog(TestRecoverStripedFile.class);
   
@@ -383,4 +386,33 @@ public class TestRecoverStripedFile {
   private LocatedBlocks getLocatedBlocks(Path file) throws IOException {
     return fs.getClient().getLocatedBlocks(file.toString(), 0, Long.MAX_VALUE);
   }
+
+  /*
+   * Tests that processErasureCodingTasks should not throw exceptions out due to
+   * invalid ECTask submission.
+   */
+  @Test
+  public void testProcessErasureCodingTasksSubmitionShouldSucceed()
+      throws Exception {
+    DataNode dataNode = cluster.dataNodes.get(0).datanode;
+
+    // Pack invalid(dummy) parameters in ecTasks. Irrespective of parameters, each task
+    // thread pool submission should succeed, so that it will not prevent
+    // processing other tasks in the list if any exceptions.
+    int size = cluster.dataNodes.size();
+    short[] liveIndices = new short[size];
+    DatanodeInfo[] dataDNs = new DatanodeInfo[size + 1];
+    DatanodeStorageInfo targetDnInfos_1 = BlockManagerTestUtil
+        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
+            new DatanodeStorage("s01"));
+    DatanodeStorageInfo[] dnStorageInfo = new DatanodeStorageInfo[] {
+        targetDnInfos_1 };
+
+    BlockECRecoveryInfo invalidECInfo = new BlockECRecoveryInfo(
+        new ExtendedBlock("bp-id", 123456), dataDNs, dnStorageInfo, liveIndices,
+        ErasureCodingPolicyManager.getSystemDefaultPolicy());
+    List<BlockECRecoveryInfo> ecTasks = new ArrayList<BlockECRecoveryInfo>();
+    ecTasks.add(invalidECInfo);
+    dataNode.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
+  }
 }


[32/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
index 7a7cd24,0000000..dabae2c
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@@ -1,1013 -1,0 +1,1014 @@@
 +/**
 + * 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.hdfs.server.datanode.erasurecode;
 +
 +import java.io.BufferedOutputStream;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.OutputStream;
 +import java.net.InetSocketAddress;
 +import java.net.Socket;
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.BitSet;
 +import java.util.Collection;
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.concurrent.Callable;
 +import java.util.concurrent.CompletionService;
 +import java.util.concurrent.ExecutorCompletionService;
 +import java.util.concurrent.Future;
 +import java.util.concurrent.LinkedBlockingQueue;
 +import java.util.concurrent.SynchronousQueue;
 +import java.util.concurrent.ThreadPoolExecutor;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.StorageType;
 +import org.apache.hadoop.hdfs.BlockReader;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.DFSPacket;
 +import org.apache.hadoop.hdfs.DFSUtil;
++import org.apache.hadoop.hdfs.DFSUtilClient;
 +import org.apache.hadoop.hdfs.RemoteBlockReader2;
 +import org.apache.hadoop.hdfs.net.Peer;
 +import org.apache.hadoop.hdfs.net.TcpPeerServer;
 +import org.apache.hadoop.hdfs.protocol.DatanodeID;
 +import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 +import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 +import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 +import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 +import org.apache.hadoop.hdfs.server.datanode.DataNode;
 +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 +import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
 +import org.apache.hadoop.io.IOUtils;
 +import org.apache.hadoop.io.erasurecode.CodecUtil;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 +import org.apache.hadoop.net.NetUtils;
 +import org.apache.hadoop.security.token.Token;
 +import org.apache.hadoop.util.Daemon;
 +import org.apache.hadoop.util.DataChecksum;
 +
 +import com.google.common.base.Preconditions;
 +
 +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.convertIndex4Decode;
 +
 +/**
 + * ErasureCodingWorker handles the erasure coding recovery work commands. These
 + * commands would be issued from Namenode as part of Datanode's heart beat
 + * response. BPOfferService delegates the work to this class for handling EC
 + * commands.
 + */
 +public final class ErasureCodingWorker {
 +  private static final Log LOG = DataNode.LOG;
 +  
 +  private final DataNode datanode; 
 +  private final Configuration conf;
 +
 +  private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL;
 +  private ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
 +  private final int STRIPED_READ_TIMEOUT_MILLIS;
 +  private final int STRIPED_READ_BUFFER_SIZE;
 +
 +  public ErasureCodingWorker(Configuration conf, DataNode datanode) {
 +    this.datanode = datanode;
 +    this.conf = conf;
 +
 +    STRIPED_READ_TIMEOUT_MILLIS = conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
 +    initializeStripedReadThreadPool(conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_KEY, 
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT));
 +    STRIPED_READ_BUFFER_SIZE = conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT);
 +
 +    initializeStripedBlkRecoveryThreadPool(conf.getInt(
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY,
 +        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT));
 +  }
 +  
 +  private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) {
 +    return CodecUtil.createRSRawDecoder(conf, numDataUnits, numParityUnits);
 +  }
 +
 +  private void initializeStripedReadThreadPool(int num) {
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Using striped reads; pool threads=" + num);
 +    }
 +    STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
 +        TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
 +        new Daemon.DaemonFactory() {
 +      private final AtomicInteger threadIndex = new AtomicInteger(0);
 +
 +      @Override
 +      public Thread newThread(Runnable r) {
 +        Thread t = super.newThread(r);
 +        t.setName("stripedRead-" + threadIndex.getAndIncrement());
 +        return t;
 +      }
 +    }, new ThreadPoolExecutor.CallerRunsPolicy() {
 +      @Override
 +      public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {
 +        LOG.info("Execution for striped reading rejected, "
 +            + "Executing in current thread");
 +        // will run in the current thread
 +        super.rejectedExecution(runnable, e);
 +      }
 +    });
 +    STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
 +  }
 +
 +  private void initializeStripedBlkRecoveryThreadPool(int num) {
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Using striped block recovery; pool threads=" + num);
 +    }
 +    STRIPED_BLK_RECOVERY_THREAD_POOL = new ThreadPoolExecutor(2, num, 60,
 +        TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
 +        new Daemon.DaemonFactory() {
 +          private final AtomicInteger threadIdx = new AtomicInteger(0);
 +
 +          @Override
 +          public Thread newThread(Runnable r) {
 +            Thread t = super.newThread(r);
 +            t.setName("stripedBlockRecovery-" + threadIdx.getAndIncrement());
 +            return t;
 +          }
 +        });
 +    STRIPED_BLK_RECOVERY_THREAD_POOL.allowCoreThreadTimeOut(true);
 +  }
 +
 +  /**
 +   * Handles the Erasure Coding recovery work commands.
 +   * 
 +   * @param ecTasks
 +   *          BlockECRecoveryInfo
 +   */
 +  public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
 +    for (BlockECRecoveryInfo recoveryInfo : ecTasks) {
 +      try {
 +        STRIPED_BLK_RECOVERY_THREAD_POOL
 +            .submit(new ReconstructAndTransferBlock(recoveryInfo));
 +      } catch (Throwable e) {
 +        LOG.warn("Failed to recover striped block "
 +            + recoveryInfo.getExtendedBlock().getLocalBlock(), e);
 +      }
 +    }
 +  }
 +
 +  /**
 +   * ReconstructAndTransferBlock recover one or more missed striped block in the
 +   * striped block group, the minimum number of live striped blocks should be
 +   * no less than data block number.
 +   * 
 +   * | <- Striped Block Group -> |
 +   *  blk_0      blk_1       blk_2(*)   blk_3   ...   <- A striped block group
 +   *    |          |           |          |  
 +   *    v          v           v          v 
 +   * +------+   +------+   +------+   +------+
 +   * |cell_0|   |cell_1|   |cell_2|   |cell_3|  ...    
 +   * +------+   +------+   +------+   +------+     
 +   * |cell_4|   |cell_5|   |cell_6|   |cell_7|  ...
 +   * +------+   +------+   +------+   +------+
 +   * |cell_8|   |cell_9|   |cell10|   |cell11|  ...
 +   * +------+   +------+   +------+   +------+
 +   *  ...         ...       ...         ...
 +   *  
 +   * 
 +   * We use following steps to recover striped block group, in each round, we
 +   * recover <code>bufferSize</code> data until finish, the 
 +   * <code>bufferSize</code> is configurable and may be less or larger than 
 +   * cell size:
 +   * step1: read <code>bufferSize</code> data from minimum number of sources 
 +   *        required by recovery.
 +   * step2: decode data for targets.
 +   * step3: transfer data to targets.
 +   * 
 +   * In step1, try to read <code>bufferSize</code> data from minimum number
 +   * of sources , if there is corrupt or stale sources, read from new source
 +   * will be scheduled. The best sources are remembered for next round and 
 +   * may be updated in each round.
 +   * 
 +   * In step2, typically if source blocks we read are all data blocks, we 
 +   * need to call encode, and if there is one parity block, we need to call
 +   * decode. Notice we only read once and recover all missed striped block 
 +   * if they are more than one.
 +   * 
 +   * In step3, send the recovered data to targets by constructing packet 
 +   * and send them directly. Same as continuous block replication, we 
 +   * don't check the packet ack. Since the datanode doing the recovery work
 +   * are one of the source datanodes, so the recovered data are sent 
 +   * remotely.
 +   * 
 +   * There are some points we can do further improvements in next phase:
 +   * 1. we can read the block file directly on the local datanode, 
 +   *    currently we use remote block reader. (Notice short-circuit is not
 +   *    a good choice, see inline comments).
 +   * 2. We need to check the packet ack for EC recovery? Since EC recovery
 +   *    is more expensive than continuous block replication, it needs to 
 +   *    read from several other datanodes, should we make sure the 
 +   *    recovered result received by targets? 
 +   */
 +  private class ReconstructAndTransferBlock implements Runnable {
 +    private final int dataBlkNum;
 +    private final int parityBlkNum;
 +    private final int cellSize;
 +    
 +    private RawErasureDecoder decoder;
 +
 +    // Striped read buffer size
 +    private int bufferSize;
 +
 +    private final ExtendedBlock blockGroup;
 +    private final int minRequiredSources;
 +    // position in striped internal block
 +    private long positionInBlock;
 +
 +    // sources
 +    private final short[] liveIndices;
 +    private final DatanodeInfo[] sources;
 +
 +    private final List<StripedReader> stripedReaders;
 +
 +    // The buffers and indices for striped blocks whose length is 0
 +    private ByteBuffer[] zeroStripeBuffers;
 +    private short[] zeroStripeIndices;
 +
 +    // targets
 +    private final DatanodeInfo[] targets;
 +    private final StorageType[] targetStorageTypes;
 +
 +    private final short[] targetIndices;
 +    private final ByteBuffer[] targetBuffers;
 +
 +    private final Socket[] targetSockets;
 +    private final DataOutputStream[] targetOutputStreams;
 +    private final DataInputStream[] targetInputStreams;
 +
 +    private final long[] blockOffset4Targets;
 +    private final long[] seqNo4Targets;
 +
 +    private final static int WRITE_PACKET_SIZE = 64 * 1024;
 +    private DataChecksum checksum;
 +    private int maxChunksPerPacket;
 +    private byte[] packetBuf;
 +    private byte[] checksumBuf;
 +    private int bytesPerChecksum;
 +    private int checksumSize;
 +
 +    private final CachingStrategy cachingStrategy;
 +
 +    private final Map<Future<Void>, Integer> futures = new HashMap<>();
 +    private final CompletionService<Void> readService =
 +        new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL);
 +
 +    ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) {
 +      ErasureCodingPolicy ecPolicy = recoveryInfo.getErasureCodingPolicy();
 +      dataBlkNum = ecPolicy.getNumDataUnits();
 +      parityBlkNum = ecPolicy.getNumParityUnits();
 +      cellSize = ecPolicy.getCellSize();
 +
 +      blockGroup = recoveryInfo.getExtendedBlock();
 +      final int cellsNum = (int)((blockGroup.getNumBytes() - 1) / cellSize + 1);
 +      minRequiredSources = Math.min(cellsNum, dataBlkNum);
 +
 +      liveIndices = recoveryInfo.getLiveBlockIndices();
 +      sources = recoveryInfo.getSourceDnInfos();
 +      stripedReaders = new ArrayList<>(sources.length);
 +
 +      Preconditions.checkArgument(liveIndices.length >= minRequiredSources,
 +          "No enough live striped blocks.");
 +      Preconditions.checkArgument(liveIndices.length == sources.length,
 +          "liveBlockIndices and source dns should match");
 +
 +      if (minRequiredSources < dataBlkNum) {
 +        zeroStripeBuffers = 
 +            new ByteBuffer[dataBlkNum - minRequiredSources];
 +        zeroStripeIndices = new short[dataBlkNum - minRequiredSources];
 +      }
 +
 +      targets = recoveryInfo.getTargetDnInfos();
 +      targetStorageTypes = recoveryInfo.getTargetStorageTypes();
 +      targetIndices = new short[targets.length];
 +      targetBuffers = new ByteBuffer[targets.length];
 +
 +      Preconditions.checkArgument(targetIndices.length <= parityBlkNum,
 +          "Too much missed striped blocks.");
 +
 +      targetSockets = new Socket[targets.length];
 +      targetOutputStreams = new DataOutputStream[targets.length];
 +      targetInputStreams = new DataInputStream[targets.length];
 +
 +      blockOffset4Targets = new long[targets.length];
 +      seqNo4Targets = new long[targets.length];
 +
 +      for (int i = 0; i < targets.length; i++) {
 +        blockOffset4Targets[i] = 0;
 +        seqNo4Targets[i] = 0;
 +      }
 +
 +      getTargetIndices();
 +      cachingStrategy = CachingStrategy.newDefaultStrategy();
 +    }
 +
 +    private ByteBuffer allocateBuffer(int length) {
 +      return ByteBuffer.allocate(length);
 +    }
 +
 +    private ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) {
 +      return StripedBlockUtil.constructInternalBlock(blockGroup, cellSize,
 +          dataBlkNum, i);
 +    }
 +
 +    private long getBlockLen(ExtendedBlock blockGroup, int i) { 
 +      return StripedBlockUtil.getInternalBlockLength(blockGroup.getNumBytes(),
 +          cellSize, dataBlkNum, i);
 +    }
 +
 +    /**
 +     * StripedReader is used to read from one source DN, it contains a block
 +     * reader, buffer and striped block index.
 +     * Only allocate StripedReader once for one source, and the StripedReader
 +     * has the same array order with sources. Typically we only need to allocate
 +     * minimum number (minRequiredSources) of StripedReader, and allocate
 +     * new for new source DN if some existing DN invalid or slow.
 +     * If some source DN is corrupt, set the corresponding blockReader to 
 +     * null and will never read from it again.
 +     *  
 +     * @param i the array index of sources
 +     * @param offsetInBlock offset for the internal block
 +     * @return StripedReader
 +     */
 +    private StripedReader addStripedReader(int i, long offsetInBlock) {
 +      StripedReader reader = new StripedReader(liveIndices[i]);
 +      stripedReaders.add(reader);
 +
 +      BlockReader blockReader = newBlockReader(
 +          getBlock(blockGroup, liveIndices[i]), offsetInBlock, sources[i]);
 +      if (blockReader != null) {
 +        initChecksumAndBufferSizeIfNeeded(blockReader);
 +        reader.blockReader = blockReader;
 +      }
 +      reader.buffer = allocateBuffer(bufferSize);
 +      return reader;
 +    }
 +
 +    @Override
 +    public void run() {
 +      datanode.incrementXmitsInProgress();
 +      try {
 +        // Store the array indices of source DNs we have read successfully.
 +        // In each iteration of read, the success list may be updated if
 +        // some source DN is corrupted or slow. And use the updated success
 +        // list of DNs for next iteration read.
 +        int[] success = new int[minRequiredSources];
 +
 +        int nsuccess = 0;
 +        for (int i = 0; 
 +            i < sources.length && nsuccess < minRequiredSources; i++) {
 +          StripedReader reader = addStripedReader(i, 0);
 +          if (reader.blockReader != null) {
 +            success[nsuccess++] = i;
 +          }
 +        }
 +
 +        if (nsuccess < minRequiredSources) {
 +          String error = "Can't find minimum sources required by "
 +              + "recovery, block id: " + blockGroup.getBlockId();
 +          throw new IOException(error);
 +        }
 +
 +        if (zeroStripeBuffers != null) {
 +          for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +            zeroStripeBuffers[i] = allocateBuffer(bufferSize);
 +          }
 +        }
 +
 +        for (int i = 0; i < targets.length; i++) {
 +          targetBuffers[i] = allocateBuffer(bufferSize);
 +        }
 +
 +        checksumSize = checksum.getChecksumSize();
 +        int chunkSize = bytesPerChecksum + checksumSize;
 +        maxChunksPerPacket = Math.max(
 +            (WRITE_PACKET_SIZE - PacketHeader.PKT_MAX_HEADER_LEN)/chunkSize, 1);
 +        int maxPacketSize = chunkSize * maxChunksPerPacket 
 +            + PacketHeader.PKT_MAX_HEADER_LEN;
 +
 +        packetBuf = new byte[maxPacketSize];
 +        checksumBuf = new byte[checksumSize * (bufferSize / bytesPerChecksum)];
 +
 +        // targetsStatus store whether some target is success, it will record
 +        // any failed target once, if some target failed (invalid DN or transfer
 +        // failed), will not transfer data to it any more.
 +        boolean[] targetsStatus = new boolean[targets.length];
 +        if (initTargetStreams(targetsStatus) == 0) {
 +          String error = "All targets are failed.";
 +          throw new IOException(error);
 +        }
 +
 +        long firstStripedBlockLength = getBlockLen(blockGroup, 0);
 +        while (positionInBlock < firstStripedBlockLength) {
 +          int toRead = Math.min(
 +              bufferSize, (int)(firstStripedBlockLength - positionInBlock));
 +          // step1: read from minimum source DNs required for reconstruction.
 +          //   The returned success list is the source DNs we do real read from
 +          success = readMinimumStripedData4Recovery(success);
 +
 +          // step2: decode to reconstruct targets
 +          long remaining = firstStripedBlockLength - positionInBlock;
 +          int toRecoverLen = remaining < bufferSize ? 
 +              (int)remaining : bufferSize;
 +          recoverTargets(success, targetsStatus, toRecoverLen);
 +
 +          // step3: transfer data
 +          if (transferData2Targets(targetsStatus) == 0) {
 +            String error = "Transfer failed for all targets.";
 +            throw new IOException(error);
 +          }
 +
 +          clearBuffers();
 +          positionInBlock += toRead;
 +        }
 +
 +        endTargetBlocks(targetsStatus);
 +
 +        // Currently we don't check the acks for packets, this is similar as
 +        // block replication.
 +      } catch (Throwable e) {
 +        LOG.warn("Failed to recover striped block: " + blockGroup, e);
 +      } finally {
 +        datanode.decrementXmitsInProgress();
 +        // close block readers
 +        for (StripedReader stripedReader : stripedReaders) {
 +          closeBlockReader(stripedReader.blockReader);
 +        }
 +        for (int i = 0; i < targets.length; i++) {
 +          IOUtils.closeStream(targetOutputStreams[i]);
 +          IOUtils.closeStream(targetInputStreams[i]);
 +          IOUtils.closeStream(targetSockets[i]);
 +        }
 +      }
 +    }
 +
 +    // init checksum from block reader
 +    private void initChecksumAndBufferSizeIfNeeded(BlockReader blockReader) {
 +      if (checksum == null) {
 +        checksum = blockReader.getDataChecksum();
 +        bytesPerChecksum = checksum.getBytesPerChecksum();
 +        // The bufferSize is flat to divide bytesPerChecksum
 +        int readBufferSize = STRIPED_READ_BUFFER_SIZE;
 +        bufferSize = readBufferSize < bytesPerChecksum ? bytesPerChecksum :
 +          readBufferSize - readBufferSize % bytesPerChecksum;
 +      } else {
 +        assert blockReader.getDataChecksum().equals(checksum);
 +      }
 +    }
 +
 +    private void getTargetIndices() {
 +      BitSet bitset = new BitSet(dataBlkNum + parityBlkNum);
 +      for (int i = 0; i < sources.length; i++) {
 +        bitset.set(liveIndices[i]);
 +      }
 +      int m = 0;
 +      int k = 0;
 +      for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {
 +        if (!bitset.get(i)) {
 +          if (getBlockLen(blockGroup, i) > 0) {
 +            if (m < targets.length) {
 +              targetIndices[m++] = (short)i;
 +            }
 +          } else {
 +            zeroStripeIndices[k++] = (short)i;
 +          }
 +        }
 +      }
 +    }
 +
 +    private long getReadLength(int index) {
 +      long blockLen = getBlockLen(blockGroup, index);
 +      long remaining = blockLen - positionInBlock;
 +      return remaining > bufferSize ? bufferSize : remaining;
 +    }
 +
 +    /**
 +     * Read from minimum source DNs required for reconstruction in the iteration.
 +     * First try the success list which we think they are the best DNs
 +     * If source DN is corrupt or slow, try to read some other source DN, 
 +     * and will update the success list. 
 +     * 
 +     * Remember the updated success list and return it for following 
 +     * operations and next iteration read.
 +     * 
 +     * @param success the initial success list of source DNs we think best
 +     * @return updated success list of source DNs we do real read
 +     * @throws IOException
 +     */
 +    private int[] readMinimumStripedData4Recovery(final int[] success)
 +        throws IOException {
 +      int nsuccess = 0;
 +      int[] newSuccess = new int[minRequiredSources];
 +      BitSet used = new BitSet(sources.length);
 +      /*
 +       * Read from minimum source DNs required, the success list contains
 +       * source DNs which we think best.
 +       */
 +      for (int i = 0; i < minRequiredSources; i++) {
 +        StripedReader reader = stripedReaders.get(success[i]);
 +        if (getReadLength(liveIndices[success[i]]) > 0) {
 +          Callable<Void> readCallable = readFromBlock(
 +              reader.blockReader, reader.buffer);
 +          Future<Void> f = readService.submit(readCallable);
 +          futures.put(f, success[i]);
 +        } else {
 +          // If the read length is 0, we don't need to do real read
 +          reader.buffer.position(0);
 +          newSuccess[nsuccess++] = success[i];
 +        }
 +        used.set(success[i]);
 +      }
 +
 +      while (!futures.isEmpty()) {
 +        try {
 +          StripingChunkReadResult result =
 +              StripedBlockUtil.getNextCompletedStripedRead(
 +                  readService, futures, STRIPED_READ_TIMEOUT_MILLIS);
 +          int resultIndex = -1;
 +          if (result.state == StripingChunkReadResult.SUCCESSFUL) {
 +            resultIndex = result.index;
 +          } else if (result.state == StripingChunkReadResult.FAILED) {
 +            // If read failed for some source DN, we should not use it anymore 
 +            // and schedule read from another source DN.
 +            StripedReader failedReader = stripedReaders.get(result.index);
 +            closeBlockReader(failedReader.blockReader);
 +            failedReader.blockReader = null;
 +            resultIndex = scheduleNewRead(used);
 +          } else if (result.state == StripingChunkReadResult.TIMEOUT) {
 +            // If timeout, we also schedule a new read.
 +            resultIndex = scheduleNewRead(used);
 +          }
 +          if (resultIndex >= 0) {
 +            newSuccess[nsuccess++] = resultIndex;
 +            if (nsuccess >= minRequiredSources) {
 +              // cancel remaining reads if we read successfully from minimum
 +              // number of source DNs required by reconstruction.
 +              cancelReads(futures.keySet());
 +              futures.clear();
 +              break;
 +            }
 +          }
 +        } catch (InterruptedException e) {
 +          LOG.info("Read data interrupted.", e);
 +          break;
 +        }
 +      }
 +
 +      if (nsuccess < minRequiredSources) {
 +        String error = "Can't read data from minimum number of sources "
 +            + "required by reconstruction, block id: " + blockGroup.getBlockId();
 +        throw new IOException(error);
 +      }
 +
 +      return newSuccess;
 +    }
 +    
 +    private void paddingBufferToLen(ByteBuffer buffer, int len) {
 +      int toPadding = len - buffer.position();
 +      for (int i = 0; i < toPadding; i++) {
 +        buffer.put((byte) 0);
 +      }
 +    }
 +    
 +    // Initialize decoder
 +    private void initDecoderIfNecessary() {
 +      if (decoder == null) {
 +        decoder = newDecoder(dataBlkNum, parityBlkNum);
 +      }
 +    }
 +
 +    private int[] getErasedIndices(boolean[] targetsStatus) {
 +      int[] result = new int[targets.length];
 +      int m = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          result[m++] = convertIndex4Decode(targetIndices[i], 
 +              dataBlkNum, parityBlkNum);
 +        }
 +      }
 +      return Arrays.copyOf(result, m);
 +    }
 +
 +    private void recoverTargets(int[] success, boolean[] targetsStatus,
 +        int toRecoverLen) {
 +      initDecoderIfNecessary();
 +      ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum];
 +      for (int i = 0; i < success.length; i++) {
 +        StripedReader reader = stripedReaders.get(success[i]);
 +        ByteBuffer buffer = reader.buffer;
 +        paddingBufferToLen(buffer, toRecoverLen);
 +        inputs[convertIndex4Decode(reader.index, dataBlkNum, parityBlkNum)] = 
 +            (ByteBuffer)buffer.flip();
 +      }
 +      if (success.length < dataBlkNum) {
 +        for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +          ByteBuffer buffer = zeroStripeBuffers[i];
 +          paddingBufferToLen(buffer, toRecoverLen);
 +          int index = convertIndex4Decode(zeroStripeIndices[i], dataBlkNum,
 +              parityBlkNum);
 +          inputs[index] = (ByteBuffer)buffer.flip();
 +        }
 +      }
 +      int[] erasedIndices = getErasedIndices(targetsStatus);
 +      ByteBuffer[] outputs = new ByteBuffer[erasedIndices.length];
 +      int m = 0;
 +      for (int i = 0; i < targetBuffers.length; i++) {
 +        if (targetsStatus[i]) {
 +          outputs[m++] = targetBuffers[i];
 +          outputs[i].limit(toRecoverLen);
 +        }
 +      }
 +      decoder.decode(inputs, erasedIndices, outputs);
 +
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          long blockLen = getBlockLen(blockGroup, targetIndices[i]);
 +          long remaining = blockLen - positionInBlock;
 +          if (remaining < 0) {
 +            targetBuffers[i].limit(0);
 +          } else if (remaining < toRecoverLen) {
 +            targetBuffers[i].limit((int)remaining);
 +          }
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Schedule a read from some new source DN if some DN is corrupted
 +     * or slow, this is called from the read iteration.
 +     * Initially we may only have <code>minRequiredSources</code> number of 
 +     * StripedReader.
 +     * If the position is at the end of target block, don't need to do 
 +     * real read, and return the array index of source DN, otherwise -1.
 +     * 
 +     * @param used the used source DNs in this iteration.
 +     * @return the array index of source DN if don't need to do real read.
 +     */
 +    private int scheduleNewRead(BitSet used) {
 +      StripedReader reader = null;
 +      // step1: initially we may only have <code>minRequiredSources</code>
 +      // number of StripedReader, and there may be some source DNs we never 
 +      // read before, so will try to create StripedReader for one new source DN
 +      // and try to read from it. If found, go to step 3.
 +      int m = stripedReaders.size();
 +      while (reader == null && m < sources.length) {
 +        reader = addStripedReader(m, positionInBlock);
 +        if (getReadLength(liveIndices[m]) > 0) {
 +          if (reader.blockReader == null) {
 +            reader = null;
 +            m++;
 +          }
 +        } else {
 +          used.set(m);
 +          return m;
 +        }
 +      }
 +
 +      // step2: if there is no new source DN we can use, try to find a source 
 +      // DN we ever read from but because some reason, e.g., slow, it
 +      // is not in the success DN list at the begin of this iteration, so 
 +      // we have not tried it in this iteration. Now we have a chance to 
 +      // revisit it again.
 +      for (int i = 0; reader == null && i < stripedReaders.size(); i++) {
 +        if (!used.get(i)) {
 +          StripedReader r = stripedReaders.get(i);
 +          if (getReadLength(liveIndices[i]) > 0) {
 +            closeBlockReader(r.blockReader);
 +            r.blockReader = newBlockReader(
 +                getBlock(blockGroup, liveIndices[i]), positionInBlock,
 +                sources[i]);
 +            if (r.blockReader != null) {
 +              m = i;
 +              reader = r;
 +            }
 +          } else {
 +            used.set(i);
 +            r.buffer.position(0);
 +            return i;
 +          }
 +        }
 +      }
 +
 +      // step3: schedule if find a correct source DN and need to do real read.
 +      if (reader != null) {
 +        Callable<Void> readCallable = readFromBlock(
 +            reader.blockReader, reader.buffer);
 +        Future<Void> f = readService.submit(readCallable);
 +        futures.put(f, m);
 +        used.set(m);
 +      }
 +
 +      return -1;
 +    }
 +
 +    // cancel all reads.
 +    private void cancelReads(Collection<Future<Void>> futures) {
 +      for (Future<Void> future : futures) {
 +        future.cancel(true);
 +      }
 +    }
 +
 +    private Callable<Void> readFromBlock(final BlockReader reader,
 +        final ByteBuffer buf) {
 +      return new Callable<Void>() {
 +
 +        @Override
 +        public Void call() throws Exception {
 +          try {
 +            actualReadFromBlock(reader, buf);
 +            return null;
 +          } catch (IOException e) {
 +            LOG.info(e.getMessage());
 +            throw e;
 +          }
 +        }
 +
 +      };
 +    }
 +
 +    /**
 +     * Read bytes from block
 +     */
 +    private void actualReadFromBlock(BlockReader reader, ByteBuffer buf)
 +        throws IOException {
 +      int len = buf.remaining();
 +      int n = 0;
 +      while (n < len) {
 +        int nread = reader.read(buf);
 +        if (nread <= 0) {
 +          break;
 +        }
 +        n += nread;
 +      }
 +    }
 +
 +    // close block reader
 +    private void closeBlockReader(BlockReader blockReader) {
 +      try {
 +        if (blockReader != null) {
 +          blockReader.close();
 +        }
 +      } catch (IOException e) {
 +        // ignore
 +      }
 +    }
 +
 +    private InetSocketAddress getSocketAddress4Transfer(DatanodeInfo dnInfo) {
 +      return NetUtils.createSocketAddr(dnInfo.getXferAddr(
 +          datanode.getDnConf().getConnectToDnViaHostname()));
 +    }
 +
 +    private BlockReader newBlockReader(final ExtendedBlock block, 
 +        long offsetInBlock, DatanodeInfo dnInfo) {
 +      if (offsetInBlock >= block.getNumBytes()) {
 +        return null;
 +      }
 +      try {
 +        InetSocketAddress dnAddr = getSocketAddress4Transfer(dnInfo);
 +        Token<BlockTokenIdentifier> blockToken = datanode.getBlockAccessToken(
 +            block, EnumSet.of(BlockTokenIdentifier.AccessMode.READ));
 +        /*
 +         * This can be further improved if the replica is local, then we can
 +         * read directly from DN and need to check the replica is FINALIZED
 +         * state, notice we should not use short-circuit local read which
 +         * requires config for domain-socket in UNIX or legacy config in Windows.
 +         */
 +        return RemoteBlockReader2.newBlockReader(
 +            "dummy", block, blockToken, offsetInBlock, 
 +            block.getNumBytes() - offsetInBlock, true,
 +            "", newConnectedPeer(block, dnAddr, blockToken, dnInfo), dnInfo,
 +            null, cachingStrategy);
 +      } catch (IOException e) {
 +        return null;
 +      }
 +    }
 +
 +    private Peer newConnectedPeer(ExtendedBlock b, InetSocketAddress addr,
 +        Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
 +        throws IOException {
 +      Peer peer = null;
 +      boolean success = false;
 +      Socket sock = null;
 +      final int socketTimeout = datanode.getDnConf().getSocketTimeout(); 
 +      try {
 +        sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
 +        NetUtils.connect(sock, addr, socketTimeout);
-         peer = TcpPeerServer.peerFromSocketAndKey(datanode.getSaslClient(), 
++        peer = DFSUtilClient.peerFromSocketAndKey(datanode.getSaslClient(),
 +            sock, datanode.getDataEncryptionKeyFactoryForBlock(b),
 +            blockToken, datanodeId);
 +        peer.setReadTimeout(socketTimeout);
 +        success = true;
 +        return peer;
 +      } finally {
 +        if (!success) {
 +          IOUtils.cleanup(LOG, peer);
 +          IOUtils.closeSocket(sock);
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Send data to targets
 +     */
 +    private int transferData2Targets(boolean[] targetsStatus) {
 +      int nsuccess = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          boolean success = false;
 +          try {
 +            ByteBuffer buffer = targetBuffers[i];
 +            
 +            if (buffer.remaining() == 0) {
 +              continue;
 +            }
 +
 +            checksum.calculateChunkedSums(
 +                buffer.array(), 0, buffer.remaining(), checksumBuf, 0);
 +
 +            int ckOff = 0;
 +            while (buffer.remaining() > 0) {
 +              DFSPacket packet = new DFSPacket(packetBuf, maxChunksPerPacket,
 +                  blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, false);
 +              int maxBytesToPacket = maxChunksPerPacket * bytesPerChecksum;
 +              int toWrite = buffer.remaining() > maxBytesToPacket ?
 +                  maxBytesToPacket : buffer.remaining();
 +              int ckLen = ((toWrite - 1) / bytesPerChecksum + 1) * checksumSize;
 +              packet.writeChecksum(checksumBuf, ckOff, ckLen);
 +              ckOff += ckLen;
 +              packet.writeData(buffer, toWrite);
 +
 +              // Send packet
 +              packet.writeTo(targetOutputStreams[i]);
 +
 +              blockOffset4Targets[i] += toWrite;
 +              nsuccess++;
 +              success = true;
 +            }
 +          } catch (IOException e) {
 +            LOG.warn(e.getMessage());
 +          }
 +          targetsStatus[i] = success;
 +        }
 +      }
 +      return nsuccess;
 +    }
 +
 +    /**
 +     * clear all buffers
 +     */
 +    private void clearBuffers() {
 +      for (StripedReader stripedReader : stripedReaders) {
 +        if (stripedReader.buffer != null) {
 +          stripedReader.buffer.clear();
 +        }
 +      }
 +
 +      if (zeroStripeBuffers != null) {
 +        for (int i = 0; i < zeroStripeBuffers.length; i++) {
 +          zeroStripeBuffers[i].clear();
 +        }
 +      }
 +
 +      for (int i = 0; i < targetBuffers.length; i++) {
 +        if (targetBuffers[i] != null) {
 +          cleanBuffer(targetBuffers[i]);
 +        }
 +      }
 +    }
 +    
 +    private ByteBuffer cleanBuffer(ByteBuffer buffer) {
 +      Arrays.fill(buffer.array(), (byte) 0);
 +      return (ByteBuffer)buffer.clear();
 +    }
 +
 +    // send an empty packet to mark the end of the block
 +    private void endTargetBlocks(boolean[] targetsStatus) {
 +      for (int i = 0; i < targets.length; i++) {
 +        if (targetsStatus[i]) {
 +          try {
 +            DFSPacket packet = new DFSPacket(packetBuf, 0, 
 +                blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, true);
 +            packet.writeTo(targetOutputStreams[i]);
 +            targetOutputStreams[i].flush();
 +          } catch (IOException e) {
 +            LOG.warn(e.getMessage());
 +          }
 +        }
 +      }
 +    }
 +
 +    /**
 +     * Initialize  output/input streams for transferring data to target
 +     * and send create block request. 
 +     */
 +    private int initTargetStreams(boolean[] targetsStatus) {
 +      int nsuccess = 0;
 +      for (int i = 0; i < targets.length; i++) {
 +        Socket socket = null;
 +        DataOutputStream out = null;
 +        DataInputStream in = null;
 +        boolean success = false;
 +        try {
 +          InetSocketAddress targetAddr = 
 +              getSocketAddress4Transfer(targets[i]);
 +          socket = datanode.newSocket();
 +          NetUtils.connect(socket, targetAddr, 
 +              datanode.getDnConf().getSocketTimeout());
 +          socket.setSoTimeout(datanode.getDnConf().getSocketTimeout());
 +
 +          ExtendedBlock block = getBlock(blockGroup, targetIndices[i]);
 +          Token<BlockTokenIdentifier> blockToken = 
 +              datanode.getBlockAccessToken(block,
 +                  EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
 +
 +          long writeTimeout = datanode.getDnConf().getSocketWriteTimeout();
 +          OutputStream unbufOut = NetUtils.getOutputStream(socket, writeTimeout);
 +          InputStream unbufIn = NetUtils.getInputStream(socket);
 +          DataEncryptionKeyFactory keyFactory =
 +            datanode.getDataEncryptionKeyFactoryForBlock(block);
 +          IOStreamPair saslStreams = datanode.getSaslClient().socketSend(
 +              socket, unbufOut, unbufIn, keyFactory, blockToken, targets[i]);
 +
 +          unbufOut = saslStreams.out;
 +          unbufIn = saslStreams.in;
 +
 +          out = new DataOutputStream(new BufferedOutputStream(unbufOut,
 +              DFSUtil.getSmallBufferSize(conf)));
 +          in = new DataInputStream(unbufIn);
 +
 +          DatanodeInfo source = new DatanodeInfo(datanode.getDatanodeId());
 +          new Sender(out).writeBlock(block, targetStorageTypes[i], 
 +              blockToken, "", new DatanodeInfo[]{targets[i]}, 
 +              new StorageType[]{targetStorageTypes[i]}, source, 
 +              BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, 0, 
 +              checksum, cachingStrategy, false, false, null);
 +
 +          targetSockets[i] = socket;
 +          targetOutputStreams[i] = out;
 +          targetInputStreams[i] = in;
 +          nsuccess++;
 +          success = true;
 +        } catch (Throwable e) {
 +          LOG.warn(e.getMessage());
 +        } finally {
 +          if (!success) {
 +            IOUtils.closeStream(out);
 +            IOUtils.closeStream(in);
 +            IOUtils.closeStream(socket);
 +          }
 +        }
 +        targetsStatus[i] = success;
 +      }
 +      return nsuccess;
 +    }
 +  }
 +
 +  private static class StripedReader {
 +    private final short index; // internal block index
 +    private BlockReader blockReader;
 +    private ByteBuffer buffer;
 +
 +    private StripedReader(short index) {
 +      this.index = index;
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index a115138,0ae739c..34b28e4
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@@ -39,10 -39,8 +39,9 @@@ import org.apache.hadoop.fs.permission.
  import org.apache.hadoop.fs.StorageType;
  import org.apache.hadoop.fs.XAttr;
  import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
- import org.apache.hadoop.hdfs.protocolPB.PBHelper;
  import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
@@@ -334,21 -331,13 +333,21 @@@ public final class FSImageFormatPBINod
        INodeSection.INodeFile f = n.getFile();
        List<BlockProto> bp = f.getBlocksList();
        short replication = (short) f.getReplication();
 +      boolean isStriped = f.getIsStriped();
        LoaderContext state = parent.getLoaderContext();
 +      ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
  
        BlockInfo[] blocks = new BlockInfo[bp.size()];
 -      for (int i = 0, e = bp.size(); i < e; ++i) {
 -        blocks[i] =
 -            new BlockInfoContiguous(PBHelperClient.convert(bp.get(i)), replication);
 +      for (int i = 0; i < bp.size(); ++i) {
 +        BlockProto b = bp.get(i);
 +        if (isStriped) {
-           blocks[i] = new BlockInfoStriped(PBHelper.convert(b), ecPolicy);
++          blocks[i] = new BlockInfoStriped(PBHelperClient.convert(b), ecPolicy);
 +        } else {
-           blocks[i] = new BlockInfoContiguous(PBHelper.convert(b),
++          blocks[i] = new BlockInfoContiguous(PBHelperClient.convert(b),
 +              replication);
 +        }
        }
 +
        final PermissionStatus permissions = loadPermission(f.getPermission(),
            parent.getLoaderContext().getStringTable());
  
@@@ -654,11 -632,10 +653,11 @@@
      private void save(OutputStream out, INodeFile n) throws IOException {
        INodeSection.INodeFile.Builder b = buildINodeFile(n,
            parent.getSaverContext());
 +      BlockInfo[] blocks = n.getBlocks();
  
 -      if (n.getBlocks() != null) {
 +      if (blocks != null) {
          for (Block block : n.getBlocks()) {
-           b.addBlocks(PBHelper.convert(block));
+           b.addBlocks(PBHelperClient.convert(block));
          }
        }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index ed52ca4,75b6be9..b6b151c
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@@ -4714,26 -4654,8 +4713,8 @@@ public class FSNamesystem implements Na
        && safeMode.isOn();
    }
  
-   /**
-    * Check if replication queues are to be populated
-    * @return true when node is HAState.Active and not in the very first safemode
-    */
-   @Override
-   public boolean isPopulatingReplQueues() {
-     if (!shouldPopulateReplQueues()) {
-       return false;
-     }
-     return initializedReplQueues;
-   }
- 
-   private boolean shouldPopulateReplQueues() {
-     if(haContext == null || haContext.getState() == null)
-       return false;
-     return haContext.getState().shouldPopulateReplQueues();
-   }
- 
    @Override
 -  public void incrementSafeBlockCount(int replication) {
 +  public void incrementSafeBlockCount(int storageNum, BlockInfo storedBlock) {
      // safeMode is volatile, and may be set to null at any time
      SafeModeInfo safeMode = this.safeMode;
      if (safeMode == null)
@@@ -6233,11 -6150,11 +6222,16 @@@
      return cacheManager;
    }
  
 +  /** @return the ErasureCodingPolicyManager. */
 +  public ErasureCodingPolicyManager getErasureCodingPolicyManager() {
 +    return ecPolicyManager;
 +  }
 +
+   @Override
+   public HAContext getHAContext() {
+     return haContext;
+   }
+ 
    @Override  // NameNodeMXBean
    public String getCorruptFiles() {
      List<String> list = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 6f7b702,8565522..c765edc
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@@ -664,37 -608,12 +664,24 @@@ public class INodeFile extends INodeWit
      return counts;
    }
  
 +  /**
 +   * Compute quota of striped file. Note that currently EC files do not support
 +   * append/hflush/hsync, thus the file length recorded in snapshots should be
 +   * the same with the current file length.
 +   */
 +  public final QuotaCounts computeQuotaUsageWithStriped(
 +      BlockStoragePolicy bsp, QuotaCounts counts) {
 +    counts.addNameSpace(1);
 +    counts.add(storagespaceConsumed(bsp));
 +    return counts;
 +  }
 +
    @Override
    public final ContentSummaryComputationContext computeContentSummary(
-       final ContentSummaryComputationContext summary) {
+       int snapshotId, final ContentSummaryComputationContext summary) {
      final ContentCounts counts = summary.getCounts();
-     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
-     final long fileLen;
-     if (sf == null) {
-       fileLen = computeFileSize();
-       counts.addContent(Content.FILE, 1);
-     } else {
-       final FileDiffList diffs = sf.getDiffs();
-       final int n = diffs.asList().size();
-       counts.addContent(Content.FILE, n);
-       if (n > 0 && sf.isCurrentFileDeleted()) {
-         fileLen =  diffs.getLast().getFileSize();
-       } else {
-         fileLen = computeFileSize();
-       }
-     }
+     counts.addContent(Content.FILE, 1);
+     final long fileLen = computeFileSize(snapshotId);
      counts.addContent(Content.LENGTH, fileLen);
      counts.addContent(Content.DISKSPACE, storagespaceConsumed(null)
          .getStorageSpace());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index e1702d9,5bc4033..b1012c2
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@@ -64,4 -52,5 +65,6 @@@ public interface Namesystem extends RwL
    boolean isInSnapshot(BlockInfo blockUC);
  
    CacheManager getCacheManager();
++
+   HAContext getHAContext();
 -}
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
index 252844c,06a8219..98deed2
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
@@@ -39,15 -39,11 +39,12 @@@ public interface SafeMode 
     */
    public boolean isInStartupSafeMode();
  
-   /** Check whether replication queues are being populated. */
-   public boolean isPopulatingReplQueues();
-     
    /**
     * Increment number of blocks that reached minimal replication.
 -   * @param replication current replication 
 +   * @param replication current replication
 +   * @param storedBlock current stored Block
     */
 -  public void incrementSafeBlockCount(int replication);
 +  public void incrementSafeBlockCount(int replication, BlockInfo storedBlock);
  
    /** Decrement number of blocks that reached minimal replication. */
    public void decrementSafeBlockCount(BlockInfo b);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index 450d981,cf21411..ae23783
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@@ -243,15 -242,13 +243,15 @@@ public class FSImageFormatPBSnapshot 
          FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
              pbf.getFileSize());
          List<BlockProto> bpl = pbf.getBlocksList();
 +        // in file diff there can only be contiguous blocks
          BlockInfo[] blocks = new BlockInfo[bpl.size()];
          for(int j = 0, e = bpl.size(); j < e; ++j) {
-           Block blk = PBHelper.convert(bpl.get(j));
+           Block blk = PBHelperClient.convert(bpl.get(j));
            BlockInfo storedBlock = bm.getStoredBlock(blk);
            if(storedBlock == null) {
 -            storedBlock = bm.addBlockCollection(
 -                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
 +            storedBlock = (BlockInfoContiguous) fsn.getBlockManager()
 +                .addBlockCollectionWithCheck(new BlockInfoContiguous(blk,
 +                    copy.getFileReplication()), file);
            }
            blocks[j] = storedBlock;
          }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
index 6c06a8d,0000000..0499a2b
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
@@@ -1,38 -1,0 +1,39 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * <p/>
 + * http://www.apache.org/licenses/LICENSE-2.0
 + * <p/>
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.cli;
 +
 +import org.apache.hadoop.cli.util.CLICommandErasureCodingCli;
 +import org.apache.hadoop.cli.util.CLICommandTypes;
 +import org.apache.hadoop.cli.util.CLITestCmd;
 +import org.apache.hadoop.cli.util.CommandExecutor;
 +import org.apache.hadoop.cli.util.ErasureCodingCliCmdExecutor;
++import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.hdfs.tools.erasurecode.ECCli;
 +
 +public class CLITestCmdErasureCoding extends CLITestCmd {
 +  public CLITestCmdErasureCoding(String str, CLICommandTypes type) {
 +    super(str, type);
 +  }
 +
 +  @Override
-   public CommandExecutor getExecutor(String tag) throws IllegalArgumentException {
++  public CommandExecutor getExecutor(String tag, Configuration conf) throws IllegalArgumentException {
 +    if (getType() instanceof CLICommandErasureCodingCli)
 +      return new ErasureCodingCliCmdExecutor(tag, new ECCli());
-     return super.getExecutor(tag);
++    return super.getExecutor(tag, conf);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
index dfefb66,0000000..29ec98e
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
@@@ -1,115 -1,0 +1,115 @@@
 +
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.hadoop.cli;
 +
 +import org.apache.hadoop.cli.util.CLICommand;
 +import org.apache.hadoop.cli.util.CLICommandErasureCodingCli;
 +import org.apache.hadoop.cli.util.CommandExecutor.Result;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.MiniDFSCluster;
 +import org.junit.After;
 +import org.junit.Before;
 +import org.junit.Test;
 +import org.xml.sax.SAXException;
 +
 +public class TestErasureCodingCLI extends CLITestHelper {
 +  private final int NUM_OF_DATANODES = 3;
 +  private MiniDFSCluster dfsCluster = null;
 +  private FileSystem fs = null;
 +  private String namenode = null;
 +
 +  @Before
 +  @Override
 +  public void setUp() throws Exception {
 +    super.setUp();
 +
 +    dfsCluster = new MiniDFSCluster.Builder(conf)
 +        .numDataNodes(NUM_OF_DATANODES).build();
 +    dfsCluster.waitClusterUp();
 +    namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///");
 +
 +    username = System.getProperty("user.name");
 +
 +    fs = dfsCluster.getFileSystem();
 +  }
 +
 +  @Override
 +  protected String getTestFile() {
 +    return "testErasureCodingConf.xml";
 +  }
 +
 +  @After
 +  @Override
 +  public void tearDown() throws Exception {
 +    if (fs != null) {
 +      fs.close();
 +    }
 +    if (dfsCluster != null) {
 +      dfsCluster.shutdown();
 +    }
 +    Thread.sleep(2000);
 +    super.tearDown();
 +  }
 +
 +  @Override
 +  protected String expandCommand(final String cmd) {
 +    String expCmd = cmd;
 +    expCmd = expCmd.replaceAll("NAMENODE", namenode);
 +    expCmd = expCmd.replaceAll("#LF#", System.getProperty("line.separator"));
 +    expCmd = super.expandCommand(expCmd);
 +    return expCmd;
 +  }
 +
 +  @Override
 +  protected TestConfigFileParser getConfigParser() {
 +    return new TestErasureCodingAdmin();
 +  }
 +
 +  private class TestErasureCodingAdmin extends
 +      CLITestHelper.TestConfigFileParser {
 +    @Override
 +    public void endElement(String uri, String localName, String qName)
 +        throws SAXException {
 +      if (qName.equals("ec-admin-command")) {
 +        if (testCommands != null) {
 +          testCommands.add(new CLITestCmdErasureCoding(charString,
 +              new CLICommandErasureCodingCli()));
 +        } else if (cleanupCommands != null) {
 +          cleanupCommands.add(new CLITestCmdErasureCoding(charString,
 +              new CLICommandErasureCodingCli()));
 +        }
 +      } else {
 +        super.endElement(uri, localName, qName);
 +      }
 +    }
 +  }
 +
 +  @Override
 +  protected Result execute(CLICommand cmd) throws Exception {
-     return cmd.getExecutor(namenode).executeCommand(cmd.getCmd());
++    return cmd.getExecutor(namenode, conf).executeCommand(cmd.getCmd());
 +  }
 +
 +  @Test
 +  @Override
 +  public void testAll() {
 +    super.testAll();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 3551055,a7e80ca..12d4811
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@@ -66,14 -66,9 +66,15 @@@ import java.util.Set
  import java.util.UUID;
  import java.util.concurrent.TimeoutException;
  import java.util.concurrent.atomic.AtomicBoolean;
 +import com.google.common.base.Charsets;
 +import com.google.common.base.Joiner;
 +import com.google.common.base.Preconditions;
 +import com.google.common.base.Supplier;
 +import com.google.common.collect.Lists;
 +import com.google.common.collect.Maps;
  
  import org.apache.commons.io.FileUtils;
+ import org.apache.commons.lang.UnhandledException;
  import org.apache.commons.logging.Log;
  import org.apache.commons.logging.LogFactory;
  import org.apache.hadoop.conf.Configuration;
@@@ -141,10 -133,8 +142,11 @@@ import org.apache.hadoop.hdfs.server.na
  import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
  import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
  import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 +import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
  import org.apache.hadoop.hdfs.tools.DFSAdmin;
+ import org.apache.hadoop.hdfs.tools.JMXGet;
  import org.apache.hadoop.io.IOUtils;
  import org.apache.hadoop.io.nativeio.NativeIO;
  import org.apache.hadoop.net.NetUtils;
@@@ -1870,150 -1858,21 +1872,168 @@@ public class DFSTestUtil 
      }
    }
  
 +  public static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock(
 +      Block block, BlockStatus blockStatus, DatanodeStorage storage) {
 +    ReceivedDeletedBlockInfo[] receivedBlocks = new ReceivedDeletedBlockInfo[1];
 +    receivedBlocks[0] = new ReceivedDeletedBlockInfo(block, blockStatus, null);
 +    StorageReceivedDeletedBlocks[] reports = new StorageReceivedDeletedBlocks[1];
 +    reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks);
 +    return reports;
 +  }
 +
 +  /**
 +   * Creates the metadata of a file in striped layout. This method only
 +   * manipulates the NameNode state without injecting data to DataNode.
 +   * You should disable periodical heartbeat before use this.
 +   *  @param file Path of the file to create
 +   * @param dir Parent path of the file
 +   * @param numBlocks Number of striped block groups to add to the file
 +   * @param numStripesPerBlk Number of striped cells in each block
 +   * @param toMkdir
 +   */
 +  public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir,
 +      int numBlocks, int numStripesPerBlk, boolean toMkdir) throws Exception {
 +    DistributedFileSystem dfs = cluster.getFileSystem();
 +    // If outer test already set EC policy, dir should be left as null
 +    if (toMkdir) {
 +      assert dir != null;
 +      dfs.mkdirs(dir);
 +      try {
 +        dfs.getClient().setErasureCodingPolicy(dir.toString(), null);
 +      } catch (IOException e) {
 +        if (!e.getMessage().contains("non-empty directory")) {
 +          throw e;
 +        }
 +      }
 +    }
 +
 +    FSDataOutputStream out = null;
 +    try {
 +      out = dfs.create(file, (short) 1); // create an empty file
 +
 +      FSNamesystem ns = cluster.getNamesystem();
 +      FSDirectory fsdir = ns.getFSDirectory();
 +      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
 +
 +      ExtendedBlock previous = null;
 +      for (int i = 0; i < numBlocks; i++) {
 +        Block newBlock = addStripedBlockToFile(cluster.getDataNodes(), dfs, ns,
 +            file.toString(), fileNode, dfs.getClient().getClientName(),
 +            previous, numStripesPerBlk);
 +        previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
 +      }
 +
 +      dfs.getClient().namenode.complete(file.toString(),
 +          dfs.getClient().getClientName(), previous, fileNode.getId());
 +    } finally {
 +      IOUtils.cleanup(null, out);
 +    }
 +  }
 +
 +  /**
 +   * Adds a striped block group to a file. This method only manipulates NameNode
 +   * states of the file and the block without injecting data to DataNode.
 +   * It does mimic block reports.
 +   * You should disable periodical heartbeat before use this.
 +   * @param dataNodes List DataNodes to host the striped block group
 +   * @param previous Previous block in the file
 +   * @param numStripes Number of stripes in each block group
 +   * @return The added block group
 +   */
 +  public static Block addStripedBlockToFile(List<DataNode> dataNodes,
 +      DistributedFileSystem fs, FSNamesystem ns, String file, INodeFile fileNode,
 +      String clientName, ExtendedBlock previous, int numStripes)
 +      throws Exception {
 +    fs.getClient().namenode.addBlock(file, clientName, previous, null,
 +        fileNode.getId(), null);
 +
 +    final BlockInfo lastBlock = fileNode.getLastBlock();
 +    final int groupSize = fileNode.getPreferredBlockReplication();
 +    assert dataNodes.size() >= groupSize;
 +    // 1. RECEIVING_BLOCK IBR
 +    for (int i = 0; i < groupSize; i++) {
 +      DataNode dn = dataNodes.get(i);
 +      final Block block = new Block(lastBlock.getBlockId() + i, 0,
 +          lastBlock.getGenerationStamp());
 +      DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
 +      StorageReceivedDeletedBlocks[] reports = DFSTestUtil
 +          .makeReportForReceivedBlock(block,
 +              ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
 +      for (StorageReceivedDeletedBlocks report : reports) {
 +        ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
 +      }
 +    }
 +
 +    // 2. RECEIVED_BLOCK IBR
 +    for (int i = 0; i < groupSize; i++) {
 +      DataNode dn = dataNodes.get(i);
 +      final Block block = new Block(lastBlock.getBlockId() + i,
 +          numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp());
 +      DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
 +      StorageReceivedDeletedBlocks[] reports = DFSTestUtil
 +          .makeReportForReceivedBlock(block,
 +              ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +      for (StorageReceivedDeletedBlocks report : reports) {
 +        ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
 +      }
 +    }
 +
 +    lastBlock.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS);
 +    return lastBlock;
 +  }
 +
 +  /**
 +   * Because currently DFSStripedOutputStream does not support hflush/hsync,
 +   * tests can use this method to flush all the buffered data to DataNodes.
 +   */
 +  public static ExtendedBlock flushInternal(DFSStripedOutputStream out)
 +      throws IOException {
 +    out.flushInternal();
 +    return out.getBlock();
 +  }
 +
 +  /**
 +   * Verify that blocks in striped block group are on different nodes, and every
 +   * internal blocks exists.
 +   */
 +  public static void verifyLocatedStripedBlocks(LocatedBlocks lbs,
 +       int groupSize) {
 +    for (LocatedBlock lb : lbs.getLocatedBlocks()) {
 +      assert lb instanceof LocatedStripedBlock;
 +      HashSet<DatanodeInfo> locs = new HashSet<>();
 +      for (DatanodeInfo datanodeInfo : lb.getLocations()) {
 +        locs.add(datanodeInfo);
 +      }
 +      assertEquals(groupSize, lb.getLocations().length);
 +      assertEquals(groupSize, locs.size());
 +
 +      // verify that every internal blocks exists
 +      int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices();
 +      assertEquals(groupSize, blockIndices.length);
 +      HashSet<Integer> found = new HashSet<>();
 +      for (int index : blockIndices) {
 +        assert index >=0;
 +        found.add(index);
 +      }
 +      assertEquals(groupSize, found.size());
 +    }
 +  }
++
+   public static void waitForMetric(final JMXGet jmx, final String metricName, final int expectedValue)
+       throws TimeoutException, InterruptedException {
+     GenericTestUtils.waitFor(new Supplier<Boolean>() {
+       @Override
+       public Boolean get() {
+         try {
+           final int currentValue = Integer.parseInt(jmx.getValue(metricName));
+           LOG.info("Waiting for " + metricName +
+                        " to reach value " + expectedValue +
+                        ", current value = " + currentValue);
+           return currentValue == expectedValue;
+         } catch (Exception e) {
+           throw new UnhandledException("Test failed due to unexpected exception", e);
+         }
+       }
+     }, 1000, Integer.MAX_VALUE);
+   }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
index 50f98a3,0000000..c28bff8
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
@@@ -1,160 -1,0 +1,163 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hdfs;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.junit.Assert;
++import org.junit.Ignore;
 +import org.junit.Test;
 +
 +import java.io.IOException;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
 +import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
 +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
 +import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks;
 +
 +public class TestWriteStripedFileWithFailure {
 +  public static final Log LOG = LogFactory
 +      .getLog(TestWriteStripedFileWithFailure.class);
 +  private static MiniDFSCluster cluster;
 +  private static FileSystem fs;
 +  private static Configuration conf = new HdfsConfiguration();
 +  private final int smallFileLength = blockSize * dataBlocks - 123;
 +  private final int largeFileLength = blockSize * dataBlocks + 123;
 +  private final int[] fileLengths = {smallFileLength, largeFileLength};
 +
 +  public void setup() throws IOException {
 +    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
 +    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
 +    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
 +    fs = cluster.getFileSystem();
 +  }
 +
 +  public void tearDown() throws IOException {
 +    if (cluster != null) {
 +      cluster.shutdown();
 +    }
 +  }
 +
 +  // Test writing file with some Datanodes failure
++  // TODO: enable this test after HDFS-8704 and HDFS-9040
++  @Ignore
 +  @Test(timeout = 300000)
 +  public void testWriteStripedFileWithDNFailure() throws IOException {
 +    for (int fileLength : fileLengths) {
 +      for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) {
 +        for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) {
 +          try {
 +            // setup a new cluster with no dead datanode
 +            setup();
 +            writeFileWithDNFailure(fileLength, dataDelNum, parityDelNum);
 +          } catch (IOException ioe) {
 +            String fileType = fileLength < (blockSize * dataBlocks) ?
 +                "smallFile" : "largeFile";
 +            LOG.error("Failed to write file with DN failure:"
 +                + " fileType = "+ fileType
 +                + ", dataDelNum = " + dataDelNum
 +                + ", parityDelNum = " + parityDelNum);
 +            throw ioe;
 +          } finally {
 +            // tear down the cluster
 +            tearDown();
 +          }
 +        }
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Test writing a file with shutting down some DNs(data DNs or parity DNs or both).
 +   * @param fileLength file length
 +   * @param dataDNFailureNum the shutdown number of data DNs
 +   * @param parityDNFailureNum the shutdown number of parity DNs
 +   * @throws IOException
 +   */
 +  private void writeFileWithDNFailure(int fileLength,
 +      int dataDNFailureNum, int parityDNFailureNum) throws IOException {
 +    String fileType = fileLength < (blockSize * dataBlocks) ?
 +        "smallFile" : "largeFile";
 +    String src = "/dnFailure_" + dataDNFailureNum + "_" + parityDNFailureNum
 +        + "_" + fileType;
 +    LOG.info("writeFileWithDNFailure: file = " + src
 +        + ", fileType = " + fileType
 +        + ", dataDNFailureNum = " + dataDNFailureNum
 +        + ", parityDNFailureNum = " + parityDNFailureNum);
 +
 +    Path srcPath = new Path(src);
 +    final AtomicInteger pos = new AtomicInteger();
 +    final FSDataOutputStream out = fs.create(srcPath);
 +    final DFSStripedOutputStream stripedOut
 +        = (DFSStripedOutputStream)out.getWrappedStream();
 +
 +    int[] dataDNFailureIndices = StripedFileTestUtil.randomArray(0, dataBlocks,
 +        dataDNFailureNum);
 +    Assert.assertNotNull(dataDNFailureIndices);
 +    int[] parityDNFailureIndices = StripedFileTestUtil.randomArray(dataBlocks,
 +        dataBlocks + parityBlocks, parityDNFailureNum);
 +    Assert.assertNotNull(parityDNFailureIndices);
 +
 +    int[] failedDataNodes = new int[dataDNFailureNum + parityDNFailureNum];
 +    System.arraycopy(dataDNFailureIndices, 0, failedDataNodes,
 +        0, dataDNFailureIndices.length);
 +    System.arraycopy(parityDNFailureIndices, 0, failedDataNodes,
 +        dataDNFailureIndices.length, parityDNFailureIndices.length);
 +
 +    final int killPos = fileLength/2;
 +    for (; pos.get() < fileLength; ) {
 +      final int i = pos.getAndIncrement();
 +      if (i == killPos) {
 +        for(int failedDn : failedDataNodes) {
 +          StripedFileTestUtil.killDatanode(cluster, stripedOut, failedDn, pos);
 +        }
 +      }
 +      write(out, i);
 +    }
 +    out.close();
 +
 +    // make sure the expected number of Datanode have been killed
 +    int dnFailureNum = dataDNFailureNum + parityDNFailureNum;
 +    Assert.assertEquals(cluster.getDataNodes().size(), numDNs - dnFailureNum);
 +
 +    byte[] smallBuf = new byte[1024];
 +    byte[] largeBuf = new byte[fileLength + 100];
 +    final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
 +    StripedFileTestUtil.verifyLength(fs, srcPath, fileLength);
 +    StripedFileTestUtil.verifySeek(fs, srcPath, fileLength);
 +    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
 +        smallBuf);
 +    StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf);
 +
 +    // delete the file
 +    fs.delete(srcPath, true);
 +  }
 +
 +  void write(FSDataOutputStream out, int i) throws IOException {
 +    try {
 +      out.write(StripedFileTestUtil.getByte(i));
 +    } catch (IOException e) {
 +      throw new IOException("Failed at i=" + i, e);
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1080c373/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
index 0a27614,851e5b9..00a4575
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@@ -516,16 -489,16 +516,16 @@@ public class TestPBHelper 
    @Test
    public void testConvertLocatedBlock() {
      LocatedBlock lb = createLocatedBlock();
-     LocatedBlockProto lbProto = PBHelper.convertLocatedBlock(lb);
-     LocatedBlock lb2 = PBHelper.convertLocatedBlockProto(lbProto);
 -    LocatedBlockProto lbProto = PBHelperClient.convert(lb);
 -    LocatedBlock lb2 = PBHelperClient.convert(lbProto);
++    LocatedBlockProto lbProto = PBHelperClient.convertLocatedBlock(lb);
++    LocatedBlock lb2 = PBHelperClient.convertLocatedBlockProto(lbProto);
      compare(lb,lb2);
    }
  
    @Test
    public void testConvertLocatedBlockNoStorageMedia() {
      LocatedBlock lb = createLocatedBlockNoStorageMedia();
-     LocatedBlockProto lbProto = PBHelper.convertLocatedBlock(lb);
-     LocatedBlock lb2 = PBHelper.convertLocatedBlockProto(lbProto);
 -    LocatedBlockProto lbProto = PBHelperClient.convert(lb);
 -    LocatedBlock lb2 = PBHelperClient.convert(lbProto);
++    LocatedBlockProto lbProto = PBHelperClient.convertLocatedBlock(lb);
++    LocatedBlock lb2 = PBHelperClient.convertLocatedBlockProto(lbProto);
      compare(lb,lb2);
    }
  
@@@ -535,8 -508,8 +535,8 @@@
      for (int i=0;i<3;i++) {
        lbl.add(createLocatedBlock());
      }
-     List<LocatedBlockProto> lbpl = PBHelper.convertLocatedBlocks2(lbl);
-     List<LocatedBlock> lbl2 = PBHelper.convertLocatedBlocks(lbpl);
 -    List<LocatedBlockProto> lbpl = PBHelperClient.convertLocatedBlock2(lbl);
 -    List<LocatedBlock> lbl2 = PBHelperClient.convertLocatedBlock(lbpl);
++    List<LocatedBlockProto> lbpl = PBHelperClient.convertLocatedBlocks2(lbl);
++    List<LocatedBlock> lbl2 = PBHelperClient.convertLocatedBlocks(lbpl);
      assertEquals(lbl.size(), lbl2.size());
      for (int i=0;i<lbl.size();i++) {
        compare(lbl.get(i), lbl2.get(2));
@@@ -549,8 -522,8 +549,8 @@@
      for (int i=0;i<3;i++) {
        lbl[i] = createLocatedBlock();
      }
-     LocatedBlockProto [] lbpl = PBHelper.convertLocatedBlocks(lbl);
-     LocatedBlock [] lbl2 = PBHelper.convertLocatedBlocks(lbpl);
 -    LocatedBlockProto [] lbpl = PBHelperClient.convertLocatedBlock(lbl);
 -    LocatedBlock [] lbl2 = PBHelperClient.convertLocatedBlock(lbpl);
++    LocatedBlockProto [] lbpl = PBHelperClient.convertLocatedBlocks(lbl);
++    LocatedBlock [] lbl2 = PBHelperClient.convertLocatedBlocks(lbpl);
      assertEquals(lbl.length, lbl2.length);
      for (int i=0;i<lbl.length;i++) {
        compare(lbl[i], lbl2[i]);
@@@ -664,99 -637,6 +664,99 @@@
          .setType(AclEntryType.OTHER).build();
      AclStatus s = new AclStatus.Builder().owner("foo").group("bar").addEntry(e)
          .build();
-     Assert.assertEquals(s, PBHelper.convert(PBHelper.convert(s)));
+     Assert.assertEquals(s, PBHelperClient.convert(PBHelperClient.convert(s)));
    }
 +  
 +  @Test
 +  public void testBlockECRecoveryCommand() {
 +    DatanodeInfo[] dnInfos0 = new DatanodeInfo[] {
 +        DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
 +    DatanodeStorageInfo targetDnInfos_0 = BlockManagerTestUtil
 +        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
 +            new DatanodeStorage("s00"));
 +    DatanodeStorageInfo targetDnInfos_1 = BlockManagerTestUtil
 +        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
 +            new DatanodeStorage("s01"));
 +    DatanodeStorageInfo[] targetDnInfos0 = new DatanodeStorageInfo[] {
 +        targetDnInfos_0, targetDnInfos_1 };
 +    short[] liveBlkIndices0 = new short[2];
 +    BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo(
 +        new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0,
 +        liveBlkIndices0, ErasureCodingPolicyManager.getSystemDefaultPolicy());
 +    DatanodeInfo[] dnInfos1 = new DatanodeInfo[] {
 +        DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
 +    DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil
 +        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
 +            new DatanodeStorage("s02"));
 +    DatanodeStorageInfo targetDnInfos_3 = BlockManagerTestUtil
 +        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
 +            new DatanodeStorage("s03"));
 +    DatanodeStorageInfo[] targetDnInfos1 = new DatanodeStorageInfo[] {
 +        targetDnInfos_2, targetDnInfos_3 };
 +    short[] liveBlkIndices1 = new short[2];
 +    BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo(
 +        new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1,
 +        liveBlkIndices1, ErasureCodingPolicyManager.getSystemDefaultPolicy());
 +    List<BlockECRecoveryInfo> blkRecoveryInfosList = new ArrayList<BlockECRecoveryInfo>();
 +    blkRecoveryInfosList.add(blkECRecoveryInfo0);
 +    blkRecoveryInfosList.add(blkECRecoveryInfo1);
 +    BlockECRecoveryCommand blkECRecoveryCmd = new BlockECRecoveryCommand(
 +        DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY, blkRecoveryInfosList);
 +    BlockECRecoveryCommandProto blkECRecoveryCmdProto = PBHelper
 +        .convert(blkECRecoveryCmd);
 +    blkECRecoveryCmd = PBHelper.convert(blkECRecoveryCmdProto);
 +    Iterator<BlockECRecoveryInfo> iterator = blkECRecoveryCmd.getECTasks()
 +        .iterator();
 +    assertBlockECRecoveryInfoEquals(blkECRecoveryInfo0, iterator.next());
 +    assertBlockECRecoveryInfoEquals(blkECRecoveryInfo1, iterator.next());
 +  }
 +
 +  private void assertBlockECRecoveryInfoEquals(
 +      BlockECRecoveryInfo blkECRecoveryInfo1,
 +      BlockECRecoveryInfo blkECRecoveryInfo2) {
 +    assertEquals(blkECRecoveryInfo1.getExtendedBlock(),
 +        blkECRecoveryInfo2.getExtendedBlock());
 +
 +    DatanodeInfo[] sourceDnInfos1 = blkECRecoveryInfo1.getSourceDnInfos();
 +    DatanodeInfo[] sourceDnInfos2 = blkECRecoveryInfo2.getSourceDnInfos();
 +    assertDnInfosEqual(sourceDnInfos1, sourceDnInfos2);
 +
 +    DatanodeInfo[] targetDnInfos1 = blkECRecoveryInfo1.getTargetDnInfos();
 +    DatanodeInfo[] targetDnInfos2 = blkECRecoveryInfo2.getTargetDnInfos();
 +    assertDnInfosEqual(targetDnInfos1, targetDnInfos2);
 +
 +    String[] targetStorageIDs1 = blkECRecoveryInfo1.getTargetStorageIDs();
 +    String[] targetStorageIDs2 = blkECRecoveryInfo2.getTargetStorageIDs();
 +    assertEquals(targetStorageIDs1.length, targetStorageIDs2.length);
 +    for (int i = 0; i < targetStorageIDs1.length; i++) {
 +      assertEquals(targetStorageIDs1[i], targetStorageIDs2[i]);
 +    }
 +
 +    short[] liveBlockIndices1 = blkECRecoveryInfo1.getLiveBlockIndices();
 +    short[] liveBlockIndices2 = blkECRecoveryInfo2.getLiveBlockIndices();
 +    for (int i = 0; i < liveBlockIndices1.length; i++) {
 +      assertEquals(liveBlockIndices1[i], liveBlockIndices2[i]);
 +    }
 +    
 +    ErasureCodingPolicy ecPolicy1 = blkECRecoveryInfo1.getErasureCodingPolicy();
 +    ErasureCodingPolicy ecPolicy2 = blkECRecoveryInfo2.getErasureCodingPolicy();
 +    // Compare ECPolicies same as default ECPolicy as we used system default
 +    // ECPolicy used in this test
 +    compareECPolicies(ErasureCodingPolicyManager.getSystemDefaultPolicy(), ecPolicy1);
 +    compareECPolicies(ErasureCodingPolicyManager.getSystemDefaultPolicy(), ecPolicy2);
 +  }
 +
 +  private void compareECPolicies(ErasureCodingPolicy ecPolicy1, ErasureCodingPolicy ecPolicy2) {
 +    assertEquals(ecPolicy1.getName(), ecPolicy2.getName());
 +    assertEquals(ecPolicy1.getNumDataUnits(), ecPolicy2.getNumDataUnits());
 +    assertEquals(ecPolicy1.getNumParityUnits(), ecPolicy2.getNumParityUnits());
 +  }
 +
 +  private void assertDnInfosEqual(DatanodeInfo[] dnInfos1,
 +      DatanodeInfo[] dnInfos2) {
 +    assertEquals(dnInfos1.length, dnInfos2.length);
 +    for (int i = 0; i < dnInfos1.length; i++) {
 +      compare(dnInfos1[i], dnInfos2[i]);
 +    }
 +  }
  }


[17/50] [abbrv] hadoop git commit: HDFS-8937. Erasure coding: do not throw exception when setting replication factor on EC files. Contributed by Gao Rui.

Posted by wa...@apache.org.
HDFS-8937. Erasure coding: do not throw exception when setting replication factor on EC files. Contributed by Gao Rui.


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

Branch: refs/heads/trunk
Commit: ddf4e785475affead2f7c070b9f151de0fcb9024
Parents: 53358fe
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Sep 2 11:45:45 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Sep 2 11:45:45 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt  |  3 +++
 .../apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java   |  9 +++------
 .../org/apache/hadoop/hdfs/TestErasureCodingZones.java    | 10 ++--------
 3 files changed, 8 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ddf4e785/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 28cc34a..fb464bf 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -406,3 +406,6 @@
 
     HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC
     to use BlockUnderConstructionFeature. (Jing Zhao via waltersu4549)
+
+    HDFS-8937. Erasure coding: do not throw exception when setting replication on
+    EC file. (Gao Rui via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ddf4e785/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 4bed13e..46e172d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -405,15 +405,12 @@ public class FSDirAttrOp {
     final BlockManager bm = fsd.getBlockManager();
     final INodesInPath iip = fsd.getINodesInPath4Write(src, true);
     final INode inode = iip.getLastINode();
-    if (inode == null || !inode.isFile()) {
+    if (inode == null || !inode.isFile() || inode.asFile().isStriped()) {
+      // TODO we do not support replication on stripe layout files yet
       return null;
     }
-    INodeFile file = inode.asFile();
-    if (file.isStriped()) {
-      throw new UnsupportedActionException(
-          "Cannot set replication to a file with striped blocks");
-    }
 
+    INodeFile file = inode.asFile();
     // Make sure the directory has sufficient quotas
     short oldBR = file.getPreferredBlockReplication();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ddf4e785/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
index a878501..b68aab9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
@@ -166,14 +166,8 @@ public class TestErasureCodingZones {
     fs.create(fooFile, FsPermission.getFileDefault(), true,
         conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
         (short)0, fs.getDefaultBlockSize(fooFile), null);
-
-    try {
-      fs.setReplication(fooFile, (short) 3);
-      fail("Shouldn't allow to set replication to a file with striped blocks");
-    } catch (IOException e) {
-      assertExceptionContains(
-          "Cannot set replication to a file with striped blocks", e);
-    }
+    // set replication should be a no-op
+    fs.setReplication(fooFile, (short) 3);
   }
 
   @Test


[08/50] [abbrv] hadoop git commit: HDFS-8838. Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream when the data length is small. Contributed by Tsz Wo Nicholas Sze.

Posted by wa...@apache.org.
HDFS-8838. Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream when the data length is small. Contributed by Tsz Wo Nicholas Sze.


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

Branch: refs/heads/trunk
Commit: 067ec8c2b14fb0929dc348b763383838e06ff8a5
Parents: 6b6a63b
Author: Walter Su <wa...@apache.org>
Authored: Thu Aug 27 09:09:52 2015 +0800
Committer: Walter Su <wa...@apache.org>
Committed: Thu Aug 27 09:11:23 2015 +0800

----------------------------------------------------------------------
 .../apache/hadoop/util/ShutdownHookManager.java |   6 +
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |   6 +-
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  56 +++---
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  43 ++--
 .../apache/hadoop/hdfs/StripedDataStreamer.java | 138 ++++++++++---
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |   3 +
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |   2 +
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |   2 +-
 .../TestDFSStripedOutputStreamWithFailure.java  | 195 +++++++++++++++----
 ...estDFSStripedOutputStreamWithFailure000.java |  22 +++
 ...estDFSStripedOutputStreamWithFailure010.java |  22 +++
 12 files changed, 384 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
index 989c96a..85533db 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
@@ -187,4 +187,10 @@ public class ShutdownHookManager {
     return shutdownInProgress.get();
   }
 
+  /**
+   * clear all registered shutdownHooks.
+   */
+  public void clearShutdownHooks() {
+    hooks.clear();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index dad997a..8b25e68 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -400,3 +400,6 @@
 
     HDFS-8220. Erasure Coding: StripedDataStreamer fails to handle the
     blocklocations which doesn't satisfy BlockGroupSize. (Rakesh R via zhz)
+
+    HDFS-8838. Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream
+    when the data length is small. (szetszwo via waltersu4549)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 00f3a65..1654a26 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -406,13 +406,13 @@ public class DFSOutputStream extends FSOutputSummer
     if (currentPacket == null) {
       currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
           .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" +
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("WriteChunk allocating new packet seqno=" +
             currentPacket.getSeqno() +
             ", src=" + src +
             ", packetSize=" + packetSize +
             ", chunksPerPacket=" + chunksPerPacket +
-            ", bytesCurBlock=" + getStreamer().getBytesCurBlock());
+            ", bytesCurBlock=" + getStreamer().getBytesCurBlock() + ", " + this);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 4ca8fe6..d3a054a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -170,15 +170,18 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       }
 
       final boolean atBlockGroupBoundary = s0.getBytesCurBlock() == 0 && b0.getNumBytes() > 0;
+
       final ExtendedBlock block = new ExtendedBlock(b0);
-      long numBytes = b0.getNumBytes();
-      for (int i = 1; i < numDataBlocks; i++) {
+      long numBytes = atBlockGroupBoundary? b0.getNumBytes(): s0.getBytesCurBlock();
+      for (int i = 1; i < numAllBlocks; i++) {
         final StripedDataStreamer si = getStripedDataStreamer(i);
         final ExtendedBlock bi = si.getBlock();
         if (bi != null && bi.getGenerationStamp() > block.getGenerationStamp()) {
           block.setGenerationStamp(bi.getGenerationStamp());
         }
-        numBytes += atBlockGroupBoundary? bi.getNumBytes(): si.getBytesCurBlock();
+        if (i < numDataBlocks) {
+          numBytes += atBlockGroupBoundary? bi.getNumBytes(): si.getBytesCurBlock();
+        }
       }
       block.setNumBytes(numBytes);
       if (LOG.isDebugEnabled()) {
@@ -318,8 +321,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     return (StripedDataStreamer)streamer;
   }
 
-  private synchronized StripedDataStreamer setCurrentStreamer(int newIdx)
-      throws IOException {
+  private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) {
     // backup currentPacket for current streamer
     int oldIdx = streamers.indexOf(streamer);
     if (oldIdx >= 0) {
@@ -349,11 +351,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   }
 
 
-  private void checkStreamers() throws IOException {
+  private void checkStreamers(boolean setExternalError) throws IOException {
     int count = 0;
     for(StripedDataStreamer s : streamers) {
       if (!s.isFailed()) {
-        if (s.getBlock() != null) {
+        if (setExternalError && s.getBlock() != null) {
           s.getErrorState().initExternalError();
         }
         count++;
@@ -369,11 +371,16 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     }
   }
 
-  private void handleStreamerFailure(String err,
-                                     Exception e) throws IOException {
+  private void handleStreamerFailure(String err, Exception e)
+      throws IOException {
+    handleStreamerFailure(err, e, true);
+  }
+
+  private void handleStreamerFailure(String err, Exception e,
+      boolean setExternalError) throws IOException {
     LOG.warn("Failed: " + err + ", " + this, e);
     getCurrentStreamer().setFailed(true);
-    checkStreamers();
+    checkStreamers(setExternalError);
     currentPacket = null;
   }
 
@@ -505,10 +512,10 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     return sum;
   }
 
-  private void writeParityCellsForLastStripe() throws IOException {
+  private boolean generateParityCellsForLastStripe() {
     final long currentBlockGroupBytes = getCurrentSumBytes();
     if (currentBlockGroupBytes % stripeDataSize() == 0) {
-      return;
+      return false;
     }
 
     final int firstCellSize =
@@ -530,8 +537,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       }
       buffers[i].flip();
     }
-
-    writeParityCells();
+    return true;
   }
 
   void writeParityCells() throws IOException {
@@ -603,12 +609,14 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       // flush from all upper layers
       try {
         flushBuffer();
-        // if the last stripe is incomplete, generate and write parity cells
-        writeParityCellsForLastStripe();
-        enqueueAllCurrentPackets();
       } catch(Exception e) {
-        handleStreamerFailure("closeImpl", e);
+        handleStreamerFailure("flushBuffer " + getCurrentStreamer(), e);
+      }
+      // if the last stripe is incomplete, generate and write parity cells
+      if (generateParityCellsForLastStripe()) {
+        writeParityCells();
       }
+      enqueueAllCurrentPackets();
 
       for (int i = 0; i < numAllBlocks; i++) {
         final StripedDataStreamer s = setCurrentStreamer(i);
@@ -620,7 +628,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
             // flush all data to Datanode
             flushInternal();
           } catch(Exception e) {
-            handleStreamerFailure("closeImpl", e);
+            handleStreamerFailure("flushInternal " + s, e, false);
           }
         }
       }
@@ -643,9 +651,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   private void enqueueAllCurrentPackets() throws IOException {
     int idx = streamers.indexOf(getCurrentStreamer());
     for(int i = 0; i < streamers.size(); i++) {
-      setCurrentStreamer(i);
-      if (currentPacket != null) {
-        enqueueCurrentPacket();
+      final StripedDataStreamer si = setCurrentStreamer(i);
+      if (!si.isFailed() && currentPacket != null) {
+        try {
+          enqueueCurrentPacket();
+        } catch (IOException e) {
+          handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e, false);
+        }
       }
     }
     setCurrentStreamer(idx);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index c78199e..bbcdd1f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -173,7 +173,7 @@ class DataStreamer extends Daemon {
     packets.clear();
   }
   
-  static class LastExceptionInStreamer {
+  class LastExceptionInStreamer {
     private IOException thrown;
 
     synchronized void set(Throwable t) {
@@ -191,7 +191,8 @@ class DataStreamer extends Daemon {
       if (thrown != null) {
         if (LOG.isTraceEnabled()) {
           // wrap and print the exception to know when the check is called
-          LOG.trace("Got Exception while checking", new Throwable(thrown));
+          LOG.trace("Got Exception while checking, " + DataStreamer.this,
+              new Throwable(thrown));
         }
         final IOException e = thrown;
         if (resetToNull) {
@@ -584,16 +585,13 @@ class DataStreamer extends Daemon {
         }
 
         // get new block from namenode.
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("stage=" + stage + ", " + this);
+        }
         if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Allocating new block " + this);
-          }
           setPipeline(nextBlockOutputStream());
           initDataStreaming();
         } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Append to block " + block);
-          }
           setupPipelineForAppendOrRecovery();
           if (streamerClosed) {
             continue;
@@ -639,8 +637,7 @@ class DataStreamer extends Daemon {
         }
 
         if (LOG.isDebugEnabled()) {
-          LOG.debug("DataStreamer block " + block +
-              " sending packet " + one);
+          LOG.debug(this + " sending " + one);
         }
 
         // write out data to remote datanode
@@ -1426,16 +1423,21 @@ class DataStreamer extends Daemon {
   /** update pipeline at the namenode */
   ExtendedBlock updatePipeline(long newGS) throws IOException {
     final ExtendedBlock newBlock = newBlock(block, newGS);
-    return callUpdatePipeline(block, newBlock);
+    return callUpdatePipeline(block, newBlock, nodes, storageIDs);
   }
 
-  ExtendedBlock callUpdatePipeline(ExtendedBlock oldBlock, ExtendedBlock newBlock)
+  ExtendedBlock callUpdatePipeline(ExtendedBlock oldBlock, ExtendedBlock newBlock,
+      DatanodeInfo[] newNodes, String[] newStorageIDs)
       throws IOException {
     dfsClient.namenode.updatePipeline(dfsClient.clientName, oldBlock, newBlock,
-        nodes, storageIDs);
+        newNodes, newStorageIDs);
     return newBlock;
   }
 
+  int getNumBlockWriteRetry() {
+    return dfsClient.getConf().getNumBlockWriteRetry();
+  }
+
   /**
    * Open a DataStreamer to a DataNode so that it can be written to.
    * This happens when a file is created and each time a new block is allocated.
@@ -1446,7 +1448,7 @@ class DataStreamer extends Daemon {
     LocatedBlock lb = null;
     DatanodeInfo[] nodes = null;
     StorageType[] storageTypes = null;
-    int count = dfsClient.getConf().getNumBlockWriteRetry();
+    int count = getNumBlockWriteRetry();
     boolean success = false;
     ExtendedBlock oldBlock = block;
     do {
@@ -1502,7 +1504,7 @@ class DataStreamer extends Daemon {
     String firstBadLink = "";
     boolean checkRestart = false;
     if (LOG.isDebugEnabled()) {
-      LOG.debug("pipeline = " + Arrays.asList(nodes));
+      LOG.debug("pipeline = " + Arrays.toString(nodes) + ", " + this);
     }
 
     // persist blocks on namenode on next flush
@@ -1574,7 +1576,7 @@ class DataStreamer extends Daemon {
         errorState.reset();
       } catch (IOException ie) {
         if (!errorState.isRestartingNode()) {
-          LOG.info("Exception in createBlockOutputStream", ie);
+          LOG.info("Exception in createBlockOutputStream " + this, ie);
         }
         if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
           LOG.info("Will fetch a new encryption key and retry, "
@@ -1649,7 +1651,7 @@ class DataStreamer extends Daemon {
     }
   }
 
-  protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
+  LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
       throws IOException {
     final DfsClientConf conf = dfsClient.getConf(); 
     int retries = conf.getNumBlockWriteLocateFollowingRetry();
@@ -1755,6 +1757,10 @@ class DataStreamer extends Daemon {
     return nodes;
   }
 
+  String[] getStorageIDs() {
+    return storageIDs;
+  }
+
   /**
    * return the token of the block
    *
@@ -1933,7 +1939,6 @@ class DataStreamer extends Daemon {
 
   @Override
   public String toString() {
-    return  (block == null? null: block.getLocalBlock())
-        + "@" + Arrays.toString(getNodes());
+    return block == null? "block==null": "" + block.getLocalBlock();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
index f533bf9..a20caa5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator;
@@ -39,6 +40,8 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * This class extends {@link DataStreamer} to support writing striped blocks
  * to datanodes.
@@ -58,13 +61,13 @@ public class StripedDataStreamer extends DataStreamer {
    * @param <T> the queue entry type.
    */
   static abstract class ConcurrentPoll<T> {
-    private final MultipleBlockingQueue<T> queue;
+    final MultipleBlockingQueue<T> queue;
 
     ConcurrentPoll(MultipleBlockingQueue<T> queue) {
       this.queue = queue;
     }
 
-    T poll(final int i) throws IOException {
+    T poll(final int i) throws InterruptedIOException {
       for(;;) {
         synchronized(queue) {
           final T polled = queue.poll(i);
@@ -72,18 +75,17 @@ public class StripedDataStreamer extends DataStreamer {
             return polled;
           }
           if (isReady2Populate()) {
-            populate();
-            return queue.poll(i);
+            try {
+              populate();
+              return queue.poll(i);
+            } catch(IOException ioe) {
+              LOG.warn("Failed to populate, " + this, ioe);
+            }
           }
         }
 
         // sleep and then retry.
-        try {
-          Thread.sleep(100);
-        } catch(InterruptedException ie) {
-          throw DFSUtil.toInterruptedIOException(
-              "Sleep interrupted during poll", ie);
-        }
+        sleep(100, "poll");
       }
     }
 
@@ -94,6 +96,15 @@ public class StripedDataStreamer extends DataStreamer {
     abstract void populate() throws IOException;
   }
 
+  private static void sleep(long ms, String op) throws InterruptedIOException {
+    try {
+      Thread.sleep(ms);
+    } catch(InterruptedException ie) {
+      throw DFSUtil.toInterruptedIOException(
+          "Sleep interrupted during " + op, ie);
+    }
+  }
+
   private final Coordinator coordinator;
   private final int index;
   private volatile boolean failed;
@@ -135,11 +146,14 @@ public class StripedDataStreamer extends DataStreamer {
   }
 
   @Override
-  protected LocatedBlock locateFollowingBlock(final DatanodeInfo[] excludedNodes)
+  int getNumBlockWriteRetry() {
+    return 0;
+  }
+
+  @Override
+  LocatedBlock locateFollowingBlock(final DatanodeInfo[] excludedNodes)
       throws IOException {
-    final MultipleBlockingQueue<LocatedBlock> followingBlocks
-        = coordinator.getFollowingBlocks();
-    return new ConcurrentPoll<LocatedBlock>(followingBlocks) {
+    return new ConcurrentPoll<LocatedBlock>(coordinator.getFollowingBlocks()) {
       @Override
       boolean isReady2Populate() {
         return super.isReady2Populate()
@@ -194,18 +208,24 @@ public class StripedDataStreamer extends DataStreamer {
             si.endBlock();
             si.close(true);
           } else {
-            followingBlocks.offer(i, blocks[i]);
+            queue.offer(i, blocks[i]);
           }
         }
       }
     }.poll(index);
   }
 
+  @VisibleForTesting
+  LocatedBlock peekFollowingBlock() {
+    return coordinator.getFollowingBlocks().peek(index);
+  }
+
   @Override
   LocatedBlock updateBlockForPipeline() throws IOException {
-    final MultipleBlockingQueue<LocatedBlock> newBlocks
-        = coordinator.getNewBlocks();
-    return new ConcurrentPoll<LocatedBlock>(newBlocks) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("updateBlockForPipeline(), " + this);
+    }
+    return new ConcurrentPoll<LocatedBlock>(coordinator.getNewBlocks()) {
       @Override
       void populate() throws IOException {
         final ExtendedBlock bg = coordinator.getBlockGroup();
@@ -224,10 +244,22 @@ public class StripedDataStreamer extends DataStreamer {
             final LocatedBlock lb = new LocatedBlock(newBlock(bi, newGS),
                 null, null, null, -1, updated.isCorrupt(), null);
             lb.setBlockToken(updatedBlks[i].getBlockToken());
-            newBlocks.offer(i, lb);
+            queue.offer(i, lb);
           } else {
-            final LocatedBlock lb = coordinator.getFollowingBlocks().peek(i);
-            lb.getBlock().setGenerationStamp(newGS);
+            final MultipleBlockingQueue<LocatedBlock> followingBlocks
+                = coordinator.getFollowingBlocks();
+            synchronized(followingBlocks) {
+              final LocatedBlock lb = followingBlocks.peek(i);
+              if (lb != null) {
+                lb.getBlock().setGenerationStamp(newGS);
+                si.getErrorState().reset();
+                continue;
+              }
+            }
+
+            //streamer i just have polled the block, sleep and retry.
+            sleep(100, "updateBlockForPipeline, " + this);
+            i--;
           }
         }
       }
@@ -236,21 +268,64 @@ public class StripedDataStreamer extends DataStreamer {
 
   @Override
   ExtendedBlock updatePipeline(final long newGS) throws IOException {
-    final MultipleBlockingQueue<ExtendedBlock> updateBlocks
-        = coordinator.getUpdateBlocks();
-    return new ConcurrentPoll<ExtendedBlock>(updateBlocks) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("updatePipeline(newGS=" + newGS + "), " + this);
+    }
+    return new ConcurrentPoll<ExtendedBlock>(coordinator.getUpdateBlocks()) {
       @Override
       void populate() throws IOException {
+        final MultipleBlockingQueue<LocatedBlock> followingBlocks
+            = coordinator.getFollowingBlocks();
         final ExtendedBlock bg = coordinator.getBlockGroup();
         final ExtendedBlock newBG = newBlock(bg, newGS);
-        final ExtendedBlock updated = callUpdatePipeline(bg, newBG);
-        for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) {
-          StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
-          if (si.isFailed()) {
-            continue; // skipping failed data streamer
+
+        final int n = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
+        final DatanodeInfo[] newNodes = new DatanodeInfo[n];
+        final String[] newStorageIDs = new String[n];
+        for (int i = 0; i < n; i++) {
+          final StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
+          DatanodeInfo[] nodes = si.getNodes();
+          String[] storageIDs = si.getStorageIDs();
+          if (nodes == null || storageIDs == null) {
+            synchronized(followingBlocks) {
+              final LocatedBlock lb = followingBlocks.peek(i);
+              if (lb != null) {
+                nodes = lb.getLocations();
+                storageIDs = lb.getStorageIDs();
+              }
+            }
+          }
+          if (nodes != null && storageIDs != null) {
+            newNodes[i] = nodes[0];
+            newStorageIDs[i] = storageIDs[0];
+          } else {
+            //streamer i just have polled the block, sleep and retry.
+            sleep(100, "updatePipeline, " + this);
+            i--;
           }
+        }
+        final ExtendedBlock updated = callUpdatePipeline(bg, newBG, newNodes,
+            newStorageIDs);
+
+        for (int i = 0; i < n; i++) {
+          final StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
           final ExtendedBlock bi = si.getBlock();
-          updateBlocks.offer(i, newBlock(bi, updated.getGenerationStamp()));
+          if (bi != null) {
+            queue.offer(i, newBlock(bi, updated.getGenerationStamp()));
+          } else if (!si.isFailed()) {
+            synchronized(followingBlocks) {
+              final LocatedBlock lb = followingBlocks.peek(i);
+              if (lb != null) {
+                lb.getBlock().setGenerationStamp(newGS);
+                si.getErrorState().reset();
+                continue;
+              }
+            }
+
+            //streamer i just have polled the block, sleep and retry.
+            sleep(100, "updatePipeline, " + this);
+            i--;
+          }
         }
       }
     }.poll(index);
@@ -258,7 +333,6 @@ public class StripedDataStreamer extends DataStreamer {
 
   @Override
   public String toString() {
-    return "#" + index + ": failed? " + Boolean.toString(failed).charAt(0)
-        + ", " + super.toString();
+    return "#" + index + ": " + (failed? "failed, ": "") + super.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index b9466f6..ffd8fbc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -134,6 +134,9 @@ class FSDirWriteFileOp {
     FSNamesystem fsn = fsd.getFSNamesystem();
     final INodeFile file = fsn.checkLease(src, holder, inode, fileId);
     Preconditions.checkState(file.isUnderConstruction());
+    if (file.isStriped()) {
+      return; // do not abandon block for striped file
+    }
 
     Block localBlock = ExtendedBlock.getLocalBlock(b);
     fsd.writeLock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 65e26df..59daba4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -120,6 +120,7 @@ import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.ToolRunner;
 
 import com.google.common.base.Joiner;
@@ -1867,6 +1868,7 @@ public class MiniDFSCluster {
         nameNode = null;
       }
     }
+    ShutdownHookManager.get().clearShutdownHooks();
     if (base_dir != null) {
       if (deleteDfsDir) {
         base_dir.delete();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index 5cab978..35e7e6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
@@ -169,6 +168,7 @@ public class TestDFSStripedOutputStream {
   }
 
   private void testOneFile(String src, int writeBytes) throws IOException {
+    src += "_" + writeBytes;
     Path testPath = new Path(src);
 
     byte[] bytes = generateBytes(writeBytes);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index f65d0c7..e8e562b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
@@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;
@@ -71,6 +73,38 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   private static final int FLUSH_POS
       = 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
+  static {
+    System.out.println("NUM_DATA_BLOCKS  = " + NUM_DATA_BLOCKS);
+    System.out.println("NUM_PARITY_BLOCKS= " + NUM_PARITY_BLOCKS);
+    System.out.println("CELL_SIZE        = " + CELL_SIZE
+        + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(CELL_SIZE, "B", 2) + ")");
+    System.out.println("BLOCK_SIZE       = " + BLOCK_SIZE
+        + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(BLOCK_SIZE, "B", 2) + ")");
+    System.out.println("BLOCK_GROUP_SIZE = " + BLOCK_GROUP_SIZE
+        + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(BLOCK_GROUP_SIZE, "B", 2) + ")");
+  }
+
+  static List<Integer> newLengths() {
+    final List<Integer> lengths = new ArrayList<>();
+    lengths.add(FLUSH_POS + 2);
+    for(int b = 0; b <= 2; b++) {
+      for(int c = 0; c < STRIPES_PER_BLOCK*NUM_DATA_BLOCKS; c++) {
+        for(int delta = -1; delta <= 1; delta++) {
+          final int length = b*BLOCK_GROUP_SIZE + c*CELL_SIZE + delta;
+          System.out.println(lengths.size() + ": length=" + length
+              + ", (b, c, d) = (" + b + ", " + c + ", " + delta + ")");
+          lengths.add(length);
+        }
+      }
+    }
+    return lengths;
+  }
+
+  private static final List<Integer> LENGTHS = newLengths();
+
+  static int getLength(int i) {
+    return LENGTHS.get(i);
+  }
 
   private MiniDFSCluster cluster;
   private DistributedFileSystem dfs;
@@ -96,32 +130,25 @@ public class TestDFSStripedOutputStreamWithFailure {
     return (byte)pos;
   }
 
-  private void initConf(Configuration conf){
+  private HdfsConfiguration newHdfsConfiguration() {
+    final HdfsConfiguration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    return conf;
   }
 
-  private void initConfWithBlockToken(Configuration conf) {
-    conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
-    conf.setInt("ipc.client.connect.max.retries", 0);
-    // Set short retry timeouts so this test runs faster
-    conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
-  }
-
-  @Test(timeout=240000)
-  public void testDatanodeFailure() throws Exception {
-    final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
-    HdfsConfiguration conf = new HdfsConfiguration();
-    initConf(conf);
+  void runTest(final int length) {
+    final HdfsConfiguration conf = newHdfsConfiguration();
     for (int dn = 0; dn < 9; dn++) {
       try {
         setup(conf);
-        cluster.startDataNodes(conf, 1, true, null, null);
-        cluster.waitActive();
-        runTest(new Path(dir, "file" + dn), length, length / 2, dn, false);
+        runTest(length, dn, false, conf);
       } catch (Exception e) {
-        LOG.error("failed, dn=" + dn + ", length=" + length);
-        throw e;
+        final String err = "failed, dn=" + dn + ", length=" + length
+            + StringUtils.stringifyException(e);
+        LOG.error(err);
+        Assert.fail(err);
       } finally {
         tearDown();
       }
@@ -129,17 +156,23 @@ public class TestDFSStripedOutputStreamWithFailure {
   }
 
   @Test(timeout=240000)
+  public void testDatanodeFailure56() throws Exception {
+    runTest(getLength(56));
+  }
+
+  @Test(timeout=240000)
   public void testBlockTokenExpired() throws Exception {
     final int length = NUM_DATA_BLOCKS * (BLOCK_SIZE - CELL_SIZE);
-    HdfsConfiguration conf = new HdfsConfiguration();
-    initConf(conf);
-    initConfWithBlockToken(conf);
+    final HdfsConfiguration conf = newHdfsConfiguration();
+
+    conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
+    conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
+    // Set short retry timeouts so this test runs faster
+    conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
     for (int dn = 0; dn < 9; dn += 2) {
       try {
         setup(conf);
-        cluster.startDataNodes(conf, 1, true, null, null);
-        cluster.waitActive();
-        runTest(new Path(dir, "file" + dn), length, length / 2, dn, true);
+        runTest(length, dn, true, conf);
       } catch (Exception e) {
         LOG.error("failed, dn=" + dn + ", length=" + length);
         throw e;
@@ -229,19 +262,41 @@ public class TestDFSStripedOutputStreamWithFailure {
     }
   }
 
-  private void runTest(final Path p, final int length, final int killPos,
-      final int dnIndex, final boolean tokenExpire) throws Exception {
-    LOG.info("p=" + p + ", length=" + length + ", killPos=" + killPos
-        + ", dnIndex=" + dnIndex);
-    Preconditions.checkArgument(killPos < length);
-    Preconditions.checkArgument(killPos > FLUSH_POS);
+  private void runTest(final int length, final int dnIndex,
+      final boolean tokenExpire, final HdfsConfiguration conf) {
+    try {
+      runTest(length, length/2, dnIndex, tokenExpire, conf);
+    } catch(Exception e) {
+      LOG.info("FAILED", e);
+      Assert.fail(StringUtils.stringifyException(e));
+    }
+  }
+
+  private void runTest(final int length, final int killPos,
+      final int dnIndex, final boolean tokenExpire,
+      final HdfsConfiguration conf) throws Exception {
+    if (killPos <= FLUSH_POS) {
+      LOG.warn("killPos=" + killPos + " <= FLUSH_POS=" + FLUSH_POS
+          + ", length=" + length + ", dnIndex=" + dnIndex);
+      return; //skip test
+    }
+    Preconditions.checkArgument(length > killPos,
+        "length=%s <= killPos=%s", length, killPos);
+
+    // start a datanode now, will kill one later
+    cluster.startDataNodes(conf, 1, true, null, null);
+    cluster.waitActive();
+
+    final Path p = new Path(dir, "dn" + dnIndex + "len" + length + "kill" +  killPos);
     final String fullPath = p.toString();
+    LOG.info("fullPath=" + fullPath);
 
-    final NameNode nn = cluster.getNameNode();
-    final BlockManager bm = nn.getNamesystem().getBlockManager();
-    final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
 
     if (tokenExpire) {
+      final NameNode nn = cluster.getNameNode();
+      final BlockManager bm = nn.getNamesystem().getBlockManager();
+      final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
+
       // set a short token lifetime (1 second)
       SecurityTestUtil.setBlockTokenLifetime(sm, 1000L);
     }
@@ -265,7 +320,7 @@ public class TestDFSStripedOutputStreamWithFailure {
           waitTokenExpires(out);
         }
 
-        StripedFileTestUtil.killDatanode(cluster, stripedOut, dnIndex, pos);
+        killDatanode(cluster, stripedOut, dnIndex, pos);
         killed = true;
       }
 
@@ -301,6 +356,40 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   }
 
+  static DatanodeInfo getDatanodes(StripedDataStreamer streamer) {
+    for(;;) {
+      DatanodeInfo[] datanodes = streamer.getNodes();
+      if (datanodes == null) {
+        // try peeking following block.
+        final LocatedBlock lb = streamer.peekFollowingBlock();
+        if (lb != null) {
+          datanodes = lb.getLocations();
+        }
+      }
+
+      if (datanodes != null) {
+        Assert.assertEquals(1, datanodes.length);
+        Assert.assertNotNull(datanodes[0]);
+        return datanodes[0];
+      }
+
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException ie) {
+        Assert.fail(StringUtils.stringifyException(ie));
+        return null;
+      }
+    }
+  }
+
+  static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out,
+      final int dnIndex, final AtomicInteger pos) {
+    final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex);
+    final DatanodeInfo datanode = getDatanodes(s);
+    LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos);
+    cluster.stopDataNode(datanode.getXferAddr());
+  }
+
   static void checkData(DistributedFileSystem dfs, String src, int length,
       int killedDnIndex, long oldGS) throws IOException {
     List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
@@ -314,7 +403,7 @@ public class TestDFSStripedOutputStreamWithFailure {
       final long gs = firstBlock.getBlock().getGenerationStamp();
       final String s = "gs=" + gs + ", oldGS=" + oldGS;
       LOG.info(s);
-      Assert.assertTrue(s, gs > oldGS);
+      Assert.assertTrue(s, gs >= oldGS);
 
       LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
           (LocatedStripedBlock) firstBlock,
@@ -342,7 +431,7 @@ public class TestDFSStripedOutputStreamWithFailure {
         final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS
             + (j <= lastCellIndex? 1: 0);
         final int blockSize = numCellInBlock*CELL_SIZE
-            + (isLastGroup && i == lastCellIndex? lastCellSize - CELL_SIZE: 0);
+            + (isLastGroup && j == lastCellIndex? lastCellSize - CELL_SIZE: 0);
 
         final byte[] blockBytes = new byte[blockSize];
         if (i < NUM_DATA_BLOCKS) {
@@ -352,7 +441,8 @@ public class TestDFSStripedOutputStreamWithFailure {
         }
 
         final LocatedBlock lb = blockList.get(i);
-        LOG.info("XXX i=" + i + ", lb=" + lb);
+        LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock
+            + ", blockSize=" + blockSize + ", lb=" + lb);
         if (lb == null) {
           continue;
         }
@@ -410,4 +500,35 @@ public class TestDFSStripedOutputStreamWithFailure {
       }
     }
   }
+
+  public static abstract class TestBase {
+    static final long TIMEOUT = 240000;
+
+    int getBase() {
+      final String name = getClass().getSimpleName();
+      int i = name.length() - 1;
+      for(; i >= 0 && Character.isDigit(name.charAt(i)); i--);
+      return Integer.parseInt(name.substring(i + 1));
+    }
+
+    private final TestDFSStripedOutputStreamWithFailure test
+        = new TestDFSStripedOutputStreamWithFailure();
+    private void run(int offset) {
+      final int i = offset + getBase();
+      final int length = getLength(i);
+      System.out.println("Run test " + i + ", length=" + length);
+      test.runTest(length);
+    }
+
+    @Test(timeout=TIMEOUT) public void test0() {run(0);}
+    @Test(timeout=TIMEOUT) public void test1() {run(1);}
+    @Test(timeout=TIMEOUT) public void test2() {run(2);}
+    @Test(timeout=TIMEOUT) public void test3() {run(3);}
+    @Test(timeout=TIMEOUT) public void test4() {run(4);}
+    @Test(timeout=TIMEOUT) public void test5() {run(5);}
+    @Test(timeout=TIMEOUT) public void test6() {run(6);}
+    @Test(timeout=TIMEOUT) public void test7() {run(7);}
+    @Test(timeout=TIMEOUT) public void test8() {run(8);}
+    @Test(timeout=TIMEOUT) public void test9() {run(9);}
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java
new file mode 100644
index 0000000..b4fb1b8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.TestDFSStripedOutputStreamWithFailure.TestBase;
+
+public class TestDFSStripedOutputStreamWithFailure000 extends TestBase {}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/067ec8c2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java
new file mode 100644
index 0000000..8489c3d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.TestDFSStripedOutputStreamWithFailure.TestBase;
+
+public class TestDFSStripedOutputStreamWithFailure010 extends TestBase {}
\ No newline at end of file


[41/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
index 0000000,0000000..264c532
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@@ -1,0 -1,0 +1,952 @@@
++/**
++ * 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.hdfs.util;
++
++import com.google.common.annotations.VisibleForTesting;
++
++import org.apache.hadoop.classification.InterfaceAudience;
++import org.apache.hadoop.fs.StorageType;
++import org.apache.hadoop.hdfs.DFSClient;
++import org.apache.hadoop.hdfs.DFSStripedOutputStream;
++import org.apache.hadoop.hdfs.protocol.Block;
++import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
++import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
++import org.apache.hadoop.hdfs.protocol.LocatedBlock;
++import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
++
++import com.google.common.base.Preconditions;
++import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
++import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
++import org.apache.hadoop.security.token.Token;
++
++import java.nio.ByteBuffer;
++import java.util.*;
++import java.io.IOException;
++import java.util.concurrent.CancellationException;
++import java.util.concurrent.CompletionService;
++import java.util.concurrent.ExecutionException;
++import java.util.concurrent.Future;
++import java.util.concurrent.TimeUnit;
++
++/**
++ * When accessing a file in striped layout, operations on logical byte ranges
++ * in the file need to be mapped to physical byte ranges on block files stored
++ * on DataNodes. This utility class facilities this mapping by defining and
++ * exposing a number of striping-related concepts. The most basic ones are
++ * illustrated in the following diagram. Unless otherwise specified, all
++ * range-related calculations are inclusive (the end offset of the previous
++ * range should be 1 byte lower than the start offset of the next one).
++ *
++ *  | <----  Block Group ----> |   <- Block Group: logical unit composing
++ *  |                          |        striped HDFS files.
++ *  blk_0      blk_1       blk_2   <- Internal Blocks: each internal block
++ *    |          |           |          represents a physically stored local
++ *    v          v           v          block file
++ * +------+   +------+   +------+
++ * |cell_0|   |cell_1|   |cell_2|  <- {@link StripingCell} represents the
++ * +------+   +------+   +------+       logical order that a Block Group should
++ * |cell_3|   |cell_4|   |cell_5|       be accessed: cell_0, cell_1, ...
++ * +------+   +------+   +------+
++ * |cell_6|   |cell_7|   |cell_8|
++ * +------+   +------+   +------+
++ * |cell_9|
++ * +------+  <- A cell contains cellSize bytes of data
++ */
++@InterfaceAudience.Private
++public class StripedBlockUtil {
++
++  /**
++   * This method parses a striped block group into individual blocks.
++   *
++   * @param bg The striped block group
++   * @param cellSize The size of a striping cell
++   * @param dataBlkNum The number of data blocks
++   * @return An array containing the blocks in the group
++   */
++  public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg,
++      int cellSize, int dataBlkNum, int parityBlkNum) {
++    int locatedBGSize = bg.getBlockIndices().length;
++    LocatedBlock[] lbs = new LocatedBlock[dataBlkNum + parityBlkNum];
++    for (short i = 0; i < locatedBGSize; i++) {
++      final int idx = bg.getBlockIndices()[i];
++      // for now we do not use redundant replica of an internal block
++      if (idx < (dataBlkNum + parityBlkNum) && lbs[idx] == null) {
++        lbs[idx] = constructInternalBlock(bg, i, cellSize,
++            dataBlkNum, idx);
++      }
++    }
++    return lbs;
++  }
++
++  /**
++   * This method creates an internal block at the given index of a block group
++   *
++   * @param idxInReturnedLocs The index in the stored locations in the
++   *                          {@link LocatedStripedBlock} object
++   * @param idxInBlockGroup The logical index in the striped block group
++   * @return The constructed internal block
++   */
++  public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg,
++      int idxInReturnedLocs, int cellSize, int dataBlkNum,
++      int idxInBlockGroup) {
++    final ExtendedBlock blk = constructInternalBlock(
++        bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup);
++    final LocatedBlock locatedBlock;
++    if (idxInReturnedLocs < bg.getLocations().length) {
++      locatedBlock = new LocatedBlock(blk,
++          new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
++          new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
++          new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
++          bg.getStartOffset(), bg.isCorrupt(), null);
++    } else {
++      locatedBlock = new LocatedBlock(blk, null, null, null,
++          bg.getStartOffset(), bg.isCorrupt(), null);
++    }
++    Token<BlockTokenIdentifier>[] blockTokens = bg.getBlockTokens();
++    if (idxInReturnedLocs < blockTokens.length) {
++      locatedBlock.setBlockToken(blockTokens[idxInReturnedLocs]);
++    }
++    return locatedBlock;
++  }
++
++  /**
++   * This method creates an internal {@link ExtendedBlock} at the given index
++   * of a block group.
++   */
++  public static ExtendedBlock constructInternalBlock(ExtendedBlock blockGroup,
++      int cellSize, int dataBlkNum, int idxInBlockGroup) {
++    ExtendedBlock block = new ExtendedBlock(blockGroup);
++    block.setBlockId(blockGroup.getBlockId() + idxInBlockGroup);
++    block.setNumBytes(getInternalBlockLength(blockGroup.getNumBytes(),
++        cellSize, dataBlkNum, idxInBlockGroup));
++    return block;
++  }
++
++  /**
++   * Get the size of an internal block at the given index of a block group
++   *
++   * @param dataSize Size of the block group only counting data blocks
++   * @param cellSize The size of a striping cell
++   * @param numDataBlocks The number of data blocks
++   * @param i The logical index in the striped block group
++   * @return The size of the internal block at the specified index
++   */
++  public static long getInternalBlockLength(long dataSize,
++      int cellSize, int numDataBlocks, int i) {
++    Preconditions.checkArgument(dataSize >= 0);
++    Preconditions.checkArgument(cellSize > 0);
++    Preconditions.checkArgument(numDataBlocks > 0);
++    Preconditions.checkArgument(i >= 0);
++    // Size of each stripe (only counting data blocks)
++    final int stripeSize = cellSize * numDataBlocks;
++    // If block group ends at stripe boundary, each internal block has an equal
++    // share of the group
++    final int lastStripeDataLen = (int)(dataSize % stripeSize);
++    if (lastStripeDataLen == 0) {
++      return dataSize / numDataBlocks;
++    }
++
++    final int numStripes = (int) ((dataSize - 1) / stripeSize + 1);
++    return (numStripes - 1L)*cellSize
++        + lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i);
++  }
++
++  private static int lastCellSize(int size, int cellSize, int numDataBlocks,
++      int i) {
++    if (i < numDataBlocks) {
++      // parity block size (i.e. i >= numDataBlocks) is the same as 
++      // the first data block size (i.e. i = 0).
++      size -= i*cellSize;
++      if (size < 0) {
++        size = 0;
++      }
++    }
++    return size > cellSize? cellSize: size;
++  }
++
++  /**
++   * Given a byte's offset in an internal block, calculate the offset in
++   * the block group
++   */
++  public static long offsetInBlkToOffsetInBG(int cellSize, int dataBlkNum,
++      long offsetInBlk, int idxInBlockGroup) {
++    int cellIdxInBlk = (int) (offsetInBlk / cellSize);
++    return cellIdxInBlk * cellSize * dataBlkNum // n full stripes before offset
++        + idxInBlockGroup * cellSize // m full cells before offset
++        + offsetInBlk % cellSize; // partial cell
++  }
++
++  /**
++   * Get the next completed striped read task
++   *
++   * @return {@link StripingChunkReadResult} indicating the status of the read task
++   *          succeeded, and the block index of the task. If the method times
++   *          out without getting any completed read tasks, -1 is returned as
++   *          block index.
++   * @throws InterruptedException
++   */
++  public static StripingChunkReadResult getNextCompletedStripedRead(
++      CompletionService<Void> readService, Map<Future<Void>, Integer> futures,
++      final long timeoutMillis) throws InterruptedException {
++    Preconditions.checkArgument(!futures.isEmpty());
++    Future<Void> future = null;
++    try {
++      if (timeoutMillis > 0) {
++        future = readService.poll(timeoutMillis, TimeUnit.MILLISECONDS);
++      } else {
++        future = readService.take();
++      }
++      if (future != null) {
++        future.get();
++        return new StripingChunkReadResult(futures.remove(future),
++            StripingChunkReadResult.SUCCESSFUL);
++      } else {
++        return new StripingChunkReadResult(StripingChunkReadResult.TIMEOUT);
++      }
++    } catch (ExecutionException e) {
++      if (DFSClient.LOG.isDebugEnabled()) {
++        DFSClient.LOG.debug("ExecutionException " + e);
++      }
++      return new StripingChunkReadResult(futures.remove(future),
++          StripingChunkReadResult.FAILED);
++    } catch (CancellationException e) {
++      return new StripingChunkReadResult(futures.remove(future),
++          StripingChunkReadResult.CANCELLED);
++    }
++  }
++
++  /**
++   * Get the total usage of the striped blocks, which is the total of data
++   * blocks and parity blocks
++   *
++   * @param numDataBlkBytes
++   *          Size of the block group only counting data blocks
++   * @param dataBlkNum
++   *          The number of data blocks
++   * @param parityBlkNum
++   *          The number of parity blocks
++   * @param cellSize
++   *          The size of a striping cell
++   * @return The total usage of data blocks and parity blocks
++   */
++  public static long spaceConsumedByStripedBlock(long numDataBlkBytes,
++      int dataBlkNum, int parityBlkNum, int cellSize) {
++    int parityIndex = dataBlkNum + 1;
++    long numParityBlkBytes = getInternalBlockLength(numDataBlkBytes, cellSize,
++        dataBlkNum, parityIndex) * parityBlkNum;
++    return numDataBlkBytes + numParityBlkBytes;
++  }
++
++  /**
++   * Initialize the decoding input buffers based on the chunk states in an
++   * {@link AlignedStripe}. For each chunk that was not initially requested,
++   * schedule a new fetch request with the decoding input buffer as transfer
++   * destination.
++   */
++  public static byte[][] initDecodeInputs(AlignedStripe alignedStripe,
++      int dataBlkNum, int parityBlkNum) {
++    byte[][] decodeInputs =
++        new byte[dataBlkNum + parityBlkNum][(int) alignedStripe.getSpanInBlock()];
++    // read the full data aligned stripe
++    for (int i = 0; i < dataBlkNum; i++) {
++      if (alignedStripe.chunks[i] == null) {
++        final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum);
++        alignedStripe.chunks[i] = new StripingChunk(decodeInputs[decodeIndex]);
++        alignedStripe.chunks[i].addByteArraySlice(0,
++            (int) alignedStripe.getSpanInBlock());
++      }
++    }
++    return decodeInputs;
++  }
++
++  /**
++   * Some fetched {@link StripingChunk} might be stored in original application
++   * buffer instead of prepared decode input buffers. Some others are beyond
++   * the range of the internal blocks and should correspond to all zero bytes.
++   * When all pending requests have returned, this method should be called to
++   * finalize decode input buffers.
++   */
++  public static void finalizeDecodeInputs(final byte[][] decodeInputs,
++      int dataBlkNum, int parityBlkNum, AlignedStripe alignedStripe) {
++    for (int i = 0; i < alignedStripe.chunks.length; i++) {
++      final StripingChunk chunk = alignedStripe.chunks[i];
++      final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum);
++      if (chunk != null && chunk.state == StripingChunk.FETCHED) {
++        chunk.copyTo(decodeInputs[decodeIndex]);
++      } else if (chunk != null && chunk.state == StripingChunk.ALLZERO) {
++        Arrays.fill(decodeInputs[decodeIndex], (byte) 0);
++      } else {
++        decodeInputs[decodeIndex] = null;
++      }
++    }
++  }
++
++  /**
++   * Currently decoding requires parity chunks are before data chunks.
++   * The indices are opposite to what we store in NN. In future we may
++   * improve the decoding to make the indices order the same as in NN.
++   *
++   * @param index The index to convert
++   * @param dataBlkNum The number of data blocks
++   * @param parityBlkNum The number of parity blocks
++   * @return converted index
++   */
++  public static int convertIndex4Decode(int index, int dataBlkNum,
++      int parityBlkNum) {
++    return index < dataBlkNum ? index + parityBlkNum : index - dataBlkNum;
++  }
++
++  public static int convertDecodeIndexBack(int index, int dataBlkNum,
++      int parityBlkNum) {
++    return index < parityBlkNum ? index + dataBlkNum : index - parityBlkNum;
++  }
++
++  /**
++   * Decode based on the given input buffers and erasure coding policy.
++   */
++  public static void decodeAndFillBuffer(final byte[][] decodeInputs,
++      AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum,
++      RawErasureDecoder decoder) {
++    // Step 1: prepare indices and output buffers for missing data units
++    int[] decodeIndices = new int[parityBlkNum];
++    int pos = 0;
++    for (int i = 0; i < dataBlkNum; i++) {
++      if (alignedStripe.chunks[i] != null &&
++          alignedStripe.chunks[i].state == StripingChunk.MISSING){
++        decodeIndices[pos++] = convertIndex4Decode(i, dataBlkNum, parityBlkNum);
++      }
++    }
++    decodeIndices = Arrays.copyOf(decodeIndices, pos);
++    byte[][] decodeOutputs =
++        new byte[decodeIndices.length][(int) alignedStripe.getSpanInBlock()];
++
++    // Step 2: decode into prepared output buffers
++    decoder.decode(decodeInputs, decodeIndices, decodeOutputs);
++
++    // Step 3: fill original application buffer with decoded data
++    for (int i = 0; i < decodeIndices.length; i++) {
++      int missingBlkIdx = convertDecodeIndexBack(decodeIndices[i],
++          dataBlkNum, parityBlkNum);
++      StripingChunk chunk = alignedStripe.chunks[missingBlkIdx];
++      if (chunk.state == StripingChunk.MISSING) {
++        chunk.copyFrom(decodeOutputs[i]);
++      }
++    }
++  }
++
++  /**
++   * Similar functionality with {@link #divideByteRangeIntoStripes}, but is used
++   * by stateful read and uses ByteBuffer as reading target buffer. Besides the
++   * read range is within a single stripe thus the calculation logic is simpler.
++   */
++  public static AlignedStripe[] divideOneStripe(ErasureCodingPolicy ecPolicy,
++      int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup,
++      long rangeEndInBlockGroup, ByteBuffer buf) {
++    final int dataBlkNum = ecPolicy.getNumDataUnits();
++    // Step 1: map the byte range to StripingCells
++    StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize,
++        blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup);
++
++    // Step 2: get the unmerged ranges on each internal block
++    VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize,
++        cells);
++
++    // Step 3: merge into stripes
++    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges);
++
++    // Step 4: calculate each chunk's position in destination buffer. Since the
++    // whole read range is within a single stripe, the logic is simpler here.
++    int bufOffset = (int) (rangeStartInBlockGroup % ((long) cellSize * dataBlkNum));
++    for (StripingCell cell : cells) {
++      long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
++      long cellEnd = cellStart + cell.size - 1;
++      for (AlignedStripe s : stripes) {
++        long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1;
++        long overlapStart = Math.max(cellStart, s.getOffsetInBlock());
++        long overlapEnd = Math.min(cellEnd, stripeEnd);
++        int overLapLen = (int) (overlapEnd - overlapStart + 1);
++        if (overLapLen > 0) {
++          Preconditions.checkState(s.chunks[cell.idxInStripe] == null);
++          final int pos = (int) (bufOffset + overlapStart - cellStart);
++          buf.position(pos);
++          buf.limit(pos + overLapLen);
++          s.chunks[cell.idxInStripe] = new StripingChunk(buf.slice());
++        }
++      }
++      bufOffset += cell.size;
++    }
++
++    // Step 5: prepare ALLZERO blocks
++    prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum);
++    return stripes;
++  }
++
++  /**
++   * This method divides a requested byte range into an array of inclusive
++   * {@link AlignedStripe}.
++   * @param ecPolicy The codec policy for the file, which carries the numbers
++   *                 of data / parity blocks
++   * @param cellSize Cell size of stripe
++   * @param blockGroup The striped block group
++   * @param rangeStartInBlockGroup The byte range's start offset in block group
++   * @param rangeEndInBlockGroup The byte range's end offset in block group
++   * @param buf Destination buffer of the read operation for the byte range
++   * @param offsetInBuf Start offset into the destination buffer
++   *
++   * At most 5 stripes will be generated from each logical range, as
++   * demonstrated in the header of {@link AlignedStripe}.
++   */
++  public static AlignedStripe[] divideByteRangeIntoStripes(ErasureCodingPolicy ecPolicy,
++      int cellSize, LocatedStripedBlock blockGroup,
++      long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf,
++      int offsetInBuf) {
++
++    // Step 0: analyze range and calculate basic parameters
++    final int dataBlkNum = ecPolicy.getNumDataUnits();
++
++    // Step 1: map the byte range to StripingCells
++    StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize,
++        blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup);
++
++    // Step 2: get the unmerged ranges on each internal block
++    VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize,
++        cells);
++
++    // Step 3: merge into at most 5 stripes
++    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges);
++
++    // Step 4: calculate each chunk's position in destination buffer
++    calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf, offsetInBuf);
++
++    // Step 5: prepare ALLZERO blocks
++    prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum);
++
++    return stripes;
++  }
++
++  /**
++   * Map the logical byte range to a set of inclusive {@link StripingCell}
++   * instances, each representing the overlap of the byte range to a cell
++   * used by {@link DFSStripedOutputStream} in encoding
++   */
++  @VisibleForTesting
++  private static StripingCell[] getStripingCellsOfByteRange(ErasureCodingPolicy ecPolicy,
++      int cellSize, LocatedStripedBlock blockGroup,
++      long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
++    Preconditions.checkArgument(
++        rangeStartInBlockGroup <= rangeEndInBlockGroup &&
++            rangeEndInBlockGroup < blockGroup.getBlockSize());
++    long len = rangeEndInBlockGroup - rangeStartInBlockGroup + 1;
++    int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
++    int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize);
++    int numCells = lastCellIdxInBG - firstCellIdxInBG + 1;
++    StripingCell[] cells = new StripingCell[numCells];
++
++    final int firstCellOffset = (int) (rangeStartInBlockGroup % cellSize);
++    final int firstCellSize =
++        (int) Math.min(cellSize - (rangeStartInBlockGroup % cellSize), len);
++    cells[0] = new StripingCell(ecPolicy, firstCellSize, firstCellIdxInBG,
++        firstCellOffset);
++    if (lastCellIdxInBG != firstCellIdxInBG) {
++      final int lastCellSize = (int) (rangeEndInBlockGroup % cellSize) + 1;
++      cells[numCells - 1] = new StripingCell(ecPolicy, lastCellSize,
++          lastCellIdxInBG, 0);
++    }
++
++    for (int i = 1; i < numCells - 1; i++) {
++      cells[i] = new StripingCell(ecPolicy, cellSize, i + firstCellIdxInBG, 0);
++    }
++
++    return cells;
++  }
++
++  /**
++   * Given a logical byte range, mapped to each {@link StripingCell}, calculate
++   * the physical byte range (inclusive) on each stored internal block.
++   */
++  @VisibleForTesting
++  private static VerticalRange[] getRangesForInternalBlocks(ErasureCodingPolicy ecPolicy,
++      int cellSize, StripingCell[] cells) {
++    int dataBlkNum = ecPolicy.getNumDataUnits();
++    int parityBlkNum = ecPolicy.getNumParityUnits();
++
++    VerticalRange ranges[] = new VerticalRange[dataBlkNum + parityBlkNum];
++
++    long earliestStart = Long.MAX_VALUE;
++    long latestEnd = -1;
++    for (StripingCell cell : cells) {
++      // iterate through all cells and update the list of StripeRanges
++      if (ranges[cell.idxInStripe] == null) {
++        ranges[cell.idxInStripe] = new VerticalRange(
++            cell.idxInInternalBlk * cellSize + cell.offset, cell.size);
++      } else {
++        ranges[cell.idxInStripe].spanInBlock += cell.size;
++      }
++      VerticalRange range = ranges[cell.idxInStripe];
++      if (range.offsetInBlock < earliestStart) {
++        earliestStart = range.offsetInBlock;
++      }
++      if (range.offsetInBlock + range.spanInBlock - 1 > latestEnd) {
++        latestEnd = range.offsetInBlock + range.spanInBlock - 1;
++      }
++    }
++
++    // Each parity block should be fetched at maximum range of all data blocks
++    for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) {
++      ranges[i] = new VerticalRange(earliestStart,
++          latestEnd - earliestStart + 1);
++    }
++
++    return ranges;
++  }
++
++  /**
++   * Merge byte ranges on each internal block into a set of inclusive
++   * {@link AlignedStripe} instances.
++   */
++  private static AlignedStripe[] mergeRangesForInternalBlocks(
++      ErasureCodingPolicy ecPolicy, VerticalRange[] ranges) {
++    int dataBlkNum = ecPolicy.getNumDataUnits();
++    int parityBlkNum = ecPolicy.getNumParityUnits();
++    List<AlignedStripe> stripes = new ArrayList<>();
++    SortedSet<Long> stripePoints = new TreeSet<>();
++    for (VerticalRange r : ranges) {
++      if (r != null) {
++        stripePoints.add(r.offsetInBlock);
++        stripePoints.add(r.offsetInBlock + r.spanInBlock);
++      }
++    }
++
++    long prev = -1;
++    for (long point : stripePoints) {
++      if (prev >= 0) {
++        stripes.add(new AlignedStripe(prev, point - prev,
++            dataBlkNum + parityBlkNum));
++      }
++      prev = point;
++    }
++    return stripes.toArray(new AlignedStripe[stripes.size()]);
++  }
++
++  private static void calcualteChunkPositionsInBuf(int cellSize,
++      AlignedStripe[] stripes, StripingCell[] cells, byte[] buf,
++      int offsetInBuf) {
++    /**
++     *     | <--------------- AlignedStripe --------------->|
++     *
++     *     |<- length_0 ->|<--  length_1  -->|<- length_2 ->|
++     * +------------------+------------------+----------------+
++     * |    cell_0_0_0    |    cell_3_1_0    |   cell_6_2_0   |  <- blk_0
++     * +------------------+------------------+----------------+
++     *   _/                \_______________________
++     *  |                                          |
++     *  v offset_0                                 v offset_1
++     * +----------------------------------------------------------+
++     * |  cell_0_0_0 |  cell_1_0_1 and cell_2_0_2  |cell_3_1_0 ...|   <- buf
++     * |  (partial)  |    (from blk_1 and blk_2)   |              |
++     * +----------------------------------------------------------+
++     *
++     * Cell indexing convention defined in {@link StripingCell}
++     */
++    int done = 0;
++    for (StripingCell cell : cells) {
++      long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
++      long cellEnd = cellStart + cell.size - 1;
++      for (AlignedStripe s : stripes) {
++        long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1;
++        long overlapStart = Math.max(cellStart, s.getOffsetInBlock());
++        long overlapEnd = Math.min(cellEnd, stripeEnd);
++        int overLapLen = (int) (overlapEnd - overlapStart + 1);
++        if (overLapLen <= 0) {
++          continue;
++        }
++        if (s.chunks[cell.idxInStripe] == null) {
++          s.chunks[cell.idxInStripe] = new StripingChunk(buf);
++        }
++        s.chunks[cell.idxInStripe].addByteArraySlice(
++            (int)(offsetInBuf + done + overlapStart - cellStart), overLapLen);
++      }
++      done += cell.size;
++    }
++  }
++
++  /**
++   * If a {@link StripingChunk} maps to a byte range beyond an internal block's
++   * size, the chunk should be treated as zero bytes in decoding.
++   */
++  private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup,
++      AlignedStripe[] stripes, int cellSize, int dataBlkNum) {
++    for (AlignedStripe s : stripes) {
++      for (int i = 0; i < dataBlkNum; i++) {
++        long internalBlkLen = getInternalBlockLength(blockGroup.getBlockSize(),
++            cellSize, dataBlkNum, i);
++        if (internalBlkLen <= s.getOffsetInBlock()) {
++          Preconditions.checkState(s.chunks[i] == null);
++          s.chunks[i] = new StripingChunk(StripingChunk.ALLZERO);
++        }
++      }
++    }
++  }
++
++  /**
++   * Cell is the unit of encoding used in {@link DFSStripedOutputStream}. This
++   * size impacts how a logical offset in the file or block group translates
++   * to physical byte offset in a stored internal block. The StripingCell util
++   * class facilitates this calculation. Each StripingCell is inclusive with
++   * its start and end offsets -- e.g., the end logical offset of cell_0_0_0
++   * should be 1 byte lower than the start logical offset of cell_1_0_1.
++   *
++   *  | <------- Striped Block Group -------> |
++   *    blk_0          blk_1          blk_2
++   *      |              |              |
++   *      v              v              v
++   * +----------+   +----------+   +----------+
++   * |cell_0_0_0|   |cell_1_0_1|   |cell_2_0_2|
++   * +----------+   +----------+   +----------+
++   * |cell_3_1_0|   |cell_4_1_1|   |cell_5_1_2| <- {@link #idxInBlkGroup} = 5
++   * +----------+   +----------+   +----------+    {@link #idxInInternalBlk} = 1
++   *                                               {@link #idxInStripe} = 2
++   * A StripingCell is a special instance of {@link StripingChunk} whose offset
++   * and size align with the cell used when writing data.
++   * TODO: consider parity cells
++   */
++  @VisibleForTesting
++  static class StripingCell {
++    final ErasureCodingPolicy ecPolicy;
++    /** Logical order in a block group, used when doing I/O to a block group */
++    final int idxInBlkGroup;
++    final int idxInInternalBlk;
++    final int idxInStripe;
++    /**
++     * When a logical byte range is mapped to a set of cells, it might
++     * partially overlap with the first and last cells. This field and the
++     * {@link #size} variable represent the start offset and size of the
++     * overlap.
++     */
++    final int offset;
++    final int size;
++
++    StripingCell(ErasureCodingPolicy ecPolicy, int cellSize, int idxInBlkGroup,
++        int offset) {
++      this.ecPolicy = ecPolicy;
++      this.idxInBlkGroup = idxInBlkGroup;
++      this.idxInInternalBlk = idxInBlkGroup / ecPolicy.getNumDataUnits();
++      this.idxInStripe = idxInBlkGroup -
++          this.idxInInternalBlk * ecPolicy.getNumDataUnits();
++      this.offset = offset;
++      this.size = cellSize;
++    }
++  }
++
++  /**
++   * Given a requested byte range on a striped block group, an AlignedStripe
++   * represents an inclusive {@link VerticalRange} that is aligned with both
++   * the byte range and boundaries of all internal blocks. As illustrated in
++   * the diagram, any given byte range on a block group leads to 1~5
++   * AlignedStripe's.
++   *
++   * |<-------- Striped Block Group -------->|
++   * blk_0   blk_1   blk_2      blk_3   blk_4
++   *                 +----+  |  +----+  +----+
++   *                 |full|  |  |    |  |    | <- AlignedStripe0:
++   *         +----+  |~~~~|  |  |~~~~|  |~~~~|      1st cell is partial
++   *         |part|  |    |  |  |    |  |    | <- AlignedStripe1: byte range
++   * +----+  +----+  +----+  |  |~~~~|  |~~~~|      doesn't start at 1st block
++   * |full|  |full|  |full|  |  |    |  |    |
++   * |cell|  |cell|  |cell|  |  |    |  |    | <- AlignedStripe2 (full stripe)
++   * |    |  |    |  |    |  |  |    |  |    |
++   * +----+  +----+  +----+  |  |~~~~|  |~~~~|
++   * |full|  |part|          |  |    |  |    | <- AlignedStripe3: byte range
++   * |~~~~|  +----+          |  |~~~~|  |~~~~|      doesn't end at last block
++   * |    |                  |  |    |  |    | <- AlignedStripe4:
++   * +----+                  |  +----+  +----+      last cell is partial
++   *                         |
++   * <---- data blocks ----> | <--- parity --->
++   *
++   * An AlignedStripe is the basic unit of reading from a striped block group,
++   * because within the AlignedStripe, all internal blocks can be processed in
++   * a uniform manner.
++   *
++   * The coverage of an AlignedStripe on an internal block is represented as a
++   * {@link StripingChunk}.
++   *
++   * To simplify the logic of reading a logical byte range from a block group,
++   * a StripingChunk is either completely in the requested byte range or
++   * completely outside the requested byte range.
++   */
++  public static class AlignedStripe {
++    public VerticalRange range;
++    /** status of each chunk in the stripe */
++    public final StripingChunk[] chunks;
++    public int fetchedChunksNum = 0;
++    public int missingChunksNum = 0;
++
++    public AlignedStripe(long offsetInBlock, long length, int width) {
++      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
++      this.range = new VerticalRange(offsetInBlock, length);
++      this.chunks = new StripingChunk[width];
++    }
++
++    public boolean include(long pos) {
++      return range.include(pos);
++    }
++
++    public long getOffsetInBlock() {
++      return range.offsetInBlock;
++    }
++
++    public long getSpanInBlock() {
++      return range.spanInBlock;
++    }
++
++    @Override
++    public String toString() {
++      return "Offset=" + range.offsetInBlock + ", length=" + range.spanInBlock +
++          ", fetchedChunksNum=" + fetchedChunksNum +
++          ", missingChunksNum=" + missingChunksNum;
++    }
++  }
++
++  /**
++   * A simple utility class representing an arbitrary vertical inclusive range
++   * starting at {@link #offsetInBlock} and lasting for {@link #spanInBlock}
++   * bytes in an internal block. Note that VerticalRange doesn't necessarily
++   * align with {@link StripingCell}.
++   *
++   * |<- Striped Block Group ->|
++   *  blk_0
++   *    |
++   *    v
++   * +-----+
++   * |~~~~~| <-- {@link #offsetInBlock}
++   * |     |  ^
++   * |     |  |
++   * |     |  | {@link #spanInBlock}
++   * |     |  v
++   * |~~~~~| ---
++   * |     |
++   * +-----+
++   */
++  public static class VerticalRange {
++    /** start offset in the block group (inclusive) */
++    public long offsetInBlock;
++    /** length of the stripe range */
++    public long spanInBlock;
++
++    public VerticalRange(long offsetInBlock, long length) {
++      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
++      this.offsetInBlock = offsetInBlock;
++      this.spanInBlock = length;
++    }
++
++    /** whether a position is in the range */
++    public boolean include(long pos) {
++      return pos >= offsetInBlock && pos < offsetInBlock + spanInBlock;
++    }
++  }
++
++  /**
++   * Indicates the coverage of an {@link AlignedStripe} on an internal block,
++   * and the state of the chunk in the context of the read request.
++   *
++   * |<---------------- Striped Block Group --------------->|
++   *   blk_0        blk_1        blk_2          blk_3   blk_4
++   *                           +---------+  |  +----+  +----+
++   *     null         null     |REQUESTED|  |  |null|  |null| <- AlignedStripe0
++   *              +---------+  |---------|  |  |----|  |----|
++   *     null     |REQUESTED|  |REQUESTED|  |  |null|  |null| <- AlignedStripe1
++   * +---------+  +---------+  +---------+  |  +----+  +----+
++   * |REQUESTED|  |REQUESTED|    ALLZERO    |  |null|  |null| <- AlignedStripe2
++   * +---------+  +---------+               |  +----+  +----+
++   * <----------- data blocks ------------> | <--- parity --->
++   */
++  public static class StripingChunk {
++    /** Chunk has been successfully fetched */
++    public static final int FETCHED = 0x01;
++    /** Chunk has encountered failed when being fetched */
++    public static final int MISSING = 0x02;
++    /** Chunk being fetched (fetching task is in-flight) */
++    public static final int PENDING = 0x04;
++    /**
++     * Chunk is requested either by application or for decoding, need to
++     * schedule read task
++     */
++    public static final int REQUESTED = 0X08;
++    /**
++     * Internal block is short and has no overlap with chunk. Chunk considered
++     * all-zero bytes in codec calculations.
++     */
++    public static final int ALLZERO = 0X0f;
++
++    /**
++     * If a chunk is completely in requested range, the state transition is:
++     * REQUESTED (when AlignedStripe created) -> PENDING -> {FETCHED | MISSING}
++     * If a chunk is completely outside requested range (including parity
++     * chunks), state transition is:
++     * null (AlignedStripe created) -> REQUESTED (upon failure) -> PENDING ...
++     */
++    public int state = REQUESTED;
++
++    public final ChunkByteArray byteArray;
++    public final ByteBuffer byteBuffer;
++
++    public StripingChunk(byte[] buf) {
++      this.byteArray = new ChunkByteArray(buf);
++      byteBuffer = null;
++    }
++
++    public StripingChunk(ByteBuffer buf) {
++      this.byteArray = null;
++      this.byteBuffer = buf;
++    }
++
++    public StripingChunk(int state) {
++      this.byteArray = null;
++      this.byteBuffer = null;
++      this.state = state;
++    }
++
++    public void addByteArraySlice(int offset, int length) {
++      assert byteArray != null;
++      byteArray.offsetsInBuf.add(offset);
++      byteArray.lengthsInBuf.add(length);
++    }
++
++    void copyTo(byte[] target) {
++      assert byteArray != null;
++      byteArray.copyTo(target);
++    }
++
++    void copyFrom(byte[] src) {
++      assert byteArray != null;
++      byteArray.copyFrom(src);
++    }
++  }
++
++  public static class ChunkByteArray {
++    private final byte[] buf;
++    private final List<Integer> offsetsInBuf;
++    private final List<Integer> lengthsInBuf;
++
++    ChunkByteArray(byte[] buf) {
++      this.buf = buf;
++      this.offsetsInBuf = new ArrayList<>();
++      this.lengthsInBuf = new ArrayList<>();
++    }
++
++    public int[] getOffsets() {
++      int[] offsets = new int[offsetsInBuf.size()];
++      for (int i = 0; i < offsets.length; i++) {
++        offsets[i] = offsetsInBuf.get(i);
++      }
++      return offsets;
++    }
++
++    public int[] getLengths() {
++      int[] lens = new int[this.lengthsInBuf.size()];
++      for (int i = 0; i < lens.length; i++) {
++        lens[i] = this.lengthsInBuf.get(i);
++      }
++      return lens;
++    }
++
++    public byte[] buf() {
++      return buf;
++    }
++
++    void copyTo(byte[] target) {
++      int posInBuf = 0;
++      for (int i = 0; i < offsetsInBuf.size(); i++) {
++        System.arraycopy(buf, offsetsInBuf.get(i),
++            target, posInBuf, lengthsInBuf.get(i));
++        posInBuf += lengthsInBuf.get(i);
++      }
++    }
++
++    void copyFrom(byte[] src) {
++      int srcPos = 0;
++      for (int j = 0; j < offsetsInBuf.size(); j++) {
++        System.arraycopy(src, srcPos, buf, offsetsInBuf.get(j),
++            lengthsInBuf.get(j));
++        srcPos += lengthsInBuf.get(j);
++      }
++    }
++  }
++
++  /**
++   * This class represents result from a striped read request.
++   * If the task was successful or the internal computation failed,
++   * an index is also returned.
++   */
++  public static class StripingChunkReadResult {
++    public static final int SUCCESSFUL = 0x01;
++    public static final int FAILED = 0x02;
++    public static final int TIMEOUT = 0x04;
++    public static final int CANCELLED = 0x08;
++
++    public final int index;
++    public final int state;
++
++    public StripingChunkReadResult(int state) {
++      Preconditions.checkArgument(state == TIMEOUT,
++          "Only timeout result should return negative index.");
++      this.index = -1;
++      this.state = state;
++    }
++
++    public StripingChunkReadResult(int index, int state) {
++      Preconditions.checkArgument(state != TIMEOUT,
++          "Timeout result should return negative index.");
++      this.index = index;
++      this.state = state;
++    }
++
++    @Override
++    public String toString() {
++      return "(index=" + index + ", state =" + state + ")";
++    }
++  }
++
++  /**
++   * Check if the information such as IDs and generation stamps in block-i
++   * match the block group.
++   */
++  public static void checkBlocks(ExtendedBlock blockGroup,
++      int i, ExtendedBlock blocki) throws IOException {
++    if (!blocki.getBlockPoolId().equals(blockGroup.getBlockPoolId())) {
++      throw new IOException("Block pool IDs mismatched: block" + i + "="
++          + blocki + ", expected block group=" + blockGroup);
++    }
++    if (blocki.getBlockId() - i != blockGroup.getBlockId()) {
++      throw new IOException("Block IDs mismatched: block" + i + "="
++          + blocki + ", expected block group=" + blockGroup);
++    }
++    if (blocki.getGenerationStamp() != blockGroup.getGenerationStamp()) {
++      throw new IOException("Generation stamps mismatched: block" + i + "="
++          + blocki + ", expected block group=" + blockGroup);
++    }
++  }
++
++  public static int getBlockIndex(Block reportedBlock) {
++    long BLOCK_GROUP_INDEX_MASK = 15;
++    return (int) (reportedBlock.getBlockId() &
++        BLOCK_GROUP_INDEX_MASK);
++  }
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index b28ab42,0e2d541..d35fb57
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@@ -648,3 -444,3 +478,11 @@@ message RollingUpgradeStatusProto 
    required string blockPoolId = 1;
    optional bool finalized = 2 [default = false];
  }
++
++
++/**
++ * A list of storage IDs.
++ */
++message StorageUuidsProto {
++  repeated string storageUuids = 1;
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 8874c4d,b631955..0166029
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@@ -402,14 -397,11 +400,19 @@@ public class DFSConfigKeys extends Comm
    public static final int     DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT = 21600;
    public static final String  DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY = "dfs.datanode.directoryscan.threads";
    public static final int     DFS_DATANODE_DIRECTORYSCAN_THREADS_DEFAULT = 1;
 +  public static final String  DFS_DATANODE_STRIPED_READ_THREADS_KEY = "dfs.datanode.stripedread.threads";
 +  public static final int     DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT = 20;
 +  public static final String  DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.stripedread.buffer.size";
 +  public static final int     DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 64 * 1024;
 +  public static final String  DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY = "dfs.datanode.stripedread.timeout.millis";
 +  public static final int     DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT = 5000; //5s
 +  public static final String  DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY = "dfs.datanode.striped.blockrecovery.threads.size";
 +  public static final int     DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT = 8;
+   public static final String
+       DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY =
+       "dfs.datanode.directoryscan.throttle.limit.ms.per.sec";
+   public static final int
+       DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_DEFAULT = 1000;
    public static final String  DFS_DATANODE_DNS_INTERFACE_KEY = "dfs.datanode.dns.interface";
    public static final String  DFS_DATANODE_DNS_INTERFACE_DEFAULT = "default";
    public static final String  DFS_DATANODE_DNS_NAMESERVER_KEY = "dfs.datanode.dns.nameserver";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 7f721f0,5b11ac2..b0ea7ce
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@@ -1442,34 -1439,5 +1441,4 @@@ public class DFSUtil 
          .createKeyProviderCryptoExtension(keyProvider);
      return cryptoProvider;
    }
- 
-   public static int getIoFileBufferSize(Configuration conf) {
-     return conf.getInt(
-       CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
-       CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
-   }
- 
-   public static int getSmallBufferSize(Configuration conf) {
-     return Math.min(getIoFileBufferSize(conf) / 2, 512);
-   }
- 
-   /**
-    * Probe for HDFS Encryption being enabled; this uses the value of
-    * the option {@link DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI},
-    * returning true if that property contains a non-empty, non-whitespace
-    * string.
-    * @param conf configuration to probe
-    * @return true if encryption is considered enabled.
-    */
-   public static boolean isHDFSEncryptionEnabled(Configuration conf) {
-     return !conf.getTrimmed(
-         DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "").isEmpty();
-   }
--
-   public static InterruptedIOException toInterruptedIOException(String message,
-       InterruptedException e) {
-     final InterruptedIOException iioe = new InterruptedIOException(message);
-     iioe.initCause(e);
-     return iioe;
-   }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 524248c,75b3811..05c498f
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@@ -47,36 -45,27 +47,35 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
  import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
++import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamespaceInfoProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
  import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
- import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockKeyProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockWithLocationsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlocksWithLocationsProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointCommandProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointSignatureProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.ExportedBlockKeysProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeCommandProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto.NamenodeRoleProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamespaceInfoProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RecoveringBlockProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogManifestProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.ReplicaStateProto;
+ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageInfoProto;
 -import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageUuidsProto;
  import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
  import org.apache.hadoop.hdfs.security.token.block.BlockKey;
  import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 9228bec,2646089..43ddf74
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@@ -364,10 -368,9 +370,9 @@@ public class DataNode extends Reconfigu
    private String supergroup;
    private boolean isPermissionEnabled;
    private String dnUserName = null;
- 
-   private SpanReceiverHost spanReceiverHost;
--
 +  private ErasureCodingWorker ecWorker;
+   final Tracer tracer;
+   private final TracerConfigurationManager tracerConfigurationManager;
    private static final int NUM_CORES = Runtime.getRuntime()
        .availableProcessors();
    private static final double CONGESTION_RATIO = 1.5;
@@@ -3289,12 -3287,8 +3320,12 @@@
    @Override
    public void removeSpanReceiver(long id) throws IOException {
      checkSuperuserPrivilege();
-     spanReceiverHost.removeSpanReceiver(id);
+     tracerConfigurationManager.removeSpanReceiver(id);
    }
 +  
 +  public ErasureCodingWorker getErasureCodingWorker(){
 +    return ecWorker;
 +  }
  
    /**
     * Get timeout value of each OOB type from configuration


[05/50] [abbrv] hadoop git commit: Merge commit '456e901a4c5c639267ee87b8e5f1319f256d20c2' (HDFS-6407. Add sorting and pagination in the datanode tab of the NN Web UI. Contributed by Haohui Mai.) into HDFS-7285-merge

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 0000000,7d32568..fb10e9c
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@@ -1,0 -1,873 +1,880 @@@
+ /**
+  * 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.
+  */
+ 
+ /**
+  * These .proto interfaces are private and stable.
+  * Please see http://wiki.apache.org/hadoop/Compatibility
+  * for what changes are allowed for a *stable* .proto interface.
+  */
+ 
+ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+ option java_outer_classname = "ClientNamenodeProtocolProtos";
+ option java_generic_services = true;
+ option java_generate_equals_and_hash = true;
+ package hadoop.hdfs;
+ 
+ import "Security.proto";
+ import "hdfs.proto";
+ import "acl.proto";
+ import "xattr.proto";
+ import "encryption.proto";
+ import "inotify.proto";
++import "erasurecoding.proto";
+ 
+ /**
+  * The ClientNamenodeProtocol Service defines the interface between a client 
+  * (as runnign inside a MR Task) and the Namenode.
+  * See org.apache.hadoop.hdfs.protocol.ClientProtocol for the javadoc 
+  * for each of the methods.
+  * The exceptions declared in the above class also apply to this protocol.
+  * Exceptions are unwrapped and thrown by the  PB libraries.
+  */
+ 
+ message GetBlockLocationsRequestProto {
+   required string src = 1;     // file name
+   required uint64 offset = 2;  // range start offset
+   required uint64 length = 3;  // range length
+ }
+ 
+ message GetBlockLocationsResponseProto {
+   optional LocatedBlocksProto locations = 1;
+ }
+ 
+ message GetServerDefaultsRequestProto { // No parameters
+ }
+ 
+ message GetServerDefaultsResponseProto {
+   required FsServerDefaultsProto serverDefaults = 1;
+ }
+ 
+ enum CreateFlagProto {
+   CREATE = 0x01;    // Create a file
+   OVERWRITE = 0x02; // Truncate/overwrite a file. Same as POSIX O_TRUNC
+   APPEND = 0x04;    // Append to a file
+   LAZY_PERSIST = 0x10; // File with reduced durability guarantees.
+   NEW_BLOCK = 0x20; // Write data to a new block when appending
+ }
+ 
+ message CreateRequestProto {
+   required string src = 1;
+   required FsPermissionProto masked = 2;
+   required string clientName = 3;
+   required uint32 createFlag = 4;  // bits set using CreateFlag
+   required bool createParent = 5;
+   required uint32 replication = 6; // Short: Only 16 bits used
+   required uint64 blockSize = 7;
+   repeated CryptoProtocolVersionProto cryptoProtocolVersion = 8;
+ }
+ 
+ message CreateResponseProto {
+   optional HdfsFileStatusProto fs = 1;
+ }
+ 
+ message AppendRequestProto {
+   required string src = 1;
+   required string clientName = 2;
+   optional uint32 flag = 3; // bits set using CreateFlag
+ }
+ 
+ message AppendResponseProto {
+   optional LocatedBlockProto block = 1;
+   optional HdfsFileStatusProto stat = 2;
+ }
+ 
+ message SetReplicationRequestProto {
+   required string src = 1;
+   required uint32 replication = 2; // Short: Only 16 bits used
+ }
+ 
+ message SetReplicationResponseProto {
+   required bool result = 1;
+ }
+ 
+ message SetStoragePolicyRequestProto {
+   required string src = 1;
+   required string policyName = 2;
+ }
+ 
+ message SetStoragePolicyResponseProto { // void response
+ }
+ 
+ message GetStoragePolicyRequestProto {
+   required string path = 1;
+ }
+ 
+ message GetStoragePolicyResponseProto {
+   required BlockStoragePolicyProto storagePolicy = 1;
+ }
+ 
+ message GetStoragePoliciesRequestProto { // void request
+ }
+ 
+ message GetStoragePoliciesResponseProto {
+   repeated BlockStoragePolicyProto policies = 1;
+ }
+ 
+ message SetPermissionRequestProto {
+   required string src = 1;
+   required FsPermissionProto permission = 2;
+ }
+ 
+ message SetPermissionResponseProto { // void response
+ }
+ 
+ message SetOwnerRequestProto {
+   required string src = 1;
+   optional string username = 2;
+   optional string groupname = 3;
+ }
+ 
+ message SetOwnerResponseProto { // void response
+ }
+ 
+ message AbandonBlockRequestProto {
+   required ExtendedBlockProto b = 1;
+   required string src = 2;
+   required string holder = 3;
+   optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
+ }
+ 
+ message AbandonBlockResponseProto { // void response
+ }
+ 
+ message AddBlockRequestProto {
+   required string src = 1;
+   required string clientName = 2;
+   optional ExtendedBlockProto previous = 3;
+   repeated DatanodeInfoProto excludeNodes = 4;
+   optional uint64 fileId = 5 [default = 0];  // default as a bogus id
+   repeated string favoredNodes = 6; //the set of datanodes to use for the block
+ }
+ 
+ message AddBlockResponseProto {
+   required LocatedBlockProto block = 1;
+ }
+ 
+ message GetAdditionalDatanodeRequestProto {
+   required string src = 1;
+   required ExtendedBlockProto blk = 2;
+   repeated DatanodeInfoProto existings = 3;
+   repeated DatanodeInfoProto excludes = 4;
+   required uint32 numAdditionalNodes = 5;
+   required string clientName = 6;
+   repeated string existingStorageUuids = 7;
+   optional uint64 fileId = 8 [default = 0];  // default to GRANDFATHER_INODE_ID
+ }
+ 
+ message GetAdditionalDatanodeResponseProto {
+   required LocatedBlockProto block = 1;
+ }
+ 
+ message CompleteRequestProto {
+   required string src = 1;
+   required string clientName = 2;
+   optional ExtendedBlockProto last = 3;
+   optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
+ }
+ 
+ message CompleteResponseProto {
+   required bool result = 1;
+ }
+ 
+ message ReportBadBlocksRequestProto {
+   repeated LocatedBlockProto blocks = 1;
+ }
+ 
+ message ReportBadBlocksResponseProto { // void response
+ }
+ 
+ message ConcatRequestProto {
+   required string trg = 1;
+   repeated string srcs = 2;
+ }
+ 
+ message ConcatResponseProto { // void response
+ }
+ 
+ message TruncateRequestProto {
+   required string src = 1;
+   required uint64 newLength = 2;
+   required string clientName = 3;
+ }
+ 
+ message TruncateResponseProto {
+   required bool result = 1;
+ }
+ 
+ message RenameRequestProto {
+   required string src = 1;
+   required string dst = 2;
+ }
+ 
+ message RenameResponseProto {
+   required bool result = 1;
+ }
+ 
+ 
+ message Rename2RequestProto {
+   required string src = 1;
+   required string dst = 2;
+   required bool overwriteDest = 3;
+ }
+ 
+ message Rename2ResponseProto { // void response
+ }
+ 
+ message DeleteRequestProto {
+   required string src = 1;
+   required bool recursive = 2;
+ }
+ 
+ message DeleteResponseProto {
+     required bool result = 1;
+ }
+ 
+ message MkdirsRequestProto {
+   required string src = 1;
+   required FsPermissionProto masked = 2;
+   required bool createParent = 3;
+ }
+ message MkdirsResponseProto {
+     required bool result = 1;
+ }
+ 
+ message GetListingRequestProto {
+   required string src = 1;
+   required bytes startAfter = 2;
+   required bool needLocation = 3;
+ }
+ message GetListingResponseProto {
+   optional DirectoryListingProto dirList = 1;
+ }
+ 
+ message GetSnapshottableDirListingRequestProto { // no input parameters
+ }
+ message GetSnapshottableDirListingResponseProto {
+   optional SnapshottableDirectoryListingProto snapshottableDirList = 1;
+ }
+ 
+ message GetSnapshotDiffReportRequestProto {
+   required string snapshotRoot = 1;
+   required string fromSnapshot = 2;
+   required string toSnapshot = 3;
+ }
+ message GetSnapshotDiffReportResponseProto {
+   required SnapshotDiffReportProto diffReport = 1;
+ }
+ 
+ message RenewLeaseRequestProto {
+   required string clientName = 1;
+ }
+ 
+ message RenewLeaseResponseProto { //void response
+ }
+ 
+ message RecoverLeaseRequestProto {
+   required string src = 1;
+   required string clientName = 2;
+ }
+ message RecoverLeaseResponseProto {
+   required bool result = 1;
+ }
+ 
+ message GetFsStatusRequestProto { // no input paramters
+ }
+ 
+ message GetFsStatsResponseProto {
+   required uint64 capacity = 1;
+   required uint64 used = 2;
+   required uint64 remaining = 3;
+   required uint64 under_replicated = 4;
+   required uint64 corrupt_blocks = 5;
+   required uint64 missing_blocks = 6;
+   optional uint64 missing_repl_one_blocks = 7;
+ }
+ 
+ enum DatanodeReportTypeProto {  // type of the datanode report
+   ALL = 1;
+   LIVE = 2;
+   DEAD = 3;
+   DECOMMISSIONING = 4;
+ }
+ 
+ message GetDatanodeReportRequestProto {
+   required DatanodeReportTypeProto type = 1;
+ }
+ 
+ message GetDatanodeReportResponseProto {
+   repeated DatanodeInfoProto di = 1;
+ }
+ 
+ message GetDatanodeStorageReportRequestProto {
+   required DatanodeReportTypeProto type = 1;
+ }
+ 
+ message DatanodeStorageReportProto {
+   required DatanodeInfoProto datanodeInfo = 1;
+   repeated StorageReportProto storageReports = 2;
+ }
+ 
+ message GetDatanodeStorageReportResponseProto {
+   repeated DatanodeStorageReportProto datanodeStorageReports = 1;
+ }
+ 
+ message GetPreferredBlockSizeRequestProto {
+   required string filename = 1;
+ }
+ 
+ message GetPreferredBlockSizeResponseProto {
+   required uint64 bsize = 1;
+ }
+ 
+ enum SafeModeActionProto {
+   SAFEMODE_LEAVE = 1;
+   SAFEMODE_ENTER = 2;
+   SAFEMODE_GET = 3;
+ }
+ 
+ message SetSafeModeRequestProto {
+   required SafeModeActionProto action = 1;
+   optional bool checked = 2 [default = false];
+ }
+ 
+ message SetSafeModeResponseProto {
+   required bool result = 1;
+ }
+ 
+ message SaveNamespaceRequestProto {
+   optional uint64 timeWindow = 1 [default = 0];
+   optional uint64 txGap = 2 [default = 0];
+ }
+ 
+ message SaveNamespaceResponseProto { // void response
+   optional bool saved = 1 [default = true];
+ }
+ 
+ message RollEditsRequestProto { // no parameters
+ }
+ 
+ message RollEditsResponseProto { // response
+   required uint64 newSegmentTxId = 1;
+ }
+ 
+ message RestoreFailedStorageRequestProto {
+   required string arg = 1;
+ }
+ 
+ message RestoreFailedStorageResponseProto {
+     required bool result = 1;
+ }
+ 
+ message RefreshNodesRequestProto { // no parameters
+ }
+ 
+ message RefreshNodesResponseProto { // void response
+ }
+ 
+ message FinalizeUpgradeRequestProto { // no parameters
+ }
+ 
+ message FinalizeUpgradeResponseProto { // void response
+ }
+ 
+ enum RollingUpgradeActionProto {
+   QUERY = 1;
+   START = 2;
+   FINALIZE = 3;
+ }
+ 
+ message RollingUpgradeRequestProto {
+   required RollingUpgradeActionProto action = 1;
+ }
+ 
+ message RollingUpgradeInfoProto {
+   required RollingUpgradeStatusProto status = 1;
+   required uint64 startTime = 2;
+   required uint64 finalizeTime = 3;
+   required bool createdRollbackImages = 4;
+ }
+ 
+ message RollingUpgradeResponseProto {
+   optional RollingUpgradeInfoProto rollingUpgradeInfo= 1;
+ }
+ 
+ message ListCorruptFileBlocksRequestProto {
+   required string path = 1;
+   optional string cookie = 2;
+ }
+ 
+ message ListCorruptFileBlocksResponseProto {
+   required CorruptFileBlocksProto corrupt = 1;
+ }
+ 
+ message MetaSaveRequestProto {
+   required string filename = 1;
+ }
+ 
+ message MetaSaveResponseProto { // void response
+ }
+ 
+ message GetFileInfoRequestProto {
+   required string src = 1;
+ }
+ 
+ message GetFileInfoResponseProto {
+   optional HdfsFileStatusProto fs = 1;
+ }
+ 
+ message IsFileClosedRequestProto {
+   required string src = 1;
+ }
+ 
+ message IsFileClosedResponseProto {
+   required bool result = 1;
+ }
+ 
+ message CacheDirectiveInfoProto {
+   optional int64 id = 1;
+   optional string path = 2;
+   optional uint32 replication = 3;
+   optional string pool = 4;
+   optional CacheDirectiveInfoExpirationProto expiration = 5;
+ }
+ 
+ message CacheDirectiveInfoExpirationProto {
+   required int64 millis = 1;
+   required bool isRelative = 2;
+ }
+ 
+ message CacheDirectiveStatsProto {
+   required int64 bytesNeeded = 1;
+   required int64 bytesCached = 2;
+   required int64 filesNeeded = 3;
+   required int64 filesCached = 4;
+   required bool hasExpired = 5;
+ }
+ 
+ enum CacheFlagProto {
+   FORCE = 0x01;    // Ignore pool resource limits
+ }
+ 
+ message AddCacheDirectiveRequestProto {
+   required CacheDirectiveInfoProto info = 1;
+   optional uint32 cacheFlags = 2;  // bits set using CacheFlag
+ }
+ 
+ message AddCacheDirectiveResponseProto {
+   required int64 id = 1;
+ }
+ 
+ message ModifyCacheDirectiveRequestProto {
+   required CacheDirectiveInfoProto info = 1;
+   optional uint32 cacheFlags = 2;  // bits set using CacheFlag
+ }
+ 
+ message ModifyCacheDirectiveResponseProto {
+ }
+ 
+ message RemoveCacheDirectiveRequestProto {
+   required int64 id = 1;
+ }
+ 
+ message RemoveCacheDirectiveResponseProto {
+ }
+ 
+ message ListCacheDirectivesRequestProto {
+   required int64 prevId = 1;
+   required CacheDirectiveInfoProto filter = 2;
+ }
+ 
+ message CacheDirectiveEntryProto {
+   required CacheDirectiveInfoProto info = 1;
+   required CacheDirectiveStatsProto stats = 2;
+ }
+ 
+ message ListCacheDirectivesResponseProto {
+   repeated CacheDirectiveEntryProto elements = 1;
+   required bool hasMore = 2;
+ }
+ 
+ message CachePoolInfoProto {
+   optional string poolName = 1;
+   optional string ownerName = 2;
+   optional string groupName = 3;
+   optional int32 mode = 4;
+   optional int64 limit = 5;
+   optional int64 maxRelativeExpiry = 6;
+ }
+ 
+ message CachePoolStatsProto {
+   required int64 bytesNeeded = 1;
+   required int64 bytesCached = 2;
+   required int64 bytesOverlimit = 3;
+   required int64 filesNeeded = 4;
+   required int64 filesCached = 5;
+ }
+ 
+ message AddCachePoolRequestProto {
+   required CachePoolInfoProto info = 1;
+ }
+ 
+ message AddCachePoolResponseProto { // void response
+ }
+ 
+ message ModifyCachePoolRequestProto {
+   required CachePoolInfoProto info = 1;
+ }
+ 
+ message ModifyCachePoolResponseProto { // void response
+ }
+ 
+ message RemoveCachePoolRequestProto {
+   required string poolName = 1;
+ }
+ 
+ message RemoveCachePoolResponseProto { // void response
+ }
+ 
+ message ListCachePoolsRequestProto {
+   required string prevPoolName = 1;
+ }
+ 
+ message ListCachePoolsResponseProto {
+   repeated CachePoolEntryProto entries = 1;
+   required bool hasMore = 2;
+ }
+ 
+ message CachePoolEntryProto {
+   required CachePoolInfoProto info = 1;
+   required CachePoolStatsProto stats = 2;
+ }
+ 
+ message GetFileLinkInfoRequestProto {
+   required string src = 1;
+ }
+ 
+ message GetFileLinkInfoResponseProto {
+   optional HdfsFileStatusProto fs = 1;
+ }
+ 
+ message GetContentSummaryRequestProto {
+   required string path = 1;
+ }
+ 
+ message GetContentSummaryResponseProto {
+   required ContentSummaryProto summary = 1;
+ }
+ 
+ message SetQuotaRequestProto {
+   required string path = 1;
+   required uint64 namespaceQuota = 2;
+   required uint64 storagespaceQuota = 3;
+   optional StorageTypeProto storageType = 4;
+ }
+ 
+ message SetQuotaResponseProto { // void response
+ }
+ 
+ message FsyncRequestProto {
+   required string src = 1;
+   required string client = 2;
+   optional sint64 lastBlockLength = 3 [default = -1];
+   optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
+ }
+ 
+ message FsyncResponseProto { // void response
+ }
+ 
+ message SetTimesRequestProto {
+   required string src = 1;
+   required uint64 mtime = 2;
+   required uint64 atime = 3;
+ }
+ 
+ message SetTimesResponseProto { // void response
+ }
+ 
+ message CreateSymlinkRequestProto {
+   required string target = 1;
+   required string link = 2;
+   required FsPermissionProto dirPerm = 3;
+   required bool createParent = 4;
+ }
+ 
+ message CreateSymlinkResponseProto { // void response
+ }
+ 
+ message GetLinkTargetRequestProto {
+   required string path = 1;
+ }
+ message GetLinkTargetResponseProto {
+   optional string targetPath = 1;
+ }
+ 
+ message UpdateBlockForPipelineRequestProto {
+   required ExtendedBlockProto block = 1;
+   required string clientName = 2;
+ }
+ 
+ message UpdateBlockForPipelineResponseProto {
+   required LocatedBlockProto block = 1;
+ }
+ 
+ message UpdatePipelineRequestProto {
+   required string clientName = 1;
+   required ExtendedBlockProto oldBlock = 2;
+   required ExtendedBlockProto newBlock = 3;
+   repeated DatanodeIDProto newNodes = 4;
+   repeated string storageIDs = 5;
+ }
+ 
+ message UpdatePipelineResponseProto { // void response
+ }
+ 
+ message SetBalancerBandwidthRequestProto {
+   required int64 bandwidth = 1;
+ }
+ 
+ message SetBalancerBandwidthResponseProto { // void response
+ }
+ 
+ message GetDataEncryptionKeyRequestProto { // no parameters
+ }
+ 
+ message GetDataEncryptionKeyResponseProto {
+   optional DataEncryptionKeyProto dataEncryptionKey = 1;
+ }
+ 
+ message CreateSnapshotRequestProto {
+   required string snapshotRoot = 1;
+   optional string snapshotName = 2;
+ }
+ 
+ message CreateSnapshotResponseProto {
+   required string snapshotPath = 1;
+ }
+ 
+ message RenameSnapshotRequestProto {
+   required string snapshotRoot = 1;
+   required string snapshotOldName = 2;
+   required string snapshotNewName = 3;
+ }
+ 
+ message RenameSnapshotResponseProto { // void response
+ }
+ 
+ message AllowSnapshotRequestProto {
+   required string snapshotRoot = 1;
+ }
+ 
+ message AllowSnapshotResponseProto {
+ }
+ 
+ message DisallowSnapshotRequestProto {
+   required string snapshotRoot = 1;
+ }
+ 
+ message DisallowSnapshotResponseProto {
+ }
+ 
+ message DeleteSnapshotRequestProto {
+   required string snapshotRoot = 1;
+   required string snapshotName = 2;
+ }
+ 
+ message DeleteSnapshotResponseProto { // void response
+ }
+ 
+ message CheckAccessRequestProto {
+   required string path = 1;
+   required AclEntryProto.FsActionProto mode = 2;
+ }
+ 
+ message CheckAccessResponseProto { // void response
+ }
+ 
+ message GetCurrentEditLogTxidRequestProto {
+ }
+ 
+ message GetCurrentEditLogTxidResponseProto {
+   required int64 txid = 1;
+ }
+ 
+ message GetEditsFromTxidRequestProto {
+   required int64 txid = 1;
+ }
+ 
+ message GetEditsFromTxidResponseProto {
+   required EventsListProto eventsList = 1;
+ }
+ 
+ service ClientNamenodeProtocol {
+   rpc getBlockLocations(GetBlockLocationsRequestProto)
+       returns(GetBlockLocationsResponseProto);
+   rpc getServerDefaults(GetServerDefaultsRequestProto)
+       returns(GetServerDefaultsResponseProto);
+   rpc create(CreateRequestProto)returns(CreateResponseProto);
+   rpc append(AppendRequestProto) returns(AppendResponseProto);
+   rpc setReplication(SetReplicationRequestProto)
+       returns(SetReplicationResponseProto);
+   rpc setStoragePolicy(SetStoragePolicyRequestProto)
+       returns(SetStoragePolicyResponseProto);
+   rpc getStoragePolicy(GetStoragePolicyRequestProto)
+       returns(GetStoragePolicyResponseProto);
+   rpc getStoragePolicies(GetStoragePoliciesRequestProto)
+       returns(GetStoragePoliciesResponseProto);
+   rpc setPermission(SetPermissionRequestProto)
+       returns(SetPermissionResponseProto);
+   rpc setOwner(SetOwnerRequestProto) returns(SetOwnerResponseProto);
+   rpc abandonBlock(AbandonBlockRequestProto) returns(AbandonBlockResponseProto);
+   rpc addBlock(AddBlockRequestProto) returns(AddBlockResponseProto);
+   rpc getAdditionalDatanode(GetAdditionalDatanodeRequestProto)
+       returns(GetAdditionalDatanodeResponseProto);
+   rpc complete(CompleteRequestProto) returns(CompleteResponseProto);
+   rpc reportBadBlocks(ReportBadBlocksRequestProto)
+       returns(ReportBadBlocksResponseProto);
+   rpc concat(ConcatRequestProto) returns(ConcatResponseProto);
+   rpc truncate(TruncateRequestProto) returns(TruncateResponseProto);
+   rpc rename(RenameRequestProto) returns(RenameResponseProto);
+   rpc rename2(Rename2RequestProto) returns(Rename2ResponseProto);
+   rpc delete(DeleteRequestProto) returns(DeleteResponseProto);
+   rpc mkdirs(MkdirsRequestProto) returns(MkdirsResponseProto);
+   rpc getListing(GetListingRequestProto) returns(GetListingResponseProto);
+   rpc renewLease(RenewLeaseRequestProto) returns(RenewLeaseResponseProto);
+   rpc recoverLease(RecoverLeaseRequestProto)
+       returns(RecoverLeaseResponseProto);
+   rpc getFsStats(GetFsStatusRequestProto) returns(GetFsStatsResponseProto);
+   rpc getDatanodeReport(GetDatanodeReportRequestProto)
+       returns(GetDatanodeReportResponseProto);
+   rpc getDatanodeStorageReport(GetDatanodeStorageReportRequestProto)
+       returns(GetDatanodeStorageReportResponseProto);
+   rpc getPreferredBlockSize(GetPreferredBlockSizeRequestProto)
+       returns(GetPreferredBlockSizeResponseProto);
+   rpc setSafeMode(SetSafeModeRequestProto)
+       returns(SetSafeModeResponseProto);
+   rpc saveNamespace(SaveNamespaceRequestProto)
+       returns(SaveNamespaceResponseProto);
+   rpc rollEdits(RollEditsRequestProto)
+       returns(RollEditsResponseProto);
+   rpc restoreFailedStorage(RestoreFailedStorageRequestProto)
+       returns(RestoreFailedStorageResponseProto);
+   rpc refreshNodes(RefreshNodesRequestProto) returns(RefreshNodesResponseProto);
+   rpc finalizeUpgrade(FinalizeUpgradeRequestProto)
+       returns(FinalizeUpgradeResponseProto);
+   rpc rollingUpgrade(RollingUpgradeRequestProto)
+       returns(RollingUpgradeResponseProto);
+   rpc listCorruptFileBlocks(ListCorruptFileBlocksRequestProto)
+       returns(ListCorruptFileBlocksResponseProto);
+   rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);
+   rpc getFileInfo(GetFileInfoRequestProto) returns(GetFileInfoResponseProto);
+   rpc addCacheDirective(AddCacheDirectiveRequestProto)
+       returns (AddCacheDirectiveResponseProto);
+   rpc modifyCacheDirective(ModifyCacheDirectiveRequestProto)
+       returns (ModifyCacheDirectiveResponseProto);
+   rpc removeCacheDirective(RemoveCacheDirectiveRequestProto)
+       returns (RemoveCacheDirectiveResponseProto);
+   rpc listCacheDirectives(ListCacheDirectivesRequestProto)
+       returns (ListCacheDirectivesResponseProto);
+   rpc addCachePool(AddCachePoolRequestProto)
+       returns(AddCachePoolResponseProto);
+   rpc modifyCachePool(ModifyCachePoolRequestProto)
+       returns(ModifyCachePoolResponseProto);
+   rpc removeCachePool(RemoveCachePoolRequestProto)
+       returns(RemoveCachePoolResponseProto);
+   rpc listCachePools(ListCachePoolsRequestProto)
+       returns(ListCachePoolsResponseProto);
+   rpc getFileLinkInfo(GetFileLinkInfoRequestProto)
+       returns(GetFileLinkInfoResponseProto);
+   rpc getContentSummary(GetContentSummaryRequestProto)
+       returns(GetContentSummaryResponseProto);
+   rpc setQuota(SetQuotaRequestProto) returns(SetQuotaResponseProto);
+   rpc fsync(FsyncRequestProto) returns(FsyncResponseProto);
+   rpc setTimes(SetTimesRequestProto) returns(SetTimesResponseProto);
+   rpc createSymlink(CreateSymlinkRequestProto)
+       returns(CreateSymlinkResponseProto);
+   rpc getLinkTarget(GetLinkTargetRequestProto)
+       returns(GetLinkTargetResponseProto);
+   rpc updateBlockForPipeline(UpdateBlockForPipelineRequestProto)
+       returns(UpdateBlockForPipelineResponseProto);
+   rpc updatePipeline(UpdatePipelineRequestProto)
+       returns(UpdatePipelineResponseProto);
+   rpc getDelegationToken(hadoop.common.GetDelegationTokenRequestProto)
+       returns(hadoop.common.GetDelegationTokenResponseProto);
+   rpc renewDelegationToken(hadoop.common.RenewDelegationTokenRequestProto)
+       returns(hadoop.common.RenewDelegationTokenResponseProto);
+   rpc cancelDelegationToken(hadoop.common.CancelDelegationTokenRequestProto)
+       returns(hadoop.common.CancelDelegationTokenResponseProto);
+   rpc setBalancerBandwidth(SetBalancerBandwidthRequestProto)
+       returns(SetBalancerBandwidthResponseProto);
+   rpc getDataEncryptionKey(GetDataEncryptionKeyRequestProto)
+       returns(GetDataEncryptionKeyResponseProto);
+   rpc createSnapshot(CreateSnapshotRequestProto)
+       returns(CreateSnapshotResponseProto);
+   rpc renameSnapshot(RenameSnapshotRequestProto)
+       returns(RenameSnapshotResponseProto);
+   rpc allowSnapshot(AllowSnapshotRequestProto)
+       returns(AllowSnapshotResponseProto);
+   rpc disallowSnapshot(DisallowSnapshotRequestProto)
+       returns(DisallowSnapshotResponseProto);   
+   rpc getSnapshottableDirListing(GetSnapshottableDirListingRequestProto)
+       returns(GetSnapshottableDirListingResponseProto);
+   rpc deleteSnapshot(DeleteSnapshotRequestProto)
+       returns(DeleteSnapshotResponseProto);
+   rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)
+       returns(GetSnapshotDiffReportResponseProto);
+   rpc isFileClosed(IsFileClosedRequestProto)
+       returns(IsFileClosedResponseProto);
+   rpc modifyAclEntries(ModifyAclEntriesRequestProto)
+       returns(ModifyAclEntriesResponseProto);
+   rpc removeAclEntries(RemoveAclEntriesRequestProto)
+       returns(RemoveAclEntriesResponseProto);
+   rpc removeDefaultAcl(RemoveDefaultAclRequestProto)
+       returns(RemoveDefaultAclResponseProto);
+   rpc removeAcl(RemoveAclRequestProto)
+       returns(RemoveAclResponseProto);
+   rpc setAcl(SetAclRequestProto)
+       returns(SetAclResponseProto);
+   rpc getAclStatus(GetAclStatusRequestProto)
+       returns(GetAclStatusResponseProto);
+   rpc setXAttr(SetXAttrRequestProto)
+       returns(SetXAttrResponseProto);
+   rpc getXAttrs(GetXAttrsRequestProto)
+       returns(GetXAttrsResponseProto);
+   rpc listXAttrs(ListXAttrsRequestProto)
+       returns(ListXAttrsResponseProto);
+   rpc removeXAttr(RemoveXAttrRequestProto)
+       returns(RemoveXAttrResponseProto);
+   rpc checkAccess(CheckAccessRequestProto)
+       returns(CheckAccessResponseProto);
+   rpc createEncryptionZone(CreateEncryptionZoneRequestProto)
+       returns(CreateEncryptionZoneResponseProto);
+   rpc listEncryptionZones(ListEncryptionZonesRequestProto)
+       returns(ListEncryptionZonesResponseProto);
+   rpc getEZForPath(GetEZForPathRequestProto)
+       returns(GetEZForPathResponseProto);
++  rpc createErasureCodingZone(CreateErasureCodingZoneRequestProto)
++      returns(CreateErasureCodingZoneResponseProto);
+   rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto)
+       returns(GetCurrentEditLogTxidResponseProto);
+   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
+       returns(GetEditsFromTxidResponseProto);
++  rpc getErasureCodingPolicies(GetErasureCodingPoliciesRequestProto)
++      returns(GetErasureCodingPoliciesResponseProto);
++  rpc getErasureCodingZone(GetErasureCodingZoneRequestProto)
++      returns(GetErasureCodingZoneResponseProto);
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
index 0000000,0000000..d27f782
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
@@@ -1,0 -1,0 +1,68 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     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.
++ */
++ 
++option java_package = "org.apache.hadoop.hdfs.protocol.proto";
++option java_outer_classname = "ErasureCodingProtos";
++option java_generate_equals_and_hash = true;
++package hadoop.hdfs;
++
++import "hdfs.proto";
++
++/**
++ * ErasureCodingZone
++ */
++message ErasureCodingZoneProto {
++  required string dir = 1;
++  required ErasureCodingPolicyProto ecPolicy = 2;
++}
++
++message CreateErasureCodingZoneRequestProto {
++  required string src = 1;
++  optional ErasureCodingPolicyProto ecPolicy = 2;
++}
++
++message CreateErasureCodingZoneResponseProto {
++}
++
++message GetErasureCodingPoliciesRequestProto { // void request
++}
++
++message GetErasureCodingPoliciesResponseProto {
++  repeated ErasureCodingPolicyProto ecPolicies = 1;
++}
++
++message GetErasureCodingZoneRequestProto {
++  required string src = 1; // path to get the zone info
++}
++
++message GetErasureCodingZoneResponseProto {
++  optional ErasureCodingZoneProto ECZone = 1;
++}
++
++/**
++ * Block erasure coding recovery info
++ */
++message BlockECRecoveryInfoProto {
++  required ExtendedBlockProto block = 1;
++  required DatanodeInfosProto sourceDnInfos = 2;
++  required DatanodeInfosProto targetDnInfos = 3;
++  required StorageUuidsProto targetStorageUuids = 4;
++  required StorageTypesProto targetStorageTypes = 5;
++  repeated uint32 liveBlockIndices = 6;
++  required ErasureCodingPolicyProto ecPolicy = 7;
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index 0000000,86fb462..63fe90c
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@@ -1,0 -1,611 +1,648 @@@
+ /**
+  * 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.
+  */
+ 
+ /**
+  * These .proto interfaces are private and stable.
+  * Please see http://wiki.apache.org/hadoop/Compatibility
+  * for what changes are allowed for a *stable* .proto interface.
+  */
+ 
+ // This file contains protocol buffers that are used throughout HDFS -- i.e.
+ // by the client, server, and data transfer protocols.
+ 
+ 
+ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+ option java_outer_classname = "HdfsProtos";
+ option java_generate_equals_and_hash = true;
+ package hadoop.hdfs;
+ 
+ import "Security.proto";
+ 
+ /**
+  * Extended block idenfies a block
+  */
+ message ExtendedBlockProto {
+   required string poolId = 1;   // Block pool id - gloablly unique across clusters
+   required uint64 blockId = 2;  // the local id within a pool
+   required uint64 generationStamp = 3;
+   optional uint64 numBytes = 4 [default = 0];  // len does not belong in ebid 
+                                                // here for historical reasons
+ }
+ 
+ /**
+  * Identifies a Datanode
+  */
+ message DatanodeIDProto {
+   required string ipAddr = 1;    // IP address
+   required string hostName = 2;  // hostname
+   required string datanodeUuid = 3;     // UUID assigned to the Datanode. For
+                                         // upgraded clusters this is the same
+                                         // as the original StorageID of the
+                                         // Datanode.
+   required uint32 xferPort = 4;  // data streaming port
+   required uint32 infoPort = 5;  // datanode http port
+   required uint32 ipcPort = 6;   // ipc server port
+   optional uint32 infoSecurePort = 7 [default = 0]; // datanode https port
+ }
+ 
+ /**
+  * Datanode local information
+  */
+ message DatanodeLocalInfoProto {
+   required string softwareVersion = 1;
+   required string configVersion = 2;
+   required uint64 uptime = 3;
+ }
+ 
+ /**
+  * DatanodeInfo array
+  */
+ message DatanodeInfosProto {
+   repeated DatanodeInfoProto datanodes = 1;
+ }
+ 
+ /**
+  * The status of a Datanode
+  */
+ message DatanodeInfoProto {
+   required DatanodeIDProto id = 1;
+   optional uint64 capacity = 2 [default = 0];
+   optional uint64 dfsUsed = 3 [default = 0];
+   optional uint64 remaining = 4 [default = 0];
+   optional uint64 blockPoolUsed = 5 [default = 0];
+   optional uint64 lastUpdate = 6 [default = 0];
+   optional uint32 xceiverCount = 7 [default = 0];
+   optional string location = 8;
+   enum AdminState {
+     NORMAL = 0;
+     DECOMMISSION_INPROGRESS = 1;
+     DECOMMISSIONED = 2;
+   }
+ 
+   optional AdminState adminState = 10 [default = NORMAL];
+   optional uint64 cacheCapacity = 11 [default = 0];
+   optional uint64 cacheUsed = 12 [default = 0];
+   optional uint64 lastUpdateMonotonic = 13 [default = 0];
+ }
+ 
+ /**
+  * Represents a storage available on the datanode
+  */
+ message DatanodeStorageProto {
+   enum StorageState {
+     NORMAL = 0;
+     READ_ONLY_SHARED = 1;
+   }
+ 
+   required string storageUuid = 1;
+   optional StorageState state = 2 [default = NORMAL];
+   optional StorageTypeProto storageType = 3 [default = DISK];
+ }
+ 
+ message StorageReportProto {
+   required string storageUuid = 1 [ deprecated = true ];
+   optional bool failed = 2 [ default = false ];
+   optional uint64 capacity = 3 [ default = 0 ];
+   optional uint64 dfsUsed = 4 [ default = 0 ];
+   optional uint64 remaining = 5 [ default = 0 ];
+   optional uint64 blockPoolUsed = 6 [ default = 0 ];
+   optional DatanodeStorageProto storage = 7; // supersedes StorageUuid
+ }
+ 
+ /**
+  * Summary of a file or directory
+  */
+ message ContentSummaryProto {
+   required uint64 length = 1;
+   required uint64 fileCount = 2;
+   required uint64 directoryCount = 3;
+   required uint64 quota = 4;
+   required uint64 spaceConsumed = 5;
+   required uint64 spaceQuota = 6;
+   optional StorageTypeQuotaInfosProto typeQuotaInfos = 7;
+ }
+ 
+ /**
+  * Storage type quota and usage information of a file or directory
+  */
+ message StorageTypeQuotaInfosProto {
+   repeated StorageTypeQuotaInfoProto typeQuotaInfo = 1;
+ }
+ 
+ message StorageTypeQuotaInfoProto {
+   required StorageTypeProto type = 1;
+   required uint64 quota = 2;
+   required uint64 consumed = 3;
+ }
+ 
+ /**
+  * Contains a list of paths corresponding to corrupt files and a cookie
+  * used for iterative calls to NameNode.listCorruptFileBlocks.
+  *
+  */
+ message CorruptFileBlocksProto {
+  repeated string files = 1;
+  required string   cookie = 2;
+ }
+ 
+ /**
+  * File or Directory permision - same spec as posix
+  */
+ message FsPermissionProto {
+   required uint32 perm = 1;       // Actually a short - only 16bits used
+ }
+ 
+ /**
+  * Types of recognized storage media.
+  */
+ enum StorageTypeProto {
+   DISK = 1;
+   SSD = 2;
+   ARCHIVE = 3;
+   RAM_DISK = 4;
+ }
+ 
+ /**
+  * A list of storage types. 
+  */
+ message StorageTypesProto {
+   repeated StorageTypeProto storageTypes = 1;
+ }
+ 
+ /**
+  * Block replica storage policy.
+  */
+ message BlockStoragePolicyProto {
+   required uint32 policyId = 1;
+   required string name = 2;
+   // a list of storage types for storing the block replicas when creating a
+   // block.
+   required StorageTypesProto creationPolicy = 3;
+   // A list of storage types for creation fallback storage.
+   optional StorageTypesProto creationFallbackPolicy = 4;
+   optional StorageTypesProto replicationFallbackPolicy = 5;
+ }
+ 
+ /**
+  * A list of storage IDs. 
+  */
+ message StorageUuidsProto {
+   repeated string storageUuids = 1;
+ }
+ 
+ /**
+  * A LocatedBlock gives information about a block and its location.
+  */ 
+ message LocatedBlockProto {
+   required ExtendedBlockProto b  = 1;
+   required uint64 offset = 2;           // offset of first byte of block in the file
+   repeated DatanodeInfoProto locs = 3;  // Locations ordered by proximity to client ip
+   required bool corrupt = 4;            // true if all replicas of a block are corrupt, else false
+                                         // If block has few corrupt replicas, they are filtered and 
+                                         // their locations are not part of this object
+ 
+   required hadoop.common.TokenProto blockToken = 5;
+   repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
+   repeated StorageTypeProto storageTypes = 7;
+   repeated string storageIDs = 8;
++
++  // striped block related fields
++  repeated uint32 blockIndex = 9; // used for striped block to indicate block index for each storage
++  repeated hadoop.common.TokenProto blockTokens = 10; // each internal block has a block token
+ }
+ 
+ message DataEncryptionKeyProto {
+   required uint32 keyId = 1;
+   required string blockPoolId = 2;
+   required bytes nonce = 3;
+   required bytes encryptionKey = 4;
+   required uint64 expiryDate = 5;
+   optional string encryptionAlgorithm = 6;
+ }
+ 
+ /**
+  * Cipher suite.
+  */
+ enum CipherSuiteProto {
+     UNKNOWN = 1;
+     AES_CTR_NOPADDING = 2;
+ }
+ 
+ /**
+  * Crypto protocol version used to access encrypted files.
+  */
+ enum CryptoProtocolVersionProto {
+     UNKNOWN_PROTOCOL_VERSION = 1;
+     ENCRYPTION_ZONES = 2;
+ }
+ 
+ /**
+  * Encryption information for a file.
+  */
+ message FileEncryptionInfoProto {
+   required CipherSuiteProto suite = 1;
+   required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
+   required bytes key = 3;
+   required bytes iv = 4;
+   required string keyName = 5;
+   required string ezKeyVersionName = 6;
+ }
+ 
+ /**
+  * Encryption information for an individual
+  * file within an encryption zone
+  */
+ message PerFileEncryptionInfoProto {
+   required bytes key = 1;
+   required bytes iv = 2;
+   required string ezKeyVersionName = 3;
+ }
+ 
+ /**
+  * Encryption information for an encryption
+  * zone
+  */
+ message ZoneEncryptionInfoProto {
+   required CipherSuiteProto suite = 1;
+   required CryptoProtocolVersionProto cryptoProtocolVersion = 2;
+   required string keyName = 3;
+ }
+ 
+ /**
+  * Cipher option
+  */
+ message CipherOptionProto {
+   required CipherSuiteProto suite = 1;
+   optional bytes inKey = 2;
+   optional bytes inIv = 3;
+   optional bytes outKey = 4;
+   optional bytes outIv = 5;
+ }
+ 
+ /**
+  * A set of file blocks and their locations.
+  */
+ message LocatedBlocksProto {
+   required uint64 fileLength = 1;
+   repeated LocatedBlockProto blocks = 2;
+   required bool underConstruction = 3;
+   optional LocatedBlockProto lastBlock = 4;
+   required bool isLastBlockComplete = 5;
+   optional FileEncryptionInfoProto fileEncryptionInfo = 6;
++
++  // Optional field for erasure coding
++  optional ErasureCodingPolicyProto ecPolicy = 7;
++}
++
++/**
++ * ECSchema options entry
++ */
++message ECSchemaOptionEntryProto {
++  required string key = 1;
++  required string value = 2;
++}
++
++/**
++ * ECSchema for erasurecoding
++ */
++message ECSchemaProto {
++  required string codecName = 1;
++  required uint32 dataUnits = 2;
++  required uint32 parityUnits = 3;
++  repeated ECSchemaOptionEntryProto options = 4;
++}
++
++message ErasureCodingPolicyProto {
++  required string name = 1;
++  required ECSchemaProto schema = 2;
++  required uint32 cellSize = 3;
+ }
+ 
+ /**
+  * Status of a file, directory or symlink
+  * Optionally includes a file's block locations if requested by client on the rpc call.
+  */
+ message HdfsFileStatusProto {
+   enum FileType {
+     IS_DIR = 1;
+     IS_FILE = 2;
+     IS_SYMLINK = 3;
+   }
+   required FileType fileType = 1;
+   required bytes path = 2;          // local name of inode encoded java UTF8
+   required uint64 length = 3;
+   required FsPermissionProto permission = 4;
+   required string owner = 5;
+   required string group = 6;
+   required uint64 modification_time = 7;
+   required uint64 access_time = 8;
+ 
+   // Optional fields for symlink
+   optional bytes symlink = 9;             // if symlink, target encoded java UTF8 
+ 
+   // Optional fields for file
+   optional uint32 block_replication = 10 [default = 0]; // only 16bits used
+   optional uint64 blocksize = 11 [default = 0];
+   optional LocatedBlocksProto locations = 12;  // suppled only if asked by client
+ 
+   // Optional field for fileId
+   optional uint64 fileId = 13 [default = 0]; // default as an invalid id
+   optional int32 childrenNum = 14 [default = -1];
+   // Optional field for file encryption
+   optional FileEncryptionInfoProto fileEncryptionInfo = 15;
+ 
+   optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
 -} 
++
++  // Optional field for erasure coding
++  optional ErasureCodingPolicyProto ecPolicy = 17;
++}
+ 
+ /**
+  * Checksum algorithms/types used in HDFS
+  * Make sure this enum's integer values match enum values' id properties defined
+  * in org.apache.hadoop.util.DataChecksum.Type
+  */
+ enum ChecksumTypeProto {
+   CHECKSUM_NULL = 0;
+   CHECKSUM_CRC32 = 1;
+   CHECKSUM_CRC32C = 2;
+ }
+ 
+ /**
+  * HDFS Server Defaults
+  */
+ message FsServerDefaultsProto {
+   required uint64 blockSize = 1;
+   required uint32 bytesPerChecksum = 2;
+   required uint32 writePacketSize = 3;
+   required uint32 replication = 4; // Actually a short - only 16 bits used
+   required uint32 fileBufferSize = 5;
+   optional bool encryptDataTransfer = 6 [default = false];
+   optional uint64 trashInterval = 7 [default = 0];
+   optional ChecksumTypeProto checksumType = 8 [default = CHECKSUM_CRC32];
+ }
+ 
+ 
+ /**
+  * Directory listing
+  */
+ message DirectoryListingProto {
+   repeated HdfsFileStatusProto partialListing = 1;
+   required uint32 remainingEntries  = 2;
+ }
+ 
+ /**
+  * Status of a snapshottable directory: besides the normal information for 
+  * a directory status, also include snapshot quota, number of snapshots, and
+  * the full path of the parent directory. 
+  */
+ message SnapshottableDirectoryStatusProto {
+   required HdfsFileStatusProto dirStatus = 1;
+ 
+   // Fields specific for snapshottable directory
+   required uint32 snapshot_quota = 2;
+   required uint32 snapshot_number = 3;
+   required bytes parent_fullpath = 4;
+ }
+ 
+ /**
+  * Snapshottable directory listing
+  */
+ message SnapshottableDirectoryListingProto {
+   repeated SnapshottableDirectoryStatusProto snapshottableDirListing = 1;
+ }
+ 
+ /**
+  * Snapshot diff report entry
+  */
+ message SnapshotDiffReportEntryProto {
+   required bytes fullpath = 1;
+   required string modificationLabel = 2;
+   optional bytes targetPath = 3;
+ }
+ 
+ /**
+  * Snapshot diff report
+  */
+ message SnapshotDiffReportProto {
+   // full path of the directory where snapshots were taken
+   required string snapshotRoot = 1;
+   required string fromSnapshot = 2;
+   required string toSnapshot = 3;
+   repeated SnapshotDiffReportEntryProto diffReportEntries = 4;
+ }
+ 
+ /**
+  * Common node information shared by all the nodes in the cluster
+  */
+ message StorageInfoProto {
+   required uint32 layoutVersion = 1; // Layout version of the file system
+   required uint32 namespceID = 2;    // File system namespace ID
+   required string clusterID = 3;     // ID of the cluster
+   required uint64 cTime = 4;         // File system creation time
+ }
+ 
+ /**
+  * Information sent by a namenode to identify itself to the primary namenode.
+  */
+ message NamenodeRegistrationProto {
+   required string rpcAddress = 1;    // host:port of the namenode RPC address
+   required string httpAddress = 2;   // host:port of the namenode http server
+   enum NamenodeRoleProto {
+     NAMENODE = 1;
+     BACKUP = 2;
+     CHECKPOINT = 3;
+   }
+   required StorageInfoProto storageInfo = 3;  // Node information
+   optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
+ }
+ 
+ /**
+  * Unique signature to identify checkpoint transactions.
+  */
+ message CheckpointSignatureProto {
+   required string blockPoolId = 1;
+   required uint64 mostRecentCheckpointTxId = 2;
+   required uint64 curSegmentTxId = 3;
+   required StorageInfoProto storageInfo = 4;
+ }
+ 
+ /**
+  * Command sent from one namenode to another namenode.
+  */
+ message NamenodeCommandProto {
+   enum Type {
+     NamenodeCommand = 0;      // Base command
+     CheckPointCommand = 1;    // Check point command
+   }
+   required uint32 action = 1;
+   required Type type = 2;
+   optional CheckpointCommandProto checkpointCmd = 3;
+ }
+ 
+ /**
+  * Command returned from primary to checkpointing namenode.
+  * This command has checkpoint signature that identifies
+  * checkpoint transaction and is needed for further
+  * communication related to checkpointing.
+  */
+ message CheckpointCommandProto {
+   // Unique signature to identify checkpoint transation
+   required CheckpointSignatureProto signature = 1; 
+ 
+   // If true, return transfer image to primary upon the completion of checkpoint
+   required bool needToReturnImage = 2;
+ }
+ 
+ /**
+  * Block information
+  *
+  * Please be wary of adding additional fields here, since INodeFiles
+  * need to fit in PB's default max message size of 64MB.
+  * We restrict the max # of blocks per file
+  * (dfs.namenode.fs-limits.max-blocks-per-file), but it's better
+  * to avoid changing this.
+  */
+ message BlockProto {
+   required uint64 blockId = 1;
+   required uint64 genStamp = 2;
+   optional uint64 numBytes = 3 [default = 0];
+ }
+ 
+ /**
+  * Block and datanodes where is it located
+  */
+ message BlockWithLocationsProto {
+   required BlockProto block = 1;   // Block
+   repeated string datanodeUuids = 2; // Datanodes with replicas of the block
+   repeated string storageUuids = 3;  // Storages with replicas of the block
+   repeated StorageTypeProto storageTypes = 4;
++
++  optional bytes indices = 5;
++  optional uint32 dataBlockNum = 6;
+ }
+ 
+ /**
+  * List of block with locations
+  */
+ message BlocksWithLocationsProto {
+   repeated BlockWithLocationsProto blocks = 1;
+ }
+ 
+ /**
+  * Editlog information with available transactions
+  */
+ message RemoteEditLogProto {
+   required uint64 startTxId = 1;  // Starting available edit log transaction
+   required uint64 endTxId = 2;    // Ending available edit log transaction
+   optional bool isInProgress = 3 [default = false];
+ }
+ 
+ /**
+  * Enumeration of editlogs available on a remote namenode
+  */
+ message RemoteEditLogManifestProto {
+   repeated RemoteEditLogProto logs = 1;
+ }
+ 
+ /**
+  * Namespace information that describes namespace on a namenode
+  */
+ message NamespaceInfoProto {
+   required string buildVersion = 1;         // Software revision version (e.g. an svn or git revision)
+   required uint32 unused = 2;               // Retained for backward compatibility
+   required string blockPoolID = 3;          // block pool used by the namespace
+   required StorageInfoProto storageInfo = 4;// Node information
+   required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
+   optional uint64 capabilities = 6 [default = 0]; // feature flags
+ }
+ 
+ /**
+  * Block access token information
+  */
+ message BlockKeyProto {
+   required uint32 keyId = 1;      // Key identifier
+   required uint64 expiryDate = 2; // Expiry time in milliseconds
+   optional bytes keyBytes = 3;    // Key secret
+ }
+ 
+ /**
+  * Current key and set of block keys at the namenode.
+  */
+ message ExportedBlockKeysProto {
+   required bool isBlockTokenEnabled = 1;
+   required uint64 keyUpdateInterval = 2;
+   required uint64 tokenLifeTime = 3;
+   required BlockKeyProto currentKey = 4;
+   repeated BlockKeyProto allKeys = 5;
+ }
+ 
+ /**
+  * State of a block replica at a datanode
+  */
+ enum ReplicaStateProto {
+   FINALIZED = 0;  // State of a replica when it is not modified
+   RBW = 1;        // State of replica that is being written to
+   RWR = 2;        // State of replica that is waiting to be recovered
+   RUR = 3;        // State of replica that is under recovery
+   TEMPORARY = 4;  // State of replica that is created for replication
+ }
+ 
+ /**
+  * Block that needs to be recovered with at a given location
+  */
+ message RecoveringBlockProto {
+   required uint64 newGenStamp = 1;        // New genstamp post recovery
+   required LocatedBlockProto block = 2;   // Block to be recovered
+   optional BlockProto truncateBlock = 3;  // New block for recovery (truncate)
+ }
+ 
+ /**
+  * void request
+  */
+ message VersionRequestProto {
+ }
+ 
+ /**
+  * Version response from namenode.
+  */
+ message VersionResponseProto {
+   required NamespaceInfoProto info = 1;
+ }
+ 
+ /**
+  * Information related to a snapshot
+  * TODO: add more information
+  */
+ message SnapshotInfoProto {
+   required string snapshotName = 1;
+   required string snapshotRoot = 2;
+   required FsPermissionProto permission = 3;
+   required string owner = 4;
+   required string group = 5;
+   required string createTime = 6;
+   // TODO: do we need access time?
+ }
+ 
+ /**
+  * Rolling upgrade status
+  */
+ message RollingUpgradeStatusProto {
+   required string blockPoolId = 1;
+   optional bool finalized = 2 [default = false];
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index 5ee7f4d,852b040..8b1ede8
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@@ -15,47 -15,46 +15,47 @@@
  # See the License for the specific language governing permissions and
  # limitations under the License.
  
+ MYNAME="${BASH_SOURCE-$0}"
+ 
  function hadoop_usage
  {
-   echo "Usage: hdfs [--config confdir] [--daemon (start|stop|status)]"
-   echo "           [--loglevel loglevel] COMMAND"
-   echo "       where COMMAND is one of:"
-   echo "  balancer             run a cluster balancing utility"
-   echo "  cacheadmin           configure the HDFS cache"
-   echo "  classpath            prints the class path needed to get the"
-   echo "                       Hadoop jar and the required libraries"
-   echo "  crypto               configure HDFS encryption zones"
-   echo "  datanode             run a DFS datanode"
-   echo "  dfs                  run a filesystem command on the file system"
-   echo "  dfsadmin             run a DFS admin client"
-   echo "  erasurecode          configure HDFS erasure coding zones"
-   echo "  fetchdt              fetch a delegation token from the NameNode"
-   echo "  fsck                 run a DFS filesystem checking utility"
-   echo "  getconf              get config values from configuration"
-   echo "  groups               get the groups which users belong to"
-   echo "  haadmin              run a DFS HA admin client"
-   echo "  jmxget               get JMX exported values from NameNode or DataNode."
-   echo "  journalnode          run the DFS journalnode"
-   echo "  lsSnapshottableDir   list all snapshottable dirs owned by the current user"
-   echo "                               Use -help to see options"
-   echo "  mover                run a utility to move block replicas across"
-   echo "                       storage types"
-   echo "  namenode             run the DFS namenode"
-   echo "                               Use -format to initialize the DFS filesystem"
-   echo "  nfs3                 run an NFS version 3 gateway"
-   echo "  oev                  apply the offline edits viewer to an edits file"
-   echo "  oiv                  apply the offline fsimage viewer to an fsimage"
-   echo "  oiv_legacy           apply the offline fsimage viewer to a legacy fsimage"
-   echo "  portmap              run a portmap service"
-   echo "  secondarynamenode    run the DFS secondary namenode"
-   echo "  snapshotDiff         diff two snapshots of a directory or diff the"
-   echo "                       current directory contents with a snapshot"
-   echo "  storagepolicies      list/get/set block storage policies"
-   echo "  version              print the version"
-   echo "  zkfc                 run the ZK Failover Controller daemon"
-   echo ""
-   echo "Most commands print help when invoked w/o parameters."
-   # There are also debug commands, but they don't show up in this listing.
+   hadoop_add_option "--buildpaths" "attempt to add class files from build tree"
+   hadoop_add_option "--daemon (start|status|stop)" "operate on a daemon"
+   hadoop_add_option "--hostnames list[,of,host,names]" "hosts to use in slave mode"
+   hadoop_add_option "--loglevel level" "set the log4j level for this command"
+   hadoop_add_option "--hosts filename" "list of hosts to use in slave mode"
+   hadoop_add_option "--slaves" "turn on slave mode"
+ 
+   hadoop_add_subcommand "balancer" "run a cluster balancing utility"
+   hadoop_add_subcommand "cacheadmin" "configure the HDFS cache"
+   hadoop_add_subcommand "classpath" "prints the class path needed to get the hadoop jar and the required libraries"
+   hadoop_add_subcommand "crypto" "configure HDFS encryption zones"
+   hadoop_add_subcommand "datanode" "run a DFS datanode"
+   hadoop_add_subcommand "debug" "run a Debug Admin to execute HDFS debug commands"
+   hadoop_add_subcommand "dfs" "run a filesystem command on the file system"
+   hadoop_add_subcommand "dfsadmin" "run a DFS admin client"
++  hadoop_add_subcommand "erasurecode" "run a HDFS ErasureCoding CLI"
+   hadoop_add_subcommand "fetchdt" "fetch a delegation token from the NameNode"
+   hadoop_add_subcommand "fsck" "run a DFS filesystem checking utility"
+   hadoop_add_subcommand "getconf" "get config values from configuration"
+   hadoop_add_subcommand "groups" "get the groups which users belong to"
+   hadoop_add_subcommand "haadmin" "run a DFS HA admin client"
+   hadoop_add_subcommand "jmxget" "get JMX exported values from NameNode or DataNode."
+   hadoop_add_subcommand "journalnode" "run the DFS journalnode"
+   hadoop_add_subcommand "lsSnapshottableDir" "list all snapshottable dirs owned by the current user"
+   hadoop_add_subcommand "mover" "run a utility to move block replicas across storage types"
+   hadoop_add_subcommand "namenode" "run the DFS namenode"
+   hadoop_add_subcommand "nfs3" "run an NFS version 3 gateway"
+   hadoop_add_subcommand "oev" "apply the offline edits viewer to an edits file"
+   hadoop_add_subcommand "oiv" "apply the offline fsimage viewer to an fsimage"
+   hadoop_add_subcommand "oiv_legacy" "apply the offline fsimage viewer to a legacy fsimage"
+   hadoop_add_subcommand "portmap" "run a portmap service"
+   hadoop_add_subcommand "secondarynamenode" "run the DFS secondary namenode"
+   hadoop_add_subcommand "snapshotDiff" "diff two snapshots of a directory or diff the current directory contents with a snapshot"
+   hadoop_add_subcommand "storagepolicies" "list/get/set block storage policies"
+   hadoop_add_subcommand "version" "print the version"
+   hadoop_add_subcommand "zkfc" "run the ZK Failover Controller daemon"
+   hadoop_generate_usage "${MYNAME}" false
  }
  
  # let's locate libexec...

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index af23d56,1af3a49..5eba08a
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@@ -25,7 -24,6 +24,7 @@@ import org.apache.hadoop.classification
  import org.apache.hadoop.fs.CommonConfigurationKeys;
  import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolarent;
++import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
  import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
  import org.apache.hadoop.http.HttpConfig;
  
@@@ -435,8 -434,6 +443,8 @@@ public class DFSConfigKeys extends Comm
    public static final Class<BlockPlacementPolicyDefault> DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT = BlockPlacementPolicyDefault.class;
    public static final String  DFS_REPLICATION_MAX_KEY = "dfs.replication.max";
    public static final int     DFS_REPLICATION_MAX_DEFAULT = 512;
 +  public static final String DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY = "dfs.block.placement.ec.classname";
-   public static final Class<BlockPlacementPolicyRackFaultTolarent> DFS_BLOCK_PLACEMENT_EC_CLASSNAME_DEFAULT = BlockPlacementPolicyRackFaultTolarent.class;
++  public static final Class<BlockPlacementPolicyRackFaultTolerant> DFS_BLOCK_PLACEMENT_EC_CLASSNAME_DEFAULT = BlockPlacementPolicyRackFaultTolerant.class;
  
    public static final String  DFS_DF_INTERVAL_KEY = "dfs.df.interval";
    public static final int     DFS_DF_INTERVAL_DEFAULT = 60000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 6c3f0ee,7f3722f..35c4f9a
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@@ -1139,13 -1140,28 +1139,14 @@@ implements ByteBufferReadable, CanSetDr
    }
  
    /**
 -   * Used when reading contiguous blocks
 -   */
 -  private void actualGetFromOneDataNode(final DNAddrPair datanode,
 -      LocatedBlock block, final long start, final long end, byte[] buf,
 -      int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
 -      throws IOException {
 -    final int length = (int) (end - start + 1);
 -    actualGetFromOneDataNode(datanode, block, start, end, buf,
 -        new int[]{offset}, new int[]{length}, corruptedBlockMap);
 -  }
 -
 -  /**
     * Read data from one DataNode.
--   * @param datanode the datanode from which to read data
--   * @param block the located block containing the requested data
--   * @param startInBlk the startInBlk offset of the block
--   * @param endInBlk the endInBlk offset of the block
--   * @param buf the given byte array into which the data is read
-    * @param offset the offset in buf
 -   * @param offsets the data may be read into multiple segments of the buf
 -   *                (when reading a striped block). this array indicates the
 -   *                offset of each buf segment.
 -   * @param lengths the length of each buf segment
++   *
++   * @param datanode          the datanode from which to read data
++   * @param block             the located block containing the requested data
++   * @param startInBlk        the startInBlk offset of the block
++   * @param endInBlk          the endInBlk offset of the block
++   * @param buf               the given byte array into which the data is read
++   * @param offset            the offset in buf
     * @param corruptedBlockMap map recording list of datanodes with corrupted
     *                          block replica
     */
@@@ -1188,7 -1208,7 +1189,7 @@@
          throw new IOException(msg);
        } catch (IOException e) {
          if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
--          DFSClient.LOG.info("Will fetch a new encryption key and retry, " 
++          DFSClient.LOG.info("Will fetch a new encryption key and retry, "
                + "encryption key was invalid when connecting to " + datanode.addr
                + " : " + e);
            // The encryption key used is invalid.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 373ebdf,c16aef2..00f3a65
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@@ -137,7 -136,7 +136,7 @@@ public class DFSOutputStream extends FS
      }
  
      return new DFSPacket(buf, chunksPerPkt, offsetInBlock, seqno,
--                         getChecksumSize(), lastPacketInBlock);
++        getChecksumSize(), lastPacketInBlock);
    }
  
    @Override
@@@ -166,7 -165,7 +165,7 @@@
      return value;
    }
  
--  /** 
++  /**
     * @return the object for computing checksum.
     *         The type is NULL if checksum is not computed.
     */
@@@ -179,7 -178,7 +178,7 @@@
      }
      return checksum;
    }
-- 
++
    private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
        HdfsFileStatus stat, DataChecksum checksum) throws IOException {
      super(getChecksum4Compute(checksum, stat));
@@@ -195,7 -194,7 +194,7 @@@
        DFSClient.LOG.debug(
            "Set non-null progress callback on DFSOutputStream " + src);
      }
--    
++
      this.bytesPerChecksum = checksum.getBytesPerChecksum();
      if (bytesPerChecksum <= 0) {
        throw new HadoopIllegalArgumentException(
@@@ -289,7 -282,7 +288,7 @@@
    private DFSOutputStream(DFSClient dfsClient, String src,
        EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
        HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
--          throws IOException {
++      throws IOException {
      this(dfsClient, src, progress, stat, checksum);
      initialFileSize = stat.getLen(); // length of file when opened
      this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK);
@@@ -357,9 -350,6 +356,9 @@@
        String[] favoredNodes) throws IOException {
      TraceScope scope =
          dfsClient.getPathTraceScope("newStreamForAppend", src);
- 	if(stat.getReplication() == 0) {
++    if(stat.getErasureCodingPolicy() != null) {
 +      throw new IOException("Not support appending to a striping layout file yet.");
 +    }
      try {
        final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
            progress, lastBlock, stat, checksum, favoredNodes);
@@@ -405,10 -395,10 +404,10 @@@
      }
  
      if (currentPacket == null) {
-       currentPacket = createPacket(packetSize, chunksPerPacket, 
-           streamer.getBytesCurBlock(), streamer.getAndIncCurrentSeqno(), false);
+       currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
+           .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
        if (DFSClient.LOG.isDebugEnabled()) {
--        DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" + 
++        DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" +
              currentPacket.getSeqno() +
              ", src=" + src +
              ", packetSize=" + packetSize +
@@@ -420,11 -410,12 +419,11 @@@
      currentPacket.writeChecksum(checksum, ckoff, cklen);
      currentPacket.writeData(b, offset, len);
      currentPacket.incNumChunks();
-     streamer.incBytesCurBlock(len);
+     getStreamer().incBytesCurBlock(len);
  
      // If packet is full, enqueue it for transmission
 -    //
      if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
-         streamer.getBytesCurBlock() == blockSize) {
+         getStreamer().getBytesCurBlock() == blockSize) {
        enqueueCurrentPacketFull();
      }
    }
@@@ -435,13 -426,10 +434,10 @@@
    }
  
    void enqueueCurrentPacketFull() throws IOException {
-     if (LOG.isDebugEnabled()) {
-       LOG.debug("enqueue full " + currentPacket + ", src=" + src
-           + ", bytesCurBlock=" + streamer.getBytesCurBlock()
-           + ", blockSize=" + blockSize
-           + ", appendChunk=" + streamer.getAppendChunk()
-           + ", " + streamer);
-     }
+     LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
 -        + " appendChunk={}, {}", currentPacket, src, getStreamer()
 -        .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
++            + " appendChunk={}, {}", currentPacket, src, getStreamer()
++            .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
+         getStreamer());
      enqueueCurrentPacket();
      adjustChunkBoundary();
      endBlock();
@@@ -487,7 -475,7 +483,7 @@@
        lastFlushOffset = 0;
      }
    }
--  
++
    /**
     * Flushes out to all replicas of the block. The data is in the buffers
     * of the DNs but not necessarily in the DN's OS buffers.
@@@ -519,16 -507,16 +515,16 @@@
        scope.close();
      }
    }
--  
++
    /**
     * The expected semantics is all data have flushed out to all replicas 
     * and all replicas have done posix fsync equivalent - ie the OS has 
     * flushed it to the disk device (but the disk may have it in its cache).
--   * 
++   *
     * Note that only the current block is flushed to the disk device.
     * To guarantee durable sync across block boundaries the stream should
     * be created with {@link CreateFlag#SYNC_BLOCK}.
--   * 
++   *
     * @param syncFlags
     *          Indicate the semantic of the sync. Currently used to specify
     *          whether or not to update the block length in NameNode.
@@@ -545,7 -533,7 +541,7 @@@
  
    /**
     * Flush/Sync buffered data to DataNodes.
--   * 
++   *
     * @param isSync
     *          Whether or not to require all replicas to flush data to the disk
     *          device
@@@ -686,7 -679,7 +687,7 @@@
    /**
     * Note that this is not a public API;
     * use {@link HdfsDataOutputStream#getCurrentBlockReplication()} instead.
--   * 
++   *
     * @return the number of valid replicas of the current block
     */
    public synchronized int getCurrentBlockReplication() throws IOException {
@@@ -701,7 -694,7 +702,7 @@@
      }
      return currentNodes.length;
    }
--  
++
    /**
     * Waits till all existing data is flushed and confirmations 
     * received from datanodes. 
@@@ -723,9 -716,9 +724,9 @@@
    }
  
    protected synchronized void start() {
-     streamer.start();
+     getStreamer().start();
    }
--  
++
    /**
     * Aborts this output stream and releases any system 
     * resources associated with this stream.
@@@ -763,7 -756,7 +764,7 @@@
        setClosed();
      }
    }
--  
++
    /**
     * Closes this output stream and releases any system 
     * resources associated with this stream.
@@@ -894,7 -887,7 +895,7 @@@
      do {
        prevStrategy = this.cachingStrategy.get();
        nextStrategy = new CachingStrategy.Builder(prevStrategy).
--                        setDropBehind(dropBehind).build();
++          setDropBehind(dropBehind).build();
      } while (!this.cachingStrategy.compareAndSet(prevStrategy, nextStrategy));
    }
  
@@@ -908,8 -901,10 +909,15 @@@
      return fileId;
    }
  
+   /**
+    * Returns the data streamer object.
+    */
+   protected DataStreamer getStreamer() {
+     return streamer;
+   }
++
 +  @Override
 +  public String toString() {
 +    return getClass().getSimpleName() + ":" + streamer;
 +  }
  }


[24/50] [abbrv] hadoop git commit: HDFS-8899. Erasure Coding: use threadpool for EC recovery tasks on DataNode. Contributed by Rakesh R.

Posted by wa...@apache.org.
HDFS-8899. Erasure Coding: use threadpool for EC recovery tasks on DataNode. Contributed by Rakesh R.

Change-Id: I9429706ae3c9b10a9274c07b98da6ed54cce192b


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

Branch: refs/heads/trunk
Commit: ced438a4bf50fe0ac9072c128e18249e6742956a
Parents: c2ebab6
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Tue Sep 15 10:43:13 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue Sep 15 10:43:13 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 ++
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  4 ++-
 .../erasurecode/ErasureCodingWorker.java        | 33 ++++++++++++++++----
 3 files changed, 33 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ced438a4/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 39b5adc..acf62cb 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -424,3 +424,6 @@
 
     HDFS-7351. Document the HDFS Erasure Coding feature.
     (umamahesh and Zhe Zhang via wang)
+
+    HDFS-8899. Erasure Coding: use threadpool for EC recovery tasks on DataNode.
+    (Rakesh R via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ced438a4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index c4dd496..f7cda18 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -400,7 +400,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.stripedread.buffer.size";
   public static final int     DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 64 * 1024;
   public static final String  DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_KEY = "dfs.datanode.stripedread.threshold.millis";
-  public static final int     DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT = 5000; //5s 
+  public static final int     DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_DEFAULT = 5000; //5s
+  public static final String  DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY = "dfs.datanode.striped.blockrecovery.threads.size";
+  public static final int     DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT = 8;
   public static final String  DFS_DATANODE_DNS_INTERFACE_KEY = "dfs.datanode.dns.interface";
   public static final String  DFS_DATANODE_DNS_INTERFACE_DEFAULT = "default";
   public static final String  DFS_DATANODE_DNS_NAMESERVER_KEY = "dfs.datanode.dns.nameserver";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ced438a4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
index f6a5ece..56b54f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@ -38,6 +38,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -92,6 +93,7 @@ public final class ErasureCodingWorker {
   private final DataNode datanode; 
   private final Configuration conf;
 
+  private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL;
   private ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
   private final int STRIPED_READ_THRESHOLD_MILLIS;
   private final int STRIPED_READ_BUFFER_SIZE;
@@ -109,6 +111,10 @@ public final class ErasureCodingWorker {
     STRIPED_READ_BUFFER_SIZE = conf.getInt(
         DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY,
         DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT);
+
+    initializeStripedBlkRecoveryThreadPool(conf.getInt(
+        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY,
+        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT));
   }
   
   private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) {
@@ -142,6 +148,25 @@ public final class ErasureCodingWorker {
     STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
   }
 
+  private void initializeStripedBlkRecoveryThreadPool(int num) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using striped block recovery; pool threads=" + num);
+    }
+    STRIPED_BLK_RECOVERY_THREAD_POOL = new ThreadPoolExecutor(2, num, 60,
+        TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
+        new Daemon.DaemonFactory() {
+          private final AtomicInteger threadIdx = new AtomicInteger(0);
+
+          @Override
+          public Thread newThread(Runnable r) {
+            Thread t = super.newThread(r);
+            t.setName("stripedBlockRecovery-" + threadIdx.getAndIncrement());
+            return t;
+          }
+        });
+    STRIPED_BLK_RECOVERY_THREAD_POOL.allowCoreThreadTimeOut(true);
+  }
+
   /**
    * Handles the Erasure Coding recovery work commands.
    * 
@@ -150,12 +175,8 @@ public final class ErasureCodingWorker {
    */
   public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
     for (BlockECRecoveryInfo recoveryInfo : ecTasks) {
-      try {
-        new Daemon(new ReconstructAndTransferBlock(recoveryInfo)).start();
-      } catch (Throwable e) {
-        LOG.warn("Failed to recover striped block " + 
-            recoveryInfo.getExtendedBlock().getLocalBlock(), e);
-      }
+      STRIPED_BLK_RECOVERY_THREAD_POOL.submit(new ReconstructAndTransferBlock(
+          recoveryInfo));
     }
   }
 


[37/50] [abbrv] hadoop git commit: HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from ErasureCodingPolicy. Contributed by Vinayakumar B.

Posted by wa...@apache.org.
HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from ErasureCodingPolicy. Contributed by Vinayakumar B.

Change-Id: Ic56da0b426f47c63dac440aef6f5fc8554f6cf13


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

Branch: refs/heads/trunk
Commit: c09dc258a8f64fab852bf6f26187163480dbee3c
Parents: 7bff8ca
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Wed Sep 23 13:34:09 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Wed Sep 23 13:34:09 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsConstants.java     | 11 ------
 .../src/main/proto/hdfs.proto                   |  1 +
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  5 ++-
 .../apache/hadoop/hdfs/StripedDataStreamer.java | 34 ++++++++---------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  3 +-
 .../hadoop/hdfs/server/balancer/Dispatcher.java | 13 ++++---
 .../blockmanagement/BlockInfoStriped.java       | 10 +++--
 .../server/blockmanagement/BlockManager.java    | 10 ++---
 .../apache/hadoop/hdfs/server/mover/Mover.java  |  3 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  | 11 ++++--
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 11 ++++--
 .../server/protocol/BlocksWithLocations.java    |  9 ++++-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  4 +-
 .../apache/hadoop/hdfs/StripedFileTestUtil.java | 40 ++++++++++----------
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |  7 ++--
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |  7 ++--
 .../TestDFSStripedOutputStreamWithFailure.java  |  8 ++--
 .../hdfs/TestReadStripedFileWithDecoding.java   |  6 +--
 .../TestReadStripedFileWithMissingBlocks.java   |  6 +--
 .../hadoop/hdfs/TestRecoverStripedFile.java     |  6 +--
 .../hdfs/TestSafeModeWithStripedFile.java       |  6 +--
 .../hadoop/hdfs/TestWriteReadStripedFile.java   |  4 +-
 .../hdfs/TestWriteStripedFileWithFailure.java   |  4 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |  4 +-
 .../hdfs/server/balancer/TestBalancer.java      |  7 ++--
 .../blockmanagement/TestBlockInfoStriped.java   |  4 +-
 .../TestBlockTokenWithDFSStriped.java           |  8 ++--
 .../TestSequentialBlockGroupId.java             |  8 ++--
 .../hadoop/hdfs/server/mover/TestMover.java     |  7 ++--
 .../TestAddOverReplicatedStripedBlocks.java     |  9 +++--
 .../server/namenode/TestAddStripedBlocks.java   | 13 ++++---
 .../server/namenode/TestFSEditLogLoader.java    |  9 +++--
 .../hdfs/server/namenode/TestFSImage.java       |  9 +++--
 .../namenode/TestQuotaWithStripedBlocks.java    |  5 ++-
 .../namenode/TestRecoverStripedBlocks.java      |  8 ++--
 .../server/namenode/TestStripedINodeFile.java   |  5 ++-
 ...TestOfflineImageViewerWithStripedBlocks.java |  8 ++--
 .../hadoop/hdfs/util/TestStripedBlockUtil.java  | 10 ++---
 38 files changed, 177 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index a8bb740..0453d1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -86,17 +86,6 @@ public final class HdfsConstants {
   public static final String CLIENT_NAMENODE_PROTOCOL_NAME =
       "org.apache.hadoop.hdfs.protocol.ClientProtocol";
 
-  /*
-   * These values correspond to the values used by the system default erasure
-   * coding policy.
-   * TODO: get these values from ec policy of the associated INodeFile
-   */
-
-  public static final byte NUM_DATA_BLOCKS = 6;
-  public static final byte NUM_PARITY_BLOCKS = 3;
-  // The chunk size for striped block which is used by erasure coding
-  public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024;
-
   // Timeouts for communicating with DataNode for streaming writes/reads
   public static final int READ_TIMEOUT = 60 * 1000;
   public static final int READ_TIMEOUT_EXTENSION = 5 * 1000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index 77c64b9..b28ab42 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -536,6 +536,7 @@ message BlockWithLocationsProto {
 
   optional bytes indices = 5;
   optional uint32 dataBlockNum = 6;
+  optional uint32 cellSize = 7;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index b79ce64..d62dbac 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -446,4 +446,7 @@
     getAllErasureCodingPolicies. (Rakesh R via zhz)
 
     HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too
-    verbose and hurt performance. (Rui Li via Kai Zheng)
\ No newline at end of file
+    verbose and hurt performance. (Rui Li via Kai Zheng)
+
+    HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from
+    ErasureCodingPolicy (Vinayakumar B via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
index a2b6a67..2f83f7c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -18,25 +18,20 @@
 
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator;
 import org.apache.hadoop.hdfs.DFSStripedOutputStream.MultipleBlockingQueue;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 
@@ -109,6 +104,8 @@ public class StripedDataStreamer extends DataStreamer {
   private final Coordinator coordinator;
   private final int index;
   private volatile boolean failed;
+  private final ECSchema schema;
+  private final int cellSize;
 
   StripedDataStreamer(HdfsFileStatus stat,
                       DFSClient dfsClient, String src,
@@ -120,6 +117,8 @@ public class StripedDataStreamer extends DataStreamer {
         byteArrayManage, favoredNodes);
     this.index = index;
     this.coordinator = coordinator;
+    this.schema = stat.getErasureCodingPolicy().getSchema();
+    this.cellSize = stat.getErasureCodingPolicy().getCellSize();
   }
 
   int getIndex() {
@@ -135,7 +134,7 @@ public class StripedDataStreamer extends DataStreamer {
   }
 
   private boolean isParityStreamer() {
-    return index >= NUM_DATA_BLOCKS;
+    return index >= schema.getNumDataUnits();
   }
 
   @Override
@@ -168,7 +167,7 @@ public class StripedDataStreamer extends DataStreamer {
         if (block != null) {
           // set numByte for the previous block group
           long bytes = 0;
-          for (int i = 0; i < NUM_DATA_BLOCKS; i++) {
+          for (int i = 0; i < schema.getNumDataUnits(); i++) {
             final ExtendedBlock b = coordinator.takeEndBlock(i);
             StripedBlockUtil.checkBlocks(index, block, i, b);
             bytes += b.getNumBytes();
@@ -183,15 +182,15 @@ public class StripedDataStreamer extends DataStreamer {
 
         final LocatedBlock lb = StripedDataStreamer.super.locateFollowingBlock(
             excludedNodes);
-        if (lb.getLocations().length < HdfsConstants.NUM_DATA_BLOCKS) {
+        if (lb.getLocations().length < schema.getNumDataUnits()) {
           throw new IOException(
               "Failed to get datablocks number of nodes from namenode: blockGroupSize= "
-                  + (HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS)
+                  + (schema.getNumDataUnits() + schema.getNumParityUnits())
                   + ", blocks.length= " + lb.getLocations().length);
         }
-        final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
-            (LocatedStripedBlock)lb,
-            BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+        final LocatedBlock[] blocks =
+            StripedBlockUtil.parseStripedBlockGroup((LocatedStripedBlock) lb,
+                cellSize, schema.getNumDataUnits(), schema.getNumParityUnits());
 
         for (int i = 0; i < blocks.length; i++) {
           StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
@@ -233,9 +232,10 @@ public class StripedDataStreamer extends DataStreamer {
         final LocatedBlock updated = callUpdateBlockForPipeline(bg);
         final long newGS = updated.getBlock().getGenerationStamp();
         final LocatedBlock[] updatedBlks = StripedBlockUtil
-            .parseStripedBlockGroup((LocatedStripedBlock) updated,
-                BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
-        for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) {
+            .parseStripedBlockGroup((LocatedStripedBlock) updated, cellSize,
+                schema.getNumDataUnits(), schema.getNumParityUnits());
+        for (int i = 0; i < schema.getNumDataUnits()
+            + schema.getNumParityUnits(); i++) {
           StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
           if (si.isFailed()) {
             continue; // skipping failed data streamer
@@ -280,7 +280,7 @@ public class StripedDataStreamer extends DataStreamer {
         final ExtendedBlock bg = coordinator.getBlockGroup();
         final ExtendedBlock newBG = newBlock(bg, newGS);
 
-        final int n = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
+        final int n = schema.getNumDataUnits() + schema.getNumParityUnits();
         final DatanodeInfo[] newNodes = new DatanodeInfo[n];
         final String[] newStorageIDs = new String[n];
         for (int i = 0; i < n; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index ece9984..524248c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -192,6 +192,7 @@ public class PBHelper {
       StripedBlockWithLocations sblk = (StripedBlockWithLocations) blk;
       builder.setIndices(PBHelperClient.getByteString(sblk.getIndices()));
       builder.setDataBlockNum(sblk.getDataBlockNum());
+      builder.setCellSize(sblk.getCellSize());
     }
     return builder.build();
   }
@@ -207,7 +208,7 @@ public class PBHelper {
         PBHelperClient.convertStorageTypes(storageTypes, storageUuids.size()));
     if (b.hasIndices()) {
       blk = new StripedBlockWithLocations(blk, b.getIndices().toByteArray(),
-          (short) b.getDataBlockNum());
+          (short) b.getDataBlockNum(), b.getCellSize());
     }
     return blk;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index 07c3c01..4c4fc27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -415,11 +415,14 @@ public class Dispatcher {
 
     final byte[] indices;
     final short dataBlockNum;
+    final int cellSize;
 
-    public DBlockStriped(Block block, byte[] indices, short dataBlockNum) {
+    public DBlockStriped(Block block, byte[] indices, short dataBlockNum,
+        int cellSize) {
       super(block);
       this.indices = indices;
       this.dataBlockNum = dataBlockNum;
+      this.cellSize = cellSize;
     }
 
     public DBlock getInternalBlock(StorageGroup storage) {
@@ -429,8 +432,8 @@ public class Dispatcher {
       }
       byte idxInGroup = indices[idxInLocs];
       long blkId = getBlock().getBlockId() + idxInGroup;
-      long numBytes = getInternalBlockLength(getNumBytes(),
-          HdfsConstants.BLOCK_STRIPED_CELL_SIZE, dataBlockNum, idxInGroup);
+      long numBytes = getInternalBlockLength(getNumBytes(), cellSize,
+          dataBlockNum, idxInGroup);
       Block blk = new Block(getBlock());
       blk.setBlockId(blkId);
       blk.setNumBytes(numBytes);
@@ -717,8 +720,8 @@ public class Dispatcher {
           bytesReceived += sblkLocs.getBlock().getNumBytes() /
               sblkLocs.getDataBlockNum();
           block = new DBlockStriped(sblkLocs.getBlock(), sblkLocs.getIndices(),
-              sblkLocs.getDataBlockNum());
-        } else{
+              sblkLocs.getDataBlockNum(), sblkLocs.getCellSize());
+        } else {
           bytesReceived += blkLocs.getBlock().getNumBytes();
           block = new DBlock(blkLocs.getBlock());
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index df48655..d1067b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -22,8 +22,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-
 /**
  * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
  *
@@ -65,6 +63,10 @@ public class BlockInfoStriped extends BlockInfo {
     return (short) ecPolicy.getNumParityUnits();
   }
 
+  public int getCellSize() {
+    return ecPolicy.getCellSize();
+  }
+
   /**
    * If the block is committed/completed and its length is less than a full
    * stripe, it returns the the number of actual data blocks.
@@ -73,7 +75,7 @@ public class BlockInfoStriped extends BlockInfo {
   public short getRealDataBlockNum() {
     if (isComplete() || getBlockUCState() == BlockUCState.COMMITTED) {
       return (short) Math.min(getDataBlockNum(),
-          (getNumBytes() - 1) / BLOCK_STRIPED_CELL_SIZE + 1);
+          (getNumBytes() - 1) / ecPolicy.getCellSize() + 1);
     } else {
       return getDataBlockNum();
     }
@@ -200,7 +202,7 @@ public class BlockInfoStriped extends BlockInfo {
     // `getNumBytes` is the total of actual data block size.
     return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(),
         ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits(),
-        BLOCK_STRIPED_CELL_SIZE);
+        ecPolicy.getCellSize());
     }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index e7f9262..18bfc41 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -92,7 +92,6 @@ import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
 
 import org.apache.hadoop.metrics2.util.MBeans;
@@ -2554,10 +2553,9 @@ public class BlockManager implements BlockStatsMXBean {
               BlockIdManager.convertToStripedID(reported.getBlockId());
           BlockInfoStriped stripedBlock = (BlockInfoStriped) storedBlock;
           int reportedBlkIdx = BlockIdManager.getBlockIndex(reported);
-          wrongSize = reported.getNumBytes() !=
-              getInternalBlockLength(stripedBlock.getNumBytes(),
-                  BLOCK_STRIPED_CELL_SIZE,
-                  stripedBlock.getDataBlockNum(), reportedBlkIdx);
+          wrongSize = reported.getNumBytes() != getInternalBlockLength(
+              stripedBlock.getNumBytes(), stripedBlock.getCellSize(),
+              stripedBlock.getDataBlockNum(), reportedBlkIdx);
         } else {
           wrongSize = storedBlock.getNumBytes() != reported.getNumBytes();
         }
@@ -3413,7 +3411,7 @@ public class BlockManager implements BlockStatsMXBean {
               (byte) blockStriped.getStorageBlockIndex(locations.get(i));
         }
         results.add(new StripedBlockWithLocations(blkWithLocs, indices,
-            blockStriped.getDataBlockNum()));
+            blockStriped.getDataBlockNum(), blockStriped.getCellSize()));
         // approximate size
         return block.getNumBytes() / blockStriped.getDataBlockNum();
       }else{

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index 34d92d0..1e07cad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -185,7 +185,8 @@ public class Mover {
       for (int i = 0; i < indices.length; i++) {
         indices[i] = (byte) lsb.getBlockIndices()[i];
       }
-      db = new DBlockStriped(blk, indices, (short) ecPolicy.getNumDataUnits());
+      db = new DBlockStriped(blk, indices, (short) ecPolicy.getNumDataUnits(),
+          ecPolicy.getCellSize());
     } else {
       db = new DBlock(blk);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 867b451..cfc939e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -204,9 +204,14 @@ class FSDirWriteFileOp {
     clientMachine = pendingFile.getFileUnderConstructionFeature()
         .getClientMachine();
     isStriped = pendingFile.isStriped();
-    numTargets = isStriped ?
-        HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS :
-        pendingFile.getFileReplication();
+    ErasureCodingPolicy ecPolicy = null;
+    if (isStriped) {
+      ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(fsn, src);
+      numTargets = (short) (ecPolicy.getSchema().getNumDataUnits()
+          + ecPolicy.getSchema().getNumParityUnits());
+    } else {
+      numTargets = pendingFile.getFileReplication();
+    }
     storagePolicyID = pendingFile.getStoragePolicyID();
     return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
                                       clientMachine, isStriped);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index c765edc..242c456 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@@ -388,8 +388,13 @@ public class INodeFile extends INodeWithAdditionalFields
       }
       max = maxInSnapshot > max ? maxInSnapshot : max;
     }
-    return isStriped() ?
-        HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max;
+    if(!isStriped()){
+      return max;
+    }
+    // TODO support more policies based on policyId
+    ErasureCodingPolicy ecPolicy =
+        ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits());
   }
 
   /** Set the replication factor of this file. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java
index 0507faf..b6eddb6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java
@@ -95,16 +95,17 @@ public class BlocksWithLocations {
   public static class StripedBlockWithLocations extends BlockWithLocations {
     final byte[] indices;
     final short dataBlockNum;
+    final int cellSize;
 
     public StripedBlockWithLocations(BlockWithLocations blk, byte[] indices,
-         short dataBlockNum) {
+         short dataBlockNum, int cellSize) {
       super(blk.getBlock(), blk.getDatanodeUuids(), blk.getStorageIDs(),
           blk.getStorageTypes());
       Preconditions.checkArgument(
           blk.getDatanodeUuids().length == indices.length);
       this.indices = indices;
       this.dataBlockNum = dataBlockNum;
-
+      this.cellSize = cellSize;
     }
 
     public byte[] getIndices() {
@@ -114,6 +115,10 @@ public class BlocksWithLocations {
     public short getDataBlockNum() {
       return dataBlockNum;
     }
+
+    public int getCellSize() {
+      return cellSize;
+    }
   }
 
   private final BlockWithLocations[] blocks;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 12d4811..274d319 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -166,8 +166,8 @@ import org.junit.Assume;
 import org.mockito.internal.util.reflection.Whitebox;
 
 import com.google.common.annotations.VisibleForTesting;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
 
 /** Utilities for HDFS tests */
 public class DFSTestUtil {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
index ded86e7..8d4a0cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.web.ByteRangeInputStream;
@@ -38,18 +37,19 @@ import java.util.Random;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-
 public class StripedFileTestUtil {
   public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class);
+  /*
+   * These values correspond to the values used by the system default erasure
+   * coding policy.
+   */
+  public static final short NUM_DATA_BLOCKS = (short) 6;
+  public static final short NUM_PARITY_BLOCKS = (short) 3;
+  public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024;
 
-  static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
-  static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
-
-  static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
   static final int stripesPerBlock = 4;
-  static final int blockSize = cellSize * stripesPerBlock;
-  static final int numDNs = dataBlocks + parityBlocks + 2;
+  static final int blockSize = BLOCK_STRIPED_CELL_SIZE * stripesPerBlock;
+  static final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS + 2;
 
   static final Random random = new Random();
 
@@ -85,9 +85,9 @@ public class StripedFileTestUtil {
   static void verifyPread(FileSystem fs, Path srcPath,  int fileLength,
       byte[] expected, byte[] buf) throws IOException {
     try (FSDataInputStream in = fs.open(srcPath)) {
-      int[] startOffsets = {0, 1, cellSize - 102, cellSize, cellSize + 102,
-          cellSize * (dataBlocks - 1), cellSize * (dataBlocks - 1) + 102,
-          cellSize * dataBlocks, fileLength - 102, fileLength - 1};
+      int[] startOffsets = {0, 1, BLOCK_STRIPED_CELL_SIZE - 102, BLOCK_STRIPED_CELL_SIZE, BLOCK_STRIPED_CELL_SIZE + 102,
+          BLOCK_STRIPED_CELL_SIZE * (NUM_DATA_BLOCKS - 1), BLOCK_STRIPED_CELL_SIZE * (NUM_DATA_BLOCKS - 1) + 102,
+          BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS, fileLength - 102, fileLength - 1};
       for (int startOffset : startOffsets) {
         startOffset = Math.max(0, Math.min(startOffset, fileLength - 1));
         int remaining = fileLength - startOffset;
@@ -158,21 +158,21 @@ public class StripedFileTestUtil {
       pos = 0;
       assertSeekAndRead(in, pos, fileLength);
 
-      if (fileLength > cellSize) {
+      if (fileLength > BLOCK_STRIPED_CELL_SIZE) {
         // seek to cellSize boundary
-        pos = cellSize - 1;
+        pos = BLOCK_STRIPED_CELL_SIZE - 1;
         assertSeekAndRead(in, pos, fileLength);
       }
 
-      if (fileLength > cellSize * dataBlocks) {
+      if (fileLength > BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS) {
         // seek to striped cell group boundary
-        pos = cellSize * dataBlocks - 1;
+        pos = BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS - 1;
         assertSeekAndRead(in, pos, fileLength);
       }
 
-      if (fileLength > blockSize * dataBlocks) {
+      if (fileLength > blockSize * NUM_DATA_BLOCKS) {
         // seek to striped block group boundary
-        pos = blockSize * dataBlocks - 1;
+        pos = blockSize * NUM_DATA_BLOCKS - 1;
         assertSeekAndRead(in, pos, fileLength);
       }
 
@@ -235,12 +235,12 @@ public class StripedFileTestUtil {
    * number of actual data internal blocks. Otherwise returns NUM_DATA_BLOCKS.
    */
   public static short getRealDataBlockNum(int numBytes) {
-    return (short) Math.min(dataBlocks,
+    return (short) Math.min(NUM_DATA_BLOCKS,
         (numBytes - 1) / BLOCK_STRIPED_CELL_SIZE + 1);
   }
 
   public static short getRealTotalBlockNum(int numBytes) {
-    return (short) (getRealDataBlockNum(numBytes) + parityBlocks);
+    return (short) (getRealDataBlockNum(numBytes) + NUM_PARITY_BLOCKS);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
index 6fcf644..549a744 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
@@ -24,7 +24,6 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@@ -60,9 +59,9 @@ public class TestDFSStripedInputStream {
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
   private final ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
-  private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
-  private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
-  private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private final int NUM_STRIPE_PER_BLOCK = 2;
   private final int INTERNAL_BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE;
   private final int BLOCK_GROUP_SIZE =  DATA_BLK_NUM * INTERNAL_BLOCK_SIZE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index 4263ffa..0641e8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@@ -51,13 +50,13 @@ public class TestDFSStripedOutputStream {
     GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
   }
 
-  private int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
-  private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
+  private int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
 
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
   private Configuration conf;
-  private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private final int stripesPerBlock = 4;
   private final int blockSize = cellSize * stripesPerBlock;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index afb7f95..44a29e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -65,9 +65,9 @@ public class TestDFSStripedOutputStreamWithFailure {
     GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
   }
 
-  private static final int NUM_DATA_BLOCKS = HdfsConstants.NUM_DATA_BLOCKS;
-  private static final int NUM_PARITY_BLOCKS = HdfsConstants.NUM_PARITY_BLOCKS;
-  private static final int CELL_SIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private static final int NUM_DATA_BLOCKS = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private static final int NUM_PARITY_BLOCKS = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private static final int CELL_SIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private static final int STRIPES_PER_BLOCK = 4;
   private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK;
   private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS;
@@ -255,7 +255,7 @@ public class TestDFSStripedOutputStreamWithFailure {
           DatanodeReportType.LIVE);
       assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
       Path srcPath = new Path(dir, "testAddBlockWhenNoSufficientParityNodes");
-      int fileLength = HdfsConstants.BLOCK_STRIPED_CELL_SIZE - 1000;
+      int fileLength = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE - 1000;
       final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
       DFSTestUtil.writeFile(dfs, srcPath, new String(expected));
       StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
index 68cd25e..9cf970b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
@@ -46,16 +46,16 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize;
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks;
 
 public class TestReadStripedFileWithDecoding {
   static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class);
 
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
+  private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private final int smallFileLength = blockSize * dataBlocks - 123;
   private final int largeFileLength = blockSize * dataBlocks + 123;
   private final int[] fileLengths = {smallFileLength, largeFileLength};

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
index ca376af..f521d8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
@@ -32,8 +32,6 @@ import org.junit.Test;
 import java.io.IOException;
 
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize;
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
 
 /**
@@ -46,6 +44,8 @@ public class TestReadStripedFileWithMissingBlocks {
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem fs;
   private static Configuration conf = new HdfsConfiguration();
+  private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private final int fileLength = blockSize * dataBlocks + 123;
 
   @Before
@@ -112,7 +112,7 @@ public class TestReadStripedFileWithMissingBlocks {
     }
     for (int i = 0; i < missingParityNum; i++) {
       missingDataNodes[i + missingDataNum] = i +
-          Math.min(StripedFileTestUtil.dataBlocks, dataBlocks);
+          Math.min(StripedFileTestUtil.NUM_DATA_BLOCKS, dataBlocks);
     }
     stopDataNodes(locs, missingDataNodes);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
index ed2ca7b..75dc6a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
@@ -58,9 +58,9 @@ import org.junit.Test;
 public class TestRecoverStripedFile {
   public static final Log LOG = LogFactory.getLog(TestRecoverStripedFile.class);
   
-  private static final int dataBlkNum = HdfsConstants.NUM_DATA_BLOCKS;
-  private static final int parityBlkNum = HdfsConstants.NUM_PARITY_BLOCKS;
-  private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private static final int dataBlkNum = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private static final int parityBlkNum = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private static final int blockSize = cellSize * 3;
   private static final int groupSize = dataBlkNum + parityBlkNum;
   private static final int dnNum = groupSize + parityBlkNum;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
index 9ab0834..9853b8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
@@ -40,10 +40,10 @@ import static org.junit.Assert.assertTrue;
 
 public class TestSafeModeWithStripedFile {
 
-  static final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
-  static final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
+  static final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  static final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
   static final int numDNs = DATA_BLK_NUM + PARITY_BLK_NUM;
-  static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   static final int blockSize = cellSize * 2;
 
   static MiniDFSCluster cluster;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
index 5381fca..3d51f0c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
@@ -37,8 +37,6 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize;
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.stripesPerBlock;
 
@@ -46,6 +44,8 @@ public class TestWriteReadStripedFile {
   public static final Log LOG = LogFactory.getLog(TestWriteReadStripedFile.class);
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem fs;
+  private static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private static short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
   private static Configuration conf = new HdfsConfiguration();
 
   static {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
index c28bff8..c0dca4e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
@@ -31,9 +31,7 @@ import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
 import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks;
 
 public class TestWriteStripedFileWithFailure {
   public static final Log LOG = LogFactory
@@ -41,6 +39,8 @@ public class TestWriteStripedFileWithFailure {
   private static MiniDFSCluster cluster;
   private static FileSystem fs;
   private static Configuration conf = new HdfsConfiguration();
+  private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
   private final int smallFileLength = blockSize * dataBlocks - 123;
   private final int largeFileLength = blockSize * dataBlocks + 123;
   private final int[] fileLengths = {smallFileLength, largeFileLength};

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
index 00a4575..0d51ac8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -201,7 +202,8 @@ public class TestPBHelper {
     BlockWithLocations blkLocs = new BlockWithLocations(new Block(bid, 0, 1),
         datanodeUuids, storageIDs, storageTypes);
     if (isStriped) {
-      blkLocs = new StripedBlockWithLocations(blkLocs, indices, dataBlkNum);
+      blkLocs = new StripedBlockWithLocations(blkLocs, indices, dataBlkNum,
+          StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE);
     }
     return blkLocs;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 25012e7..124bf80 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -149,10 +150,10 @@ public class TestBalancer {
     conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
   }
 
-  int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
-  int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
+  int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
   int groupSize = dataBlocks + parityBlocks;
-  private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private final static int stripesPerBlock = 4;
   static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
index 6cc1dcd..4650dce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
@@ -31,8 +31,8 @@ import java.io.DataOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.nio.ByteBuffer;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_PARITY_BLOCKS;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
index 9dc537c..ae33ffe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
@@ -31,9 +31,9 @@ import java.io.IOException;
 
 public class TestBlockTokenWithDFSStriped extends TestBlockTokenWithDFS {
 
-  private final static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
-  private final static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
-  private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final static int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final static int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private final static int stripesPerBlock = 4;
   private final static int numDNs = dataBlocks + parityBlocks + 2;
   private static MiniDFSCluster cluster;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java
index 9f4c47d..00b4e3b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -58,9 +58,9 @@ public class TestSequentialBlockGroupId {
 
   private final short REPLICATION = 1;
   private final long SEED = 0;
-  private final int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
-  private final int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
-  private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
 
   private final int stripesPerBlock = 2;
   private final int blockSize = cellSize * stripesPerBlock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index 7794f04..3a9748f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -415,9 +416,9 @@ public class TestMover {
     }
   }
 
-  int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
-  int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
-  private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private final static int stripesPerBlock = 4;
   static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
index ae2cbab..64d33a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -49,10 +50,10 @@ public class TestAddOverReplicatedStripedBlocks {
   private DistributedFileSystem fs;
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
-  private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
-  private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
-  private final short GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM;
-  private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final short GROUP_SIZE = (short) (DATA_BLK_NUM + PARITY_BLK_NUM);
+  private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private final int NUM_STRIPE_PER_BLOCK = 4;
   private final int BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE;
   private final int numDNs = GROUP_SIZE + 3;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index 4351fb9..836bff1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -57,13 +58,13 @@ import java.util.List;
 import java.util.UUID;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
 import static org.junit.Assert.assertEquals;
 
 public class TestAddStripedBlocks {
-  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
-      HdfsConstants.NUM_PARITY_BLOCKS;
+  private final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS +
+      StripedFileTestUtil.NUM_PARITY_BLOCKS);
 
   private MiniDFSCluster cluster;
   private DistributedFileSystem dfs;
@@ -163,8 +164,8 @@ public class TestAddStripedBlocks {
       boolean checkReplica) {
     assertEquals(0, block.numNodes());
     Assert.assertFalse(block.isComplete());
-    Assert.assertEquals(HdfsConstants.NUM_DATA_BLOCKS, block.getDataBlockNum());
-    Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS,
+    Assert.assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, block.getDataBlockNum());
+    Assert.assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS,
         block.getParityBlockNum());
     Assert.assertEquals(0,
         block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index 5298980..917b0f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -450,8 +451,8 @@ public class TestFSEditLogLoader {
       long blkId = 1;
       long blkNumBytes = 1024;
       long timestamp = 1426222918;
-      short blockNum = HdfsConstants.NUM_DATA_BLOCKS;
-      short parityNum = HdfsConstants.NUM_PARITY_BLOCKS;
+      short blockNum = StripedFileTestUtil.NUM_DATA_BLOCKS;
+      short parityNum = StripedFileTestUtil.NUM_PARITY_BLOCKS;
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
@@ -522,8 +523,8 @@ public class TestFSEditLogLoader {
       long blkId = 1;
       long blkNumBytes = 1024;
       long timestamp = 1426222918;
-      short blockNum = HdfsConstants.NUM_DATA_BLOCKS;
-      short parityNum = HdfsConstants.NUM_PARITY_BLOCKS;
+      short blockNum = StripedFileTestUtil.NUM_DATA_BLOCKS;
+      short parityNum = StripedFileTestUtil.NUM_PARITY_BLOCKS;
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 7565e86..e45d08d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
@@ -414,8 +415,8 @@ public class TestFSImage {
    */
   @Test
   public void testSupportBlockGroup() throws IOException {
-    final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
-        HdfsConstants.NUM_PARITY_BLOCKS;
+    final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS
+        + StripedFileTestUtil.NUM_PARITY_BLOCKS);
     final int BLOCK_SIZE = 8 * 1024 * 1024;
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -447,8 +448,8 @@ public class TestFSImage {
       BlockInfo[] blks = inode.getBlocks();
       assertEquals(1, blks.length);
       assertTrue(blks[0].isStriped());
-      assertEquals(HdfsConstants.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum());
-      assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum());
+      assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum());
+      assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum());
     } finally {
       cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
index efa5027..26f9b8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -113,9 +114,9 @@ public class TestQuotaWithStripedBlocks {
       final long actualDiskUsed = dirNode.getDirectoryWithQuotaFeature()
           .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
       // In this case the file's real size is cell size * block group size.
-      Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
+      Assert.assertEquals(StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
           actualSpaceUsed);
-      Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
+      Assert.assertEquals(StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
           actualDiskUsed);
     } finally {
       IOUtils.cleanup(null, out);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
index 3134373..6774aed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -35,16 +35,16 @@ import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECReco
 import org.junit.Test;
 import java.util.List;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_PARITY_BLOCKS;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 public class TestRecoverStripedBlocks {
   private final short GROUP_SIZE =
-      NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
+      (short) (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS);
   private MiniDFSCluster cluster;
   private final Path dirPath = new Path("/dir");
   private Path filePath = new Path(dirPath, "file");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
index c827582..9f7ea3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@@ -222,8 +223,8 @@ public class TestStripedINodeFile {
       final Path contiguousFile = new Path(parentDir, "someFile");
       final DistributedFileSystem dfs;
       final Configuration conf = new Configuration();
-      final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS
-          + HdfsConstants.NUM_PARITY_BLOCKS;
+      final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS
+          + StripedFileTestUtil.NUM_PARITY_BLOCKS);
       conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 2);
 
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
index 1e2326a..6d4d797 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@@ -44,12 +44,12 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestOfflineImageViewerWithStripedBlocks {
-  private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
-  private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
+  private static int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private static int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
 
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem fs;
-  private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private static final int stripesPerBlock = 3;
   private static final int blockSize = cellSize * stripesPerBlock;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c09dc258/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
index e61ac07..95b0135 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
@@ -21,9 +21,9 @@ package org.apache.hadoop.hdfs.util;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
@@ -77,10 +77,10 @@ import static org.junit.Assert.assertFalse;
  * TODO: test parity block logic
  */
 public class TestStripedBlockUtil {
-  private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
-  private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
-  private final short BLK_GROUP_WIDTH = DATA_BLK_NUM + PARITY_BLK_NUM;
-  private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final short BLK_GROUP_WIDTH = (short) (DATA_BLK_NUM + PARITY_BLK_NUM);
+  private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private final int FULL_STRIPE_SIZE = DATA_BLK_NUM * CELLSIZE;
   /** number of full stripes in a full block group */
   private final int BLK_GROUP_STRIPE_NUM = 16;


[45/50] [abbrv] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
index 0000000,0000000..69105a0
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@@ -1,0 -1,0 +1,972 @@@
++/**
++ * 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.hdfs;
++
++import com.google.common.base.Preconditions;
++import org.apache.hadoop.fs.ChecksumException;
++import org.apache.hadoop.fs.ReadOption;
++import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
++import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
++import org.apache.hadoop.hdfs.protocol.LocatedBlock;
++import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
++import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
++import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
++import org.apache.hadoop.hdfs.util.StripedBlockUtil;
++import org.apache.hadoop.io.ByteBufferPool;
++
++import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe;
++import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk;
++import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
++
++import org.apache.hadoop.io.IOUtils;
++import org.apache.hadoop.io.erasurecode.CodecUtil;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
++
++import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
++import org.apache.hadoop.util.DirectBufferPool;
++
++import java.io.EOFException;
++import java.io.IOException;
++import java.io.InterruptedIOException;
++import java.nio.ByteBuffer;
++import java.util.ArrayList;
++import java.util.Arrays;
++import java.util.Collections;
++import java.util.EnumSet;
++import java.util.List;
++import java.util.Set;
++import java.util.Collection;
++import java.util.Map;
++import java.util.HashMap;
++import java.util.concurrent.CompletionService;
++import java.util.concurrent.ConcurrentHashMap;
++import java.util.concurrent.ExecutorCompletionService;
++import java.util.concurrent.Callable;
++import java.util.concurrent.Future;
++
++/**
++ * DFSStripedInputStream reads from striped block groups
++ */
++public class DFSStripedInputStream extends DFSInputStream {
++
++  private static class ReaderRetryPolicy {
++    private int fetchEncryptionKeyTimes = 1;
++    private int fetchTokenTimes = 1;
++
++    void refetchEncryptionKey() {
++      fetchEncryptionKeyTimes--;
++    }
++
++    void refetchToken() {
++      fetchTokenTimes--;
++    }
++
++    boolean shouldRefetchEncryptionKey() {
++      return fetchEncryptionKeyTimes > 0;
++    }
++
++    boolean shouldRefetchToken() {
++      return fetchTokenTimes > 0;
++    }
++  }
++
++  /** Used to indicate the buffered data's range in the block group */
++  private static class StripeRange {
++    /** start offset in the block group (inclusive) */
++    final long offsetInBlock;
++    /** length of the stripe range */
++    final long length;
++
++    StripeRange(long offsetInBlock, long length) {
++      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
++      this.offsetInBlock = offsetInBlock;
++      this.length = length;
++    }
++
++    boolean include(long pos) {
++      return pos >= offsetInBlock && pos < offsetInBlock + length;
++    }
++  }
++
++  private static class BlockReaderInfo {
++    final BlockReader reader;
++    final DatanodeInfo datanode;
++    /**
++     * when initializing block readers, their starting offsets are set to the same
++     * number: the smallest internal block offsets among all the readers. This is
++     * because it is possible that for some internal blocks we have to read
++     * "backwards" for decoding purpose. We thus use this offset array to track
++     * offsets for all the block readers so that we can skip data if necessary.
++     */
++    long blockReaderOffset;
++    /**
++     * We use this field to indicate whether we should use this reader. In case
++     * we hit any issue with this reader, we set this field to true and avoid
++     * using it for the next stripe.
++     */
++    boolean shouldSkip = false;
++
++    BlockReaderInfo(BlockReader reader, DatanodeInfo dn, long offset) {
++      this.reader = reader;
++      this.datanode = dn;
++      this.blockReaderOffset = offset;
++    }
++
++    void setOffset(long offset) {
++      this.blockReaderOffset = offset;
++    }
++
++    void skip() {
++      this.shouldSkip = true;
++    }
++  }
++
++  private static final DirectBufferPool bufferPool = new DirectBufferPool();
++
++  private final BlockReaderInfo[] blockReaders;
++  private final int cellSize;
++  private final short dataBlkNum;
++  private final short parityBlkNum;
++  private final int groupSize;
++  /** the buffer for a complete stripe */
++  private ByteBuffer curStripeBuf;
++  private ByteBuffer parityBuf;
++  private final ErasureCodingPolicy ecPolicy;
++  private final RawErasureDecoder decoder;
++
++  /**
++   * indicate the start/end offset of the current buffered stripe in the
++   * block group
++   */
++  private StripeRange curStripeRange;
++  private final CompletionService<Void> readingService;
++
++  /**
++   * When warning the user of a lost block in striping mode, we remember the
++   * dead nodes we've logged. All other striping blocks on these nodes can be
++   * considered lost too, and we don't want to log a warning for each of them.
++   * This is to prevent the log from being too verbose. Refer to HDFS-8920.
++   *
++   * To minimize the overhead, we only store the datanodeUuid in this set
++   */
++  private final Set<String> warnedNodes = Collections.newSetFromMap(
++      new ConcurrentHashMap<String, Boolean>());
++
++  DFSStripedInputStream(DFSClient dfsClient, String src,
++      boolean verifyChecksum, ErasureCodingPolicy ecPolicy,
++      LocatedBlocks locatedBlocks) throws IOException {
++    super(dfsClient, src, verifyChecksum, locatedBlocks);
++
++    assert ecPolicy != null;
++    this.ecPolicy = ecPolicy;
++    this.cellSize = ecPolicy.getCellSize();
++    dataBlkNum = (short) ecPolicy.getNumDataUnits();
++    parityBlkNum = (short) ecPolicy.getNumParityUnits();
++    groupSize = dataBlkNum + parityBlkNum;
++    blockReaders = new BlockReaderInfo[groupSize];
++    curStripeRange = new StripeRange(0, 0);
++    readingService =
++        new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool());
++    decoder = CodecUtil.createRSRawDecoder(dfsClient.getConfiguration(),
++        dataBlkNum, parityBlkNum);
++    if (DFSClient.LOG.isDebugEnabled()) {
++      DFSClient.LOG.debug("Creating an striped input stream for file " + src);
++    }
++  }
++
++  private void resetCurStripeBuffer() {
++    if (curStripeBuf == null) {
++      curStripeBuf = bufferPool.getBuffer(cellSize * dataBlkNum);
++    }
++    curStripeBuf.clear();
++    curStripeRange = new StripeRange(0, 0);
++  }
++
++  private ByteBuffer getParityBuffer() {
++    if (parityBuf == null) {
++      parityBuf = bufferPool.getBuffer(cellSize * parityBlkNum);
++    }
++    parityBuf.clear();
++    return parityBuf;
++  }
++
++  /**
++   * When seeking into a new block group, create blockReader for each internal
++   * block in the group.
++   */
++  private synchronized void blockSeekTo(long target) throws IOException {
++    if (target >= getFileLength()) {
++      throw new IOException("Attempted to read past end of file");
++    }
++
++    // Will be getting a new BlockReader.
++    closeCurrentBlockReaders();
++
++    // Compute desired striped block group
++    LocatedStripedBlock targetBlockGroup = getBlockGroupAt(target);
++    // Update current position
++    this.pos = target;
++    this.blockEnd = targetBlockGroup.getStartOffset() +
++        targetBlockGroup.getBlockSize() - 1;
++    currentLocatedBlock = targetBlockGroup;
++  }
++
++  @Override
++  public synchronized void close() throws IOException {
++    super.close();
++    if (curStripeBuf != null) {
++      bufferPool.returnBuffer(curStripeBuf);
++      curStripeBuf = null;
++    }
++    if (parityBuf != null) {
++      bufferPool.returnBuffer(parityBuf);
++      parityBuf = null;
++    }
++  }
++
++  /**
++   * Extend the super method with the logic of switching between cells.
++   * When reaching the end of a cell, proceed to the next cell and read it
++   * with the next blockReader.
++   */
++  @Override
++  protected void closeCurrentBlockReaders() {
++    resetCurStripeBuffer();
++    if (blockReaders ==  null || blockReaders.length == 0) {
++      return;
++    }
++    for (int i = 0; i < groupSize; i++) {
++      closeReader(blockReaders[i]);
++      blockReaders[i] = null;
++    }
++    blockEnd = -1;
++  }
++
++  private void closeReader(BlockReaderInfo readerInfo) {
++    if (readerInfo != null) {
++//      IOUtils.cleanup(null, readerInfo.reader);
++      readerInfo.skip();
++    }
++  }
++
++  private long getOffsetInBlockGroup() {
++    return getOffsetInBlockGroup(pos);
++  }
++
++  private long getOffsetInBlockGroup(long pos) {
++    return pos - currentLocatedBlock.getStartOffset();
++  }
++
++  /**
++   * Read a new stripe covering the current position, and store the data in the
++   * {@link #curStripeBuf}.
++   */
++  private void readOneStripe(
++      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
++      throws IOException {
++    resetCurStripeBuffer();
++
++    // compute stripe range based on pos
++    final long offsetInBlockGroup = getOffsetInBlockGroup();
++    final long stripeLen = cellSize * dataBlkNum;
++    final int stripeIndex = (int) (offsetInBlockGroup / stripeLen);
++    final int stripeBufOffset = (int) (offsetInBlockGroup % stripeLen);
++    final int stripeLimit = (int) Math.min(currentLocatedBlock.getBlockSize()
++        - (stripeIndex * stripeLen), stripeLen);
++    StripeRange stripeRange = new StripeRange(offsetInBlockGroup,
++        stripeLimit - stripeBufOffset);
++
++    LocatedStripedBlock blockGroup = (LocatedStripedBlock) currentLocatedBlock;
++    AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(ecPolicy, cellSize,
++        blockGroup, offsetInBlockGroup,
++        offsetInBlockGroup + stripeRange.length - 1, curStripeBuf);
++    final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(
++        blockGroup, cellSize, dataBlkNum, parityBlkNum);
++    // read the whole stripe
++    for (AlignedStripe stripe : stripes) {
++      // Parse group to get chosen DN location
++      StripeReader sreader = new StatefulStripeReader(readingService, stripe,
++          blks, blockReaders, corruptedBlockMap);
++      sreader.readStripe();
++    }
++    curStripeBuf.position(stripeBufOffset);
++    curStripeBuf.limit(stripeLimit);
++    curStripeRange = stripeRange;
++  }
++
++  private Callable<Void> readCells(final BlockReader reader,
++      final DatanodeInfo datanode, final long currentReaderOffset,
++      final long targetReaderOffset, final ByteBufferStrategy[] strategies,
++      final ExtendedBlock currentBlock,
++      final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
++    return new Callable<Void>() {
++      @Override
++      public Void call() throws Exception {
++        // reader can be null if getBlockReaderWithRetry failed or
++        // the reader hit exception before
++        if (reader == null) {
++          throw new IOException("The BlockReader is null. " +
++              "The BlockReader creation failed or the reader hit exception.");
++        }
++        Preconditions.checkState(currentReaderOffset <= targetReaderOffset);
++        if (currentReaderOffset < targetReaderOffset) {
++          long skipped = reader.skip(targetReaderOffset - currentReaderOffset);
++          Preconditions.checkState(
++              skipped == targetReaderOffset - currentReaderOffset);
++        }
++        int result = 0;
++        for (ByteBufferStrategy strategy : strategies) {
++          result += readToBuffer(reader, datanode, strategy, currentBlock,
++              corruptedBlockMap);
++        }
++        return null;
++      }
++    };
++  }
++
++  private int readToBuffer(BlockReader blockReader,
++      DatanodeInfo currentNode, ByteBufferStrategy strategy,
++      ExtendedBlock currentBlock,
++      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
++      throws IOException {
++    final int targetLength = strategy.buf.remaining();
++    int length = 0;
++    try {
++      while (length < targetLength) {
++        int ret = strategy.doRead(blockReader, 0, 0);
++        if (ret < 0) {
++          throw new IOException("Unexpected EOS from the reader");
++        }
++        length += ret;
++      }
++      return length;
++    } catch (ChecksumException ce) {
++      DFSClient.LOG.warn("Found Checksum error for "
++          + currentBlock + " from " + currentNode
++          + " at " + ce.getPos());
++      // we want to remember which block replicas we have tried
++      addIntoCorruptedBlockMap(currentBlock, currentNode,
++          corruptedBlockMap);
++      throw ce;
++    } catch (IOException e) {
++      DFSClient.LOG.warn("Exception while reading from "
++          + currentBlock + " of " + src + " from "
++          + currentNode, e);
++      throw e;
++    }
++  }
++
++  /**
++   * Seek to a new arbitrary location
++   */
++  @Override
++  public synchronized void seek(long targetPos) throws IOException {
++    if (targetPos > getFileLength()) {
++      throw new EOFException("Cannot seek after EOF");
++    }
++    if (targetPos < 0) {
++      throw new EOFException("Cannot seek to negative offset");
++    }
++    if (closed.get()) {
++      throw new IOException("Stream is closed!");
++    }
++    if (targetPos <= blockEnd) {
++      final long targetOffsetInBlk = getOffsetInBlockGroup(targetPos);
++      if (curStripeRange.include(targetOffsetInBlk)) {
++        int bufOffset = getStripedBufOffset(targetOffsetInBlk);
++        curStripeBuf.position(bufOffset);
++        pos = targetPos;
++        return;
++      }
++    }
++    pos = targetPos;
++    blockEnd = -1;
++  }
++
++  private int getStripedBufOffset(long offsetInBlockGroup) {
++    final long stripeLen = cellSize * dataBlkNum;
++    // compute the position in the curStripeBuf based on "pos"
++    return (int) (offsetInBlockGroup % stripeLen);
++  }
++
++  @Override
++  public synchronized boolean seekToNewSource(long targetPos)
++      throws IOException {
++    return false;
++  }
++
++  @Override
++  protected synchronized int readWithStrategy(ReaderStrategy strategy,
++      int off, int len) throws IOException {
++    dfsClient.checkOpen();
++    if (closed.get()) {
++      throw new IOException("Stream closed");
++    }
++    Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap =
++        new ConcurrentHashMap<>();
++    if (pos < getFileLength()) {
++      try {
++        if (pos > blockEnd) {
++          blockSeekTo(pos);
++        }
++        int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
++        synchronized (infoLock) {
++          if (locatedBlocks.isLastBlockComplete()) {
++            realLen = (int) Math.min(realLen,
++                locatedBlocks.getFileLength() - pos);
++          }
++        }
++
++        /** Number of bytes already read into buffer */
++        int result = 0;
++        while (result < realLen) {
++          if (!curStripeRange.include(getOffsetInBlockGroup())) {
++            readOneStripe(corruptedBlockMap);
++          }
++          int ret = copyToTargetBuf(strategy, off + result, realLen - result);
++          result += ret;
++          pos += ret;
++        }
++        if (dfsClient.stats != null) {
++          dfsClient.stats.incrementBytesRead(result);
++        }
++        return result;
++      } finally {
++        // Check if need to report block replicas corruption either read
++        // was successful or ChecksumException occured.
++        reportCheckSumFailure(corruptedBlockMap,
++            currentLocatedBlock.getLocations().length);
++      }
++    }
++    return -1;
++  }
++
++  /**
++   * Copy the data from {@link #curStripeBuf} into the given buffer
++   * @param strategy the ReaderStrategy containing the given buffer
++   * @param offset the offset of the given buffer. Used only when strategy is
++   *               a ByteArrayStrategy
++   * @param length target length
++   * @return number of bytes copied
++   */
++  private int copyToTargetBuf(ReaderStrategy strategy, int offset, int length) {
++    final long offsetInBlk = getOffsetInBlockGroup();
++    int bufOffset = getStripedBufOffset(offsetInBlk);
++    curStripeBuf.position(bufOffset);
++    return strategy.copyFrom(curStripeBuf, offset,
++        Math.min(length, curStripeBuf.remaining()));
++  }
++
++  /**
++   * The super method {@link DFSInputStream#refreshLocatedBlock} refreshes
++   * cached LocatedBlock by executing {@link DFSInputStream#getBlockAt} again.
++   * This method extends the logic by first remembering the index of the
++   * internal block, and re-parsing the refreshed block group with the same
++   * index.
++   */
++  @Override
++  protected LocatedBlock refreshLocatedBlock(LocatedBlock block)
++      throws IOException {
++    int idx = StripedBlockUtil.getBlockIndex(block.getBlock().getLocalBlock());
++    LocatedBlock lb = getBlockGroupAt(block.getStartOffset());
++    // If indexing information is returned, iterate through the index array
++    // to find the entry for position idx in the group
++    LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
++    int i = 0;
++    for (; i < lsb.getBlockIndices().length; i++) {
++      if (lsb.getBlockIndices()[i] == idx) {
++        break;
++      }
++    }
++    if (DFSClient.LOG.isDebugEnabled()) {
++      DFSClient.LOG.debug("refreshLocatedBlock for striped blocks, offset="
++          + block.getStartOffset() + ". Obtained block " + lb + ", idx=" + idx);
++    }
++    return StripedBlockUtil.constructInternalBlock(
++        lsb, i, cellSize, dataBlkNum, idx);
++  }
++
++  private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException {
++    LocatedBlock lb = super.getBlockAt(offset);
++    assert lb instanceof LocatedStripedBlock : "NameNode" +
++        " should return a LocatedStripedBlock for a striped file";
++    return (LocatedStripedBlock)lb;
++  }
++
++  /**
++   * Real implementation of pread.
++   */
++  @Override
++  protected void fetchBlockByteRange(LocatedBlock block, long start,
++      long end, byte[] buf, int offset,
++      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
++      throws IOException {
++    // Refresh the striped block group
++    LocatedStripedBlock blockGroup = getBlockGroupAt(block.getStartOffset());
++
++    AlignedStripe[] stripes = StripedBlockUtil.divideByteRangeIntoStripes(
++        ecPolicy, cellSize, blockGroup, start, end, buf, offset);
++    CompletionService<Void> readService = new ExecutorCompletionService<>(
++        dfsClient.getStripedReadsThreadPool());
++    final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(
++        blockGroup, cellSize, dataBlkNum, parityBlkNum);
++    final BlockReaderInfo[] preaderInfos = new BlockReaderInfo[groupSize];
++    try {
++      for (AlignedStripe stripe : stripes) {
++        // Parse group to get chosen DN location
++        StripeReader preader = new PositionStripeReader(readService, stripe,
++            blks, preaderInfos, corruptedBlockMap);
++        preader.readStripe();
++      }
++    } finally {
++      for (BlockReaderInfo preaderInfo : preaderInfos) {
++        closeReader(preaderInfo);
++      }
++    }
++  }
++
++  @Override
++  protected void reportLostBlock(LocatedBlock lostBlock,
++      Collection<DatanodeInfo> ignoredNodes) {
++    DatanodeInfo[] nodes = lostBlock.getLocations();
++    if (nodes != null && nodes.length > 0) {
++      List<String> dnUUIDs = new ArrayList<>();
++      for (DatanodeInfo node : nodes) {
++        dnUUIDs.add(node.getDatanodeUuid());
++      }
++      if (!warnedNodes.containsAll(dnUUIDs)) {
++        DFSClient.LOG.warn(Arrays.toString(nodes) + " are unavailable and " +
++            "all striping blocks on them are lost. " +
++            "IgnoredNodes = " + ignoredNodes);
++        warnedNodes.addAll(dnUUIDs);
++      }
++    } else {
++      super.reportLostBlock(lostBlock, ignoredNodes);
++    }
++  }
++
++  /**
++   * The reader for reading a complete {@link AlignedStripe}. Note that an
++   * {@link AlignedStripe} may cross multiple stripes with cellSize width.
++   */
++  private abstract class StripeReader {
++    final Map<Future<Void>, Integer> futures = new HashMap<>();
++    final AlignedStripe alignedStripe;
++    final CompletionService<Void> service;
++    final LocatedBlock[] targetBlocks;
++    final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap;
++    final BlockReaderInfo[] readerInfos;
++
++    StripeReader(CompletionService<Void> service, AlignedStripe alignedStripe,
++        LocatedBlock[] targetBlocks, BlockReaderInfo[] readerInfos,
++        Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
++      this.service = service;
++      this.alignedStripe = alignedStripe;
++      this.targetBlocks = targetBlocks;
++      this.readerInfos = readerInfos;
++      this.corruptedBlockMap = corruptedBlockMap;
++    }
++
++    /** prepare all the data chunks */
++    abstract void prepareDecodeInputs();
++
++    /** prepare the parity chunk and block reader if necessary */
++    abstract boolean prepareParityChunk(int index) throws IOException;
++
++    abstract void decode();
++
++    void updateState4SuccessRead(StripingChunkReadResult result) {
++      Preconditions.checkArgument(
++          result.state == StripingChunkReadResult.SUCCESSFUL);
++      readerInfos[result.index].setOffset(alignedStripe.getOffsetInBlock()
++          + alignedStripe.getSpanInBlock());
++    }
++
++    private void checkMissingBlocks() throws IOException {
++      if (alignedStripe.missingChunksNum > parityBlkNum) {
++        clearFutures(futures.keySet());
++        throw new IOException(alignedStripe.missingChunksNum
++            + " missing blocks, the stripe is: " + alignedStripe);
++      }
++    }
++
++    /**
++     * We need decoding. Thus go through all the data chunks and make sure we
++     * submit read requests for all of them.
++     */
++    private void readDataForDecoding() throws IOException {
++      prepareDecodeInputs();
++      for (int i = 0; i < dataBlkNum; i++) {
++        Preconditions.checkNotNull(alignedStripe.chunks[i]);
++        if (alignedStripe.chunks[i].state == StripingChunk.REQUESTED) {
++          if (!readChunk(targetBlocks[i], i)) {
++            alignedStripe.missingChunksNum++;
++          }
++        }
++      }
++      checkMissingBlocks();
++    }
++
++    void readParityChunks(int num) throws IOException {
++      for (int i = dataBlkNum, j = 0; i < dataBlkNum + parityBlkNum && j < num;
++           i++) {
++        if (alignedStripe.chunks[i] == null) {
++          if (prepareParityChunk(i) && readChunk(targetBlocks[i], i)) {
++            j++;
++          } else {
++            alignedStripe.missingChunksNum++;
++          }
++        }
++      }
++      checkMissingBlocks();
++    }
++
++    boolean createBlockReader(LocatedBlock block, int chunkIndex)
++        throws IOException {
++      BlockReader reader = null;
++      final ReaderRetryPolicy retry = new ReaderRetryPolicy();
++      DNAddrPair dnInfo = new DNAddrPair(null, null, null);
++
++      while(true) {
++        try {
++          // the cached block location might have been re-fetched, so always
++          // get it from cache.
++          block = refreshLocatedBlock(block);
++          targetBlocks[chunkIndex] = block;
++
++          // internal block has one location, just rule out the deadNodes
++          dnInfo = getBestNodeDNAddrPair(block, null);
++          if (dnInfo == null) {
++            break;
++          }
++          reader = getBlockReader(block, alignedStripe.getOffsetInBlock(),
++              block.getBlockSize() - alignedStripe.getOffsetInBlock(),
++              dnInfo.addr, dnInfo.storageType, dnInfo.info);
++        } catch (IOException e) {
++          if (e instanceof InvalidEncryptionKeyException &&
++              retry.shouldRefetchEncryptionKey()) {
++            DFSClient.LOG.info("Will fetch a new encryption key and retry, "
++                + "encryption key was invalid when connecting to " + dnInfo.addr
++                + " : " + e);
++            dfsClient.clearDataEncryptionKey();
++            retry.refetchEncryptionKey();
++          } else if (retry.shouldRefetchToken() &&
++              tokenRefetchNeeded(e, dnInfo.addr)) {
++            fetchBlockAt(block.getStartOffset());
++            retry.refetchToken();
++          } else {
++            //TODO: handles connection issues
++            DFSClient.LOG.warn("Failed to connect to " + dnInfo.addr + " for " +
++                "block" + block.getBlock(), e);
++            // re-fetch the block in case the block has been moved
++            fetchBlockAt(block.getStartOffset());
++            addToDeadNodes(dnInfo.info);
++          }
++        }
++        if (reader != null) {
++          readerInfos[chunkIndex] = new BlockReaderInfo(reader, dnInfo.info,
++              alignedStripe.getOffsetInBlock());
++          return true;
++        }
++      }
++      return false;
++    }
++
++    private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) {
++      if (chunk.byteBuffer != null) {
++        ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer);
++        return new ByteBufferStrategy[]{strategy};
++      } else {
++        ByteBufferStrategy[] strategies =
++            new ByteBufferStrategy[chunk.byteArray.getOffsets().length];
++        for (int i = 0; i < strategies.length; i++) {
++          ByteBuffer buffer = ByteBuffer.wrap(chunk.byteArray.buf(),
++              chunk.byteArray.getOffsets()[i], chunk.byteArray.getLengths()[i]);
++          strategies[i] = new ByteBufferStrategy(buffer);
++        }
++        return strategies;
++      }
++    }
++
++    boolean readChunk(final LocatedBlock block, int chunkIndex)
++        throws IOException {
++      final StripingChunk chunk = alignedStripe.chunks[chunkIndex];
++      if (block == null) {
++        chunk.state = StripingChunk.MISSING;
++        return false;
++      }
++      if (readerInfos[chunkIndex] == null) {
++        if (!createBlockReader(block, chunkIndex)) {
++          chunk.state = StripingChunk.MISSING;
++          return false;
++        }
++      } else if (readerInfos[chunkIndex].shouldSkip) {
++        chunk.state = StripingChunk.MISSING;
++        return false;
++      }
++
++      chunk.state = StripingChunk.PENDING;
++      Callable<Void> readCallable = readCells(readerInfos[chunkIndex].reader,
++          readerInfos[chunkIndex].datanode,
++          readerInfos[chunkIndex].blockReaderOffset,
++          alignedStripe.getOffsetInBlock(), getReadStrategies(chunk),
++          block.getBlock(), corruptedBlockMap);
++
++      Future<Void> request = service.submit(readCallable);
++      futures.put(request, chunkIndex);
++      return true;
++    }
++
++    /** read the whole stripe. do decoding if necessary */
++    void readStripe() throws IOException {
++      for (int i = 0; i < dataBlkNum; i++) {
++        if (alignedStripe.chunks[i] != null &&
++            alignedStripe.chunks[i].state != StripingChunk.ALLZERO) {
++          if (!readChunk(targetBlocks[i], i)) {
++            alignedStripe.missingChunksNum++;
++          }
++        }
++      }
++      // There are missing block locations at this stage. Thus we need to read
++      // the full stripe and one more parity block.
++      if (alignedStripe.missingChunksNum > 0) {
++        checkMissingBlocks();
++        readDataForDecoding();
++        // read parity chunks
++        readParityChunks(alignedStripe.missingChunksNum);
++      }
++      // TODO: for a full stripe we can start reading (dataBlkNum + 1) chunks
++
++      // Input buffers for potential decode operation, which remains null until
++      // first read failure
++      while (!futures.isEmpty()) {
++        try {
++          StripingChunkReadResult r = StripedBlockUtil
++              .getNextCompletedStripedRead(service, futures, 0);
++          if (DFSClient.LOG.isDebugEnabled()) {
++            DFSClient.LOG.debug("Read task returned: " + r + ", for stripe "
++                + alignedStripe);
++          }
++          StripingChunk returnedChunk = alignedStripe.chunks[r.index];
++          Preconditions.checkNotNull(returnedChunk);
++          Preconditions.checkState(returnedChunk.state == StripingChunk.PENDING);
++
++          if (r.state == StripingChunkReadResult.SUCCESSFUL) {
++            returnedChunk.state = StripingChunk.FETCHED;
++            alignedStripe.fetchedChunksNum++;
++            updateState4SuccessRead(r);
++            if (alignedStripe.fetchedChunksNum == dataBlkNum) {
++              clearFutures(futures.keySet());
++              break;
++            }
++          } else {
++            returnedChunk.state = StripingChunk.MISSING;
++            // close the corresponding reader
++            closeReader(readerInfos[r.index]);
++
++            final int missing = alignedStripe.missingChunksNum;
++            alignedStripe.missingChunksNum++;
++            checkMissingBlocks();
++
++            readDataForDecoding();
++            readParityChunks(alignedStripe.missingChunksNum - missing);
++          }
++        } catch (InterruptedException ie) {
++          String err = "Read request interrupted";
++          DFSClient.LOG.error(err);
++          clearFutures(futures.keySet());
++          // Don't decode if read interrupted
++          throw new InterruptedIOException(err);
++        }
++      }
++
++      if (alignedStripe.missingChunksNum > 0) {
++        decode();
++      }
++    }
++  }
++
++  class PositionStripeReader extends StripeReader {
++    private byte[][] decodeInputs = null;
++
++    PositionStripeReader(CompletionService<Void> service,
++        AlignedStripe alignedStripe, LocatedBlock[] targetBlocks,
++        BlockReaderInfo[] readerInfos,
++        Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
++      super(service, alignedStripe, targetBlocks, readerInfos,
++          corruptedBlockMap);
++    }
++
++    @Override
++    void prepareDecodeInputs() {
++      if (decodeInputs == null) {
++        decodeInputs = StripedBlockUtil.initDecodeInputs(alignedStripe,
++            dataBlkNum, parityBlkNum);
++      }
++    }
++
++    @Override
++    boolean prepareParityChunk(int index) {
++      Preconditions.checkState(index >= dataBlkNum &&
++          alignedStripe.chunks[index] == null);
++      final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index,
++          dataBlkNum, parityBlkNum);
++      alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]);
++      alignedStripe.chunks[index].addByteArraySlice(0,
++          (int) alignedStripe.getSpanInBlock());
++      return true;
++    }
++
++    @Override
++    void decode() {
++      StripedBlockUtil.finalizeDecodeInputs(decodeInputs, dataBlkNum,
++          parityBlkNum, alignedStripe);
++      StripedBlockUtil.decodeAndFillBuffer(decodeInputs, alignedStripe,
++          dataBlkNum, parityBlkNum, decoder);
++    }
++  }
++
++  class StatefulStripeReader extends StripeReader {
++    ByteBuffer[] decodeInputs;
++
++    StatefulStripeReader(CompletionService<Void> service,
++        AlignedStripe alignedStripe, LocatedBlock[] targetBlocks,
++        BlockReaderInfo[] readerInfos,
++        Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
++      super(service, alignedStripe, targetBlocks, readerInfos,
++          corruptedBlockMap);
++    }
++
++    @Override
++    void prepareDecodeInputs() {
++      if (decodeInputs == null) {
++        decodeInputs = new ByteBuffer[dataBlkNum + parityBlkNum];
++        final ByteBuffer cur;
++        synchronized (DFSStripedInputStream.this) {
++          cur = curStripeBuf.duplicate();
++        }
++        StripedBlockUtil.VerticalRange range = alignedStripe.range;
++        for (int i = 0; i < dataBlkNum; i++) {
++          cur.limit(cur.capacity());
++          int pos = (int) (range.offsetInBlock % cellSize + cellSize * i);
++          cur.position(pos);
++          cur.limit((int) (pos + range.spanInBlock));
++          final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
++              dataBlkNum, parityBlkNum);
++          decodeInputs[decodeIndex] = cur.slice();
++          if (alignedStripe.chunks[i] == null) {
++            alignedStripe.chunks[i] = new StripingChunk(
++                decodeInputs[decodeIndex]);
++          }
++        }
++      }
++    }
++
++    @Override
++    boolean prepareParityChunk(int index) throws IOException {
++      Preconditions.checkState(index >= dataBlkNum
++          && alignedStripe.chunks[index] == null);
++      if (blockReaders[index] != null && blockReaders[index].shouldSkip) {
++        alignedStripe.chunks[index] = new StripingChunk(StripingChunk.MISSING);
++        // we have failed the block reader before
++        return false;
++      }
++      final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index,
++          dataBlkNum, parityBlkNum);
++      ByteBuffer buf = getParityBuffer().duplicate();
++      buf.position(cellSize * decodeIndex);
++      buf.limit(cellSize * decodeIndex + (int) alignedStripe.range.spanInBlock);
++      decodeInputs[decodeIndex] = buf.slice();
++      alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]);
++      return true;
++    }
++
++    @Override
++    void decode() {
++      // TODO no copy for data chunks. this depends on HADOOP-12047
++      final int span = (int) alignedStripe.getSpanInBlock();
++      for (int i = 0; i < alignedStripe.chunks.length; i++) {
++        final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
++            dataBlkNum, parityBlkNum);
++        if (alignedStripe.chunks[i] != null &&
++            alignedStripe.chunks[i].state == StripingChunk.ALLZERO) {
++          for (int j = 0; j < span; j++) {
++            decodeInputs[decodeIndex].put((byte) 0);
++          }
++          decodeInputs[decodeIndex].flip();
++        } else if (alignedStripe.chunks[i] != null &&
++            alignedStripe.chunks[i].state == StripingChunk.FETCHED) {
++          decodeInputs[decodeIndex].position(0);
++          decodeInputs[decodeIndex].limit(span);
++        }
++      }
++      int[] decodeIndices = new int[parityBlkNum];
++      int pos = 0;
++      for (int i = 0; i < alignedStripe.chunks.length; i++) {
++        if (alignedStripe.chunks[i] != null &&
++            alignedStripe.chunks[i].state == StripingChunk.MISSING) {
++          int  decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
++              dataBlkNum, parityBlkNum);
++          if (i < dataBlkNum) {
++            decodeIndices[pos++] = decodeIndex;
++          } else {
++            decodeInputs[decodeIndex] = null;
++          }
++        }
++      }
++      decodeIndices = Arrays.copyOf(decodeIndices, pos);
++
++      final int decodeChunkNum = decodeIndices.length;
++      ByteBuffer[] outputs = new ByteBuffer[decodeChunkNum];
++      for (int i = 0; i < decodeChunkNum; i++) {
++        outputs[i] = decodeInputs[decodeIndices[i]];
++        outputs[i].position(0);
++        outputs[i].limit((int) alignedStripe.range.spanInBlock);
++        decodeInputs[decodeIndices[i]] = null;
++      }
++
++      decoder.decode(decodeInputs, decodeIndices, outputs);
++    }
++  }
++
++  /**
++   * May need online read recovery, zero-copy read doesn't make
++   * sense, so don't support it.
++   */
++  @Override
++  public synchronized ByteBuffer read(ByteBufferPool bufferPool,
++      int maxLength, EnumSet<ReadOption> opts)
++          throws IOException, UnsupportedOperationException {
++    throw new UnsupportedOperationException(
++        "Not support enhanced byte buffer access.");
++  }
++
++  @Override
++  public synchronized void releaseBuffer(ByteBuffer buffer) {
++    throw new UnsupportedOperationException(
++        "Not support enhanced byte buffer access.");
++  }
++
++  /** A variation to {@link DFSInputStream#cancelAll} */
++  private void clearFutures(Collection<Future<Void>> futures) {
++    for (Future<Void> future : futures) {
++      future.cancel(false);
++    }
++    futures.clear();
++  }
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 0000000,0000000..bf4e10e
new file mode 100644
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@@ -1,0 -1,0 +1,953 @@@
++/**
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *     http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++package org.apache.hadoop.hdfs;
++
++import java.io.IOException;
++import java.io.InterruptedIOException;
++import java.nio.ByteBuffer;
++import java.nio.channels.ClosedChannelException;
++import java.util.ArrayList;
++import java.util.Arrays;
++import java.util.Collections;
++import java.util.EnumSet;
++import java.util.HashMap;
++import java.util.HashSet;
++import java.util.List;
++import java.util.Map;
++import java.util.Set;
++import java.util.concurrent.BlockingQueue;
++import java.util.concurrent.LinkedBlockingQueue;
++import java.util.concurrent.TimeUnit;
++
++import org.apache.hadoop.HadoopIllegalArgumentException;
++import org.apache.hadoop.classification.InterfaceAudience;
++import org.apache.hadoop.fs.CreateFlag;
++import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
++import org.apache.hadoop.hdfs.protocol.ClientProtocol;
++import org.apache.hadoop.hdfs.protocol.DatanodeID;
++import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
++import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
++import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
++import org.apache.hadoop.hdfs.protocol.LocatedBlock;
++import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
++import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
++import org.apache.hadoop.hdfs.util.StripedBlockUtil;
++import org.apache.hadoop.io.MultipleIOException;
++import org.apache.hadoop.io.erasurecode.CodecUtil;
++import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
++import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
++import org.apache.hadoop.util.DataChecksum;
++import org.apache.hadoop.util.Progressable;
++import org.apache.hadoop.util.Time;
++
++import com.google.common.base.Preconditions;
++import org.apache.htrace.core.TraceScope;
++
++
++/**
++ * This class supports writing files in striped layout and erasure coded format.
++ * Each stripe contains a sequence of cells.
++ */
++@InterfaceAudience.Private
++public class DFSStripedOutputStream extends DFSOutputStream {
++  static class MultipleBlockingQueue<T> {
++    private final List<BlockingQueue<T>> queues;
++
++    MultipleBlockingQueue(int numQueue, int queueSize) {
++      List<BlockingQueue<T>> list = new ArrayList<>(numQueue);
++      for (int i = 0; i < numQueue; i++) {
++        list.add(new LinkedBlockingQueue<T>(queueSize));
++      }
++      queues = Collections.synchronizedList(list);
++    }
++
++    void offer(int i, T object) {
++      final boolean b = queues.get(i).offer(object);
++      Preconditions.checkState(b, "Failed to offer " + object
++          + " to queue, i=" + i);
++    }
++
++    T take(int i) throws InterruptedIOException {
++      try {
++        return queues.get(i).take();
++      } catch(InterruptedException ie) {
++        throw DFSUtilClient.toInterruptedIOException("take interrupted, i=" + i, ie);
++      }
++    }
++
++    T takeWithTimeout(int i) throws InterruptedIOException {
++      try {
++        return queues.get(i).poll(100, TimeUnit.MILLISECONDS);
++      } catch (InterruptedException e) {
++        throw DFSUtilClient.toInterruptedIOException("take interrupted, i=" + i, e);
++      }
++    }
++
++    T poll(int i) {
++      return queues.get(i).poll();
++    }
++
++    T peek(int i) {
++      return queues.get(i).peek();
++    }
++
++    void clear() {
++      for (BlockingQueue<T> q : queues) {
++        q.clear();
++      }
++    }
++  }
++
++  /** Coordinate the communication between the streamers. */
++  static class Coordinator {
++    /**
++     * The next internal block to write to for each streamers. The
++     * DFSStripedOutputStream makes the {@link ClientProtocol#addBlock} RPC to
++     * get a new block group. The block group is split to internal blocks, which
++     * are then distributed into the queue for streamers to retrieve.
++     */
++    private final MultipleBlockingQueue<LocatedBlock> followingBlocks;
++    /**
++     * Used to sync among all the streamers before allocating a new block. The
++     * DFSStripedOutputStream uses this to make sure every streamer has finished
++     * writing the previous block.
++     */
++    private final MultipleBlockingQueue<ExtendedBlock> endBlocks;
++
++    /**
++     * The following data structures are used for syncing while handling errors
++     */
++    private final MultipleBlockingQueue<LocatedBlock> newBlocks;
++    private final Map<StripedDataStreamer, Boolean> updateStreamerMap;
++    private final MultipleBlockingQueue<Boolean> streamerUpdateResult;
++
++    Coordinator(final int numAllBlocks) {
++      followingBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
++      endBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
++      newBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
++      updateStreamerMap = Collections.synchronizedMap(
++          new HashMap<StripedDataStreamer, Boolean>(numAllBlocks));
++      streamerUpdateResult = new MultipleBlockingQueue<>(numAllBlocks, 1);
++    }
++
++    MultipleBlockingQueue<LocatedBlock> getFollowingBlocks() {
++      return followingBlocks;
++    }
++
++    MultipleBlockingQueue<LocatedBlock> getNewBlocks() {
++      return newBlocks;
++    }
++
++    void offerEndBlock(int i, ExtendedBlock block) {
++      endBlocks.offer(i, block);
++    }
++
++    void offerStreamerUpdateResult(int i, boolean success) {
++      streamerUpdateResult.offer(i, success);
++    }
++
++    boolean takeStreamerUpdateResult(int i) throws InterruptedIOException {
++      return streamerUpdateResult.take(i);
++    }
++
++    void updateStreamer(StripedDataStreamer streamer,
++        boolean success) {
++      assert !updateStreamerMap.containsKey(streamer);
++      updateStreamerMap.put(streamer, success);
++    }
++
++    void clearFailureStates() {
++      newBlocks.clear();
++      updateStreamerMap.clear();
++      streamerUpdateResult.clear();
++    }
++  }
++
++  /** Buffers for writing the data and parity cells of a stripe. */
++  class CellBuffers {
++    private final ByteBuffer[] buffers;
++    private final byte[][] checksumArrays;
++
++    CellBuffers(int numParityBlocks) throws InterruptedException{
++      if (cellSize % bytesPerChecksum != 0) {
++        throw new HadoopIllegalArgumentException("Invalid values: "
++            + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (="
++            + bytesPerChecksum + ") must divide cell size (=" + cellSize + ").");
++      }
++
++      checksumArrays = new byte[numParityBlocks][];
++      final int size = getChecksumSize() * (cellSize / bytesPerChecksum);
++      for (int i = 0; i < checksumArrays.length; i++) {
++        checksumArrays[i] = new byte[size];
++      }
++
++      buffers = new ByteBuffer[numAllBlocks];
++      for (int i = 0; i < buffers.length; i++) {
++        buffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
++      }
++    }
++
++    private ByteBuffer[] getBuffers() {
++      return buffers;
++    }
++
++    byte[] getChecksumArray(int i) {
++      return checksumArrays[i - numDataBlocks];
++    }
++
++    private int addTo(int i, byte[] b, int off, int len) {
++      final ByteBuffer buf = buffers[i];
++      final int pos = buf.position() + len;
++      Preconditions.checkState(pos <= cellSize);
++      buf.put(b, off, len);
++      return pos;
++    }
++
++    private void clear() {
++      for (int i = 0; i< numAllBlocks; i++) {
++        buffers[i].clear();
++        if (i >= numDataBlocks) {
++          Arrays.fill(buffers[i].array(), (byte) 0);
++        }
++      }
++    }
++
++    private void release() {
++      for (int i = 0; i < numAllBlocks; i++) {
++        byteArrayManager.release(buffers[i].array());
++      }
++    }
++
++    private void flipDataBuffers() {
++      for (int i = 0; i < numDataBlocks; i++) {
++        buffers[i].flip();
++      }
++    }
++  }
++
++  private final Coordinator coordinator;
++  private final CellBuffers cellBuffers;
++  private final RawErasureEncoder encoder;
++  private final List<StripedDataStreamer> streamers;
++  private final DFSPacket[] currentPackets; // current Packet of each streamer
++
++  /** Size of each striping cell, must be a multiple of bytesPerChecksum */
++  private final int cellSize;
++  private final int numAllBlocks;
++  private final int numDataBlocks;
++  private ExtendedBlock currentBlockGroup;
++  private final String[] favoredNodes;
++  private final List<StripedDataStreamer> failedStreamers;
++
++  /** Construct a new output stream for creating a file. */
++  DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
++                         EnumSet<CreateFlag> flag, Progressable progress,
++                         DataChecksum checksum, String[] favoredNodes)
++                         throws IOException {
++    super(dfsClient, src, stat, flag, progress, checksum, favoredNodes, false);
++    if (LOG.isDebugEnabled()) {
++      LOG.debug("Creating DFSStripedOutputStream for " + src);
++    }
++
++    final ErasureCodingPolicy ecPolicy = stat.getErasureCodingPolicy();
++    final int numParityBlocks = ecPolicy.getNumParityUnits();
++    cellSize = ecPolicy.getCellSize();
++    numDataBlocks = ecPolicy.getNumDataUnits();
++    numAllBlocks = numDataBlocks + numParityBlocks;
++    this.favoredNodes = favoredNodes;
++    failedStreamers = new ArrayList<>();
++
++    encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(),
++        numDataBlocks, numParityBlocks);
++
++    coordinator = new Coordinator(numAllBlocks);
++    try {
++      cellBuffers = new CellBuffers(numParityBlocks);
++    } catch (InterruptedException ie) {
++      throw DFSUtilClient.toInterruptedIOException(
++          "Failed to create cell buffers", ie);
++    }
++
++    streamers = new ArrayList<>(numAllBlocks);
++    for (short i = 0; i < numAllBlocks; i++) {
++      StripedDataStreamer streamer = new StripedDataStreamer(stat,
++          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
++          favoredNodes, i, coordinator);
++      streamers.add(streamer);
++    }
++    currentPackets = new DFSPacket[streamers.size()];
++    setCurrentStreamer(0);
++  }
++
++  StripedDataStreamer getStripedDataStreamer(int i) {
++    return streamers.get(i);
++  }
++
++  int getCurrentIndex() {
++    return getCurrentStreamer().getIndex();
++  }
++
++  private synchronized StripedDataStreamer getCurrentStreamer() {
++    return (StripedDataStreamer) streamer;
++  }
++
++  private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) {
++    // backup currentPacket for current streamer
++    if (streamer != null) {
++      int oldIdx = streamers.indexOf(getCurrentStreamer());
++      if (oldIdx >= 0) {
++        currentPackets[oldIdx] = currentPacket;
++      }
++    }
++
++    streamer = getStripedDataStreamer(newIdx);
++    currentPacket = currentPackets[newIdx];
++    adjustChunkBoundary();
++
++    return getCurrentStreamer();
++  }
++
++  /**
++   * Encode the buffers, i.e. compute parities.
++   *
++   * @param buffers data buffers + parity buffers
++   */
++  private static void encode(RawErasureEncoder encoder, int numData,
++      ByteBuffer[] buffers) {
++    final ByteBuffer[] dataBuffers = new ByteBuffer[numData];
++    final ByteBuffer[] parityBuffers = new ByteBuffer[buffers.length - numData];
++    System.arraycopy(buffers, 0, dataBuffers, 0, dataBuffers.length);
++    System.arraycopy(buffers, numData, parityBuffers, 0, parityBuffers.length);
++
++    encoder.encode(dataBuffers, parityBuffers);
++  }
++
++  /**
++   * check all the existing StripedDataStreamer and find newly failed streamers.
++   * @return The newly failed streamers.
++   * @throws IOException if less than {@link #numDataBlocks} streamers are still
++   *                     healthy.
++   */
++  private Set<StripedDataStreamer> checkStreamers() throws IOException {
++    Set<StripedDataStreamer> newFailed = new HashSet<>();
++    for(StripedDataStreamer s : streamers) {
++      if (!s.isHealthy() && !failedStreamers.contains(s)) {
++        newFailed.add(s);
++      }
++    }
++
++    final int failCount = failedStreamers.size() + newFailed.size();
++    if (LOG.isDebugEnabled()) {
++      LOG.debug("checkStreamers: " + streamers);
++      LOG.debug("healthy streamer count=" + (numAllBlocks - failCount));
++      LOG.debug("original failed streamers: " + failedStreamers);
++      LOG.debug("newly failed streamers: " + newFailed);
++    }
++    if (failCount > (numAllBlocks - numDataBlocks)) {
++      throw new IOException("Failed: the number of failed blocks = "
++          + failCount + " > the number of data blocks = "
++          + (numAllBlocks - numDataBlocks));
++    }
++    return newFailed;
++  }
++
++  private void handleStreamerFailure(String err, Exception e)
++      throws IOException {
++    LOG.warn("Failed: " + err + ", " + this, e);
++    getCurrentStreamer().getErrorState().setInternalError();
++    getCurrentStreamer().close(true);
++    checkStreamers();
++    currentPacket = null;
++  }
++
++  private void replaceFailedStreamers() {
++    assert streamers.size() == numAllBlocks;
++    for (short i = 0; i < numAllBlocks; i++) {
++      final StripedDataStreamer oldStreamer = getStripedDataStreamer(i);
++      if (!oldStreamer.isHealthy()) {
++        StripedDataStreamer streamer = new StripedDataStreamer(oldStreamer.stat,
++            dfsClient, src, oldStreamer.progress,
++            oldStreamer.checksum4WriteBlock, cachingStrategy, byteArrayManager,
++            favoredNodes, i, coordinator);
++        streamers.set(i, streamer);
++        currentPackets[i] = null;
++        if (i == 0) {
++          this.streamer = streamer;
++        }
++        streamer.start();
++      }
++    }
++  }
++
++  private void waitEndBlocks(int i) throws IOException {
++    while (getStripedDataStreamer(i).isHealthy()) {
++      final ExtendedBlock b = coordinator.endBlocks.takeWithTimeout(i);
++      if (b != null) {
++        StripedBlockUtil.checkBlocks(currentBlockGroup, i, b);
++        return;
++      }
++    }
++  }
++
++  private void allocateNewBlock() throws IOException {
++    if (currentBlockGroup != null) {
++      for (int i = 0; i < numAllBlocks; i++) {
++        // sync all the healthy streamers before writing to the new block
++        waitEndBlocks(i);
++      }
++    }
++    failedStreamers.clear();
++    // replace failed streamers
++    replaceFailedStreamers();
++
++    if (LOG.isDebugEnabled()) {
++      LOG.debug("Allocating new block group. The previous block group: "
++          + currentBlockGroup);
++    }
++
++    // TODO collect excludedNodes from all the data streamers
++    final LocatedBlock lb = addBlock(null, dfsClient, src, currentBlockGroup,
++        fileId, favoredNodes);
++    assert lb.isStriped();
++    if (lb.getLocations().length < numDataBlocks) {
++      throw new IOException("Failed to get " + numDataBlocks
++          + " nodes from namenode: blockGroupSize= " + numAllBlocks
++          + ", blocks.length= " + lb.getLocations().length);
++    }
++    // assign the new block to the current block group
++    currentBlockGroup = lb.getBlock();
++
++    final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
++        (LocatedStripedBlock) lb, cellSize, numDataBlocks,
++        numAllBlocks - numDataBlocks);
++    for (int i = 0; i < blocks.length; i++) {
++      StripedDataStreamer si = getStripedDataStreamer(i);
++      if (si.isHealthy()) { // skipping failed data streamer
++        if (blocks[i] == null) {
++          // Set exception and close streamer as there is no block locations
++          // found for the parity block.
++          LOG.warn("Failed to get block location for parity block, index=" + i);
++          si.getLastException().set(
++              new IOException("Failed to get following block, i=" + i));
++          si.getErrorState().setInternalError();
++          si.close(true);
++        } else {
++          coordinator.getFollowingBlocks().offer(i, blocks[i]);
++        }
++      }
++    }
++  }
++
++  private boolean shouldEndBlockGroup() {
++    return currentBlockGroup != null &&
++        currentBlockGroup.getNumBytes() == blockSize * numDataBlocks;
++  }
++
++  @Override
++  protected synchronized void writeChunk(byte[] bytes, int offset, int len,
++      byte[] checksum, int ckoff, int cklen) throws IOException {
++    final int index = getCurrentIndex();
++    final StripedDataStreamer current = getCurrentStreamer();
++    final int pos = cellBuffers.addTo(index, bytes, offset, len);
++    final boolean cellFull = pos == cellSize;
++
++    if (currentBlockGroup == null || shouldEndBlockGroup()) {
++      // the incoming data should belong to a new block. Allocate a new block.
++      allocateNewBlock();
++    }
++
++    currentBlockGroup.setNumBytes(currentBlockGroup.getNumBytes() + len);
++    if (current.isHealthy()) {
++      try {
++        super.writeChunk(bytes, offset, len, checksum, ckoff, cklen);
++      } catch(Exception e) {
++        handleStreamerFailure("offset=" + offset + ", length=" + len, e);
++      }
++    }
++
++    // Two extra steps are needed when a striping cell is full:
++    // 1. Forward the current index pointer
++    // 2. Generate parity packets if a full stripe of data cells are present
++    if (cellFull) {
++      int next = index + 1;
++      //When all data cells in a stripe are ready, we need to encode
++      //them and generate some parity cells. These cells will be
++      //converted to packets and put to their DataStreamer's queue.
++      if (next == numDataBlocks) {
++        cellBuffers.flipDataBuffers();
++        writeParityCells();
++        next = 0;
++        // check failure state for all the streamers. Bump GS if necessary
++        checkStreamerFailures();
++
++        // if this is the end of the block group, end each internal block
++        if (shouldEndBlockGroup()) {
++          for (int i = 0; i < numAllBlocks; i++) {
++            final StripedDataStreamer s = setCurrentStreamer(i);
++            if (s.isHealthy()) {
++              try {
++                endBlock();
++              } catch (IOException ignored) {}
++            }
++          }
++        }
++      }
++      setCurrentStreamer(next);
++    }
++  }
++
++  @Override
++  void enqueueCurrentPacketFull() throws IOException {
++    LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
++            + " appendChunk={}, {}", currentPacket, src, getStreamer()
++            .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
++        getStreamer());
++    enqueueCurrentPacket();
++    adjustChunkBoundary();
++    // no need to end block here
++  }
++
++  private Set<StripedDataStreamer> markExternalErrorOnStreamers() {
++    Set<StripedDataStreamer> healthySet = new HashSet<>();
++    for (StripedDataStreamer streamer : streamers) {
++      if (streamer.isHealthy() &&
++          streamer.getStage() == BlockConstructionStage.DATA_STREAMING) {
++        streamer.setExternalError();
++        healthySet.add(streamer);
++      }
++    }
++    return healthySet;
++  }
++
++  /**
++   * Check and handle data streamer failures. This is called only when we have
++   * written a full stripe (i.e., enqueue all packets for a full stripe), or
++   * when we're closing the outputstream.
++   */
++  private void checkStreamerFailures() throws IOException {
++    Set<StripedDataStreamer> newFailed = checkStreamers();
++    if (newFailed.size() > 0) {
++      // for healthy streamers, wait till all of them have fetched the new block
++      // and flushed out all the enqueued packets.
++      flushAllInternals();
++    }
++    // get all the current failed streamers after the flush
++    newFailed = checkStreamers();
++    while (newFailed.size() > 0) {
++      failedStreamers.addAll(newFailed);
++      coordinator.clearFailureStates();
++
++      // mark all the healthy streamers as external error
++      Set<StripedDataStreamer> healthySet = markExternalErrorOnStreamers();
++
++      // we have newly failed streamers, update block for pipeline
++      final ExtendedBlock newBG = updateBlockForPipeline(healthySet);
++
++      // wait till all the healthy streamers to
++      // 1) get the updated block info
++      // 2) create new block outputstream
++      newFailed = waitCreatingNewStreams(healthySet);
++      if (newFailed.size() + failedStreamers.size() >
++          numAllBlocks - numDataBlocks) {
++        throw new IOException(
++            "Data streamers failed while creating new block streams: "
++                + newFailed + ". There are not enough healthy streamers.");
++      }
++      for (StripedDataStreamer failedStreamer : newFailed) {
++        assert !failedStreamer.isHealthy();
++      }
++
++      // TODO we can also succeed if all the failed streamers have not taken
++      // the updated block
++      if (newFailed.size() == 0) {
++        // reset external error state of all the streamers
++        for (StripedDataStreamer streamer : healthySet) {
++          assert streamer.isHealthy();
++          streamer.getErrorState().reset();
++        }
++        updatePipeline(newBG);
++      }
++      for (int i = 0; i < numAllBlocks; i++) {
++        coordinator.offerStreamerUpdateResult(i, newFailed.size() == 0);
++      }
++    }
++  }
++
++  private int checkStreamerUpdates(Set<StripedDataStreamer> failed,
++      Set<StripedDataStreamer> streamers) {
++    for (StripedDataStreamer streamer : streamers) {
++      if (!coordinator.updateStreamerMap.containsKey(streamer)) {
++        if (!streamer.isHealthy() &&
++            coordinator.getNewBlocks().peek(streamer.getIndex()) != null) {
++          // this streamer had internal error before getting updated block
++          failed.add(streamer);
++        }
++      }
++    }
++    return coordinator.updateStreamerMap.size() + failed.size();
++  }
++
++  private Set<StripedDataStreamer> waitCreatingNewStreams(
++      Set<StripedDataStreamer> healthyStreamers) throws IOException {
++    Set<StripedDataStreamer> failed = new HashSet<>();
++    final int expectedNum = healthyStreamers.size();
++    final long socketTimeout = dfsClient.getConf().getSocketTimeout();
++    // the total wait time should be less than the socket timeout, otherwise
++    // a slow streamer may cause other streamers to timeout. here we wait for
++    // half of the socket timeout
++    long remaingTime = socketTimeout > 0 ? socketTimeout/2 : Long.MAX_VALUE;
++    final long waitInterval = 1000;
++    synchronized (coordinator) {
++      while (checkStreamerUpdates(failed, healthyStreamers) < expectedNum
++          && remaingTime > 0) {
++        try {
++          long start = Time.monotonicNow();
++          coordinator.wait(waitInterval);
++          remaingTime -= Time.monotonicNow() - start;
++        } catch (InterruptedException e) {
++          throw DFSUtilClient.toInterruptedIOException("Interrupted when waiting" +
++              " for results of updating striped streamers", e);
++        }
++      }
++    }
++    synchronized (coordinator) {
++      for (StripedDataStreamer streamer : healthyStreamers) {
++        if (!coordinator.updateStreamerMap.containsKey(streamer)) {
++          // close the streamer if it is too slow to create new connection
++          streamer.setStreamerAsClosed();
++          failed.add(streamer);
++        }
++      }
++    }
++    for (Map.Entry<StripedDataStreamer, Boolean> entry :
++        coordinator.updateStreamerMap.entrySet()) {
++      if (!entry.getValue()) {
++        failed.add(entry.getKey());
++      }
++    }
++    for (StripedDataStreamer failedStreamer : failed) {
++      healthyStreamers.remove(failedStreamer);
++    }
++    return failed;
++  }
++
++  /**
++   * Call {@link ClientProtocol#updateBlockForPipeline} and assign updated block
++   * to healthy streamers.
++   * @param healthyStreamers The healthy data streamers. These streamers join
++   *                         the failure handling.
++   */
++  private ExtendedBlock updateBlockForPipeline(
++      Set<StripedDataStreamer> healthyStreamers) throws IOException {
++    final LocatedBlock updated = dfsClient.namenode.updateBlockForPipeline(
++        currentBlockGroup, dfsClient.clientName);
++    final long newGS = updated.getBlock().getGenerationStamp();
++    ExtendedBlock newBlock = new ExtendedBlock(currentBlockGroup);
++    newBlock.setGenerationStamp(newGS);
++    final LocatedBlock[] updatedBlks = StripedBlockUtil.parseStripedBlockGroup(
++        (LocatedStripedBlock) updated, cellSize, numDataBlocks,
++        numAllBlocks - numDataBlocks);
++
++    for (int i = 0; i < numAllBlocks; i++) {
++      StripedDataStreamer si = getStripedDataStreamer(i);
++      if (healthyStreamers.contains(si)) {
++        final LocatedBlock lb = new LocatedBlock(new ExtendedBlock(newBlock),
++            null, null, null, -1, updated.isCorrupt(), null);
++        lb.setBlockToken(updatedBlks[i].getBlockToken());
++        coordinator.getNewBlocks().offer(i, lb);
++      }
++    }
++    return newBlock;
++  }
++
++  private void updatePipeline(ExtendedBlock newBG) throws IOException {
++    final DatanodeInfo[] newNodes = new DatanodeInfo[numAllBlocks];
++    final String[] newStorageIDs = new String[numAllBlocks];
++    for (int i = 0; i < numAllBlocks; i++) {
++      final StripedDataStreamer streamer = getStripedDataStreamer(i);
++      final DatanodeInfo[] nodes = streamer.getNodes();
++      final String[] storageIDs = streamer.getStorageIDs();
++      if (streamer.isHealthy() && nodes != null && storageIDs != null) {
++        newNodes[i] = nodes[0];
++        newStorageIDs[i] = storageIDs[0];
++      } else {
++        newNodes[i] = new DatanodeInfo(DatanodeID.EMPTY_DATANODE_ID);
++        newStorageIDs[i] = "";
++      }
++    }
++    dfsClient.namenode.updatePipeline(dfsClient.clientName, currentBlockGroup,
++        newBG, newNodes, newStorageIDs);
++    currentBlockGroup = newBG;
++  }
++
++  private int stripeDataSize() {
++    return numDataBlocks * cellSize;
++  }
++
++  @Override
++  public void hflush() {
++    throw new UnsupportedOperationException();
++  }
++
++  @Override
++  public void hsync() {
++    throw new UnsupportedOperationException();
++  }
++
++  @Override
++  protected synchronized void start() {
++    for (StripedDataStreamer streamer : streamers) {
++      streamer.start();
++    }
++  }
++
++  @Override
++  synchronized void abort() throws IOException {
++    if (isClosed()) {
++      return;
++    }
++    for (StripedDataStreamer streamer : streamers) {
++      streamer.getLastException().set(new IOException("Lease timeout of "
++          + (dfsClient.getConf().getHdfsTimeout()/1000) +
++          " seconds expired."));
++    }
++    closeThreads(true);
++    dfsClient.endFileLease(fileId);
++  }
++
++  @Override
++  boolean isClosed() {
++    if (closed) {
++      return true;
++    }
++    for(StripedDataStreamer s : streamers) {
++      if (!s.streamerClosed()) {
++        return false;
++      }
++    }
++    return true;
++  }
++
++  @Override
++  protected void closeThreads(boolean force) throws IOException {
++    final MultipleIOException.Builder b = new MultipleIOException.Builder();
++    try {
++      for (StripedDataStreamer streamer : streamers) {
++        try {
++          streamer.close(force);
++          streamer.join();
++          streamer.closeSocket();
++        } catch (Exception e) {
++          try {
++            handleStreamerFailure("force=" + force, e);
++          } catch (IOException ioe) {
++            b.add(ioe);
++          }
++        } finally {
++          streamer.setSocketToNull();
++        }
++      }
++    } finally {
++      setClosed();
++    }
++    final IOException ioe = b.build();
++    if (ioe != null) {
++      throw ioe;
++    }
++  }
++
++  private boolean generateParityCellsForLastStripe() {
++    final long currentBlockGroupBytes = currentBlockGroup == null ?
++        0 : currentBlockGroup.getNumBytes();
++    final long lastStripeSize = currentBlockGroupBytes % stripeDataSize();
++    if (lastStripeSize == 0) {
++      return false;
++    }
++
++    final long parityCellSize = lastStripeSize < cellSize?
++        lastStripeSize : cellSize;
++    final ByteBuffer[] buffers = cellBuffers.getBuffers();
++
++    for (int i = 0; i < numAllBlocks; i++) {
++      // Pad zero bytes to make all cells exactly the size of parityCellSize
++      // If internal block is smaller than parity block, pad zero bytes.
++      // Also pad zero bytes to all parity cells
++      final int position = buffers[i].position();
++      assert position <= parityCellSize : "If an internal block is smaller" +
++          " than parity block, then its last cell should be small than last" +
++          " parity cell";
++      for (int j = 0; j < parityCellSize - position; j++) {
++        buffers[i].put((byte) 0);
++      }
++      buffers[i].flip();
++    }
++    return true;
++  }
++
++  void writeParityCells() throws IOException {
++    final ByteBuffer[] buffers = cellBuffers.getBuffers();
++    //encode the data cells
++    encode(encoder, numDataBlocks, buffers);
++    for (int i = numDataBlocks; i < numAllBlocks; i++) {
++      writeParity(i, buffers[i], cellBuffers.getChecksumArray(i));
++    }
++    cellBuffers.clear();
++  }
++
++  void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf)
++      throws IOException {
++    final StripedDataStreamer current = setCurrentStreamer(index);
++    final int len = buffer.limit();
++
++    final long oldBytes = current.getBytesCurBlock();
++    if (current.isHealthy()) {
++      try {
++        DataChecksum sum = getDataChecksum();
++        sum.calculateChunkedSums(buffer.array(), 0, len, checksumBuf, 0);
++        for (int i = 0; i < len; i += sum.getBytesPerChecksum()) {
++          int chunkLen = Math.min(sum.getBytesPerChecksum(), len - i);
++          int ckOffset = i / sum.getBytesPerChecksum() * getChecksumSize();
++          super.writeChunk(buffer.array(), i, chunkLen, checksumBuf, ckOffset,
++              getChecksumSize());
++        }
++      } catch(Exception e) {
++        handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e);
++      }
++    }
++  }
++
++  @Override
++  void setClosed() {
++    super.setClosed();
++    for (int i = 0; i < numAllBlocks; i++) {
++      getStripedDataStreamer(i).release();
++    }
++    cellBuffers.release();
++  }
++
++  @Override
++  protected synchronized void closeImpl() throws IOException {
++    if (isClosed()) {
++      final MultipleIOException.Builder b = new MultipleIOException.Builder();
++      for(int i = 0; i < streamers.size(); i++) {
++        final StripedDataStreamer si = getStripedDataStreamer(i);
++        try {
++          si.getLastException().check(true);
++        } catch (IOException e) {
++          b.add(e);
++        }
++      }
++      final IOException ioe = b.build();
++      if (ioe != null) {
++        throw ioe;
++      }
++      return;
++    }
++
++    try {
++      // flush from all upper layers
++      flushBuffer();
++      // if the last stripe is incomplete, generate and write parity cells
++      if (generateParityCellsForLastStripe()) {
++        writeParityCells();
++      }
++      enqueueAllCurrentPackets();
++
++      // flush all the data packets
++      flushAllInternals();
++      // check failures
++      checkStreamerFailures();
++
++      for (int i = 0; i < numAllBlocks; i++) {
++        final StripedDataStreamer s = setCurrentStreamer(i);
++        if (s.isHealthy()) {
++          try {
++            if (s.getBytesCurBlock() > 0) {
++              setCurrentPacketToEmpty();
++            }
++            // flush the last "close" packet to Datanode
++            flushInternal();
++          } catch(Exception e) {
++            // TODO for both close and endBlock, we currently do not handle
++            // failures when sending the last packet. We actually do not need to
++            // bump GS for this kind of failure. Thus counting the total number
++            // of failures may be good enough.
++          }
++        }
++      }
++
++      closeThreads(false);
++      TraceScope scope = dfsClient.getTracer().newScope("completeFile");
++      try {
++        completeFile(currentBlockGroup);
++      } finally {
++        scope.close();
++      }
++      dfsClient.endFileLease(fileId);
++    } catch (ClosedChannelException ignored) {
++    } finally {
++      setClosed();
++    }
++  }
++
++  private void enqueueAllCurrentPackets() throws IOException {
++    int idx = streamers.indexOf(getCurrentStreamer());
++    for(int i = 0; i < streamers.size(); i++) {
++      final StripedDataStreamer si = setCurrentStreamer(i);
++      if (si.isHealthy() && currentPacket != null) {
++        try {
++          enqueueCurrentPacket();
++        } catch (IOException e) {
++          handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e);
++        }
++      }
++    }
++    setCurrentStreamer(idx);
++  }
++
++  void flushAllInternals() throws IOException {
++    int current = getCurrentIndex();
++
++    for (int i = 0; i < numAllBlocks; i++) {
++      final StripedDataStreamer s = setCurrentStreamer(i);
++      if (s.isHealthy()) {
++        try {
++          // flush all data to Datanode
++          flushInternal();
++        } catch(Exception e) {
++          handleStreamerFailure("flushInternal " + s, e);
++        }
++      }
++    }
++    setCurrentStreamer(current);
++  }
++
++  static void sleep(long ms, String op) throws InterruptedIOException {
++    try {
++      Thread.sleep(ms);
++    } catch(InterruptedException ie) {
++      throw DFSUtilClient.toInterruptedIOException(
++          "Sleep interrupted during " + op, ie);
++    }
++  }
++
++  @Override
++  ExtendedBlock getBlock() {
++    return currentBlockGroup;
++  }
++}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fd55202/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index 359886e,e275afb..f96ae65
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@@ -53,6 -54,6 +54,7 @@@ import org.slf4j.LoggerFactory
  
  import javax.net.SocketFactory;
  import java.io.IOException;
++import java.io.InterruptedIOException;
  import java.io.UnsupportedEncodingException;
  import java.net.InetAddress;
  import java.net.InetSocketAddress;
@@@ -628,4 -652,4 +653,11 @@@ public class DFSUtilClient 
      return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
          + namenode.getHostName() + portString);
    }
++
++  public static InterruptedIOException toInterruptedIOException(String message,
++      InterruptedException e) {
++    final InterruptedIOException iioe = new InterruptedIOException(message);
++    iioe.initCause(e);
++    return iioe;
++  }
  }


[09/50] [abbrv] hadoop git commit: HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC to use BlockUnderConstructionFeature. Contributed by Jing Zhao.

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index a7107d7..0d9d427 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -142,7 +142,6 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -204,10 +203,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
@@ -3124,28 +3122,25 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new AlreadyBeingCreatedException(message);
     case UNDER_CONSTRUCTION:
     case UNDER_RECOVERY:
-      // TODO support truncate of striped blocks
-      final BlockInfoUnderConstruction uc =
-          (BlockInfoUnderConstruction)lastBlock;
+      final BlockUnderConstructionFeature uc = lastBlock.getUnderConstructionFeature();
       // determine if last block was intended to be truncated
       Block recoveryBlock = uc.getTruncateBlock();
       boolean truncateRecovery = recoveryBlock != null;
       boolean copyOnTruncate = truncateRecovery &&
-          recoveryBlock.getBlockId() != uc.toBlock().getBlockId();
+          recoveryBlock.getBlockId() != lastBlock.getBlockId();
       assert !copyOnTruncate ||
-          recoveryBlock.getBlockId() < uc.toBlock().getBlockId() &&
-          recoveryBlock.getGenerationStamp() < uc.toBlock().
-              getGenerationStamp() &&
-          recoveryBlock.getNumBytes() > uc.toBlock().getNumBytes() :
+          recoveryBlock.getBlockId() < lastBlock.getBlockId() &&
+          recoveryBlock.getGenerationStamp() < lastBlock.getGenerationStamp() &&
+          recoveryBlock.getNumBytes() > lastBlock.getNumBytes() :
             "wrong recoveryBlock";
 
       // setup the last block locations from the blockManager if not known
       if (uc.getNumExpectedLocations() == 0) {
-        uc.setExpectedLocations(blockManager.getStorages(lastBlock));
+        uc.setExpectedLocations(lastBlock, blockManager.getStorages(lastBlock),
+            lastBlock.isStriped());
       }
 
-      if (uc.getNumExpectedLocations() == 0 &&
-          uc.toBlock().getNumBytes() == 0) {
+      if (uc.getNumExpectedLocations() == 0 && lastBlock.getNumBytes() == 0) {
         // There is no datanode reported to this block.
         // may be client have crashed before writing data to pipeline.
         // This blocks doesn't need any recovery.
@@ -3159,14 +3154,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
       // start recovery of the last block for this file
       long blockRecoveryId =
-          nextGenerationStamp(blockIdManager.isLegacyBlock(uc.toBlock()));
+          nextGenerationStamp(blockIdManager.isLegacyBlock(lastBlock));
       lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
       if(copyOnTruncate) {
-        uc.toBlock().setGenerationStamp(blockRecoveryId);
+        lastBlock.setGenerationStamp(blockRecoveryId);
       } else if(truncateRecovery) {
         recoveryBlock.setGenerationStamp(blockRecoveryId);
       }
-      uc.initializeBlockRecovery(blockRecoveryId);
+      uc.initializeBlockRecovery(lastBlock, blockRecoveryId);
       leaseManager.renewLease(lease);
       // Cannot close file right now, since the last block requires recovery.
       // This may potentially cause infinite loop in lease recovery
@@ -3371,8 +3366,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
 
       truncatedBlock = iFile.getLastBlock();
-      final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)truncatedBlock;
-      final long recoveryId = uc.getBlockRecoveryId();
+      final long recoveryId = truncatedBlock.getUnderConstructionFeature()
+          .getBlockRecoveryId();
       copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId();
       if(recoveryId != newgenerationstamp) {
         throw new IOException("The recovery id " + newgenerationstamp
@@ -5424,7 +5419,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // check the vadility of the block and lease holder name
     final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
     final BlockInfo lastBlock = pendingFile.getLastBlock();
-    final BlockInfoUnderConstruction blockinfo = (BlockInfoUnderConstruction)lastBlock;
+    assert !lastBlock.isComplete();
 
     // check new GS & length: this is not expected
     if (newBlock.getGenerationStamp() <= lastBlock.getGenerationStamp()) {
@@ -5444,12 +5439,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // Update old block with the new generation stamp and new length
     lastBlock.setNumBytes(newBlock.getNumBytes());
-    blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
+    lastBlock.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
 
     // find the DatanodeDescriptor objects
     final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
         .getDatanodeStorageInfos(newNodes, newStorageIDs);
-    blockinfo.setExpectedLocations(storages);
+    lastBlock.getUnderConstructionFeature().setExpectedLocations(lastBlock,
+        storages, lastBlock.isStriped());
 
     String src = pendingFile.getFullPathName();
     FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, logRetryCache);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 13f180a..4fa457d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 
@@ -39,12 +38,10 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
@@ -86,7 +83,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * [4-bit storagePolicyID][1-bit isStriped]
    * [11-bit replication][48-bit preferredBlockSize]
    */
-  static enum HeaderFormat {
+  enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
     REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 0),
     IS_STRIPED(REPLICATION.BITS, 1, 0),
@@ -264,25 +261,20 @@ public class INodeFile extends INodeWithAdditionalFields
     if (numBlocks() == 0) {
       throw new IOException("Failed to set last block: File is empty.");
     }
+    lastBlock.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
+        locations);
+  }
 
-    final BlockInfo ucBlock;
-    if (isStriped()) {
-      Preconditions.checkState(lastBlock.isStriped());
-      ucBlock = ((BlockInfoStriped) lastBlock)
-          .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
-    } else {
-      Preconditions.checkState(!lastBlock.isStriped());
-      ucBlock = ((BlockInfoContiguous) lastBlock)
-          .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
-    }
-    setBlock(numBlocks() - 1, ucBlock);
+  void setLastBlock(BlockInfo blk) {
+    blk.setBlockCollection(this);
+    setBlock(numBlocks() - 1, blk);
   }
 
   /**
    * Remove a block from the block list. This block should be
    * the last one on the list.
    */
-  BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
+  BlockInfo removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
     if (blocks == null || blocks.length == 0) {
@@ -293,13 +285,12 @@ public class INodeFile extends INodeWithAdditionalFields
       return null;
     }
 
-    BlockInfoUnderConstruction uc =
-        (BlockInfoUnderConstruction)blocks[size_1];
+    BlockInfo ucBlock = blocks[size_1];
     //copy to a new list
     BlockInfo[] newlist = new BlockInfo[size_1];
     System.arraycopy(blocks, 0, newlist, 0, size_1);
     setBlocks(newlist);
-    return uc;
+    return ucBlock;
   }
 
   /* End of Under-Construction Feature */
@@ -758,7 +749,7 @@ public class INodeFile extends INodeWithAdditionalFields
     //check if the last block is BlockInfoUnderConstruction
     BlockInfo lastBlk = blocks[last];
     long size = lastBlk.getNumBytes();
-    if (lastBlk instanceof BlockInfoUnderConstruction) {
+    if (!lastBlk.isComplete()) {
        if (!includesLastUcBlock) {
          size = 0;
        } else if (usePreferredBlockSize4LastUcBlock) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index 442e127..9dcd4d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -23,7 +23,6 @@ import java.util.List;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -136,7 +135,7 @@ public class FileDiffList extends
     Block dontRemoveBlock = null;
     if (lastBlock != null && lastBlock.getBlockUCState().equals(
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
-      dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
+      dontRemoveBlock = lastBlock.getUnderConstructionFeature()
           .getTruncateBlock();
     }
     // Collect the remaining blocks of the file, ignoring truncate block

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 7d895e3..6500b96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -117,7 +117,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -1650,13 +1649,11 @@ public class DFSTestUtil {
     BlockManager bm0 = nn.getNamesystem().getBlockManager();
     BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
     assertTrue("Block " + blk + " should be under construction, " +
-        "got: " + storedBlock,
-        storedBlock instanceof BlockInfoContiguousUnderConstruction);
-    BlockInfoContiguousUnderConstruction ucBlock =
-      (BlockInfoContiguousUnderConstruction)storedBlock;
+        "got: " + storedBlock, !storedBlock.isComplete());
     // We expect that the replica with the most recent heart beat will be
     // the one to be in charge of the synchronization / recovery protocol.
-    final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations();
+    final DatanodeStorageInfo[] storages = storedBlock
+        .getUnderConstructionFeature().getExpectedStorageLocations();
     DatanodeStorageInfo expectedPrimary = storages[0];
     long mostRecentLastUpdate = expectedPrimary.getDatanodeDescriptor()
         .getLastUpdateMonotonic();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
index a447aaf..c347472 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
@@ -39,25 +39,24 @@ public class TestBlockInfoUnderConstruction {
     DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
 
     dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
-    BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
-        new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP),
-        (short) 3,
-        BlockUCState.UNDER_CONSTRUCTION,
+    BlockInfoContiguous blockInfo = new BlockInfoContiguous(
+        new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3);
+    blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
         new DatanodeStorageInfo[] {s1, s2, s3});
 
     // Recovery attempt #1.
     DFSTestUtil.resetLastUpdatesWithOffset(dd1, -3 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000);
-    blockInfo.initializeBlockRecovery(1);
-    BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
+    blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 1);
+    BlockInfo[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
     // Recovery attempt #2.
     DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, -3 * 1000);
-    blockInfo.initializeBlockRecovery(2);
+    blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 2);
     blockInfoRecovery = dd1.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
@@ -65,7 +64,7 @@ public class TestBlockInfoUnderConstruction {
     DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, -3 * 1000);
-    blockInfo.initializeBlockRecovery(3);
+    blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 3);
     blockInfoRecovery = dd3.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
@@ -74,7 +73,7 @@ public class TestBlockInfoUnderConstruction {
     DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
-    blockInfo.initializeBlockRecovery(3);
+    blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 3);
     blockInfoRecovery = dd3.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 66a4681..862f14d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -727,8 +728,8 @@ public class TestBlockManager {
     // verify the storage info is correct
     assertTrue(bm.getStoredBlock(new Block(receivedBlockId)).findStorageInfo
         (ds) >= 0);
-    assertTrue(((BlockInfoContiguousUnderConstruction) bm.
-        getStoredBlock(new Block(receivingBlockId))).getNumExpectedLocations() > 0);
+    assertTrue(bm.getStoredBlock(new Block(receivingBlockId))
+        .getUnderConstructionFeature().getNumExpectedLocations() > 0);
     assertTrue(bm.getStoredBlock(new Block(receivingReceivedBlockId))
         .findStorageInfo(ds) >= 0);
     assertNull(bm.getStoredBlock(new Block(ReceivedDeletedBlockId)));
@@ -748,8 +749,8 @@ public class TestBlockManager {
 
   private BlockInfo addUcBlockToBM(long blkId) {
     Block block = new Block(blkId);
-    BlockInfoContiguousUnderConstruction blockInfo =
-        new BlockInfoContiguousUnderConstruction(block, (short) 3);
+    BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3);
+    blockInfo.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, null);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
     bm.blocksMap.addBlockCollection(blockInfo, bc);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
index c1218a0..3e233c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
@@ -172,9 +172,10 @@ public class TestHeartbeatHandling {
               dd1.getStorageInfos()[0],
               dd2.getStorageInfos()[0],
               dd3.getStorageInfos()[0]};
-          BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
-              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
-              BlockUCState.UNDER_RECOVERY, storages);
+          BlockInfo blockInfo = new BlockInfoContiguous(
+              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3);
+          blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_RECOVERY,
+              storages);
           dd1.addBlockToBeRecovered(blockInfo);
           DatanodeCommand[] cmds =
               NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
@@ -194,9 +195,10 @@ public class TestHeartbeatHandling {
           // More than the default stale interval of 30 seconds.
           DFSTestUtil.resetLastUpdatesWithOffset(dd2, -40 * 1000);
           DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
-          blockInfo = new BlockInfoContiguousUnderConstruction(
-              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
-              BlockUCState.UNDER_RECOVERY, storages);
+          blockInfo = new BlockInfoContiguous(
+              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3);
+          blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_RECOVERY,
+              storages);
           dd1.addBlockToBeRecovered(blockInfo);
           cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
           assertEquals(1, cmds.length);
@@ -215,9 +217,10 @@ public class TestHeartbeatHandling {
           // More than the default stale interval of 30 seconds.
           DFSTestUtil.resetLastUpdatesWithOffset(dd2, - 40 * 1000);
           DFSTestUtil.resetLastUpdatesWithOffset(dd3, - 80 * 1000);
-          blockInfo = new BlockInfoContiguousUnderConstruction(
-              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
-              BlockUCState.UNDER_RECOVERY, storages);
+          blockInfo = new BlockInfoContiguous(
+              new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3);
+          blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_RECOVERY,
+              storages);
           dd1.addBlockToBeRecovered(blockInfo);
           cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
           assertEquals(1, cmds.length);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index dcb852b..0172595 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -1182,7 +1182,8 @@ public class TestReplicationPolicy {
     // block under construction, the BlockManager will realize the expected
     // replication has been achieved and remove it from the under-replicated
     // queue.
-    BlockInfoContiguousUnderConstruction info = new BlockInfoContiguousUnderConstruction(block1, (short) 1);
+    BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
+    info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, null);
     BlockCollection bc = mock(BlockCollection.class);
     when(bc.getPreferredBlockReplication()).thenReturn((short)1);
     bm.addBlockCollection(info, bc);
@@ -1247,9 +1248,8 @@ public class TestReplicationPolicy {
 
     DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
         dataNodes[0], new DatanodeStorage("s1"))};
-    final BlockInfoContiguousUnderConstruction ucBlock =
-        info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
-            storageAry);
+    info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
+        storageAry);
     DatanodeStorageInfo storage = mock(DatanodeStorageInfo.class);
     DatanodeDescriptor dn = mock(DatanodeDescriptor.class);
     when(dn.isDecommissioned()).thenReturn(true);
@@ -1258,10 +1258,10 @@ public class TestReplicationPolicy {
     when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true);
     when(storage.addBlock(any(BlockInfo.class))).thenReturn
         (DatanodeStorageInfo.AddBlockResult.ADDED);
-    ucBlock.addStorage(storage, ucBlock);
+    info.addStorage(storage, info);
 
     BlockInfo lastBlk = mbc.getLastBlock();
-    when(mbc.getLastBlock()).thenReturn(lastBlk, ucBlock);
+    when(mbc.getLastBlock()).thenReturn(lastBlk, info);
 
     bm.convertLastBlockToUnderConstruction(mbc, 0L);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index fa92c64..c472cd8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -170,13 +169,13 @@ public class TestAddStripedBlocks {
     Assert.assertEquals(0,
         block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
 
-    final BlockInfoStripedUnderConstruction blockUC =
-        (BlockInfoStripedUnderConstruction) block;
     Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
-        blockUC.getBlockUCState());
+        block.getBlockUCState());
     if (checkReplica) {
-      Assert.assertEquals(GROUP_SIZE, blockUC.getNumExpectedLocations());
-      DatanodeStorageInfo[] storages = blockUC.getExpectedStorageLocations();
+      Assert.assertEquals(GROUP_SIZE,
+          block.getUnderConstructionFeature().getNumExpectedLocations());
+      DatanodeStorageInfo[] storages = block.getUnderConstructionFeature()
+          .getExpectedStorageLocations();
       for (DataNode dn : cluster.getDataNodes()) {
         Assert.assertTrue(includeDataNode(dn.getDatanodeId(), storages));
       }
@@ -205,11 +204,10 @@ public class TestAddStripedBlocks {
 
       FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
       INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
-      BlockInfoStripedUnderConstruction lastBlk =
-          (BlockInfoStripedUnderConstruction) fileNode.getLastBlock();
-      DatanodeInfo[] expectedDNs = DatanodeStorageInfo
-          .toDatanodeInfos(lastBlk.getExpectedStorageLocations());
-      int[] indices = lastBlk.getBlockIndices();
+      BlockInfoStriped lastBlk = (BlockInfoStriped) fileNode.getLastBlock();
+      DatanodeInfo[] expectedDNs = DatanodeStorageInfo.toDatanodeInfos(
+          lastBlk.getUnderConstructionFeature().getExpectedStorageLocations());
+      int[] indices = lastBlk.getUnderConstructionFeature().getBlockIndices();
 
       LocatedBlocks blks = dfs.getClient().getLocatedBlocks(file.toString(), 0L);
       Assert.assertEquals(1, blks.locatedBlockCount());
@@ -246,11 +244,10 @@ public class TestAddStripedBlocks {
       cluster.getNamesystem().getAdditionalBlock(file.toString(),
           fileNode.getId(), dfs.getClient().getClientName(), null, null, null);
       BlockInfo lastBlock = fileNode.getLastBlock();
-      BlockInfoStripedUnderConstruction ucBlock =
-          (BlockInfoStripedUnderConstruction) lastBlock;
 
-      DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
-      int[] indices = ucBlock.getBlockIndices();
+      DatanodeStorageInfo[] locs = lastBlock.getUnderConstructionFeature()
+          .getExpectedStorageLocations();
+      int[] indices = lastBlock.getUnderConstructionFeature().getBlockIndices();
       Assert.assertEquals(GROUP_SIZE, locs.length);
       Assert.assertEquals(GROUP_SIZE, indices.length);
 
@@ -272,8 +269,8 @@ public class TestAddStripedBlocks {
       }
 
       // make sure lastBlock is correct and the storages have been updated
-      locs = ucBlock.getExpectedStorageLocations();
-      indices = ucBlock.getBlockIndices();
+      locs = lastBlock.getUnderConstructionFeature().getExpectedStorageLocations();
+      indices = lastBlock.getUnderConstructionFeature().getBlockIndices();
       Assert.assertEquals(GROUP_SIZE, locs.length);
       Assert.assertEquals(GROUP_SIZE, indices.length);
       for (DatanodeStorageInfo newstorage : locs) {
@@ -307,10 +304,9 @@ public class TestAddStripedBlocks {
           bpId, reports, null);
     }
 
-    BlockInfoStripedUnderConstruction ucBlock =
-        (BlockInfoStripedUnderConstruction) lastBlock;
-    DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
-    int[] indices = ucBlock.getBlockIndices();
+    DatanodeStorageInfo[] locs = lastBlock.getUnderConstructionFeature()
+        .getExpectedStorageLocations();
+    int[] indices = lastBlock.getUnderConstructionFeature().getBlockIndices();
     Assert.assertEquals(GROUP_SIZE, locs.length);
     Assert.assertEquals(GROUP_SIZE, indices.length);
     for (i = 0; i < GROUP_SIZE; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
index f372bec..ab6cde3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -36,7 +37,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.junit.AfterClass;
@@ -156,6 +157,7 @@ public class TestBlockUnderConstruction {
   @Test
   public void testGetBlockLocations() throws IOException {
     final NamenodeProtocols namenode = cluster.getNameNodeRpc();
+    final BlockManager blockManager = cluster.getNamesystem().getBlockManager();
     final Path p = new Path(BASE_DIR, "file2.dat");
     final String src = p.toString();
     final FSDataOutputStream out = TestFileCreation.createFile(hdfs, p, 3);
@@ -170,7 +172,7 @@ public class TestBlockUnderConstruction {
       final List<LocatedBlock> blocks = lb.getLocatedBlocks();
       assertEquals(i, blocks.size());
       final Block b = blocks.get(blocks.size() - 1).getBlock().getLocalBlock();
-      assertTrue(b instanceof BlockInfoContiguousUnderConstruction);
+      assertFalse(blockManager.getStoredBlock(b).isComplete());
 
       if (++i < NUM_BLOCKS) {
         // write one more block

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
index c218b7c..35a098a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.junit.Test;
@@ -68,11 +67,13 @@ public class TestCommitBlockSynchronization {
     namesystem.dir.getINodeMap().put(file);
 
     FSNamesystem namesystemSpy = spy(namesystem);
-    BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
-        block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
+    BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 1);
+    blockInfo.convertToBlockUnderConstruction(
+        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
     blockInfo.setBlockCollection(file);
     blockInfo.setGenerationStamp(genStamp);
-    blockInfo.initializeBlockRecovery(genStamp);
+    blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo,
+        genStamp);
     doReturn(blockInfo).when(file).removeLastBlock(any(Block.class));
     doReturn(true).when(file).isUnderConstruction();
     doReturn(new BlockInfoContiguous[1]).when(file).getBlocks();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index f6aae22..7436879 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -1017,7 +1016,7 @@ public class TestFileTruncate {
           is(fsn.getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
-      long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
+      long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
           .getBlockRecoveryId();
       assertThat(blockRecoveryId, is(initialGenStamp + 1));
       fsn.getEditLog().logTruncate(
@@ -1051,7 +1050,7 @@ public class TestFileTruncate {
           is(fsn.getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
-      long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
+      long blockRecoveryId = file.getLastBlock().getUnderConstructionFeature()
           .getBlockRecoveryId();
       assertThat(blockRecoveryId, is(initialGenStamp + 1));
       fsn.getEditLog().logTruncate(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
index 8b1a11f..48b22c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
@@ -157,8 +156,9 @@ public class TestStripedINodeFile {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     Block blk = new Block(1);
-    BlockInfoStripedUnderConstruction bInfoUCStriped
-        = new BlockInfoStripedUnderConstruction(blk, testECPolicy);
+    BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy);
+    bInfoUCStriped.convertToBlockUnderConstruction(
+        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
     bInfoUCStriped.setNumBytes(100);
     inf.addBlock(bInfoUCStriped);
     assertEquals(100, inf.computeFileSize());
@@ -191,8 +191,9 @@ public class TestStripedINodeFile {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     Block blk = new Block(1);
-    BlockInfoStripedUnderConstruction bInfoUCStriped
-        = new BlockInfoStripedUnderConstruction(blk, testECPolicy);
+    BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy);
+    bInfoUCStriped.convertToBlockUnderConstruction(
+        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
     bInfoUCStriped.setNumBytes(100);
     inf.addBlock(bInfoUCStriped);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index d202fb7..c27ead5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -72,7 +72,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -752,12 +752,13 @@ public class TestRetryCacheWithHA {
     boolean checkNamenodeBeforeReturn() throws Exception {
       INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory()
           .getINode4Write(file).asFile();
-      BlockInfoContiguousUnderConstruction blkUC =
-          (BlockInfoContiguousUnderConstruction) (fileNode.getBlocks())[1];
-      int datanodeNum = blkUC.getExpectedStorageLocations().length;
+      BlockInfo blkUC = (fileNode.getBlocks())[1];
+      int datanodeNum = blkUC.getUnderConstructionFeature()
+          .getExpectedStorageLocations().length;
       for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) {
         Thread.sleep(1000);
-        datanodeNum = blkUC.getExpectedStorageLocations().length;
+        datanodeNum = blkUC.getUnderConstructionFeature()
+            .getExpectedStorageLocations().length;
       }
       return datanodeNum == 2;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/164cbe64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
index 128bd9e..9a611aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
@@ -43,10 +43,9 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
 import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -178,8 +177,7 @@ public class SnapshotTestHelper {
    * Specific information for different types of INode: 
    * {@link INodeDirectory}:childrenSize
    * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()}
-   * and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information.
-   * {@link FileWithSnapshot}: next link
+   * and {@link BlockUnderConstructionFeature#toString()} for detailed information.
    * </pre>
    * @see INode#dumpTreeRecursively()
    */


[23/50] [abbrv] hadoop git commit: HDFS-7351. Document the HDFS Erasure Coding feature. Contributed by Uma Maheswara Rao G and Zhe Zhang.

Posted by wa...@apache.org.
HDFS-7351. Document the HDFS Erasure Coding feature. Contributed by Uma Maheswara Rao G and Zhe Zhang.


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

Branch: refs/heads/trunk
Commit: c2ebab67acf421f52dfa4879261982fed2768c82
Parents: ce02b55
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Sep 15 10:16:45 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Sep 15 10:16:45 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   2 +
 .../src/site/markdown/HDFSCommands.md           |  20 +++
 .../src/site/markdown/HDFSErasureCoding.md      | 148 +++++++++++++++++++
 hadoop-project/src/site/site.xml                |   1 +
 4 files changed, 171 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2ebab67/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 47bab0b..39b5adc 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -422,3 +422,5 @@
     HDFS-8975. Erasure coding : Fix random failure in TestSafeModeWithStripedFile
     (J.Andreina via vinayakumarb)
 
+    HDFS-7351. Document the HDFS Erasure Coding feature.
+    (umamahesh and Zhe Zhang via wang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2ebab67/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 1c2026c..c7794c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -36,6 +36,7 @@ HDFS Commands Guide
     * [crypto](#crypto)
     * [datanode](#datanode)
     * [dfsadmin](#dfsadmin)
+    * [erasurecode](#erasurecode)
     * [haadmin](#haadmin)
     * [journalnode](#journalnode)
     * [mover](#mover)
@@ -379,6 +380,25 @@ Usage:
 
 Runs a HDFS dfsadmin client.
 
+### `erasurecode`
+
+Usage:
+
+       hdfs erasurecode [generic options]
+         [-setPolicy [-s <policyName>] <path>]
+         [-getPolicy <path>]
+         [-listPolicies]
+         [-usage [cmd ...]]
+         [-help [cmd ...]]
+
+| COMMAND\_OPTION | Description |
+|:---- |:---- |
+|-setPolicy| Set a specified ErasureCoding policy to a directory|
+|-getPolicy| Get ErasureCoding policy information about a specified path|
+|-listPolicies| Lists all supported ErasureCoding policies|
+
+Runs the ErasureCoding CLI. See [HDFS ErasureCoding](./HDFSErasureCoding.html#Administrative_commands) for more information on this command.
+
 ### `haadmin`
 
 Usage:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2ebab67/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
new file mode 100644
index 0000000..44c209e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -0,0 +1,148 @@
+<!---
+  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.
+-->
+
+HDFS Erasure Coding
+===================
+
+* [HDFS Erasure Coding](#HDFS_Erasure_Coding)
+    * [Purpose](#Purpose)
+    * [Background](#Background)
+    * [Architecture](#Architecture)
+    * [Hardware resources](#Hardware_resources)
+    * [Deployment](#Deployment)
+        * [Configuration details](#Configuration_details)
+        * [Deployment details](#Deployment_details)
+        * [Administrative commands](#Administrative_commands)
+
+Purpose
+-------
+  Replication is expensive -- the default 3x replication scheme has 200% overhead in storage space and other resources (e.g., network bandwidth).
+  However, for “warm” and “cold” datasets with relatively low I/O activities, secondary block replicas are rarely accessed during normal operations, but still consume the same amount of resources as the primary ones.
+
+  Therefore, a natural improvement is to use Erasure Coding (EC) in place of replication, which provides the same level of fault tolerance with much less storage space. In typical Erasure Coding(EC) setups, the storage overhead is ≤ 50%.
+
+Background
+----------
+
+  In storage systems, the most notable usage of EC is Redundant Array of Inexpensive Disks (RAID). RAID implements EC through striping, which divides logically sequential data (such as a file) into smaller units (such as bit, byte, or block) and stores consecutive units on different disks. In the rest of this guide this unit of striping distribution is termed a striping cell (or cell). For each stripe of original data cells, a certain number of parity cells are calculated and stored -- the process of which is called encoding. The error on any striping cell can be recovered through decoding calculation based on surviving data and parity cells.
+
+  Integrating the EC function with HDFS could get storage efficient deployments. It can provide similar data tolerance as traditional HDFS replication based deployments but it stores only one original replica data and parity cells.
+  In a typical case, A file with 6 blocks will actually be consume space of 6*3 = 18 blocks with replication factor 3. But with EC (6 data,3 parity) deployment, it will only consume space of 9 blocks.
+
+Architecture
+------------
+  In the context of EC, striping has several critical advantages. First, it enables online EC which bypasses the conversion phase and immediately saves storage space. Online EC also enhances sequential I/O performance by leveraging multiple disk spindles in parallel; this is especially desirable in clusters with high end networking  . Second, it naturally distributes a small file to multiple DataNodes and eliminates the need to bundle multiple files into a single coding group. This greatly simplifies file operations such as deletion, quota reporting, and migration between federated namespaces.
+
+  As in general HDFS clusters, small files could account for over 3/4 of total storage consumption. So, In this first phase of erasure coding work, HDFS supports striping model. In the near future, HDFS will supports contiguous layout as second second phase work. So this guide focuses more on striping model EC.
+
+ *  **NameNode Extensions** - Under the striping layout, a HDFS file is logically composed of block groups, each of which contains a certain number of   internal blocks.
+   To eliminate the need for NameNode to monitor all internal blocks, a new hierarchical block naming protocol is introduced, where the ID of a block group can be inferred from any of its internal blocks. This allows each block group to be managed as a new type of BlockInfo named BlockInfoStriped, which tracks its own internal blocks by attaching an index to each replica location.
+
+ *  **Client Extensions** - The basic principle behind the extensions is to allow the client node to work on multiple internal blocks in a block group in
+    parallel.
+    On the output / write path, DFSStripedOutputStream manages a set of data streamers, one for each DataNode storing an internal block in the current block group. The streamers mostly
+    work asynchronously. A coordinator takes charge of operations on the entire block group, including ending the current block group, allocating a new block group, and so forth.
+    On the input / read path, DFSStripedInputStream translates a requested logical byte range of data as ranges into internal blocks stored on DataNodes. It then issues read requests in
+    parallel. Upon failures, it issues additional read requests for decoding.
+
+ *  **DataNode Extensions** - ErasureCodingWorker(ECWorker) is for reconstructing erased erasure coding blocks and runs along with the Datanode process. Erased block details would have been found out by Namenode ReplicationMonitor thread and sent to Datanode via its heartbeat responses as discussed in the previous sections. For each reconstruction task,
+   i.e. ReconstructAndTransferBlock, it will start an internal daemon thread that performs 3 key tasks:
+
+      _1.Read the data from source nodes:_ For reading the data blocks from different source nodes, it uses a dedicated thread pool.
+         The thread pool is initialized when ErasureCodingWorker initializes. Based on the EC policy, it schedules the read requests to all source targets and ensures only to read
+         minimum required input blocks for reconstruction.
+
+      _2.Decode the data and generate the output data:_ Actual decoding/encoding is done by using RawErasureEncoder API currently.
+        All the erased data and/or parity blocks will be recovered together.
+
+     _3.Transfer the generated data blocks to target nodes:_ Once decoding is finished, it will encapsulate the output data to packets and send them to
+        target Datanodes.
+   To accommodate heterogeneous workloads, we allow files and directories in an HDFS cluster to have different replication and EC policies.
+*   **ErasureCodingPolicy**
+    Information on how to encode/decode a file is encapsulated in an ErasureCodingPolicy class. Each policy is defined by the following 2 pieces of information:
+    _1.The ECScema: This includes the numbers of data and parity blocks in an EC group (e.g., 6+3), as well as the codec algorithm (e.g., Reed-Solomon).
+
+    _2.The size of a striping cell.
+
+   Client and Datanode uses EC codec framework directly for doing the endoing/decoding work.
+
+ *  **Erasure Codec Framework**
+     We support a generic EC framework which allows system users to define, configure, and deploy multiple coding schemas such as conventional Reed-Solomon, HitchHicker and
+     so forth.
+     ErasureCoder is provided to encode or decode for a block group in the middle level, and RawErasureCoder is provided to perform the concrete algorithm calculation in the low level. ErasureCoder can
+     combine and make use of different RawErasureCoders for tradeoff. We abstracted coder type, data blocks size, parity blocks size into ECSchema. A default system schema using RS (6, 3) is built-in.
+     For the system default codec Reed-Solomon we implemented both RSRawErasureCoder in pure Java and NativeRawErasureCoder based on Intel ISA-L. Below is the performance
+     comparing for different coding chunk size. We can see that the native coder can outperform the Java coder by up to 35X.
+
+     _Intel® Storage Acceleration-Library(Intel® ISA-L)_ ISA-L is an Open Source Version and is a collection of low-level functions used in storage applications.
+     The open source version contains fast erasure codes that implement a general Reed-Solomon type encoding for blocks of data that helps protect against
+     erasure of whole blocks. The general ISA-L library contains an expanded set of functions used for data protection, hashing, encryption, etc. By
+     leveraging instruction sets like SSE, AVX and AVX2, the erasure coding functions are much optimized and outperform greatly on IA platforms. ISA-L
+     supports Linux, Windows and other platforms as well. Additionally, it also supports incremental coding so applications don’t have to wait all source
+     blocks to be available before to perform the coding, which can be used in HDFS.
+
+Hardware resources
+------------------
+  For using EC feature, you need to prepare for the following.
+    Depending on the ECSchemas used, we need to have minimum number of Datanodes available in the cluster. Example if we use ReedSolomon(6, 3) ECSchema,
+    then minimum nodes required is 9 to succeed the write. It can tolerate up to 3 failures.
+
+Deployment
+----------
+
+### Configuration details
+
+  In the EC feature, raw coders are configurable. So, users need to decide the RawCoder algorithms.
+  Configure the customized algorithms with configuration key "*io.erasurecode.codecs*".
+
+  Default Reed-Solomon based raw coders available in built, which can be configured by using the configuration key "*io.erasurecode.codec.rs.rawcoder*".
+  And also another default raw coder available if XOR based raw coder. Which could be configured by using "*io.erasurecode.codec.xor.rawcoder*"
+
+  _EarasureCodingWorker Confugurations:_
+    dfs.datanode.stripedread.threshold.millis - Threshold time for polling timeout for read service. Default value is 5000
+    dfs.datanode.stripedread.threads – Number striped read thread pool threads. Default value is 20
+    dfs.datanode.stripedread.buffer.size - Buffer size for reader service. Default value is 256 * 1024
+
+### Deployment details
+
+  With the striping model, client machine is responsible for do the EC endoing and tranferring data to the datanodes.
+  So, EC with striping model expects client machines with hghg end configurations especially of CPU and network.
+
+### Administrative commands
+ ErasureCoding command-line is provided to perform administrative commands related to ErasureCoding. This can be accessed by executing the following command.
+
+       hdfs erasurecode [generic options]
+         [-setPolicy [-s <policyName>] <path>]
+         [-getPolicy <path>]
+         [-listPolicies]
+         [-usage [cmd ...]]
+         [-help [cmd ...]]
+
+Below are the details about each command.
+
+*  **SetPolicy command**: `[-setPolicy [-s <policyName>] <path>]`
+
+    SetPolicy command is used to set an ErasureCoding policy on a directory at the specified path.
+
+      `path`: Refer to a pre-created directory in HDFS. This is a mandatory parameter.
+
+      `policyName`: This is an optional parameter, specified using ‘-s’ flag. Refer to the name of ErasureCodingPolicy to be used for encoding files under this directory. If not specified the system default ErasureCodingPolicy will be used.
+
+*  **GetPolicy command**: `[-getPolicy <path>]`
+
+     GetPolicy command is used to get details of the ErasureCoding policy of a file or directory at the specified path.
+
+*  **ListPolicies command**:  `[-listPolicies]`
+
+     Lists all supported ErasureCoding policies. For setPolicy command, one of these policies' name should be provided.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2ebab67/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 272ea7b..5c92d73 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -98,6 +98,7 @@
       <item name="Storage Policies" href="hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html"/>
       <item name="Memory Storage Support" href="hadoop-project-dist/hadoop-hdfs/MemoryStorage.html"/>
       <item name="Synthetic Load Generator" href="hadoop-project-dist/hadoop-hdfs/SLGUserGuide.html"/>
+      <item name="Erasure Coding" href="hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html"/>
     </menu>
 
     <menu name="MapReduce" inherit="top">


[39/50] [abbrv] hadoop git commit: HDFS-9040. Erasure coding: coordinate data streamers in DFSStripedOutputStream. Contributed by Jing Zhao and Walter Su.

Posted by wa...@apache.org.
HDFS-9040. Erasure coding: coordinate data streamers in DFSStripedOutputStream. Contributed by Jing Zhao and Walter Su.


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

Branch: refs/heads/trunk
Commit: 6419900ac24a5493827abf9b5d90373bc1043e0b
Parents: c09dc25
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Sep 28 14:40:27 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Mon Sep 28 14:40:27 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/protocol/DatanodeID.java |   2 +
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  62 +-
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 603 ++++++++++++++-----
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 212 +++----
 .../apache/hadoop/hdfs/StripedDataStreamer.java | 342 +++--------
 .../BlockUnderConstructionFeature.java          |  30 +-
 .../server/blockmanagement/DatanodeManager.java |   4 +
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  23 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  31 +-
 .../apache/hadoop/hdfs/StripedFileTestUtil.java | 213 ++++++-
 .../hadoop/hdfs/TestDFSStripedOutputStream.java | 144 +----
 .../TestDFSStripedOutputStreamWithFailure.java  | 300 ++++-----
 .../hdfs/TestWriteStripedFileWithFailure.java   |   8 +
 .../hdfs/server/balancer/TestBalancer.java      |   4 +-
 .../hadoop/hdfs/server/mover/TestMover.java     |   4 +-
 .../TestAddOverReplicatedStripedBlocks.java     |  12 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |   9 +-
 18 files changed, 1068 insertions(+), 938 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
index 6d72285..c709cbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
@@ -38,6 +38,8 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceStability.Evolving
 public class DatanodeID implements Comparable<DatanodeID> {
   public static final DatanodeID[] EMPTY_ARRAY = {};
+  public static final DatanodeID EMPTY_DATANODE_ID = new DatanodeID("null",
+      "null", "null", 0, 0, 0, 0);
 
   private String ipAddr;     // IP address
   private String hostName;   // hostname claimed by datanode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index d62dbac..6a01d61 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -450,3 +450,6 @@
 
     HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from
     ErasureCodingPolicy (Vinayakumar B via zhz)
+
+    HDFS-9040. Erasure coding: coordinate data streamers in
+    DFSStripedOutputStream. (jing9 and Walter Su)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 4923c86..e77a00a 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
@@ -212,14 +213,17 @@ public class DFSOutputStream extends FSOutputSummer
   /** Construct a new output stream for creating a file. */
   protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
       EnumSet<CreateFlag> flag, Progressable progress,
-      DataChecksum checksum, String[] favoredNodes) throws IOException {
+      DataChecksum checksum, String[] favoredNodes, boolean createStreamer)
+      throws IOException {
     this(dfsClient, src, progress, stat, checksum);
     this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
 
     computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
 
-    streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
-        cachingStrategy, byteArrayManager, favoredNodes);
+    if (createStreamer) {
+      streamer = new DataStreamer(stat, null, dfsClient, src, progress,
+          checksum, cachingStrategy, byteArrayManager, favoredNodes);
+    }
   }
 
   static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
@@ -276,7 +280,7 @@ public class DFSOutputStream extends FSOutputSummer
             flag, progress, checksum, favoredNodes);
       } else {
         out = new DFSOutputStream(dfsClient, src, stat,
-            flag, progress, checksum, favoredNodes);
+            flag, progress, checksum, favoredNodes, true);
       }
       out.start();
       return out;
@@ -476,7 +480,7 @@ public class DFSOutputStream extends FSOutputSummer
    *
    * @throws IOException
    */
-  protected void endBlock() throws IOException {
+  void endBlock() throws IOException {
     if (getStreamer().getBytesCurBlock() == blockSize) {
       setCurrentPacketToEmpty();
       enqueueCurrentPacket();
@@ -921,4 +925,52 @@ public class DFSOutputStream extends FSOutputSummer
   public String toString() {
     return getClass().getSimpleName() + ":" + streamer;
   }
+
+  static LocatedBlock addBlock(DatanodeInfo[] excludedNodes, DFSClient dfsClient,
+      String src, ExtendedBlock prevBlock, long fileId, String[] favoredNodes)
+      throws IOException {
+    final DfsClientConf conf = dfsClient.getConf();
+    int retries = conf.getNumBlockWriteLocateFollowingRetry();
+    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
+    long localstart = Time.monotonicNow();
+    while (true) {
+      try {
+        return dfsClient.namenode.addBlock(src, dfsClient.clientName, prevBlock,
+            excludedNodes, fileId, favoredNodes);
+      } catch (RemoteException e) {
+        IOException ue = e.unwrapRemoteException(FileNotFoundException.class,
+            AccessControlException.class,
+            NSQuotaExceededException.class,
+            DSQuotaExceededException.class,
+            QuotaByStorageTypeExceededException.class,
+            UnresolvedPathException.class);
+        if (ue != e) {
+          throw ue; // no need to retry these exceptions
+        }
+        if (NotReplicatedYetException.class.getName().equals(e.getClassName())) {
+          if (retries == 0) {
+            throw e;
+          } else {
+            --retries;
+            LOG.info("Exception while adding a block", e);
+            long elapsed = Time.monotonicNow() - localstart;
+            if (elapsed > 5000) {
+              LOG.info("Waiting for replication for " + (elapsed / 1000)
+                  + " seconds");
+            }
+            try {
+              LOG.warn("NotReplicatedYetException sleeping " + src
+                  + " retries left " + retries);
+              Thread.sleep(sleeptime);
+              sleeptime *= 2;
+            } catch (InterruptedException ie) {
+              LOG.warn("Caught exception", ie);
+            }
+          }
+        } else {
+          throw e;
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index d3a054a..c145a2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -25,23 +25,34 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
 import org.apache.htrace.Sampler;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
@@ -59,23 +70,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     private final List<BlockingQueue<T>> queues;
 
     MultipleBlockingQueue(int numQueue, int queueSize) {
-      queues = new ArrayList<>(numQueue);
+      List<BlockingQueue<T>> list = new ArrayList<>(numQueue);
       for (int i = 0; i < numQueue; i++) {
-        queues.add(new LinkedBlockingQueue<T>(queueSize));
+        list.add(new LinkedBlockingQueue<T>(queueSize));
       }
-    }
-
-    boolean isEmpty() {
-      for(int i = 0; i < queues.size(); i++) {
-        if (!queues.get(i).isEmpty()) {
-          return false;
-        }
-      }
-      return true;
-    }
-
-    int numQueues() {
-      return queues.size();
+      queues = Collections.synchronizedList(list);
     }
 
     void offer(int i, T object) {
@@ -92,6 +91,14 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       }
     }
 
+    T takeWithTimeout(int i) throws InterruptedIOException {
+      try {
+        return queues.get(i).poll(100, TimeUnit.MILLISECONDS);
+      } catch (InterruptedException e) {
+        throw DFSUtil.toInterruptedIOException("take interrupted, i=" + i, e);
+      }
+    }
+
     T poll(int i) {
       return queues.get(i).poll();
     }
@@ -99,23 +106,44 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     T peek(int i) {
       return queues.get(i).peek();
     }
+
+    void clear() {
+      for (BlockingQueue<T> q : queues) {
+        q.clear();
+      }
+    }
   }
 
   /** Coordinate the communication between the streamers. */
-  class Coordinator {
+  static class Coordinator {
+    /**
+     * The next internal block to write to for each streamers. The
+     * DFSStripedOutputStream makes the {@link ClientProtocol#addBlock} RPC to
+     * get a new block group. The block group is split to internal blocks, which
+     * are then distributed into the queue for streamers to retrieve.
+     */
     private final MultipleBlockingQueue<LocatedBlock> followingBlocks;
+    /**
+     * Used to sync among all the streamers before allocating a new block. The
+     * DFSStripedOutputStream uses this to make sure every streamer has finished
+     * writing the previous block.
+     */
     private final MultipleBlockingQueue<ExtendedBlock> endBlocks;
 
+    /**
+     * The following data structures are used for syncing while handling errors
+     */
     private final MultipleBlockingQueue<LocatedBlock> newBlocks;
-    private final MultipleBlockingQueue<ExtendedBlock> updateBlocks;
+    private final Map<StripedDataStreamer, Boolean> updateStreamerMap;
+    private final MultipleBlockingQueue<Boolean> streamerUpdateResult;
 
-    Coordinator(final DfsClientConf conf, final int numDataBlocks,
-        final int numAllBlocks) {
+    Coordinator(final int numAllBlocks) {
       followingBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
-      endBlocks = new MultipleBlockingQueue<>(numDataBlocks, 1);
-
+      endBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
       newBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
-      updateBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
+      updateStreamerMap = Collections.synchronizedMap(
+          new HashMap<StripedDataStreamer, Boolean>(numAllBlocks));
+      streamerUpdateResult = new MultipleBlockingQueue<>(numAllBlocks, 1);
     }
 
     MultipleBlockingQueue<LocatedBlock> getFollowingBlocks() {
@@ -126,68 +154,28 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       return newBlocks;
     }
 
-    MultipleBlockingQueue<ExtendedBlock> getUpdateBlocks() {
-      return updateBlocks;
-    }
-
-    StripedDataStreamer getStripedDataStreamer(int i) {
-      return DFSStripedOutputStream.this.getStripedDataStreamer(i);
-    }
-
     void offerEndBlock(int i, ExtendedBlock block) {
       endBlocks.offer(i, block);
     }
 
-    ExtendedBlock takeEndBlock(int i) throws InterruptedIOException {
-      return endBlocks.take(i);
+    void offerStreamerUpdateResult(int i, boolean success) {
+      streamerUpdateResult.offer(i, success);
     }
 
-    boolean hasAllEndBlocks() {
-      for(int i = 0; i < endBlocks.numQueues(); i++) {
-        if (endBlocks.peek(i) == null) {
-          return false;
-        }
-      }
-      return true;
+    boolean takeStreamerUpdateResult(int i) throws InterruptedIOException {
+      return streamerUpdateResult.take(i);
     }
 
-    void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) {
-      ExtendedBlock b = endBlocks.peek(i);
-      if (b == null) {
-        // streamer just has failed, put end block and continue
-        b = block;
-        offerEndBlock(i, b);
-      }
-      b.setNumBytes(newBytes);
+    void updateStreamer(StripedDataStreamer streamer,
+        boolean success) {
+      assert !updateStreamerMap.containsKey(streamer);
+      updateStreamerMap.put(streamer, success);
     }
 
-    /** @return a block representing the entire block group. */
-    ExtendedBlock getBlockGroup() {
-      final StripedDataStreamer s0 = getStripedDataStreamer(0);
-      final ExtendedBlock b0 = s0.getBlock();
-      if (b0 == null) {
-        return null;
-      }
-
-      final boolean atBlockGroupBoundary = s0.getBytesCurBlock() == 0 && b0.getNumBytes() > 0;
-
-      final ExtendedBlock block = new ExtendedBlock(b0);
-      long numBytes = atBlockGroupBoundary? b0.getNumBytes(): s0.getBytesCurBlock();
-      for (int i = 1; i < numAllBlocks; i++) {
-        final StripedDataStreamer si = getStripedDataStreamer(i);
-        final ExtendedBlock bi = si.getBlock();
-        if (bi != null && bi.getGenerationStamp() > block.getGenerationStamp()) {
-          block.setGenerationStamp(bi.getGenerationStamp());
-        }
-        if (i < numDataBlocks) {
-          numBytes += atBlockGroupBoundary? bi.getNumBytes(): si.getBytesCurBlock();
-        }
-      }
-      block.setNumBytes(numBytes);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("getBlockGroup: " + block + ", numBytes=" + block.getNumBytes());
-      }
-      return block;
+    void clearFailureStates() {
+      newBlocks.clear();
+      updateStreamerMap.clear();
+      streamerUpdateResult.clear();
     }
   }
 
@@ -263,18 +251,16 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   private final int cellSize;
   private final int numAllBlocks;
   private final int numDataBlocks;
-
-  @Override
-  ExtendedBlock getBlock() {
-    return coordinator.getBlockGroup();
-  }
+  private ExtendedBlock currentBlockGroup;
+  private final String[] favoredNodes;
+  private final List<StripedDataStreamer> failedStreamers;
 
   /** Construct a new output stream for creating a file. */
   DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
                          EnumSet<CreateFlag> flag, Progressable progress,
                          DataChecksum checksum, String[] favoredNodes)
                          throws IOException {
-    super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
+    super(dfsClient, src, stat, flag, progress, checksum, favoredNodes, false);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Creating DFSStripedOutputStream for " + src);
     }
@@ -284,12 +270,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     cellSize = ecPolicy.getCellSize();
     numDataBlocks = ecPolicy.getNumDataUnits();
     numAllBlocks = numDataBlocks + numParityBlocks;
+    this.favoredNodes = favoredNodes;
+    failedStreamers = new ArrayList<>();
 
     encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(),
         numDataBlocks, numParityBlocks);
 
-    coordinator = new Coordinator(dfsClient.getConf(),
-        numDataBlocks, numAllBlocks);
+    coordinator = new Coordinator(numAllBlocks);
     try {
       cellBuffers = new CellBuffers(numParityBlocks);
     } catch (InterruptedException ie) {
@@ -297,14 +284,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
           "Failed to create cell buffers", ie);
     }
 
-    List<StripedDataStreamer> s = new ArrayList<>(numAllBlocks);
+    streamers = new ArrayList<>(numAllBlocks);
     for (short i = 0; i < numAllBlocks; i++) {
       StripedDataStreamer streamer = new StripedDataStreamer(stat,
           dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
           favoredNodes, i, coordinator);
-      s.add(streamer);
+      streamers.add(streamer);
     }
-    streamers = Collections.unmodifiableList(s);
     currentPackets = new DFSPacket[streamers.size()];
     setCurrentStreamer(0);
   }
@@ -318,17 +304,19 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   }
 
   private synchronized StripedDataStreamer getCurrentStreamer() {
-    return (StripedDataStreamer)streamer;
+    return (StripedDataStreamer) streamer;
   }
 
   private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) {
     // backup currentPacket for current streamer
-    int oldIdx = streamers.indexOf(streamer);
-    if (oldIdx >= 0) {
-      currentPackets[oldIdx] = currentPacket;
+    if (streamer != null) {
+      int oldIdx = streamers.indexOf(getCurrentStreamer());
+      if (oldIdx >= 0) {
+        currentPackets[oldIdx] = currentPacket;
+      }
     }
 
-    streamer = streamers.get(newIdx);
+    streamer = getStripedDataStreamer(newIdx);
     currentPacket = currentPackets[newIdx];
     adjustChunkBoundary();
 
@@ -350,40 +338,127 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     encoder.encode(dataBuffers, parityBuffers);
   }
 
-
-  private void checkStreamers(boolean setExternalError) throws IOException {
-    int count = 0;
+  /**
+   * check all the existing StripedDataStreamer and find newly failed streamers.
+   * @return The newly failed streamers.
+   * @throws IOException if less than {@link #numDataBlocks} streamers are still
+   *                     healthy.
+   */
+  private Set<StripedDataStreamer> checkStreamers() throws IOException {
+    Set<StripedDataStreamer> newFailed = new HashSet<>();
     for(StripedDataStreamer s : streamers) {
-      if (!s.isFailed()) {
-        if (setExternalError && s.getBlock() != null) {
-          s.getErrorState().initExternalError();
-        }
-        count++;
+      if (!s.isHealthy() && !failedStreamers.contains(s)) {
+        newFailed.add(s);
       }
     }
+
+    final int failCount = failedStreamers.size() + newFailed.size();
     if (LOG.isDebugEnabled()) {
       LOG.debug("checkStreamers: " + streamers);
-      LOG.debug("count=" + count);
+      LOG.debug("healthy streamer count=" + (numAllBlocks - failCount));
+      LOG.debug("original failed streamers: " + failedStreamers);
+      LOG.debug("newly failed streamers: " + newFailed);
     }
-    if (count < numDataBlocks) {
-      throw new IOException("Failed: the number of remaining blocks = "
-          + count + " < the number of data blocks = " + numDataBlocks);
+    if (failCount > (numAllBlocks - numDataBlocks)) {
+      throw new IOException("Failed: the number of failed blocks = "
+          + failCount + " > the number of data blocks = "
+          + (numAllBlocks - numDataBlocks));
     }
+    return newFailed;
   }
 
   private void handleStreamerFailure(String err, Exception e)
       throws IOException {
-    handleStreamerFailure(err, e, true);
-  }
-
-  private void handleStreamerFailure(String err, Exception e,
-      boolean setExternalError) throws IOException {
     LOG.warn("Failed: " + err + ", " + this, e);
-    getCurrentStreamer().setFailed(true);
-    checkStreamers(setExternalError);
+    getCurrentStreamer().getErrorState().setInternalError();
+    getCurrentStreamer().close(true);
+    checkStreamers();
     currentPacket = null;
   }
 
+  private void replaceFailedStreamers() {
+    assert streamers.size() == numAllBlocks;
+    for (short i = 0; i < numAllBlocks; i++) {
+      final StripedDataStreamer oldStreamer = getStripedDataStreamer(i);
+      if (!oldStreamer.isHealthy()) {
+        StripedDataStreamer streamer = new StripedDataStreamer(oldStreamer.stat,
+            dfsClient, src, oldStreamer.progress,
+            oldStreamer.checksum4WriteBlock, cachingStrategy, byteArrayManager,
+            favoredNodes, i, coordinator);
+        streamers.set(i, streamer);
+        currentPackets[i] = null;
+        if (i == 0) {
+          this.streamer = streamer;
+        }
+        streamer.start();
+      }
+    }
+  }
+
+  private void waitEndBlocks(int i) throws IOException {
+    while (getStripedDataStreamer(i).isHealthy()) {
+      final ExtendedBlock b = coordinator.endBlocks.takeWithTimeout(i);
+      if (b != null) {
+        StripedBlockUtil.checkBlocks(currentBlockGroup, i, b);
+        return;
+      }
+    }
+  }
+
+  private void allocateNewBlock() throws IOException {
+    if (currentBlockGroup != null) {
+      for (int i = 0; i < numAllBlocks; i++) {
+        // sync all the healthy streamers before writing to the new block
+        waitEndBlocks(i);
+      }
+    }
+    failedStreamers.clear();
+    // replace failed streamers
+    replaceFailedStreamers();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Allocating new block group. The previous block group: "
+          + currentBlockGroup);
+    }
+
+    // TODO collect excludedNodes from all the data streamers
+    final LocatedBlock lb = addBlock(null, dfsClient, src, currentBlockGroup,
+        fileId, favoredNodes);
+    assert lb.isStriped();
+    if (lb.getLocations().length < numDataBlocks) {
+      throw new IOException("Failed to get " + numDataBlocks
+          + " nodes from namenode: blockGroupSize= " + numAllBlocks
+          + ", blocks.length= " + lb.getLocations().length);
+    }
+    // assign the new block to the current block group
+    currentBlockGroup = lb.getBlock();
+
+    final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
+        (LocatedStripedBlock) lb, cellSize, numDataBlocks,
+        numAllBlocks - numDataBlocks);
+    for (int i = 0; i < blocks.length; i++) {
+      StripedDataStreamer si = getStripedDataStreamer(i);
+      if (si.isHealthy()) { // skipping failed data streamer
+        if (blocks[i] == null) {
+          // Set exception and close streamer as there is no block locations
+          // found for the parity block.
+          LOG.warn("Failed to get block location for parity block, index=" + i);
+          si.getLastException().set(
+              new IOException("Failed to get following block, i=" + i));
+          si.getErrorState().setInternalError();
+          si.close(true);
+        } else {
+          coordinator.getFollowingBlocks().offer(i, blocks[i]);
+        }
+      }
+    }
+  }
+
+  private boolean shouldEndBlockGroup() {
+    return currentBlockGroup != null &&
+        currentBlockGroup.getNumBytes() == blockSize * numDataBlocks;
+  }
+
   @Override
   protected synchronized void writeChunk(byte[] bytes, int offset, int len,
       byte[] checksum, int ckoff, int cklen) throws IOException {
@@ -392,8 +467,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     final int pos = cellBuffers.addTo(index, bytes, offset, len);
     final boolean cellFull = pos == cellSize;
 
-    final long oldBytes = current.getBytesCurBlock();
-    if (!current.isFailed()) {
+    if (currentBlockGroup == null || shouldEndBlockGroup()) {
+      // the incoming data should belong to a new block. Allocate a new block.
+      allocateNewBlock();
+    }
+
+    currentBlockGroup.setNumBytes(currentBlockGroup.getNumBytes() + len);
+    if (current.isHealthy()) {
       try {
         super.writeChunk(bytes, offset, len, checksum, ckoff, cklen);
       } catch(Exception e) {
@@ -401,12 +481,6 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       }
     }
 
-    if (current.isFailed()) {
-      final long newBytes = oldBytes + len;
-      coordinator.setBytesEndBlock(index, newBytes, current.getBlock());
-      current.setBytesCurBlock(newBytes);
-    }
-
     // Two extra steps are needed when a striping cell is full:
     // 1. Forward the current index pointer
     // 2. Generate parity packets if a full stripe of data cells are present
@@ -419,11 +493,209 @@ public class DFSStripedOutputStream extends DFSOutputStream {
         cellBuffers.flipDataBuffers();
         writeParityCells();
         next = 0;
+        // check failure state for all the streamers. Bump GS if necessary
+        checkStreamerFailures();
+
+        // if this is the end of the block group, end each internal block
+        if (shouldEndBlockGroup()) {
+          for (int i = 0; i < numAllBlocks; i++) {
+            final StripedDataStreamer s = setCurrentStreamer(i);
+            if (s.isHealthy()) {
+              try {
+                endBlock();
+              } catch (IOException ignored) {}
+            }
+          }
+        }
       }
       setCurrentStreamer(next);
     }
   }
 
+  @Override
+  void enqueueCurrentPacketFull() throws IOException {
+    LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
+            + " appendChunk={}, {}", currentPacket, src, getStreamer()
+            .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
+        getStreamer());
+    enqueueCurrentPacket();
+    adjustChunkBoundary();
+    // no need to end block here
+  }
+
+  private Set<StripedDataStreamer> markExternalErrorOnStreamers() {
+    Set<StripedDataStreamer> healthySet = new HashSet<>();
+    for (StripedDataStreamer streamer : streamers) {
+      if (streamer.isHealthy() &&
+          streamer.getStage() == BlockConstructionStage.DATA_STREAMING) {
+        streamer.setExternalError();
+        healthySet.add(streamer);
+      }
+    }
+    return healthySet;
+  }
+
+  /**
+   * Check and handle data streamer failures. This is called only when we have
+   * written a full stripe (i.e., enqueue all packets for a full stripe), or
+   * when we're closing the outputstream.
+   */
+  private void checkStreamerFailures() throws IOException {
+    Set<StripedDataStreamer> newFailed = checkStreamers();
+    if (newFailed.size() > 0) {
+      // for healthy streamers, wait till all of them have fetched the new block
+      // and flushed out all the enqueued packets.
+      flushAllInternals();
+    }
+    // get all the current failed streamers after the flush
+    newFailed = checkStreamers();
+    while (newFailed.size() > 0) {
+      failedStreamers.addAll(newFailed);
+      coordinator.clearFailureStates();
+
+      // mark all the healthy streamers as external error
+      Set<StripedDataStreamer> healthySet = markExternalErrorOnStreamers();
+
+      // we have newly failed streamers, update block for pipeline
+      final ExtendedBlock newBG = updateBlockForPipeline(healthySet);
+
+      // wait till all the healthy streamers to
+      // 1) get the updated block info
+      // 2) create new block outputstream
+      newFailed = waitCreatingNewStreams(healthySet);
+      if (newFailed.size() + failedStreamers.size() >
+          numAllBlocks - numDataBlocks) {
+        throw new IOException(
+            "Data streamers failed while creating new block streams: "
+                + newFailed + ". There are not enough healthy streamers.");
+      }
+      for (StripedDataStreamer failedStreamer : newFailed) {
+        assert !failedStreamer.isHealthy();
+      }
+
+      // TODO we can also succeed if all the failed streamers have not taken
+      // the updated block
+      if (newFailed.size() == 0) {
+        // reset external error state of all the streamers
+        for (StripedDataStreamer streamer : healthySet) {
+          assert streamer.isHealthy();
+          streamer.getErrorState().reset();
+        }
+        updatePipeline(newBG);
+      }
+      for (int i = 0; i < numAllBlocks; i++) {
+        coordinator.offerStreamerUpdateResult(i, newFailed.size() == 0);
+      }
+    }
+  }
+
+  private int checkStreamerUpdates(Set<StripedDataStreamer> failed,
+      Set<StripedDataStreamer> streamers) {
+    for (StripedDataStreamer streamer : streamers) {
+      if (!coordinator.updateStreamerMap.containsKey(streamer)) {
+        if (!streamer.isHealthy() &&
+            coordinator.getNewBlocks().peek(streamer.getIndex()) != null) {
+          // this streamer had internal error before getting updated block
+          failed.add(streamer);
+        }
+      }
+    }
+    return coordinator.updateStreamerMap.size() + failed.size();
+  }
+
+  private Set<StripedDataStreamer> waitCreatingNewStreams(
+      Set<StripedDataStreamer> healthyStreamers) throws IOException {
+    Set<StripedDataStreamer> failed = new HashSet<>();
+    final int expectedNum = healthyStreamers.size();
+    final long socketTimeout = dfsClient.getConf().getSocketTimeout();
+    // the total wait time should be less than the socket timeout, otherwise
+    // a slow streamer may cause other streamers to timeout. here we wait for
+    // half of the socket timeout
+    long remaingTime = socketTimeout > 0 ? socketTimeout/2 : Long.MAX_VALUE;
+    final long waitInterval = 1000;
+    synchronized (coordinator) {
+      while (checkStreamerUpdates(failed, healthyStreamers) < expectedNum
+          && remaingTime > 0) {
+        try {
+          long start = Time.monotonicNow();
+          coordinator.wait(waitInterval);
+          remaingTime -= Time.monotonicNow() - start;
+        } catch (InterruptedException e) {
+          throw DFSUtil.toInterruptedIOException("Interrupted when waiting" +
+              " for results of updating striped streamers", e);
+        }
+      }
+    }
+    synchronized (coordinator) {
+      for (StripedDataStreamer streamer : healthyStreamers) {
+        if (!coordinator.updateStreamerMap.containsKey(streamer)) {
+          // close the streamer if it is too slow to create new connection
+          streamer.setStreamerAsClosed();
+          failed.add(streamer);
+        }
+      }
+    }
+    for (Map.Entry<StripedDataStreamer, Boolean> entry :
+        coordinator.updateStreamerMap.entrySet()) {
+      if (!entry.getValue()) {
+        failed.add(entry.getKey());
+      }
+    }
+    for (StripedDataStreamer failedStreamer : failed) {
+      healthyStreamers.remove(failedStreamer);
+    }
+    return failed;
+  }
+
+  /**
+   * Call {@link ClientProtocol#updateBlockForPipeline} and assign updated block
+   * to healthy streamers.
+   * @param healthyStreamers The healthy data streamers. These streamers join
+   *                         the failure handling.
+   */
+  private ExtendedBlock updateBlockForPipeline(
+      Set<StripedDataStreamer> healthyStreamers) throws IOException {
+    final LocatedBlock updated = dfsClient.namenode.updateBlockForPipeline(
+        currentBlockGroup, dfsClient.clientName);
+    final long newGS = updated.getBlock().getGenerationStamp();
+    ExtendedBlock newBlock = new ExtendedBlock(currentBlockGroup);
+    newBlock.setGenerationStamp(newGS);
+    final LocatedBlock[] updatedBlks = StripedBlockUtil.parseStripedBlockGroup(
+        (LocatedStripedBlock) updated, cellSize, numDataBlocks,
+        numAllBlocks - numDataBlocks);
+
+    for (int i = 0; i < numAllBlocks; i++) {
+      StripedDataStreamer si = getStripedDataStreamer(i);
+      if (healthyStreamers.contains(si)) {
+        final LocatedBlock lb = new LocatedBlock(new ExtendedBlock(newBlock),
+            null, null, null, -1, updated.isCorrupt(), null);
+        lb.setBlockToken(updatedBlks[i].getBlockToken());
+        coordinator.getNewBlocks().offer(i, lb);
+      }
+    }
+    return newBlock;
+  }
+
+  private void updatePipeline(ExtendedBlock newBG) throws IOException {
+    final DatanodeInfo[] newNodes = new DatanodeInfo[numAllBlocks];
+    final String[] newStorageIDs = new String[numAllBlocks];
+    for (int i = 0; i < numAllBlocks; i++) {
+      final StripedDataStreamer streamer = getStripedDataStreamer(i);
+      final DatanodeInfo[] nodes = streamer.getNodes();
+      final String[] storageIDs = streamer.getStorageIDs();
+      if (streamer.isHealthy() && nodes != null && storageIDs != null) {
+        newNodes[i] = nodes[0];
+        newStorageIDs[i] = storageIDs[0];
+      } else {
+        newNodes[i] = new DatanodeInfo(DatanodeID.EMPTY_DATANODE_ID);
+        newStorageIDs[i] = "";
+      }
+    }
+    dfsClient.namenode.updatePipeline(dfsClient.clientName, currentBlockGroup,
+        newBG, newNodes, newStorageIDs);
+    currentBlockGroup = newBG;
+  }
+
   private int stripeDataSize() {
     return numDataBlocks * cellSize;
   }
@@ -500,28 +772,16 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     }
   }
 
-  /**
-   * Simply add bytesCurBlock together. Note that this result is not accurately
-   * the size of the block group.
-   */
-  private long getCurrentSumBytes() {
-    long sum = 0;
-    for (int i = 0; i < numDataBlocks; i++) {
-      sum += streamers.get(i).getBytesCurBlock();
-    }
-    return sum;
-  }
-
   private boolean generateParityCellsForLastStripe() {
-    final long currentBlockGroupBytes = getCurrentSumBytes();
-    if (currentBlockGroupBytes % stripeDataSize() == 0) {
+    final long currentBlockGroupBytes = currentBlockGroup == null ?
+        0 : currentBlockGroup.getNumBytes();
+    final long lastStripeSize = currentBlockGroupBytes % stripeDataSize();
+    if (lastStripeSize == 0) {
       return false;
     }
 
-    final int firstCellSize =
-        (int)(getStripedDataStreamer(0).getBytesCurBlock() % cellSize);
-    final int parityCellSize = firstCellSize > 0 && firstCellSize < cellSize?
-        firstCellSize : cellSize;
+    final long parityCellSize = lastStripeSize < cellSize?
+        lastStripeSize : cellSize;
     final ByteBuffer[] buffers = cellBuffers.getBuffers();
 
     for (int i = 0; i < numAllBlocks; i++) {
@@ -550,13 +810,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     cellBuffers.clear();
   }
 
-  void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf
-      ) throws IOException {
+  void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf)
+      throws IOException {
     final StripedDataStreamer current = setCurrentStreamer(index);
     final int len = buffer.limit();
 
     final long oldBytes = current.getBytesCurBlock();
-    if (!current.isFailed()) {
+    if (current.isHealthy()) {
       try {
         DataChecksum sum = getDataChecksum();
         sum.calculateChunkedSums(buffer.array(), 0, len, checksumBuf, 0);
@@ -570,18 +830,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
         handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e);
       }
     }
-
-    if (current.isFailed()) {
-      final long newBytes = oldBytes + len;
-      current.setBytesCurBlock(newBytes);
-    }
   }
 
   @Override
   void setClosed() {
     super.setClosed();
     for (int i = 0; i < numAllBlocks; i++) {
-      streamers.get(i).release();
+      getStripedDataStreamer(i).release();
     }
     cellBuffers.release();
   }
@@ -607,37 +862,40 @@ public class DFSStripedOutputStream extends DFSOutputStream {
 
     try {
       // flush from all upper layers
-      try {
-        flushBuffer();
-      } catch(Exception e) {
-        handleStreamerFailure("flushBuffer " + getCurrentStreamer(), e);
-      }
+      flushBuffer();
       // if the last stripe is incomplete, generate and write parity cells
       if (generateParityCellsForLastStripe()) {
         writeParityCells();
       }
       enqueueAllCurrentPackets();
 
+      // flush all the data packets
+      flushAllInternals();
+      // check failures
+      checkStreamerFailures();
+
       for (int i = 0; i < numAllBlocks; i++) {
         final StripedDataStreamer s = setCurrentStreamer(i);
-        if (!s.isFailed()) {
+        if (s.isHealthy()) {
           try {
             if (s.getBytesCurBlock() > 0) {
               setCurrentPacketToEmpty();
             }
-            // flush all data to Datanode
+            // flush the last "close" packet to Datanode
             flushInternal();
           } catch(Exception e) {
-            handleStreamerFailure("flushInternal " + s, e, false);
+            // TODO for both close and endBlock, we currently do not handle
+            // failures when sending the last packet. We actually do not need to
+            // bump GS for this kind of failure. Thus counting the total number
+            // of failures may be good enough.
           }
         }
       }
 
       closeThreads(false);
-      final ExtendedBlock lastBlock = coordinator.getBlockGroup();
       TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
       try {
-        completeFile(lastBlock);
+        completeFile(currentBlockGroup);
       } finally {
         scope.close();
       }
@@ -652,14 +910,45 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     int idx = streamers.indexOf(getCurrentStreamer());
     for(int i = 0; i < streamers.size(); i++) {
       final StripedDataStreamer si = setCurrentStreamer(i);
-      if (!si.isFailed() && currentPacket != null) {
+      if (si.isHealthy() && currentPacket != null) {
         try {
           enqueueCurrentPacket();
         } catch (IOException e) {
-          handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e, false);
+          handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e);
         }
       }
     }
     setCurrentStreamer(idx);
   }
+
+  void flushAllInternals() throws IOException {
+    int current = getCurrentIndex();
+
+    for (int i = 0; i < numAllBlocks; i++) {
+      final StripedDataStreamer s = setCurrentStreamer(i);
+      if (s.isHealthy()) {
+        try {
+          // flush all data to Datanode
+          flushInternal();
+        } catch(Exception e) {
+          handleStreamerFailure("flushInternal " + s, e);
+        }
+      }
+    }
+    setCurrentStreamer(current);
+  }
+
+  static void sleep(long ms, String op) throws InterruptedIOException {
+    try {
+      Thread.sleep(ms);
+    } catch(InterruptedException ie) {
+      throw DFSUtil.toInterruptedIOException(
+          "Sleep interrupted during " + op, ie);
+    }
+  }
+
+  @Override
+  ExtendedBlock getBlock() {
+    return currentBlockGroup;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index c478f1c..a6eb01f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -22,7 +22,6 @@ import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SU
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InterruptedIOException;
@@ -46,16 +45,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
@@ -69,13 +64,10 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MultipleIOException;
-import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
@@ -204,9 +196,12 @@ class DataStreamer extends Daemon {
     }
   }
 
+  enum ErrorType {
+    NONE, INTERNAL, EXTERNAL
+  }
+
   static class ErrorState {
-    private boolean error = false;
-    private boolean externalError = false;
+    ErrorType error = ErrorType.NONE;
     private int badNodeIndex = -1;
     private int restartingNodeIndex = -1;
     private long restartingNodeDeadline = 0;
@@ -216,35 +211,47 @@ class DataStreamer extends Daemon {
       this.datanodeRestartTimeout = datanodeRestartTimeout;
     }
 
+    synchronized void resetInternalError() {
+      if (hasInternalError()) {
+        error = ErrorType.NONE;
+      }
+      badNodeIndex = -1;
+      restartingNodeIndex = -1;
+      restartingNodeDeadline = 0;
+    }
+
     synchronized void reset() {
-      error = false;
-      externalError = false;
+      error = ErrorType.NONE;
       badNodeIndex = -1;
       restartingNodeIndex = -1;
       restartingNodeDeadline = 0;
     }
 
-    synchronized boolean hasError() {
-      return error;
+    synchronized boolean hasInternalError() {
+      return error == ErrorType.INTERNAL;
     }
 
-    synchronized boolean hasExternalErrorOnly() {
-      return error && externalError && !isNodeMarked();
+    synchronized boolean hasExternalError() {
+      return error == ErrorType.EXTERNAL;
     }
 
-    synchronized boolean hasDatanodeError() {
-      return error && (isNodeMarked() || externalError);
+    synchronized boolean hasError() {
+      return error != ErrorType.NONE;
     }
 
-    synchronized void setError(boolean err) {
-      this.error = err;
+    synchronized boolean hasDatanodeError() {
+      return error == ErrorType.INTERNAL && isNodeMarked();
     }
 
-    synchronized void initExternalError() {
-      setError(true);
-      this.externalError = true;
+    synchronized void setInternalError() {
+      this.error = ErrorType.INTERNAL;
     }
 
+    synchronized void setExternalError() {
+      if (!hasInternalError()) {
+        this.error = ErrorType.EXTERNAL;
+      }
+    }
 
     synchronized void setBadNodeIndex(int index) {
       this.badNodeIndex = index;
@@ -306,14 +313,14 @@ class DataStreamer extends Daemon {
       }
 
       if (!isRestartingNode()) {
-        error = false;
+        error = ErrorType.NONE;
       }
       badNodeIndex = -1;
     }
 
     synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) {
       if (restartingNodeIndex >= 0) {
-        if (!error) {
+        if (error == ErrorType.NONE) {
           throw new IllegalStateException("error=false while checking" +
               " restarting node deadline");
         }
@@ -345,7 +352,7 @@ class DataStreamer extends Daemon {
 
   private volatile boolean streamerClosed = false;
   protected ExtendedBlock block; // its length is number of bytes acked
-  private Token<BlockTokenIdentifier> accessToken;
+  protected Token<BlockTokenIdentifier> accessToken;
   private DataOutputStream blockStream;
   private DataInputStream blockReplyStream;
   private ResponseProcessor response = null;
@@ -355,7 +362,7 @@ class DataStreamer extends Daemon {
   private final ErrorState errorState;
 
   private BlockConstructionStage stage;  // block construction stage
-  private long bytesSent = 0; // number of bytes that've been sent
+  protected long bytesSent = 0; // number of bytes that've been sent
   private final boolean isLazyPersistFile;
 
   /** Nodes have been used in the pipeline before and have failed. */
@@ -378,13 +385,13 @@ class DataStreamer extends Daemon {
   protected final DFSClient dfsClient;
   protected final String src;
   /** Only for DataTransferProtocol.writeBlock(..) */
-  private final DataChecksum checksum4WriteBlock;
-  private final Progressable progress;
+  final DataChecksum checksum4WriteBlock;
+  final Progressable progress;
   protected final HdfsFileStatus stat;
   // appending to existing partial block
   private volatile boolean appendChunk = false;
   // both dataQueue and ackQueue are protected by dataQueue lock
-  private final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
+  protected final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
   private final LinkedList<DFSPacket> ackQueue = new LinkedList<>();
   private final AtomicReference<CachingStrategy> cachingStrategy;
   private final ByteArrayManager byteArrayManager;
@@ -401,7 +408,7 @@ class DataStreamer extends Daemon {
       CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10;
   private int lastCongestionBackoffTime;
 
-  private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
+  protected final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
   private final String[] favoredNodes;
 
   private DataStreamer(HdfsFileStatus stat, ExtendedBlock block,
@@ -473,6 +480,10 @@ class DataStreamer extends Daemon {
     }
   }
 
+  void setAccessToken(Token<BlockTokenIdentifier> t) {
+    this.accessToken = t;
+  }
+
   private void setPipeline(LocatedBlock lb) {
     setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
   }
@@ -533,7 +544,7 @@ class DataStreamer extends Daemon {
       DFSPacket one;
       try {
         // process datanode IO errors if any
-        boolean doSleep = processDatanodeError();
+        boolean doSleep = processDatanodeOrExternalError();
 
         final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2; 
         synchronized (dataQueue) {
@@ -696,7 +707,7 @@ class DataStreamer extends Daemon {
         }
         lastException.set(e);
         assert !(e instanceof NullPointerException);
-        errorState.setError(true);
+        errorState.setInternalError();
         if (!errorState.isNodeMarked()) {
           // Not a datanode issue
           streamerClosed = true;
@@ -837,6 +848,9 @@ class DataStreamer extends Daemon {
     }
   }
 
+  void setStreamerAsClosed() {
+    streamerClosed = true;
+  }
 
   private void checkClosed() throws IOException {
     if (streamerClosed) {
@@ -857,7 +871,7 @@ class DataStreamer extends Daemon {
     }
   }
 
-  private void closeStream() {
+  void closeStream() {
     final MultipleIOException.Builder b = new MultipleIOException.Builder();
 
     if (blockStream != null) {
@@ -1037,7 +1051,7 @@ class DataStreamer extends Daemon {
         } catch (Exception e) {
           if (!responderClosed) {
             lastException.set(e);
-            errorState.setError(true);
+            errorState.setInternalError();
             errorState.markFirstNodeIfNotMarked();
             synchronized (dataQueue) {
               dataQueue.notifyAll();
@@ -1059,18 +1073,18 @@ class DataStreamer extends Daemon {
     }
   }
 
+  private boolean shouldHandleExternalError(){
+    return errorState.hasExternalError() && blockStream != null;
+  }
+
   /**
    * If this stream has encountered any errors, shutdown threads
    * and mark the stream as closed.
    *
    * @return true if it should sleep for a while after returning.
    */
-  private boolean processDatanodeError() throws IOException {
-    if (!errorState.hasDatanodeError()) {
-      return false;
-    }
-    if (errorState.hasExternalErrorOnly() && block == null) {
-      // block is not yet initialized, handle external error later.
+  private boolean processDatanodeOrExternalError() throws IOException {
+    if (!errorState.hasDatanodeError() && !shouldHandleExternalError()) {
       return false;
     }
     if (response != null) {
@@ -1103,7 +1117,8 @@ class DataStreamer extends Daemon {
         return false;
       }
     }
-    boolean doSleep = setupPipelineForAppendOrRecovery();
+
+    setupPipelineForAppendOrRecovery();
 
     if (!streamerClosed && dfsClient.clientRunning) {
       if (stage == BlockConstructionStage.PIPELINE_CLOSE) {
@@ -1135,7 +1150,7 @@ class DataStreamer extends Daemon {
       }
     }
 
-    return doSleep;
+    return false;
   }
 
   void setHflush() {
@@ -1266,7 +1281,7 @@ class DataStreamer extends Daemon {
    * This happens when a file is appended or data streaming fails
    * It keeps on trying until a pipeline is setup
    */
-  private boolean setupPipelineForAppendOrRecovery() throws IOException {
+  private void setupPipelineForAppendOrRecovery() throws IOException {
     // check number of datanodes
     if (nodes == null || nodes.length == 0) {
       String msg = "Could not get block locations. " + "Source file \""
@@ -1274,19 +1289,23 @@ class DataStreamer extends Daemon {
       LOG.warn(msg);
       lastException.set(new IOException(msg));
       streamerClosed = true;
-      return false;
+      return;
     }
+    setupPipelineInternal(nodes, storageTypes);
+  }
 
+  protected void setupPipelineInternal(DatanodeInfo[] datanodes,
+      StorageType[] nodeStorageTypes) throws IOException {
     boolean success = false;
     long newGS = 0L;
     while (!success && !streamerClosed && dfsClient.clientRunning) {
       if (!handleRestartingDatanode()) {
-        return false;
+        return;
       }
 
-      final boolean isRecovery = errorState.hasError();
+      final boolean isRecovery = errorState.hasInternalError();
       if (!handleBadDatanode()) {
-        return false;
+        return;
       }
 
       handleDatanodeReplacement();
@@ -1307,7 +1326,6 @@ class DataStreamer extends Daemon {
     if (success) {
       block = updatePipeline(newGS);
     }
-    return false; // do not sleep, continue processing
   }
 
   /**
@@ -1315,7 +1333,7 @@ class DataStreamer extends Daemon {
    * This process is repeated until the deadline or the node starts back up.
    * @return true if it should continue.
    */
-  private boolean handleRestartingDatanode() {
+  boolean handleRestartingDatanode() {
     if (errorState.isRestartingNode()) {
       // 4 seconds or the configured deadline period, whichever is shorter.
       // This is the retry interval and recovery will be retried in this
@@ -1338,7 +1356,7 @@ class DataStreamer extends Daemon {
    * Remove bad node from list of nodes if badNodeIndex was set.
    * @return true if it should continue.
    */
-  private boolean handleBadDatanode() {
+  boolean handleBadDatanode() {
     final int badNodeIndex = errorState.getBadNodeIndex();
     if (badNodeIndex >= 0) {
       if (nodes.length <= 1) {
@@ -1388,7 +1406,7 @@ class DataStreamer extends Daemon {
     }
   }
 
-  private void failPacket4Testing() {
+  void failPacket4Testing() {
     if (failPacket) { // for testing
       failPacket = false;
       try {
@@ -1400,13 +1418,8 @@ class DataStreamer extends Daemon {
     }
   }
 
-  LocatedBlock updateBlockForPipeline() throws IOException {
-    return callUpdateBlockForPipeline(block);
-  }
-
-  LocatedBlock callUpdateBlockForPipeline(ExtendedBlock newBlock) throws IOException {
-    return dfsClient.namenode.updateBlockForPipeline(
-        newBlock, dfsClient.clientName);
+  private LocatedBlock updateBlockForPipeline() throws IOException {
+    return dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName);
   }
 
   static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) {
@@ -1417,18 +1430,12 @@ class DataStreamer extends Daemon {
   /** update pipeline at the namenode */
   ExtendedBlock updatePipeline(long newGS) throws IOException {
     final ExtendedBlock newBlock = newBlock(block, newGS);
-    return callUpdatePipeline(block, newBlock, nodes, storageIDs);
-  }
-
-  ExtendedBlock callUpdatePipeline(ExtendedBlock oldBlock, ExtendedBlock newBlock,
-      DatanodeInfo[] newNodes, String[] newStorageIDs)
-      throws IOException {
-    dfsClient.namenode.updatePipeline(dfsClient.clientName, oldBlock, newBlock,
-        newNodes, newStorageIDs);
+    dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
+        nodes, storageIDs);
     return newBlock;
   }
 
-  int getNumBlockWriteRetry() {
+  private int getNumBlockWriteRetry() {
     return dfsClient.getConf().getNumBlockWriteRetry();
   }
 
@@ -1438,7 +1445,7 @@ class DataStreamer extends Daemon {
    * Must get block ID and the IDs of the destinations from the namenode.
    * Returns the list of target datanodes.
    */
-  private LocatedBlock nextBlockOutputStream() throws IOException {
+  protected LocatedBlock nextBlockOutputStream() throws IOException {
     LocatedBlock lb = null;
     DatanodeInfo[] nodes = null;
     StorageType[] storageTypes = null;
@@ -1446,9 +1453,8 @@ class DataStreamer extends Daemon {
     boolean success = false;
     ExtendedBlock oldBlock = block;
     do {
-      errorState.reset();
+      errorState.resetInternalError();
       lastException.clear();
-      success = false;
 
       DatanodeInfo[] excluded =
           excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
@@ -1488,7 +1494,7 @@ class DataStreamer extends Daemon {
   // connects to the first datanode in the pipeline
   // Returns true if success, otherwise return failure.
   //
-  private boolean createBlockOutputStream(DatanodeInfo[] nodes,
+  boolean createBlockOutputStream(DatanodeInfo[] nodes,
       StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
     if (nodes.length == 0) {
       LOG.info("nodes are empty for write pipeline of " + block);
@@ -1567,7 +1573,7 @@ class DataStreamer extends Daemon {
         assert null == blockStream : "Previous blockStream unclosed";
         blockStream = out;
         result =  true; // success
-        errorState.reset();
+        errorState.resetInternalError();
       } catch (IOException ie) {
         if (!errorState.isRestartingNode()) {
           LOG.info("Exception in createBlockOutputStream " + this, ie);
@@ -1603,7 +1609,7 @@ class DataStreamer extends Daemon {
         if (checkRestart && shouldWaitForRestart(i)) {
           errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
         }
-        errorState.setError(true);
+        errorState.setInternalError();
         lastException.set(ie);
         result =  false;  // error
       } finally {
@@ -1645,58 +1651,10 @@ class DataStreamer extends Daemon {
     }
   }
 
-  LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
+  private LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
       throws IOException {
-    final DfsClientConf conf = dfsClient.getConf(); 
-    int retries = conf.getNumBlockWriteLocateFollowingRetry();
-    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
-    while (true) {
-      long localstart = Time.monotonicNow();
-      while (true) {
-        try {
-          return dfsClient.namenode.addBlock(src, dfsClient.clientName,
-              block, excludedNodes, stat.getFileId(), favoredNodes);
-        } catch (RemoteException e) {
-          IOException ue =
-              e.unwrapRemoteException(FileNotFoundException.class,
-                  AccessControlException.class,
-                  NSQuotaExceededException.class,
-                  DSQuotaExceededException.class,
-                  QuotaByStorageTypeExceededException.class,
-                  UnresolvedPathException.class);
-          if (ue != e) {
-            throw ue; // no need to retry these exceptions
-          }
-
-
-          if (NotReplicatedYetException.class.getName().
-              equals(e.getClassName())) {
-            if (retries == 0) {
-              throw e;
-            } else {
-              --retries;
-              LOG.info("Exception while adding a block", e);
-              long elapsed = Time.monotonicNow() - localstart;
-              if (elapsed > 5000) {
-                LOG.info("Waiting for replication for "
-                    + (elapsed / 1000) + " seconds");
-              }
-              try {
-                LOG.warn("NotReplicatedYetException sleeping " + src
-                    + " retries left " + retries);
-                Thread.sleep(sleeptime);
-                sleeptime *= 2;
-              } catch (InterruptedException ie) {
-                LOG.warn("Caught exception", ie);
-              }
-            }
-          } else {
-            throw e;
-          }
-
-        }
-      }
-    }
+    return DFSOutputStream.addBlock(excludedNodes, dfsClient, src, block,
+        stat.getFileId(), favoredNodes);
   }
 
   /**
@@ -1755,6 +1713,10 @@ class DataStreamer extends Daemon {
     return storageIDs;
   }
 
+  BlockConstructionStage getStage() {
+    return stage;
+  }
+
   /**
    * return the token of the block
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
index 2f83f7c..a313ecb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -19,18 +19,15 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator;
-import org.apache.hadoop.hdfs.DFSStripedOutputStream.MultipleBlockingQueue;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
-import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
@@ -46,66 +43,8 @@ import com.google.common.annotations.VisibleForTesting;
  * other streamers.
  */
 public class StripedDataStreamer extends DataStreamer {
-  /**
-   * This class is designed for multiple threads to share a
-   * {@link MultipleBlockingQueue}. Initially, the queue is empty. The earliest
-   * thread calling poll populates entries to the queue and the other threads
-   * will wait for it. Once the entries are populated, all the threads can poll
-   * their entries.
-   *
-   * @param <T> the queue entry type.
-   */
-  static abstract class ConcurrentPoll<T> {
-    final MultipleBlockingQueue<T> queue;
-
-    ConcurrentPoll(MultipleBlockingQueue<T> queue) {
-      this.queue = queue;
-    }
-
-    T poll(final int i) throws IOException {
-      for(;;) {
-        synchronized(queue) {
-          final T polled = queue.poll(i);
-          if (polled != null) { // already populated; return polled item.
-            return polled;
-          }
-          if (isReady2Populate()) {
-            try {
-              populate();
-              return queue.poll(i);
-            } catch(IOException ioe) {
-              LOG.warn("Failed to populate, " + this, ioe);
-              throw ioe;
-            }
-          }
-        }
-
-        // sleep and then retry.
-        sleep(100, "poll");
-      }
-    }
-
-    boolean isReady2Populate() {
-      return queue.isEmpty();
-    }
-
-    abstract void populate() throws IOException;
-  }
-
-  private static void sleep(long ms, String op) throws InterruptedIOException {
-    try {
-      Thread.sleep(ms);
-    } catch(InterruptedException ie) {
-      throw DFSUtil.toInterruptedIOException(
-          "Sleep interrupted during " + op, ie);
-    }
-  }
-
   private final Coordinator coordinator;
   private final int index;
-  private volatile boolean failed;
-  private final ECSchema schema;
-  private final int cellSize;
 
   StripedDataStreamer(HdfsFileStatus stat,
                       DFSClient dfsClient, String src,
@@ -117,102 +56,59 @@ public class StripedDataStreamer extends DataStreamer {
         byteArrayManage, favoredNodes);
     this.index = index;
     this.coordinator = coordinator;
-    this.schema = stat.getErasureCodingPolicy().getSchema();
-    this.cellSize = stat.getErasureCodingPolicy().getCellSize();
   }
 
   int getIndex() {
     return index;
   }
 
-  void setFailed(boolean failed) {
-    this.failed = failed;
-  }
-
-  boolean isFailed() {
-    return failed;
-  }
-
-  private boolean isParityStreamer() {
-    return index >= schema.getNumDataUnits();
+  boolean isHealthy() {
+    return !streamerClosed() && !getErrorState().hasInternalError();
   }
 
   @Override
   protected void endBlock() {
-    if (!isParityStreamer()) {
-      coordinator.offerEndBlock(index, block);
-    }
+    coordinator.offerEndBlock(index, block);
     super.endBlock();
   }
 
-  @Override
-  int getNumBlockWriteRetry() {
-    return 0;
+  /**
+   * The upper level DFSStripedOutputStream will allocate the new block group.
+   * All the striped data streamer only needs to fetch from the queue, which
+   * should be already be ready.
+   */
+  private LocatedBlock getFollowingBlock() throws IOException {
+    if (!this.isHealthy()) {
+      // No internal block for this streamer, maybe no enough healthy DN.
+      // Throw the exception which has been set by the StripedOutputStream.
+      this.getLastException().check(false);
+    }
+    return coordinator.getFollowingBlocks().poll(index);
   }
 
   @Override
-  LocatedBlock locateFollowingBlock(final DatanodeInfo[] excludedNodes)
-      throws IOException {
-    return new ConcurrentPoll<LocatedBlock>(coordinator.getFollowingBlocks()) {
-      @Override
-      boolean isReady2Populate() {
-        return super.isReady2Populate()
-            && (block == null || coordinator.hasAllEndBlocks());
-      }
-
-      @Override
-      void populate() throws IOException {
-        getLastException().check(false);
-
-        if (block != null) {
-          // set numByte for the previous block group
-          long bytes = 0;
-          for (int i = 0; i < schema.getNumDataUnits(); i++) {
-            final ExtendedBlock b = coordinator.takeEndBlock(i);
-            StripedBlockUtil.checkBlocks(index, block, i, b);
-            bytes += b.getNumBytes();
-          }
-          block.setNumBytes(bytes);
-          block.setBlockId(block.getBlockId() - index);
-        }
-
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("locateFollowingBlock: index=" + index + ", block=" + block);
-        }
-
-        final LocatedBlock lb = StripedDataStreamer.super.locateFollowingBlock(
-            excludedNodes);
-        if (lb.getLocations().length < schema.getNumDataUnits()) {
-          throw new IOException(
-              "Failed to get datablocks number of nodes from namenode: blockGroupSize= "
-                  + (schema.getNumDataUnits() + schema.getNumParityUnits())
-                  + ", blocks.length= " + lb.getLocations().length);
-        }
-        final LocatedBlock[] blocks =
-            StripedBlockUtil.parseStripedBlockGroup((LocatedStripedBlock) lb,
-                cellSize, schema.getNumDataUnits(), schema.getNumParityUnits());
-
-        for (int i = 0; i < blocks.length; i++) {
-          StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
-          if (si.isFailed()) {
-            continue; // skipping failed data streamer
-          }
-          if (blocks[i] == null) {
-            // Set exception and close streamer as there is no block locations
-            // found for the parity block.
-            LOG.warn("Failed to get block location for parity block, index="
-                + i);
-            si.getLastException().set(
-                new IOException("Failed to get following block, i=" + i));
-            si.setFailed(true);
-            si.endBlock();
-            si.close(true);
-          } else {
-            queue.offer(i, blocks[i]);
-          }
-        }
-      }
-    }.poll(index);
+  protected LocatedBlock nextBlockOutputStream() throws IOException {
+    boolean success;
+    LocatedBlock lb = getFollowingBlock();
+    block = lb.getBlock();
+    block.setNumBytes(0);
+    bytesSent = 0;
+    accessToken = lb.getBlockToken();
+
+    DatanodeInfo[] nodes = lb.getLocations();
+    StorageType[] storageTypes = lb.getStorageTypes();
+
+    // Connect to the DataNode. If fail the internal error state will be set.
+    success = createBlockOutputStream(nodes, storageTypes, 0L, false);
+
+    if (!success) {
+      block = null;
+      final DatanodeInfo badNode = nodes[getErrorState().getBadNodeIndex()];
+      LOG.info("Excluding datanode " + badNode);
+      excludedNodes.put(badNode, badNode);
+      throw new IOException("Unable to create new block.");
+    }
+    return lb;
   }
 
   @VisibleForTesting
@@ -221,119 +117,71 @@ public class StripedDataStreamer extends DataStreamer {
   }
 
   @Override
-  LocatedBlock updateBlockForPipeline() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("updateBlockForPipeline(), " + this);
-    }
-    return new ConcurrentPoll<LocatedBlock>(coordinator.getNewBlocks()) {
-      @Override
-      void populate() throws IOException {
-        final ExtendedBlock bg = coordinator.getBlockGroup();
-        final LocatedBlock updated = callUpdateBlockForPipeline(bg);
-        final long newGS = updated.getBlock().getGenerationStamp();
-        final LocatedBlock[] updatedBlks = StripedBlockUtil
-            .parseStripedBlockGroup((LocatedStripedBlock) updated, cellSize,
-                schema.getNumDataUnits(), schema.getNumParityUnits());
-        for (int i = 0; i < schema.getNumDataUnits()
-            + schema.getNumParityUnits(); i++) {
-          StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
-          if (si.isFailed()) {
-            continue; // skipping failed data streamer
-          }
-          final ExtendedBlock bi = si.getBlock();
-          if (bi != null) {
-            final LocatedBlock lb = new LocatedBlock(newBlock(bi, newGS),
-                null, null, null, -1, updated.isCorrupt(), null);
-            lb.setBlockToken(updatedBlks[i].getBlockToken());
-            queue.offer(i, lb);
-          } else {
-            final MultipleBlockingQueue<LocatedBlock> followingBlocks
-                = coordinator.getFollowingBlocks();
-            synchronized(followingBlocks) {
-              final LocatedBlock lb = followingBlocks.peek(i);
-              if (lb != null) {
-                lb.getBlock().setGenerationStamp(newGS);
-                si.getErrorState().reset();
-                continue;
-              }
-            }
-
-            //streamer i just have polled the block, sleep and retry.
-            sleep(100, "updateBlockForPipeline, " + this);
-            i--;
-          }
-        }
+  protected void setupPipelineInternal(DatanodeInfo[] nodes,
+      StorageType[] nodeStorageTypes) throws IOException {
+    boolean success = false;
+    while (!success && !streamerClosed() && dfsClient.clientRunning) {
+      if (!handleRestartingDatanode()) {
+        return;
+      }
+      if (!handleBadDatanode()) {
+        // for striped streamer if it is datanode error then close the stream
+        // and return. no need to replace datanode
+        return;
       }
-    }.poll(index);
-  }
-
-  @Override
-  ExtendedBlock updatePipeline(final long newGS) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("updatePipeline(newGS=" + newGS + "), " + this);
-    }
-    return new ConcurrentPoll<ExtendedBlock>(coordinator.getUpdateBlocks()) {
-      @Override
-      void populate() throws IOException {
-        final MultipleBlockingQueue<LocatedBlock> followingBlocks
-            = coordinator.getFollowingBlocks();
-        final ExtendedBlock bg = coordinator.getBlockGroup();
-        final ExtendedBlock newBG = newBlock(bg, newGS);
 
-        final int n = schema.getNumDataUnits() + schema.getNumParityUnits();
-        final DatanodeInfo[] newNodes = new DatanodeInfo[n];
-        final String[] newStorageIDs = new String[n];
-        for (int i = 0; i < n; i++) {
-          final StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
-          DatanodeInfo[] nodes = si.getNodes();
-          String[] storageIDs = si.getStorageIDs();
-          if (nodes == null || storageIDs == null) {
-            synchronized(followingBlocks) {
-              final LocatedBlock lb = followingBlocks.peek(i);
-              if (lb != null) {
-                nodes = lb.getLocations();
-                storageIDs = lb.getStorageIDs();
-              }
-            }
-          }
-          if (nodes != null && storageIDs != null) {
-            newNodes[i] = nodes[0];
-            newStorageIDs[i] = storageIDs[0];
-          } else {
-            //streamer i just have polled the block, sleep and retry.
-            sleep(100, "updatePipeline, " + this);
-            i--;
-          }
+      // get a new generation stamp and an access token
+      final LocatedBlock lb = coordinator.getNewBlocks().take(index);
+      long newGS = lb.getBlock().getGenerationStamp();
+      setAccessToken(lb.getBlockToken());
+
+      // set up the pipeline again with the remaining nodes. when a striped
+      // data streamer comes here, it must be in external error state.
+      assert getErrorState().hasExternalError();
+      success = createBlockOutputStream(nodes, nodeStorageTypes, newGS, true);
+
+      failPacket4Testing();
+      getErrorState().checkRestartingNodeDeadline(nodes);
+
+      // notify coordinator the result of createBlockOutputStream
+      synchronized (coordinator) {
+        if (!streamerClosed()) {
+          coordinator.updateStreamer(this, success);
+          coordinator.notify();
+        } else {
+          success = false;
         }
-        final ExtendedBlock updated = callUpdatePipeline(bg, newBG, newNodes,
-            newStorageIDs);
-
-        for (int i = 0; i < n; i++) {
-          final StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
-          final ExtendedBlock bi = si.getBlock();
-          if (bi != null) {
-            queue.offer(i, newBlock(bi, updated.getGenerationStamp()));
-          } else if (!si.isFailed()) {
-            synchronized(followingBlocks) {
-              final LocatedBlock lb = followingBlocks.peek(i);
-              if (lb != null) {
-                lb.getBlock().setGenerationStamp(newGS);
-                si.getErrorState().reset();
-                continue;
-              }
-            }
+      }
 
-            //streamer i just have polled the block, sleep and retry.
-            sleep(100, "updatePipeline, " + this);
-            i--;
-          }
+      if (success) {
+        // wait for results of other streamers
+        success = coordinator.takeStreamerUpdateResult(index);
+        if (success) {
+          // if all succeeded, update its block using the new GS
+          block = newBlock(block, newGS);
+        } else {
+          // otherwise close the block stream and restart the recovery process
+          closeStream();
         }
+      } else {
+        // if fail, close the stream. The internal error state and last
+        // exception have already been set in createBlockOutputStream
+        // TODO: wait for restarting DataNodes during RollingUpgrade
+        closeStream();
+        setStreamerAsClosed();
       }
-    }.poll(index);
+    } // while
+  }
+
+  void setExternalError() {
+    getErrorState().setExternalError();
+    synchronized (dataQueue) {
+      dataQueue.notifyAll();
+    }
   }
 
   @Override
   public String toString() {
-    return "#" + index + ": " + (failed? "failed, ": "") + super.toString();
+    return "#" + index + ": " + (!isHealthy() ? "failed, ": "") + super.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
index 0e92779..1d4cff3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
@@ -68,16 +68,28 @@ public class BlockUnderConstructionFeature {
   /** Set expected locations */
   public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets,
       boolean isStriped) {
-    int numLocations = targets == null ? 0 : targets.length;
+    if (targets == null) {
+      return;
+    }
+    int numLocations = 0;
+    for (DatanodeStorageInfo target : targets) {
+      if (target != null) {
+        numLocations++;
+      }
+    }
+
     this.replicas = new ReplicaUnderConstruction[numLocations];
-    for(int i = 0; i < numLocations; i++) {
-      // when creating a new striped block we simply sequentially assign block
-      // index to each storage
-      Block replicaBlock = isStriped ?
-          new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
-          block;
-      replicas[i] = new ReplicaUnderConstruction(replicaBlock, targets[i],
-          ReplicaState.RBW);
+    int offset = 0;
+    for(int i = 0; i < targets.length; i++) {
+      if (targets[i] != null) {
+        // when creating a new striped block we simply sequentially assign block
+        // index to each storage
+        Block replicaBlock = isStriped ?
+            new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
+            block;
+        replicas[offset++] = new ReplicaUnderConstruction(replicaBlock,
+            targets[i], ReplicaState.RBW);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
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 b5b3b97..61c6386 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
@@ -513,6 +513,10 @@ public class DatanodeManager {
     }
     final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[datanodeID.length];
     for(int i = 0; i < datanodeID.length; i++) {
+      if (datanodeID[i].equals(DatanodeID.EMPTY_DATANODE_ID)) {
+        storages[i] = null;
+        continue;
+      }
       final DatanodeDescriptor dd = getDatanode(datanodeID[i]);
       storages[i] = dd.getStorageInfo(storageIDs[i]);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
index 5af3585..d49d39b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@ -925,22 +925,21 @@ public class StripedBlockUtil {
 
   /**
    * Check if the information such as IDs and generation stamps in block-i
-   * match block-j, where block-i and block-j are in the same group.
+   * match the block group.
    */
-  public static void checkBlocks(int j, ExtendedBlock blockj,
+  public static void checkBlocks(ExtendedBlock blockGroup,
       int i, ExtendedBlock blocki) throws IOException {
-
-    if (!blocki.getBlockPoolId().equals(blockj.getBlockPoolId())) {
-      throw new IOException("Block pool IDs mismatched: block" + j + "="
-          + blockj + ", block" + i + "=" + blocki);
+    if (!blocki.getBlockPoolId().equals(blockGroup.getBlockPoolId())) {
+      throw new IOException("Block pool IDs mismatched: block" + i + "="
+          + blocki + ", expected block group=" + blockGroup);
     }
-    if (blocki.getBlockId() - i != blockj.getBlockId() - j) {
-      throw new IOException("Block IDs mismatched: block" + j + "="
-          + blockj + ", block" + i + "=" + blocki);
+    if (blocki.getBlockId() - i != blockGroup.getBlockId()) {
+      throw new IOException("Block IDs mismatched: block" + i + "="
+          + blocki + ", expected block group=" + blockGroup);
     }
-    if (blocki.getGenerationStamp() != blockj.getGenerationStamp()) {
-      throw new IOException("Generation stamps mismatched: block" + j + "="
-          + blockj + ", block" + i + "=" + blocki);
+    if (blocki.getGenerationStamp() != blockGroup.getGenerationStamp()) {
+      throw new IOException("Generation stamps mismatched: block" + i + "="
+          + blocki + ", expected block group=" + blockGroup);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6419900a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 274d319..e621f26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -1988,35 +1988,14 @@ public class DFSTestUtil {
    */
   public static ExtendedBlock flushInternal(DFSStripedOutputStream out)
       throws IOException {
-    out.flushInternal();
+    out.flushAllInternals();
     return out.getBlock();
   }
 
-  /**
-   * Verify that blocks in striped block group are on different nodes, and every
-   * internal blocks exists.
-   */
-  public static void verifyLocatedStripedBlocks(LocatedBlocks lbs,
-       int groupSize) {
-    for (LocatedBlock lb : lbs.getLocatedBlocks()) {
-      assert lb instanceof LocatedStripedBlock;
-      HashSet<DatanodeInfo> locs = new HashSet<>();
-      for (DatanodeInfo datanodeInfo : lb.getLocations()) {
-        locs.add(datanodeInfo);
-      }
-      assertEquals(groupSize, lb.getLocations().length);
-      assertEquals(groupSize, locs.size());
-
-      // verify that every internal blocks exists
-      int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices();
-      assertEquals(groupSize, blockIndices.length);
-      HashSet<Integer> found = new HashSet<>();
-      for (int index : blockIndices) {
-        assert index >=0;
-        found.add(index);
-      }
-      assertEquals(groupSize, found.size());
-    }
+  public static ExtendedBlock flushBuffer(DFSStripedOutputStream out)
+      throws IOException {
+    out.flush();
+    return out.getBlock();
   }
 
   public static void waitForMetric(final JMXGet jmx, final String metricName, final int expectedValue)


[02/50] [abbrv] hadoop git commit: Merge commit '456e901a4c5c639267ee87b8e5f1319f256d20c2' (HDFS-6407. Add sorting and pagination in the datanode tab of the NN Web UI. Contributed by Haohui Mai.) into HDFS-7285-merge

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 2a9ab39,a5053bc..bae033b
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@@ -32,31 -29,23 +32,33 @@@ import org.apache.hadoop.security.Acces
  @InterfaceAudience.Private
  public interface Namesystem extends RwLock, SafeMode {
    /** Is this name system running? */
-   public boolean isRunning();
+   boolean isRunning();
  
    /** Check if the user has superuser privilege. */
-   public void checkSuperuserPrivilege() throws AccessControlException;
+   void checkSuperuserPrivilege() throws AccessControlException;
  
    /** @return the block pool ID */
-   public String getBlockPoolId();
+   String getBlockPoolId();
  
-   public boolean isInStandbyState();
+   boolean isInStandbyState();
  
-   public boolean isGenStampInFuture(Block block);
+   boolean isGenStampInFuture(Block block);
  
-   public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
+   void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
  
-   public void checkOperation(OperationCategory read) throws StandbyException;
+   void checkOperation(OperationCategory read) throws StandbyException;
  
-   public boolean isInSnapshot(BlockCollection bc);
 -  boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC);
++  boolean isInSnapshot(BlockCollection bc);
 +
 +  /**
 +   * Gets the ECZone for path
 +   * @param src
 +   *          - path
 +   * @return {@link ErasureCodingZone}
 +   * @throws IOException
 +   */
-   public ErasureCodingZone getErasureCodingZoneForPath(String src)
++  ErasureCodingZone getErasureCodingZoneForPath(String src)
 +      throws IOException;
+ 
+   CacheManager getCacheManager();
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index 0bcb860,64ad1f6..0162f85
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@@ -240,16 -240,13 +241,15 @@@ public class FSImageFormatPBSnapshot 
          FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
              pbf.getFileSize());
          List<BlockProto> bpl = pbf.getBlocksList();
 +        // in file diff there can only be contiguous blocks
-         BlockInfoContiguous[] blocks = new BlockInfoContiguous[bpl.size()];
+         BlockInfo[] blocks = new BlockInfo[bpl.size()];
          for(int j = 0, e = bpl.size(); j < e; ++j) {
            Block blk = PBHelper.convert(bpl.get(j));
-           BlockInfoContiguous storedBlock =
-               (BlockInfoContiguous) fsn.getBlockManager().getStoredBlock(blk);
+           BlockInfo storedBlock =  fsn.getBlockManager().getStoredBlock(blk);
            if(storedBlock == null) {
 -            storedBlock = fsn.getBlockManager().addBlockCollection(
 -                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
 +            storedBlock = (BlockInfoContiguous) fsn.getBlockManager()
 +                .addBlockCollectionWithCheck(new BlockInfoContiguous(blk,
 +                    copy.getFileReplication()), file);
            }
            blocks[j] = storedBlock;
          }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index f44ea76,6b8388e..442e127
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@@ -22,8 -22,7 +22,8 @@@ import java.util.List
  
  import org.apache.hadoop.hdfs.protocol.Block;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
  import org.apache.hadoop.hdfs.server.namenode.INode;
  import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
index bb59a17,9f8aef5..e944b81
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
@@@ -103,7 -109,100 +109,101 @@@ public class TestLayoutVersion 
      assertEquals(LAST_COMMON_FEATURE.getInfo().getLayoutVersion() - 1,
          first.getInfo().getLayoutVersion());
    }
-   
+ 
+   /**
+    * Tests expected values for minimum compatible layout version in NameNode
+    * features.  TRUNCATE, APPEND_NEW_BLOCK and QUOTA_BY_STORAGE_TYPE are all
+    * features that launched in the same release.  TRUNCATE was added first, so
+    * we expect all 3 features to have a minimum compatible layout version equal
+    * to TRUNCATE's layout version.  All features older than that existed prior
+    * to the concept of a minimum compatible layout version, so for each one, the
+    * minimum compatible layout version must be equal to itself.
+    */
+   @Test
+   public void testNameNodeFeatureMinimumCompatibleLayoutVersions() {
+     int baseLV = NameNodeLayoutVersion.Feature.TRUNCATE.getInfo()
+         .getLayoutVersion();
+     EnumSet<NameNodeLayoutVersion.Feature> compatibleFeatures = EnumSet.of(
+         NameNodeLayoutVersion.Feature.TRUNCATE,
+         NameNodeLayoutVersion.Feature.APPEND_NEW_BLOCK,
 -        NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE);
++        NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE,
++        NameNodeLayoutVersion.Feature.ERASURE_CODING);
+     for (LayoutFeature f : compatibleFeatures) {
+       assertEquals(String.format("Expected minimum compatible layout version " +
+           "%d for feature %s.", baseLV, f), baseLV,
+           f.getInfo().getMinimumCompatibleLayoutVersion());
+     }
+     List<LayoutFeature> features = new ArrayList<>();
+     features.addAll(EnumSet.allOf(LayoutVersion.Feature.class));
+     features.addAll(EnumSet.allOf(NameNodeLayoutVersion.Feature.class));
+     for (LayoutFeature f : features) {
+       if (!compatibleFeatures.contains(f)) {
+         assertEquals(String.format("Expected feature %s to have minimum " +
+             "compatible layout version set to itself.", f),
+             f.getInfo().getLayoutVersion(),
+             f.getInfo().getMinimumCompatibleLayoutVersion());
+       }
+     }
+   }
+ 
+   /**
+    * Tests that NameNode features are listed in order of minimum compatible
+    * layout version.  It would be inconsistent to have features listed out of
+    * order with respect to minimum compatible layout version, because it would
+    * imply going back in time to change compatibility logic in a software release
+    * that had already shipped.
+    */
+   @Test
+   public void testNameNodeFeatureMinimumCompatibleLayoutVersionAscending() {
+     LayoutFeature prevF = null;
+     for (LayoutFeature f : EnumSet.allOf(NameNodeLayoutVersion.Feature.class)) {
+       if (prevF != null) {
+         assertTrue(String.format("Features %s and %s not listed in order of " +
+             "minimum compatible layout version.", prevF, f),
+             f.getInfo().getMinimumCompatibleLayoutVersion() <=
+             prevF.getInfo().getMinimumCompatibleLayoutVersion());
+       } else {
+         prevF = f;
+       }
+     }
+   }
+ 
+   /**
+    * Tests that attempting to add a new NameNode feature out of order with
+    * respect to minimum compatible layout version will fail fast.
+    */
+   @Test(expected=AssertionError.class)
+   public void testNameNodeFeatureMinimumCompatibleLayoutVersionOutOfOrder() {
+     FeatureInfo ancestorF = LayoutVersion.Feature.RESERVED_REL2_4_0.getInfo();
+     LayoutFeature f = mock(LayoutFeature.class);
+     when(f.getInfo()).thenReturn(new FeatureInfo(
+         ancestorF.getLayoutVersion() - 1, ancestorF.getLayoutVersion(),
+         ancestorF.getMinimumCompatibleLayoutVersion() + 1, "Invalid feature.",
+         false));
+     Map<Integer, SortedSet<LayoutFeature>> features = new HashMap<>();
+     LayoutVersion.updateMap(features, LayoutVersion.Feature.values());
+     LayoutVersion.updateMap(features, new LayoutFeature[] { f });
+   }
+ 
+   /**
+    * Asserts the current minimum compatible layout version of the software, if a
+    * release were created from the codebase right now.  This test is meant to
+    * make developers stop and reconsider if they introduce a change that requires
+    * a new minimum compatible layout version.  This would make downgrade
+    * impossible.
+    */
+   @Test
+   public void testCurrentMinimumCompatibleLayoutVersion() {
+     int expectedMinCompatLV = NameNodeLayoutVersion.Feature.TRUNCATE.getInfo()
+         .getLayoutVersion();
+     int actualMinCompatLV = LayoutVersion.getMinimumCompatibleLayoutVersion(
+         NameNodeLayoutVersion.Feature.values());
+     assertEquals("The minimum compatible layout version has changed.  " +
+         "Downgrade to prior versions is no longer possible.  Please either " +
+         "restore compatibility, or if the incompatibility is intentional, " +
+         "then update this assertion.", expectedMinCompatLV, actualMinCompatLV);
+   }
+ 
    /**
     * Given feature {@code f}, ensures the layout version of that feature
     * supports all the features supported by it's ancestor.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 99b460a,f8f4329..ad8f204
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@@ -59,8 -66,13 +66,14 @@@ import org.apache.hadoop.hdfs.Distribut
  import org.apache.hadoop.hdfs.HdfsConfiguration;
  import org.apache.hadoop.hdfs.MiniDFSCluster;
  import org.apache.hadoop.hdfs.NameNodeProxies;
- import org.apache.hadoop.hdfs.protocol.*;
+ import org.apache.hadoop.hdfs.protocol.Block;
+ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
++import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
  import org.apache.hadoop.hdfs.server.balancer.Balancer.Cli;
  import org.apache.hadoop.hdfs.server.balancer.Balancer.Parameters;
  import org.apache.hadoop.hdfs.server.balancer.Balancer.Result;
@@@ -130,26 -142,11 +144,28 @@@ public class TestBalancer 
      conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
      conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500);
      conf.setInt(DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC, 1);
-     conf.setInt(DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES, DEFAULT_RAM_DISK_BLOCK_SIZE);
      LazyPersistTestCase.initCacheManipulator();
+ 
+     conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
    }
  
 +  int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
 +  int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
 +  int groupSize = dataBlocks + parityBlocks;
 +  private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +  private final static int stripesPerBlock = 4;
 +  static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock;
 +
 +  static void initConfWithStripe(Configuration conf) {
 +    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE);
 +    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
 +    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
 +    SimulatedFSDataset.setFactory(conf);
 +    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
 +    conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
++    conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
 +  }
 +
    /* create a file with a length of <code>fileLen</code> */
    static void createFile(MiniDFSCluster cluster, Path filePath, long fileLen,
        short replicationFactor, int nnIndex)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 3e28236,6553185..dcb852b
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@@ -1255,13 -1246,13 +1255,13 @@@ public class TestReplicationPolicy 
      when(dn.isDecommissioned()).thenReturn(true);
      when(storage.getState()).thenReturn(DatanodeStorage.State.NORMAL);
      when(storage.getDatanodeDescriptor()).thenReturn(dn);
-     when(storage.removeBlock(any(BlockInfoContiguous.class))).thenReturn(true);
-     when(storage.addBlock(any(BlockInfoContiguous.class))).thenReturn
+     when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true);
+     when(storage.addBlock(any(BlockInfo.class))).thenReturn
          (DatanodeStorageInfo.AddBlockResult.ADDED);
 -    ucBlock.addStorage(storage);
 +    ucBlock.addStorage(storage, ucBlock);
  
 -    when(mbc.setLastBlock((BlockInfo) any(), (DatanodeStorageInfo[]) any()))
 -    .thenReturn(ucBlock);
 +    BlockInfo lastBlk = mbc.getLastBlock();
 +    when(mbc.getLastBlock()).thenReturn(lastBlk, ucBlock);
  
      bm.convertLastBlockToUnderConstruction(mbc, 0L);
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index 261d397,d3d814c..2598fa8
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@@ -35,12 -34,7 +35,11 @@@ import org.apache.hadoop.hdfs.DFSUtil
  import org.apache.hadoop.hdfs.HdfsConfiguration;
  import org.apache.hadoop.hdfs.MiniDFSCluster;
  import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 +import org.apache.hadoop.hdfs.NameNodeProxies;
 +import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 +import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 +import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
- import org.apache.hadoop.hdfs.server.balancer.Dispatcher;
  import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DBlock;
  import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
  import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
@@@ -368,120 -409,4 +414,119 @@@ public class TestMover 
        cluster.shutdown();
      }
    }
 +
 +  int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
 +  int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
 +  private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +  private final static int stripesPerBlock = 4;
 +  static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock;
 +
 +  static void initConfWithStripe(Configuration conf) {
 +    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE);
 +    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
 +    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
 +    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
-     Dispatcher.setBlockMoveWaitTime(3000L);
 +  }
 +
 +  @Test(timeout = 300000)
 +  public void testMoverWithStripedFile() throws Exception {
 +    final Configuration conf = new HdfsConfiguration();
 +    initConfWithStripe(conf);
 +
 +    // start 10 datanodes
 +    int numOfDatanodes =10;
 +    int storagesPerDatanode=2;
 +    long capacity = 10 * DEFAULT_STRIPE_BLOCK_SIZE;
 +    long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
 +    for (int i = 0; i < numOfDatanodes; i++) {
 +      for(int j=0;j<storagesPerDatanode;j++){
 +        capacities[i][j]=capacity;
 +      }
 +    }
 +    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
 +        .numDataNodes(numOfDatanodes)
 +        .storagesPerDatanode(storagesPerDatanode)
 +        .storageTypes(new StorageType[][]{
 +            {StorageType.DISK, StorageType.DISK},
 +            {StorageType.DISK, StorageType.DISK},
 +            {StorageType.DISK, StorageType.DISK},
 +            {StorageType.DISK, StorageType.DISK},
 +            {StorageType.DISK, StorageType.DISK},
 +            {StorageType.DISK, StorageType.ARCHIVE},
 +            {StorageType.DISK, StorageType.ARCHIVE},
 +            {StorageType.DISK, StorageType.ARCHIVE},
 +            {StorageType.DISK, StorageType.ARCHIVE},
 +            {StorageType.DISK, StorageType.ARCHIVE}})
 +        .storageCapacities(capacities)
 +        .build();
 +
 +    try {
 +      cluster.waitActive();
 +
 +      // set "/bar" directory with HOT storage policy.
 +      ClientProtocol client = NameNodeProxies.createProxy(conf,
 +          cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy();
 +      String barDir = "/bar";
 +      client.mkdirs(barDir, new FsPermission((short) 777), true);
 +      client.setStoragePolicy(barDir,
-           HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
++          HdfsConstants.HOT_STORAGE_POLICY_NAME);
 +      // set "/bar" directory with EC zone.
 +      client.createErasureCodingZone(barDir, null);
 +
 +      // write file to barDir
 +      final String fooFile = "/bar/foo";
 +      long fileLen = 20 * DEFAULT_STRIPE_BLOCK_SIZE ;
 +      DFSTestUtil.createFile(cluster.getFileSystem(), new Path(fooFile),
 +          fileLen,(short) 3, 0);
 +
 +      // verify storage types and locations
 +      LocatedBlocks locatedBlocks =
 +          client.getBlockLocations(fooFile, 0, fileLen);
 +      for(LocatedBlock lb : locatedBlocks.getLocatedBlocks()){
 +        for( StorageType type : lb.getStorageTypes()){
 +          Assert.assertEquals(StorageType.DISK, type);
 +        }
 +      }
 +      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
 +          dataBlocks + parityBlocks);
 +
 +      // start 5 more datanodes
 +      numOfDatanodes +=5;
 +      capacities = new long[5][storagesPerDatanode];
 +      for (int i = 0; i < 5; i++) {
 +        for(int j=0;j<storagesPerDatanode;j++){
 +          capacities[i][j]=capacity;
 +        }
 +      }
 +      cluster.startDataNodes(conf, 5,
 +          new StorageType[][]{
 +              {StorageType.ARCHIVE, StorageType.ARCHIVE},
 +              {StorageType.ARCHIVE, StorageType.ARCHIVE},
 +              {StorageType.ARCHIVE, StorageType.ARCHIVE},
 +              {StorageType.ARCHIVE, StorageType.ARCHIVE},
 +              {StorageType.ARCHIVE, StorageType.ARCHIVE}},
 +          true, null, null, null,capacities, null, false, false, false, null);
 +      cluster.triggerHeartbeats();
 +
 +      // move file to ARCHIVE
 +      client.setStoragePolicy(barDir, "COLD");
 +      // run Mover
 +      int rc = ToolRunner.run(conf, new Mover.Cli(),
 +          new String[] { "-p", barDir });
 +      Assert.assertEquals("Movement to ARCHIVE should be successfull", 0, rc);
 +
 +      // verify storage types and locations
 +      locatedBlocks = client.getBlockLocations(fooFile, 0, fileLen);
 +      for(LocatedBlock lb : locatedBlocks.getLocatedBlocks()){
 +        for( StorageType type : lb.getStorageTypes()){
 +          Assert.assertEquals(StorageType.ARCHIVE, type);
 +        }
 +      }
 +      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
 +          dataBlocks + parityBlocks);
 +
 +    }finally{
 +      cluster.shutdown();
 +    }
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index 4ae10db,0000000..fa92c64
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@@ -1,433 -1,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.hdfs.server.namenode;
 +
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hdfs.DFSStripedOutputStream;
 +import org.apache.hadoop.hdfs.DFSTestUtil;
 +import org.apache.hadoop.hdfs.DistributedFileSystem;
 +import org.apache.hadoop.hdfs.HdfsConfiguration;
 +import org.apache.hadoop.hdfs.MiniDFSCluster;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 +import org.apache.hadoop.hdfs.protocol.DatanodeID;
 +import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 +import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 +import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 +import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
++import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 +import org.apache.hadoop.hdfs.server.datanode.DataNode;
 +import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
 +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 +import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 +import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 +import org.apache.hadoop.io.IOUtils;
 +import org.junit.After;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Test;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 +import static org.junit.Assert.assertEquals;
 +
 +public class TestAddStripedBlocks {
 +  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
 +      HdfsConstants.NUM_PARITY_BLOCKS;
 +
 +  private MiniDFSCluster cluster;
 +  private DistributedFileSystem dfs;
 +
 +  @Before
 +  public void setup() throws IOException {
 +    cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
 +        .numDataNodes(GROUP_SIZE).build();
 +    cluster.waitActive();
 +    dfs = cluster.getFileSystem();
 +    dfs.getClient().createErasureCodingZone("/", null);
 +  }
 +
 +  @After
 +  public void tearDown() {
 +    if (cluster != null) {
 +      cluster.shutdown();
 +    }
 +  }
 +
 +  /**
 +   * Make sure the IDs of striped blocks do not conflict
 +   */
 +  @Test
 +  public void testAllocateBlockId() throws Exception {
 +    Path testPath = new Path("/testfile");
 +    // create a file while allocates a new block
 +    DFSTestUtil.writeFile(dfs, testPath, "hello, world!");
 +    LocatedBlocks lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0);
 +    final long firstId = lb.get(0).getBlock().getBlockId();
 +    // delete the file
 +    dfs.delete(testPath, true);
 +
 +    // allocate a new block, and make sure the new block's id does not conflict
 +    // with the previous one
 +    DFSTestUtil.writeFile(dfs, testPath, "hello again");
 +    lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0);
 +    final long secondId = lb.get(0).getBlock().getBlockId();
 +    Assert.assertEquals(firstId + HdfsServerConstants.MAX_BLOCKS_IN_GROUP, secondId);
 +  }
 +
 +  private static void writeAndFlushStripedOutputStream(
 +      DFSStripedOutputStream out, int chunkSize) throws IOException {
 +    // FSOutputSummer.BUFFER_NUM_CHUNKS == 9
 +    byte[] toWrite = new byte[chunkSize * 9 + 1];
 +    out.write(toWrite);
 +    DFSTestUtil.flushInternal(out);
 +  }
 +
 +  @Test (timeout=60000)
 +  public void testAddStripedBlock() throws Exception {
 +    final Path file = new Path("/file1");
 +    // create an empty file
 +    FSDataOutputStream out = null;
 +    try {
 +      out = dfs.create(file, (short) 1);
 +      writeAndFlushStripedOutputStream(
 +          (DFSStripedOutputStream) out.getWrappedStream(),
 +          DFS_BYTES_PER_CHECKSUM_DEFAULT);
 +
 +      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
 +      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
 +
 +      BlockInfo[] blocks = fileNode.getBlocks();
 +      assertEquals(1, blocks.length);
 +      Assert.assertTrue(blocks[0].isStriped());
 +
 +      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), true);
 +
 +      // restart NameNode to check editlog
 +      cluster.restartNameNode(true);
 +      fsdir = cluster.getNamesystem().getFSDirectory();
 +      fileNode = fsdir.getINode4Write(file.toString()).asFile();
 +      blocks = fileNode.getBlocks();
 +      assertEquals(1, blocks.length);
 +      Assert.assertTrue(blocks[0].isStriped());
 +      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
 +
 +      // save namespace, restart namenode, and check
 +      dfs = cluster.getFileSystem();
 +      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
 +      dfs.saveNamespace();
 +      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
 +      cluster.restartNameNode(true);
 +      fsdir = cluster.getNamesystem().getFSDirectory();
 +      fileNode = fsdir.getINode4Write(file.toString()).asFile();
 +      blocks = fileNode.getBlocks();
 +      assertEquals(1, blocks.length);
 +      Assert.assertTrue(blocks[0].isStriped());
 +      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
 +    } finally {
 +      IOUtils.cleanup(null, out);
 +    }
 +  }
 +
 +  private void checkStripedBlockUC(BlockInfoStriped block,
 +      boolean checkReplica) {
 +    assertEquals(0, block.numNodes());
 +    Assert.assertFalse(block.isComplete());
 +    Assert.assertEquals(HdfsConstants.NUM_DATA_BLOCKS, block.getDataBlockNum());
 +    Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS,
 +        block.getParityBlockNum());
 +    Assert.assertEquals(0,
 +        block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
 +
-     final BlockInfoUnderConstructionStriped blockUC =
-         (BlockInfoUnderConstructionStriped) block;
++    final BlockInfoStripedUnderConstruction blockUC =
++        (BlockInfoStripedUnderConstruction) block;
 +    Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
 +        blockUC.getBlockUCState());
 +    if (checkReplica) {
 +      Assert.assertEquals(GROUP_SIZE, blockUC.getNumExpectedLocations());
 +      DatanodeStorageInfo[] storages = blockUC.getExpectedStorageLocations();
 +      for (DataNode dn : cluster.getDataNodes()) {
 +        Assert.assertTrue(includeDataNode(dn.getDatanodeId(), storages));
 +      }
 +    }
 +  }
 +
 +  private boolean includeDataNode(DatanodeID dn, DatanodeStorageInfo[] storages) {
 +    for (DatanodeStorageInfo storage : storages) {
 +      if (storage.getDatanodeDescriptor().equals(dn)) {
 +        return true;
 +      }
 +    }
 +    return false;
 +  }
 +
 +  @Test
 +  public void testGetLocatedStripedBlocks() throws Exception {
 +    final Path file = new Path("/file1");
 +    // create an empty file
 +    FSDataOutputStream out = null;
 +    try {
 +      out = dfs.create(file, (short) 1);
 +      writeAndFlushStripedOutputStream(
 +          (DFSStripedOutputStream) out.getWrappedStream(),
 +          DFS_BYTES_PER_CHECKSUM_DEFAULT);
 +
 +      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
 +      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
-       BlockInfoUnderConstructionStriped lastBlk =
-           (BlockInfoUnderConstructionStriped) fileNode.getLastBlock();
++      BlockInfoStripedUnderConstruction lastBlk =
++          (BlockInfoStripedUnderConstruction) fileNode.getLastBlock();
 +      DatanodeInfo[] expectedDNs = DatanodeStorageInfo
 +          .toDatanodeInfos(lastBlk.getExpectedStorageLocations());
 +      int[] indices = lastBlk.getBlockIndices();
 +
 +      LocatedBlocks blks = dfs.getClient().getLocatedBlocks(file.toString(), 0L);
 +      Assert.assertEquals(1, blks.locatedBlockCount());
 +      LocatedBlock lblk = blks.get(0);
 +
 +      Assert.assertTrue(lblk instanceof LocatedStripedBlock);
 +      DatanodeInfo[] datanodes = lblk.getLocations();
 +      int[] blockIndices = ((LocatedStripedBlock) lblk).getBlockIndices();
 +      Assert.assertEquals(GROUP_SIZE, datanodes.length);
 +      Assert.assertEquals(GROUP_SIZE, blockIndices.length);
 +      Assert.assertArrayEquals(indices, blockIndices);
 +      Assert.assertArrayEquals(expectedDNs, datanodes);
 +    } finally {
 +      IOUtils.cleanup(null, out);
 +    }
 +  }
 +
 +  /**
-    * Test BlockInfoUnderConstructionStriped#addReplicaIfNotPresent in different
++   * Test BlockInfoStripedUnderConstruction#addReplicaIfNotPresent in different
 +   * scenarios.
 +   */
 +  @Test
 +  public void testAddUCReplica() throws Exception {
 +    final Path file = new Path("/file1");
 +    final List<String> storageIDs = new ArrayList<>();
 +    // create an empty file
 +    FSDataOutputStream out = null;
 +    try {
 +      out = dfs.create(file, (short) 1);
 +
 +      // 1. create the UC striped block
 +      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
 +      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
 +      cluster.getNamesystem().getAdditionalBlock(file.toString(),
 +          fileNode.getId(), dfs.getClient().getClientName(), null, null, null);
 +      BlockInfo lastBlock = fileNode.getLastBlock();
-       BlockInfoUnderConstructionStriped ucBlock =
-           (BlockInfoUnderConstructionStriped) lastBlock;
++      BlockInfoStripedUnderConstruction ucBlock =
++          (BlockInfoStripedUnderConstruction) lastBlock;
 +
 +      DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
 +      int[] indices = ucBlock.getBlockIndices();
 +      Assert.assertEquals(GROUP_SIZE, locs.length);
 +      Assert.assertEquals(GROUP_SIZE, indices.length);
 +
 +      // 2. mimic incremental block reports and make sure the uc-replica list in
 +      // the BlockInfoUCStriped is correct
 +      int i = 0;
 +      for (DataNode dn : cluster.getDataNodes()) {
 +        final Block block = new Block(lastBlock.getBlockId() + i++,
 +            0, lastBlock.getGenerationStamp());
 +        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
 +        storageIDs.add(storage.getStorageID());
 +        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
 +            .makeReportForReceivedBlock(block, BlockStatus.RECEIVING_BLOCK,
 +                storage);
 +        for (StorageReceivedDeletedBlocks report : reports) {
 +          cluster.getNamesystem().processIncrementalBlockReport(
 +              dn.getDatanodeId(), report);
 +        }
 +      }
 +
 +      // make sure lastBlock is correct and the storages have been updated
 +      locs = ucBlock.getExpectedStorageLocations();
 +      indices = ucBlock.getBlockIndices();
 +      Assert.assertEquals(GROUP_SIZE, locs.length);
 +      Assert.assertEquals(GROUP_SIZE, indices.length);
 +      for (DatanodeStorageInfo newstorage : locs) {
 +        Assert.assertTrue(storageIDs.contains(newstorage.getStorageID()));
 +      }
 +    } finally {
 +      IOUtils.cleanup(null, out);
 +    }
 +
 +    // 3. restart the namenode. mimic the full block reports and check the
 +    // uc-replica list again
 +    cluster.restartNameNode(true);
 +    final String bpId = cluster.getNamesystem().getBlockPoolId();
 +    INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
 +        .getINode4Write(file.toString()).asFile();
 +    BlockInfo lastBlock = fileNode.getLastBlock();
 +    int i = GROUP_SIZE - 1;
 +    for (DataNode dn : cluster.getDataNodes()) {
 +      String storageID = storageIDs.get(i);
 +      final Block block = new Block(lastBlock.getBlockId() + i--,
 +          lastBlock.getGenerationStamp(), 0);
 +      DatanodeStorage storage = new DatanodeStorage(storageID);
 +      List<ReplicaBeingWritten> blocks = new ArrayList<>();
 +      ReplicaBeingWritten replica = new ReplicaBeingWritten(block, null, null,
 +          null);
 +      blocks.add(replica);
 +      BlockListAsLongs bll = BlockListAsLongs.encode(blocks);
 +      StorageBlockReport[] reports = {new StorageBlockReport(storage,
 +          bll)};
 +      cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId),
 +          bpId, reports, null);
 +    }
 +
-     BlockInfoUnderConstructionStriped ucBlock =
-         (BlockInfoUnderConstructionStriped) lastBlock;
++    BlockInfoStripedUnderConstruction ucBlock =
++        (BlockInfoStripedUnderConstruction) lastBlock;
 +    DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
 +    int[] indices = ucBlock.getBlockIndices();
 +    Assert.assertEquals(GROUP_SIZE, locs.length);
 +    Assert.assertEquals(GROUP_SIZE, indices.length);
 +    for (i = 0; i < GROUP_SIZE; i++) {
 +      Assert.assertEquals(storageIDs.get(i),
 +          locs[GROUP_SIZE - 1 - i].getStorageID());
 +      Assert.assertEquals(GROUP_SIZE - i - 1, indices[i]);
 +    }
 +  }
 +
 +  @Test
 +  public void testCheckStripedReplicaCorrupt() throws Exception {
 +    final int numBlocks = 4;
 +    final int numStripes = 4;
 +    final Path filePath = new Path("/corrupt");
 +    final FSNamesystem ns = cluster.getNameNode().getNamesystem();
 +    final BlockManager bm = ns.getBlockManager();
 +    DFSTestUtil.createStripedFile(cluster, filePath, null,
 +        numBlocks, numStripes, false);
 +
 +    INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()).
 +        asFile();
 +    Assert.assertTrue(fileNode.isStriped());
 +    BlockInfo stored = fileNode.getBlocks()[0];
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(0, ns.getCorruptReplicaBlocks());
 +
 +    // Now send a block report with correct size
 +    DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
 +    final Block reported = new Block(stored);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
 +    StorageReceivedDeletedBlocks[] reports = DFSTestUtil
 +        .makeReportForReceivedBlock(reported,
 +            ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(0).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(0, ns.getCorruptReplicaBlocks());
 +
 +    // Now send a block report with wrong size
 +    reported.setBlockId(stored.getBlockId() + 1);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE - 1);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +            ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(1).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +
 +    // Now send a parity block report with correct size
 +    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(2).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +
 +    // Now send a parity block report with wrong size
 +    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 1);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(3).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    // the total number of corrupted block info is still 1
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +    // 2 internal blocks corrupted
 +    Assert.assertEquals(2, bm.getCorruptReplicas(stored).size());
 +
 +    // Now change the size of stored block, and test verifying the last
 +    // block size
 +    stored.setNumBytes(stored.getNumBytes() + 10);
 +    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS + 2);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(4).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
 +
 +    // Now send a parity block report with correct size based on adjusted
 +    // size of stored block
 +    /** Now stored block has {@link numStripes} full stripes + a cell + 10 */
 +    stored.setNumBytes(stored.getNumBytes() + BLOCK_STRIPED_CELL_SIZE);
 +    reported.setBlockId(stored.getBlockId());
 +    reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(0).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
 +
 +    reported.setBlockId(stored.getBlockId() + 1);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 10);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(5).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
 +
 +    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
 +    reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(2).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index 92c329e,c5262d4..ec3d924
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@@ -126,4 -132,40 +132,41 @@@ public class TestDeadDatanode 
      assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
          .getAction());
    }
+ 
+   @Test
+   public void testDeadNodeAsBlockTarget() throws Exception {
+     Configuration conf = new HdfsConfiguration();
+     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500);
+     conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+     cluster.waitActive();
+ 
+     String poolId = cluster.getNamesystem().getBlockPoolId();
+     // wait for datanode to be marked live
+     DataNode dn = cluster.getDataNodes().get(0);
+     DatanodeRegistration reg = DataNodeTestUtils.getDNRegistrationForBP(cluster
+         .getDataNodes().get(0), poolId);
+     // Get the updated datanode descriptor
+     BlockManager bm = cluster.getNamesystem().getBlockManager();
+     DatanodeManager dm = bm.getDatanodeManager();
+     Node clientNode = dm.getDatanode(reg);
+ 
+     DFSTestUtil.waitForDatanodeState(cluster, reg.getDatanodeUuid(), true,
+         20000);
+ 
+     // Shutdown and wait for datanode to be marked dead
+     dn.shutdown();
+     DFSTestUtil.waitForDatanodeState(cluster, reg.getDatanodeUuid(), false,
+         20000);
+     // Get the updated datanode descriptor available in DNM
+     // choose the targets, but local node should not get selected as this is not
+     // part of the cluster anymore
+     DatanodeStorageInfo[] results = bm.chooseTarget4NewBlock("/hello", 3,
 -        clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7);
++        clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7,
++        false);
+     for (DatanodeStorageInfo datanodeStorageInfo : results) {
+       assertFalse("Dead node should not be choosen", datanodeStorageInfo
+           .getDatanodeDescriptor().equals(clientNode));
+     }
+   }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index d1c7600,df20fd6..d3689f3
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@@ -17,13 -17,11 +17,17 @@@
   */
  package org.apache.hadoop.hdfs.server.namenode;
  
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertFalse;
+ import static org.junit.Assert.assertTrue;
+ 
  import java.io.File;
 +import java.io.DataOutput;
 +import java.io.DataOutputStream;
 +import java.io.DataInput;
 +import java.io.DataInputStream;
 +import java.io.ByteArrayOutputStream;
 +import java.io.ByteArrayInputStream;
  import java.io.IOException;
  import java.util.EnumSet;
  
@@@ -51,19 -40,16 +55,16 @@@ import org.apache.hadoop.hdfs.MiniDFSCl
  import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
  import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
+ import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
  import org.apache.hadoop.hdfs.util.MD5FileUtils;
  import org.apache.hadoop.test.GenericTestUtils;
  import org.apache.hadoop.test.PathUtils;
  import org.junit.Test;
  
 +import static org.junit.Assert.assertArrayEquals;
- import static org.junit.Assert.assertEquals;
- import static org.junit.Assert.assertNotNull;
- import static org.junit.Assert.assertTrue;
- import static org.junit.Assert.assertFalse;
 +
  public class TestFSImage {
  
    private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ =
@@@ -137,123 -120,45 +138,162 @@@
      }
    }
  
 +  private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf,
 +                                               boolean isUC) throws IOException{
 +    // contruct a INode with StripedBlock for saving and loading
 +    fsn.createErasureCodingZone("/", null, false);
 +    long id = 123456789;
 +    byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
 +    PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
 +            "testuser_groups", new FsPermission((short)0x755));
 +    long mtime = 1426222916-3600;
 +    long atime = 1426222916;
 +    BlockInfoContiguous[] blks = new BlockInfoContiguous[0];
 +    short replication = 3;
 +    long preferredBlockSize = 128*1024*1024;
 +    INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
 +        blks, replication, preferredBlockSize, (byte) 0, true);
 +    ByteArrayOutputStream bs = new ByteArrayOutputStream();
 +
 +    //construct StripedBlocks for the INode
 +    BlockInfoStriped[] stripedBlks = new BlockInfoStriped[3];
 +    long stripedBlkId = 10000001;
 +    long timestamp = mtime+3600;
 +    for (int i = 0; i < stripedBlks.length; i++) {
 +      stripedBlks[i] = new BlockInfoStriped(
 +              new Block(stripedBlkId + i, preferredBlockSize, timestamp),
 +              testECPolicy);
 +      file.addBlock(stripedBlks[i]);
 +    }
 +
 +    final String client = "testClient";
 +    final String clientMachine = "testClientMachine";
 +    final String path = "testUnderConstructionPath";
 +
 +    //save the INode to byte array
 +    DataOutput out = new DataOutputStream(bs);
 +    if (isUC) {
 +      file.toUnderConstruction(client, clientMachine);
 +      FSImageSerialization.writeINodeUnderConstruction((DataOutputStream) out,
 +          file, path);
 +    } else {
 +      FSImageSerialization.writeINodeFile(file, out, false);
 +    }
 +    DataInput in = new DataInputStream(
 +            new ByteArrayInputStream(bs.toByteArray()));
 +
 +    // load the INode from the byte array
 +    INodeFile fileByLoaded;
 +    if (isUC) {
 +      fileByLoaded = FSImageSerialization.readINodeUnderConstruction(in,
 +              fsn, fsn.getFSImage().getLayoutVersion());
 +    } else {
 +      fileByLoaded = (INodeFile) new FSImageFormat.Loader(conf, fsn)
 +              .loadINodeWithLocalName(false, in, false);
 +    }
 +
 +    assertEquals(id, fileByLoaded.getId() );
 +    assertArrayEquals(isUC ? path.getBytes() : name,
 +        fileByLoaded.getLocalName().getBytes());
 +    assertEquals(permissionStatus.getUserName(),
 +        fileByLoaded.getPermissionStatus().getUserName());
 +    assertEquals(permissionStatus.getGroupName(),
 +        fileByLoaded.getPermissionStatus().getGroupName());
 +    assertEquals(permissionStatus.getPermission(),
 +        fileByLoaded.getPermissionStatus().getPermission());
 +    assertEquals(mtime, fileByLoaded.getModificationTime());
 +    assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime());
 +    // TODO for striped blocks, we currently save and load them as contiguous
 +    // blocks to/from legacy fsimage
 +    assertEquals(3, fileByLoaded.getBlocks().length);
 +    assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
 +
 +    if (isUC) {
 +      assertEquals(client,
 +          fileByLoaded.getFileUnderConstructionFeature().getClientName());
 +      assertEquals(clientMachine,
 +          fileByLoaded.getFileUnderConstructionFeature().getClientMachine());
 +    }
 +  }
 +
 +  /**
 +   * Test if a INodeFile with BlockInfoStriped can be saved by
 +   * FSImageSerialization and loaded by FSImageFormat#Loader.
 +   */
 +  @Test
 +  public void testSaveAndLoadStripedINodeFile() throws IOException{
 +    Configuration conf = new Configuration();
 +    MiniDFSCluster cluster = null;
 +    try {
 +      cluster = new MiniDFSCluster.Builder(conf).build();
 +      cluster.waitActive();
 +      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, false);
 +    } finally {
 +      if (cluster != null) {
 +        cluster.shutdown();
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Test if a INodeFileUnderConstruction with BlockInfoStriped can be
 +   * saved and loaded by FSImageSerialization
 +   */
 +  @Test
-   public void testSaveAndLoadStripedINodeFileUC() throws IOException{
++  public void testSaveAndLoadStripedINodeFileUC() throws IOException {
 +    // construct a INode with StripedBlock for saving and loading
 +    Configuration conf = new Configuration();
 +    MiniDFSCluster cluster = null;
 +    try {
 +      cluster = new MiniDFSCluster.Builder(conf).build();
 +      cluster.waitActive();
 +      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, true);
 +    } finally {
 +      if (cluster != null) {
 +        cluster.shutdown();
 +      }
 +    }
 +  }
 +
+    /**
+    * On checkpointing , stale fsimage checkpoint file should be deleted.
+    */
+   @Test
+   public void testRemovalStaleFsimageCkpt() throws IOException {
+     MiniDFSCluster cluster = null;
+     SecondaryNameNode secondary = null;
+     Configuration conf = new HdfsConfiguration();
+     try {
+       cluster = new MiniDFSCluster.Builder(conf).
+           numDataNodes(1).format(true).build();
+       conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+           "0.0.0.0:0");
+       secondary = new SecondaryNameNode(conf);
+       // Do checkpointing
+       secondary.doCheckpoint();
+       NNStorage storage = secondary.getFSImage().storage;
+       File currentDir = FSImageTestUtil.
+           getCurrentDirs(storage, NameNodeDirType.IMAGE).get(0);
+       // Create a stale fsimage.ckpt file
+       File staleCkptFile = new File(currentDir.getPath() +
+           "/fsimage.ckpt_0000000000000000002");
+       staleCkptFile.createNewFile();
+       assertTrue(staleCkptFile.exists());
+       // After checkpoint stale fsimage.ckpt file should be deleted
+       secondary.doCheckpoint();
+       assertFalse(staleCkptFile.exists());
+     } finally {
+       if (secondary != null) {
+         secondary.shutdown();
+         secondary = null;
+       }
+       if (cluster != null) {
+         cluster.shutdown();
+         cluster = null;
+       }
+     }
+   }
+ 
    /**
     * Ensure that the digest written by the saver equals to the digest of the
     * file.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 53a5d67,767f4de..f6aae22
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@@ -54,8 -54,7 +54,8 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
  import org.apache.hadoop.hdfs.protocol.LocatedBlock;
  import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
  import org.apache.hadoop.security.UserGroupInformation;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index e6f505e,8818f17..30f7f5c
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@@ -1202,16 -1198,15 +1202,16 @@@ public class TestFsck 
  
      HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication,
          blockSize, modTime, accessTime, perms, owner, group, symlink,
 -        path, fileId, numChildren, null, storagePolicy);
 -    Result res = new Result(conf);
 +        path, fileId, numChildren, null, storagePolicy, null);
 +    Result replRes = new ReplicationResult(conf);
 +    Result ecRes = new ErasureCodingResult(conf);
  
      try {
 -      fsck.check(pathString, file, res);
 +      fsck.check(pathString, file, replRes, ecRes);
      } catch (Exception e) {
-       fail("Unexpected exception "+ e.getMessage());
+       fail("Unexpected exception " + e.getMessage());
      }
 -    assertTrue(res.toString().contains("HEALTHY"));
 +    assertTrue(replRes.isHealthy());
    }
  
    /** Test fsck with symlinks in the filesystem */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
index b1bb191,0000000..de29997
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
@@@ -1,125 -1,0 +1,125 @@@
 +/**
 + * 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.hdfs.server.namenode;
 +
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.fs.StorageType;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.DFSTestUtil;
 +import org.apache.hadoop.hdfs.DistributedFileSystem;
 +import org.apache.hadoop.hdfs.MiniDFSCluster;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 +import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 +import org.apache.hadoop.io.IOUtils;
 +import org.junit.After;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Test;
 +
 +import java.io.IOException;
 +
 +/**
 + * Make sure we correctly update the quota usage with the striped blocks.
 + */
 +public class TestQuotaWithStripedBlocks {
 +  private static final int BLOCK_SIZE = 1024 * 1024;
 +  private static final long DISK_QUOTA = BLOCK_SIZE * 10;
 +  private static final ErasureCodingPolicy ecPolicy =
 +      ErasureCodingPolicyManager.getSystemDefaultPolicy();
 +  private static final int NUM_DATA_BLOCKS = ecPolicy.getNumDataUnits();
 +  private static final int NUM_PARITY_BLOCKS = ecPolicy.getNumParityUnits();
 +  private static final int GROUP_SIZE = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
 +  private static final Path ecDir = new Path("/ec");
 +
 +  private MiniDFSCluster cluster;
 +  private FSDirectory dir;
 +  private DistributedFileSystem dfs;
 +
 +  @Before
 +  public void setUp() throws IOException {
 +    final Configuration conf = new Configuration();
 +    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
 +    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE).build();
 +    cluster.waitActive();
 +
 +    dir = cluster.getNamesystem().getFSDirectory();
 +    dfs = cluster.getFileSystem();
 +
 +    dfs.mkdirs(ecDir);
 +    dfs.getClient().createErasureCodingZone(ecDir.toString(), ecPolicy);
 +    dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA);
 +    dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA);
-     dfs.setStoragePolicy(ecDir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
++    dfs.setStoragePolicy(ecDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
 +  }
 +
 +  @After
 +  public void tearDown() {
 +    if (cluster != null) {
 +      cluster.shutdown();
 +    }
 +  }
 +
 +  @Test
 +  public void testUpdatingQuotaCount() throws Exception {
 +    final Path file = new Path(ecDir, "file");
 +    FSDataOutputStream out = null;
 +
 +    try {
 +      out = dfs.create(file, (short) 1);
 +
 +      INodeFile fileNode = dir.getINode4Write(file.toString()).asFile();
 +      ExtendedBlock previous = null;
 +      // Create striped blocks which have a cell in each block.
 +      Block newBlock = DFSTestUtil.addStripedBlockToFile(cluster.getDataNodes(),
 +          dfs, cluster.getNamesystem(), file.toString(), fileNode,
 +          dfs.getClient().getClientName(), previous, 1);
 +      previous = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(),
 +          newBlock);
 +
 +      final INodeDirectory dirNode = dir.getINode4Write(ecDir.toString())
 +          .asDirectory();
 +      final long spaceUsed = dirNode.getDirectoryWithQuotaFeature()
 +          .getSpaceConsumed().getStorageSpace();
 +      final long diskUsed = dirNode.getDirectoryWithQuotaFeature()
 +          .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
 +      // When we add a new block we update the quota using the full block size.
 +      Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, spaceUsed);
 +      Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, diskUsed);
 +
 +      dfs.getClient().getNamenode().complete(file.toString(),
 +          dfs.getClient().getClientName(), previous, fileNode.getId());
 +
 +      final long actualSpaceUsed = dirNode.getDirectoryWithQuotaFeature()
 +          .getSpaceConsumed().getStorageSpace();
 +      final long actualDiskUsed = dirNode.getDirectoryWithQuotaFeature()
 +          .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
 +      // In this case the file's real size is cell size * block group size.
 +      Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
 +          actualSpaceUsed);
 +      Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
 +          actualDiskUsed);
 +    } finally {
 +      IOUtils.cleanup(null, out);
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
index 7684c1d,0000000..8b1a11f
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@@ -1,283 -1,0 +1,283 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hdfs.server.namenode;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.assertFalse;
 +
 +import java.io.IOException;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.fs.permission.FsPermission;
 +import org.apache.hadoop.fs.permission.PermissionStatus;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.DFSTestUtil;
 +import org.apache.hadoop.hdfs.DistributedFileSystem;
 +import org.apache.hadoop.hdfs.MiniDFSCluster;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
++import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 +
 +import org.junit.Test;
 +
 +/**
 + * This class tests INodeFile with striped feature.
 + */
 +public class TestStripedINodeFile {
 +  public static final Log LOG = LogFactory.getLog(TestINodeFile.class);
 +
 +  private static final PermissionStatus perm = new PermissionStatus(
 +      "userName", null, FsPermission.getDefault());
 +
 +  private final BlockStoragePolicySuite defaultSuite =
 +      BlockStoragePolicySuite.createDefaultSuite();
 +  private final BlockStoragePolicy defaultPolicy =
 +      defaultSuite.getDefaultPolicy();
 +
 +  private static final ErasureCodingPolicy testECPolicy
 +      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 +
 +  private static INodeFile createStripedINodeFile() {
 +    return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
 +        null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID, true);
 +  }
 +
 +  @Test
 +  public void testBlockStripedFeature()
 +      throws IOException, InterruptedException{
 +    INodeFile inf = createStripedINodeFile();
 +    assertTrue(inf.isStriped());
 +  }
 +
 +  @Test
 +  public void testBlockStripedTotalBlockCount() {
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    assertEquals(9, blockInfoStriped.getTotalBlockNum());
 +  }
 +
 +  @Test
 +  public void testBlockStripedLength()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    inf.addBlock(blockInfoStriped);
 +    assertEquals(1, inf.getBlocks().length);
 +  }
 +
 +  @Test
 +  public void testBlockStripedConsumedSpace()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(1);
 +    inf.addBlock(blockInfoStriped);
 +    //   0. Calculate the total bytes per stripes <Num Bytes per Stripes>
 +    //   1. Calculate the number of stripes in this block group. <Num Stripes>
 +    //   2. Calculate the last remaining length which does not make a stripe. <Last Stripe Length>
 +    //   3. Total consumed space is the total of
 +    //     a. The total of the full cells of data blocks and parity blocks.
 +    //     b. The remaining of data block which does not make a stripe.
 +    //     c. The last parity block cells. These size should be same
 +    //        to the first cell in this stripe.
 +    // So the total consumed space is the sum of
 +    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
 +    //  b. <Num Bytes> % <Num Bytes per Stripes> = 1
 +    //  c. <Last Stripe Length> * <Parity Block Num> = 1 * 3
 +    assertEquals(4, inf.storagespaceConsumedStriped().getStorageSpace());
 +    assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
 +  }
 +
 +  @Test
 +  public void testMultipleBlockStripedConsumedSpace()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk1 = new Block(1);
 +    BlockInfoStriped blockInfoStriped1
 +        = new BlockInfoStriped(blk1, testECPolicy);
 +    blockInfoStriped1.setNumBytes(1);
 +    Block blk2 = new Block(2);
 +    BlockInfoStriped blockInfoStriped2
 +        = new BlockInfoStriped(blk2, testECPolicy);
 +    blockInfoStriped2.setNumBytes(1);
 +    inf.addBlock(blockInfoStriped1);
 +    inf.addBlock(blockInfoStriped2);
 +    // This is the double size of one block in above case.
 +    assertEquals(4 * 2, inf.storagespaceConsumedStriped().getStorageSpace());
 +    assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
 +  }
 +
 +  @Test
 +  public void testBlockStripedFileSize()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(100);
 +    inf.addBlock(blockInfoStriped);
 +    // Compute file size should return actual data
 +    // size which is retained by this file.
 +    assertEquals(100, inf.computeFileSize());
 +    assertEquals(100, inf.computeFileSize(false, false));
 +  }
 +
 +  @Test
-   public void testBlockUCStripedFileSize()
++  public void testBlockStripedUCFileSize()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
-     BlockInfoUnderConstructionStriped bInfoUCStriped
-         = new BlockInfoUnderConstructionStriped(blk, testECPolicy);
++    BlockInfoStripedUnderConstruction bInfoUCStriped
++        = new BlockInfoStripedUnderConstruction(blk, testECPolicy);
 +    bInfoUCStriped.setNumBytes(100);
 +    inf.addBlock(bInfoUCStriped);
 +    assertEquals(100, inf.computeFileSize());
 +    assertEquals(0, inf.computeFileSize(false, false));
 +  }
 +
 +  @Test
 +  public void testBlockStripedComputeQuotaUsage()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(100);
 +    inf.addBlock(blockInfoStriped);
 +
 +    QuotaCounts counts =
 +        inf.computeQuotaUsageWithStriped(defaultPolicy,
 +            new QuotaCounts.Builder().build());
 +    assertEquals(1, counts.getNameSpace());
 +    // The total consumed space is the sum of
 +    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
 +    //  b. <Num Bytes> % <Num Bytes per Stripes> = 100
 +    //  c. <Last Stripe Length> * <Parity Block Num> = 100 * 3
 +    assertEquals(400, counts.getStorageSpace());
 +  }
 +
 +  @Test
-   public void testBlockUCStripedComputeQuotaUsage()
++  public void testBlockStripedUCComputeQuotaUsage()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
-     BlockInfoUnderConstructionStriped bInfoUCStriped
-         = new BlockInfoUnderConstructionStriped(blk, testECPolicy);
++    BlockInfoStripedUnderConstruction bInfoUCStriped
++        = new BlockInfoStripedUnderConstruction(blk, testECPolicy);
 +    bInfoUCStriped.setNumBytes(100);
 +    inf.addBlock(bInfoUCStriped);
 +
 +    QuotaCounts counts
 +        = inf.computeQuotaUsageWithStriped(defaultPolicy,
 +              new QuotaCounts.Builder().build());
 +    assertEquals(1024, inf.getPreferredBlockSize());
 +    assertEquals(1, counts.getNameSpace());
-     // Consumed space in the case of BlockInfoUCStriped can be calculated
++    // Consumed space in the case of BlockInfoStripedUC can be calculated
 +    // by using preferred block size. This is 1024 and total block num
 +    // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216.
 +    assertEquals(9216, counts.getStorageSpace());
 +  }
 +
 +  /**
 +   * Test the behavior of striped and contiguous block deletions.
 +   */
 +  @Test(timeout = 60000)
 +  public void testDeleteOp() throws Exception {
 +    MiniDFSCluster cluster = null;
 +    try {
 +      final int len = 1024;
 +      final Path parentDir = new Path("/parentDir");
 +      final Path zone = new Path(parentDir, "zone");
 +      final Path zoneFile = new Path(zone, "zoneFile");
 +      final Path contiguousFile = new Path(parentDir, "someFile");
 +      final DistributedFileSystem dfs;
 +      final Configuration conf = new Configuration();
 +      final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS
 +          + HdfsConstants.NUM_PARITY_BLOCKS;
 +      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 2);
 +
 +      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE)
 +          .build();
 +      cluster.waitActive();
 +
 +      FSNamesystem fsn = cluster.getNamesystem();
 +      dfs = cluster.getFileSystem();
 +      dfs.mkdirs(zone);
 +
 +      // create erasure zone
 +      dfs.createErasureCodingZone(zone, null);
 +      DFSTestUtil.createFile(dfs, zoneFile, len, (short) 1, 0xFEED);
 +      DFSTestUtil.createFile(dfs, contiguousFile, len, (short) 1, 0xFEED);
 +      final FSDirectory fsd = fsn.getFSDirectory();
 +
 +      // Case-1: Verify the behavior of striped blocks
 +      // Get blocks of striped file
 +      INode inodeStriped = fsd.getINode("/parentDir/zone/zoneFile");
 +      assertTrue("Failed to get INodeFile for /parentDir/zone/zoneFile",
 +          inodeStriped instanceof INodeFile);
 +      INodeFile inodeStripedFile = (INodeFile) inodeStriped;
 +      BlockInfo[] stripedBlks = inodeStripedFile.getBlocks();
 +      for (BlockInfo blockInfo : stripedBlks) {
 +        assertFalse("Mistakenly marked the block as deleted!",
 +            blockInfo.isDeleted());
 +      }
 +
 +      // delete erasure zone directory
 +      dfs.delete(zone, true);
 +      for (BlockInfo blockInfo : stripedBlks) {
 +        assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
 +      }
 +
 +      // Case-2: Verify the behavior of contiguous blocks
 +      // Get blocks of contiguous file
 +      INode inode = fsd.getINode("/parentDir/someFile");
 +      assertTrue("Failed to get INodeFile for /parentDir/someFile",
 +          inode instanceof INodeFile);
 +      INodeFile inodeFile = (INodeFile) inode;
 +      BlockInfo[] contiguousBlks = inodeFile.getBlocks();
 +      for (BlockInfo blockInfo : contiguousBlks) {
 +        assertFalse("Mistakenly marked the block as deleted!",
 +            blockInfo.isDeleted());
 +      }
 +
 +      // delete parent directory
 +      dfs.delete(parentDir, true);
 +      for (BlockInfo blockInfo : contiguousBlks) {
 +        assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
 +      }
 +    } finally {
 +      if (cluster != null) {
 +        cluster.shutdown();
 +      }
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
index 38d0daa,a1abd08..128bd9e
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
@@@ -43,8 -43,8 +43,9 @@@ import org.apache.hadoop.hdfs.DFSTestUt
  import org.apache.hadoop.hdfs.DistributedFileSystem;
  import org.apache.hadoop.hdfs.MiniDFSCluster;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
++import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
  import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
  import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
@@@ -175,9 -175,9 +176,9 @@@ public class SnapshotTestHelper 
     * localName (className@hashCode) parent permission group user
     * 
     * Specific information for different types of INode: 
--   * {@link INodeDirectory}:childrenSize 
-    * {@link INodeFile}: fileSize, block list. Check {@link BlockInfoContiguous#toString()}
-    * and {@link BlockInfoUnderConstructionContiguous#toString()} for detailed information.
++   * {@link INodeDirectory}:childrenSize
+    * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()}
+    * and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information.
     * {@link FileWithSnapshot}: next link
     * </pre>
     * @see INode#dumpTreeRecursively()


[07/50] [abbrv] hadoop git commit: Merge commit '456e901a4c5c639267ee87b8e5f1319f256d20c2' (HDFS-6407. Add sorting and pagination in the datanode tab of the NN Web UI. Contributed by Haohui Mai.) into HDFS-7285-merge

Posted by wa...@apache.org.
Merge commit '456e901a4c5c639267ee87b8e5f1319f256d20c2' (HDFS-6407. Add sorting and pagination in the datanode tab of the NN Web UI. Contributed by Haohui Mai.) into HDFS-7285-merge


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

Branch: refs/heads/trunk
Commit: 6b6a63bbbda920315d3d24b61ed3344a78a981b6
Parents: b57c9a3 456e901
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Wed Aug 19 22:52:32 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Mon Aug 24 12:59:26 2015 -0700

----------------------------------------------------------------------
 .gitignore                                      |     1 +
 BUILDING.txt                                    |    39 +-
 dev-support/determine-flaky-tests-hadoop.py     |    57 +-
 dev-support/docker/Dockerfile                   |     8 +
 dev-support/releasedocmaker.py                  |   405 +-
 dev-support/smart-apply-patch.sh                |    52 +-
 dev-support/test-patch.d/shellcheck.sh          |     2 +-
 dev-support/test-patch.sh                       |   310 +-
 hadoop-client/pom.xml                           |     8 +
 .../server/AuthenticationFilter.java            |    28 +-
 .../src/site/markdown/Configuration.md          |    18 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |   368 +-
 .../hadoop-common/HadoopCommon.cmake            |   207 +
 .../hadoop-common/HadoopJNI.cmake               |    97 +
 hadoop-common-project/hadoop-common/pom.xml     |    41 +-
 .../hadoop-common/src/CMakeLists.txt            |   366 +-
 .../hadoop-common/src/JNIFlags.cmake            |   124 -
 .../hadoop-common/src/main/bin/hadoop           |    62 +-
 .../hadoop-common/src/main/bin/hadoop-config.sh |    73 +-
 .../src/main/bin/hadoop-functions.sh            |   423 +-
 .../hadoop-common/src/main/bin/slaves.sh        |     3 +-
 .../main/conf/hadoop-user-functions.sh.example  |    29 +-
 .../src/main/conf/log4j.properties              |    21 +-
 .../src/main/docs/changes/ChangesFancyStyle.css |   170 -
 .../main/docs/changes/ChangesSimpleStyle.css    |    49 -
 .../src/main/docs/changes/changes2html.pl       |   286 -
 .../src/main/docs/releasenotes.html             | 29099 -----------------
 .../src/main/docs/src/documentation/README.txt  |     7 -
 .../classes/CatalogManager.properties           |    40 -
 .../main/docs/src/documentation/conf/cli.xconf  |   327 -
 .../src/documentation/content/xdocs/index.xml   |    48 -
 .../src/documentation/content/xdocs/site.xml    |   263 -
 .../src/documentation/content/xdocs/tabs.xml    |    37 -
 .../resources/images/architecture.gif           |   Bin 15461 -> 0 bytes
 .../resources/images/common-logo.jpg            |   Bin 5887 -> 0 bytes
 .../resources/images/core-logo.gif              |   Bin 6665 -> 0 bytes
 .../documentation/resources/images/favicon.ico  |   Bin 766 -> 0 bytes
 .../resources/images/hadoop-logo-big.jpg        |   Bin 127869 -> 0 bytes
 .../resources/images/hadoop-logo.jpg            |   Bin 9443 -> 0 bytes
 .../resources/images/hdfsarchitecture.gif       |   Bin 17653 -> 0 bytes
 .../resources/images/hdfsarchitecture.odg       |   Bin 41298 -> 0 bytes
 .../resources/images/hdfsarchitecture.png       |   Bin 40571 -> 0 bytes
 .../resources/images/hdfsdatanodes.gif          |   Bin 16060 -> 0 bytes
 .../resources/images/hdfsdatanodes.odg          |   Bin 37296 -> 0 bytes
 .../resources/images/hdfsdatanodes.png          |   Bin 30012 -> 0 bytes
 .../main/docs/src/documentation/skinconf.xml    |   366 -
 .../hadoop-common/src/main/docs/status.xml      |    75 -
 .../org/apache/hadoop/conf/Configuration.java   |     8 +-
 .../apache/hadoop/conf/ReconfigurableBase.java  |    14 +-
 .../crypto/key/kms/KMSClientProvider.java       |     4 +-
 .../apache/hadoop/fs/AbstractFileSystem.java    |    13 +
 .../org/apache/hadoop/fs/BlockLocation.java     |    55 +-
 .../apache/hadoop/fs/ByteBufferReadable.java    |     4 +
 .../fs/CommonConfigurationKeysPublic.java       |     6 +
 .../apache/hadoop/fs/DelegateToFileSystem.java  |    19 +-
 .../java/org/apache/hadoop/fs/FileContext.java  |    23 +
 .../java/org/apache/hadoop/fs/FileStatus.java   |    15 +-
 .../java/org/apache/hadoop/fs/FileSystem.java   |   196 +-
 .../java/org/apache/hadoop/fs/FileUtil.java     |   206 +-
 .../org/apache/hadoop/fs/FilterFileSystem.java  |     6 +
 .../java/org/apache/hadoop/fs/FilterFs.java     |     6 +
 .../main/java/org/apache/hadoop/fs/FsShell.java |    33 +-
 .../main/java/org/apache/hadoop/fs/Globber.java |    28 +-
 .../org/apache/hadoop/fs/LocatedFileStatus.java |    10 +-
 .../apache/hadoop/fs/RawLocalFileSystem.java    |   156 +-
 .../java/org/apache/hadoop/fs/StorageType.java  |     7 +-
 .../apache/hadoop/fs/TrashPolicyDefault.java    |     7 +-
 .../hadoop/fs/permission/FsPermission.java      |     7 +-
 .../hadoop/fs/permission/UmaskParser.java       |     2 +-
 .../hadoop/fs/sftp/SFTPConnectionPool.java      |   303 +
 .../apache/hadoop/fs/sftp/SFTPFileSystem.java   |   671 +
 .../apache/hadoop/fs/sftp/SFTPInputStream.java  |   130 +
 .../org/apache/hadoop/fs/sftp/package-info.java |    19 +
 .../java/org/apache/hadoop/fs/shell/Delete.java |    17 +-
 .../org/apache/hadoop/fs/shell/FsCommand.java   |    30 +-
 .../java/org/apache/hadoop/fs/shell/Ls.java     |     8 +
 .../java/org/apache/hadoop/fs/shell/Mkdir.java  |     3 +-
 .../java/org/apache/hadoop/fs/shell/Touch.java  |    84 +
 .../java/org/apache/hadoop/fs/shell/Touchz.java |    84 -
 .../org/apache/hadoop/fs/viewfs/ChRootedFs.java |     6 +
 .../org/apache/hadoop/fs/viewfs/ViewFs.java     |    15 +
 .../fs/viewfs/ViewFsLocatedFileStatus.java      |     3 +-
 .../apache/hadoop/ha/ActiveStandbyElector.java  |    20 +-
 .../apache/hadoop/ha/ZKFailoverController.java  |    63 +-
 .../main/java/org/apache/hadoop/io/MD5Hash.java |     3 +-
 .../org/apache/hadoop/io/ReadaheadPool.java     |     4 +-
 .../java/org/apache/hadoop/io/SequenceFile.java |    85 +-
 .../main/java/org/apache/hadoop/io/Text.java    |     4 +-
 .../hadoop/io/compress/bzip2/Bzip2Factory.java  |     2 +-
 .../org/apache/hadoop/io/nativeio/NativeIO.java |    98 +-
 .../apache/hadoop/io/retry/MultiException.java  |    49 +
 .../hadoop/io/retry/RetryInvocationHandler.java |    99 +-
 .../apache/hadoop/io/retry/RetryPolicies.java   |     6 +-
 .../org/apache/hadoop/io/retry/RetryUtils.java  |     7 +-
 .../org/apache/hadoop/ipc/CallQueueManager.java |    27 +-
 .../main/java/org/apache/hadoop/ipc/Client.java |     8 +-
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |     2 +-
 .../org/apache/hadoop/ipc/RpcClientUtil.java    |    24 +
 .../main/java/org/apache/hadoop/ipc/Server.java |     5 +-
 .../apache/hadoop/ipc/metrics/RpcMetrics.java   |     6 +-
 .../org/apache/hadoop/jmx/package-info.java     |     5 +-
 .../metrics2/impl/MetricsSinkAdapter.java       |     6 +-
 .../hadoop/metrics2/impl/MetricsSystemImpl.java |     2 +-
 .../org/apache/hadoop/net/NetworkTopology.java  |    96 +-
 .../hadoop/net/unix/DomainSocketWatcher.java    |    21 +-
 .../hadoop/security/LdapGroupsMapping.java      |    29 +-
 .../apache/hadoop/security/ProviderUtils.java   |    30 +
 .../apache/hadoop/security/SaslPlainServer.java |     2 +-
 .../hadoop/security/UserGroupInformation.java   |     3 +-
 .../alias/AbstractJavaKeyStoreProvider.java     |   339 +
 .../security/alias/JavaKeyStoreProvider.java    |   257 +-
 .../alias/LocalJavaKeyStoreProvider.java        |   215 +
 .../hadoop/security/token/SecretManager.java    |     2 +-
 .../org/apache/hadoop/security/token/Token.java |     2 +-
 .../hadoop/security/token/TokenIdentifier.java  |     2 +-
 .../apache/hadoop/security/token/TokenInfo.java |     2 +-
 .../hadoop/security/token/TokenRenewer.java     |     2 +-
 .../hadoop/security/token/TokenSelector.java    |     2 +-
 .../DelegationTokenAuthenticationFilter.java    |    16 +-
 .../hadoop/security/token/package-info.java     |     2 +-
 .../apache/hadoop/tracing/SpanReceiverHost.java |    45 +-
 .../java/org/apache/hadoop/util/ConfTest.java   |   368 +
 .../org/apache/hadoop/util/CpuTimeTracker.java  |   115 +
 .../org/apache/hadoop/util/DiskChecker.java     |    24 +-
 .../org/apache/hadoop/util/JvmPauseMonitor.java |    12 +-
 .../java/org/apache/hadoop/util/LineReader.java |     8 +
 .../org/apache/hadoop/util/ReflectionUtils.java |     6 +-
 .../java/org/apache/hadoop/util/RunJar.java     |     3 +-
 .../main/java/org/apache/hadoop/util/Shell.java |    22 +-
 .../org/apache/hadoop/util/StringUtils.java     |     8 +
 .../java/org/apache/hadoop/util/SysInfo.java    |   137 +
 .../org/apache/hadoop/util/SysInfoLinux.java    |   690 +
 .../org/apache/hadoop/util/SysInfoWindows.java  |   208 +
 .../apache/hadoop/util/hash/JenkinsHash.java    |    15 +-
 .../org/apache/hadoop/io/nativeio/NativeIO.c    |   150 +-
 .../org/apache/hadoop/net/unix/DomainSocket.c   |    17 +-
 .../hadoop/net/unix/DomainSocketWatcher.c       |     2 +-
 .../src/org/apache/hadoop/util/NativeCrc32.c    |     4 +-
 ...oop.security.alias.CredentialProviderFactory |     1 +
 .../src/main/resources/core-default.xml         |    44 +
 .../src/site/markdown/CLIMiniCluster.md.vm      |     2 +-
 .../src/site/markdown/CommandsManual.md         |     4 +-
 .../src/site/markdown/DeprecatedProperties.md   |     1 +
 .../src/site/markdown/FileSystemShell.md        |     4 +-
 .../hadoop-common/src/site/markdown/Metrics.md  |     7 +
 .../hadoop-common/src/site/markdown/Tracing.md  |    12 +-
 .../src/site/markdown/filesystem/filesystem.md  |     4 +
 .../apache/hadoop/conf/TestConfiguration.java   |    15 +
 .../apache/hadoop/conf/TestReconfiguration.java |    12 +-
 .../apache/hadoop/crypto/TestCryptoCodec.java   |    11 +-
 ...yptoStreamsWithOpensslAesCtrCryptoCodec.java |    11 +-
 .../apache/hadoop/fs/FCStatisticsBaseTest.java  |    56 +-
 .../fs/FileContextMainOperationsBaseTest.java   |     4 +-
 .../hadoop/fs/FileSystemContractBaseTest.java   |    11 +-
 .../org/apache/hadoop/fs/SymlinkBaseTest.java   |    45 +-
 .../org/apache/hadoop/fs/TestBlockLocation.java |    23 +-
 .../org/apache/hadoop/fs/TestFileStatus.java    |    22 +
 .../java/org/apache/hadoop/fs/TestFileUtil.java |   112 +-
 .../java/org/apache/hadoop/fs/TestFsShell.java  |    28 +
 .../apache/hadoop/fs/TestFsShellReturnCode.java |    60 +-
 .../org/apache/hadoop/fs/TestHarFileSystem.java |     3 +
 .../apache/hadoop/fs/TestLocalFileSystem.java   |    83 +-
 .../fs/TestLocalFileSystemPermission.java       |   111 +-
 .../apache/hadoop/fs/TestSymlinkLocalFS.java    |    18 +
 .../java/org/apache/hadoop/fs/TestTrash.java    |    14 +-
 .../fs/contract/AbstractContractAppendTest.java |     4 +-
 .../AbstractContractGetFileStatusTest.java      |    61 +
 .../contract/AbstractContractSetTimesTest.java  |    61 +
 .../hadoop/fs/contract/ContractOptions.java     |    12 +
 .../TestLocalFSContractGetFileStatus.java       |    33 +
 .../localfs/TestLocalFSContractSetTimes.java    |    33 +
 .../TestRawlocalContractGetFileStatus.java      |    33 +
 .../rawlocal/TestRawlocalContractSetTimes.java  |    33 +
 .../hadoop/fs/sftp/TestSFTPFileSystem.java      |   308 +
 .../hadoop/fs/shell/TestCopyPreserveFlag.java   |    63 +-
 .../org/apache/hadoop/fs/shell/TestCount.java   |     4 +-
 .../java/org/apache/hadoop/fs/shell/TestLs.java |    25 +
 .../org/apache/hadoop/fs/shell/TestMove.java    |     1 +
 .../org/apache/hadoop/ha/MiniZKFCCluster.java   |    93 +-
 .../hadoop/ha/TestActiveStandbyElector.java     |    31 +
 .../hadoop/ha/TestZKFailoverController.java     |    32 +
 .../http/TestAuthenticationSessionCookie.java   |   187 +
 .../apache/hadoop/http/TestHttpCookieFlag.java  |     2 +-
 .../hadoop/io/TestSequenceFileAppend.java       |   316 +
 .../apache/hadoop/io/nativeio/TestNativeIO.java |    90 +-
 .../hadoop/io/retry/TestDefaultRetryPolicy.java |   101 +
 .../apache/hadoop/ipc/TestCallQueueManager.java |     6 +-
 .../java/org/apache/hadoop/ipc/TestIPC.java     |    78 +-
 .../metrics2/impl/TestMetricsSystemImpl.java    |    60 +
 .../org/apache/hadoop/net/ServerSocketUtil.java |    63 +
 .../apache/hadoop/net/TestClusterTopology.java  |    75 +-
 .../net/unix/TestDomainSocketWatcher.java       |    75 +
 .../TestLdapGroupsMappingWithPosixGroup.java    |    25 +-
 .../alias/TestCredentialProviderFactory.java    |    42 +-
 .../apache/hadoop/test/GenericTestUtils.java    |    12 +
 .../apache/hadoop/tracing/SetSpanReceiver.java  |   109 +
 .../org/apache/hadoop/util/TestConfTest.java    |   204 +
 .../org/apache/hadoop/util/TestDiskChecker.java |    22 +
 .../org/apache/hadoop/util/TestStringUtils.java |     7 +-
 .../apache/hadoop/util/TestSysInfoLinux.java    |   432 +
 .../apache/hadoop/util/TestSysInfoWindows.java  |   100 +
 .../src/test/resources/contract/localfs.xml     |    10 +
 .../src/test/resources/contract/rawlocal.xml    |    10 +
 .../scripts/hadoop-functions_test_helper.bash   |    56 +
 .../src/test/scripts/hadoop_add_classpath.bats  |   100 +
 .../src/test/scripts/hadoop_add_colonpath.bats  |    96 +
 .../scripts/hadoop_add_common_to_classpath.bats |    71 +
 .../test/scripts/hadoop_add_javalibpath.bats    |    98 +
 .../src/test/scripts/hadoop_add_ldlibpath.bats  |    97 +
 .../src/test/scripts/hadoop_add_param.bats      |    49 +
 .../hadoop_add_to_classpath_userpath.bats       |    98 +
 .../src/test/scripts/hadoop_basic_init.bats     |    94 +
 .../src/test/scripts/hadoop_bootstrap.bats      |    51 +
 .../src/test/scripts/hadoop_confdir.bats        |    92 +
 .../test/scripts/hadoop_deprecate_envvar.bats   |    32 +
 .../src/test/scripts/hadoop_finalize.bats       |   206 +
 .../scripts/hadoop_finalize_catalina_opts.bats  |    56 +
 .../test/scripts/hadoop_finalize_classpath.bats |    64 +
 .../scripts/hadoop_finalize_hadoop_heap.bats    |    87 +
 .../scripts/hadoop_finalize_hadoop_opts.bats    |    52 +
 .../test/scripts/hadoop_finalize_libpaths.bats  |    30 +
 .../src/test/scripts/hadoop_java_setup.bats     |    47 +
 .../src/test/scripts/hadoop_os_tricks.bats      |    34 +
 .../src/test/scripts/hadoop_rotate_log.bats     |    52 +
 .../src/test/scripts/hadoop_shellprofile.bats   |    91 +
 .../src/test/scripts/hadoop_slaves.bats         |    37 +
 .../src/test/scripts/hadoop_ssh.bats            |    51 +
 .../scripts/hadoop_translate_cygwin_path.bats   |    48 +
 .../test/scripts/hadoop_validate_classname.bats |    26 +
 .../hadoop-common/src/test/scripts/run-bats.sh  |    43 +
 hadoop-common-project/hadoop-kms/pom.xml        |     4 +-
 .../crypto/key/kms/server/KMSMDCFilter.java     |     2 +-
 .../hadoop-kms/src/main/sbin/kms.sh             |    29 +-
 hadoop-dist/pom.xml                             |     4 +-
 .../dev-support/findbugsExcludeFile.xml         |     4 +
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  |    53 +
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |     9 +-
 .../hdfs/client/HdfsClientConfigKeys.java       |     2 +
 .../org/apache/hadoop/hdfs/inotify/Event.java   |   132 +-
 .../hdfs/protocol/BlockStoragePolicy.java       |    12 +-
 .../hadoop/hdfs/protocol/CachePoolInfo.java     |     3 -
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  1511 +
 .../hadoop/hdfs/protocol/ErasureCodingZone.java |    54 +
 .../hadoop/hdfs/protocol/HdfsConstants.java     |     3 +
 .../hdfs/server/namenode/SafeModeException.java |    38 +
 .../hadoop/hdfs/web/ByteRangeInputStream.java   |    57 +-
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |    24 +
 .../org/apache/hadoop/hdfs/web/TokenAspect.java |    16 +-
 .../hadoop/hdfs/web/URLConnectionFactory.java   |    13 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |    62 +-
 .../src/main/proto/ClientDatanodeProtocol.proto |   247 +
 .../src/main/proto/ClientNamenodeProtocol.proto |   880 +
 .../hadoop-hdfs-client/src/main/proto/acl.proto |   108 +
 .../src/main/proto/datatransfer.proto           |   304 +
 .../src/main/proto/encryption.proto             |    67 +
 .../src/main/proto/erasurecoding.proto          |    68 +
 .../src/main/proto/hdfs.proto                   |   648 +
 .../src/main/proto/inotify.proto                |   133 +
 .../src/main/proto/xattr.proto                  |    75 +
 hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml  |     8 +-
 .../http/server/HttpFSAuthenticationFilter.java |     5 +
 .../apache/hadoop/lib/servlet/ServerWebApp.java |     3 +-
 .../hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh  |    29 +-
 .../org/apache/hadoop/test/TestDirHelper.java   |     2 +-
 .../org/apache/hadoop/test/TestHdfsHelper.java  |     4 +-
 .../org/apache/hadoop/test/TestJettyHelper.java |     2 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   477 +-
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |    19 +-
 .../hadoop-hdfs/src/CMakeLists.txt              |   129 +-
 .../hadoop-hdfs/src/contrib/bkjournal/pom.xml   |     2 +-
 .../bkjournal/TestBookKeeperHACheckpoints.java  |     7 +-
 .../src/contrib/libwebhdfs/CMakeLists.txt       |     8 +-
 .../libwebhdfs/resources/FindJansson.cmake      |     9 +-
 .../hadoop-hdfs/src/main/bin/hdfs               |    97 +-
 .../hadoop-hdfs/src/main/bin/hdfs.cmd           |    13 +-
 .../hadoop-hdfs/src/main/bin/start-balancer.sh  |    12 +-
 .../hadoop-hdfs/src/main/bin/stop-balancer.sh   |    10 +-
 .../apache/hadoop/fs/BlockStorageLocation.java  |     1 +
 .../main/java/org/apache/hadoop/fs/Hdfs.java    |     5 +
 .../java/org/apache/hadoop/fs/SWebHdfs.java     |    51 +
 .../main/java/org/apache/hadoop/fs/WebHdfs.java |    51 +
 .../main/java/org/apache/hadoop/fs/package.html |    26 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |    57 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    40 +-
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |    18 +-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |   196 +-
 .../hadoop/hdfs/DFSStripedOutputStream.java     |     2 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |    25 +
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |    36 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |    43 +-
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |    12 +-
 .../hdfs/protocol/ClientDatanodeProtocol.java   |     5 +
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  1484 -
 .../hadoop/hdfs/protocol/LayoutVersion.java     |    51 +-
 ...tDatanodeProtocolServerSideTranslatorPB.java |    19 +
 .../ClientDatanodeProtocolTranslatorPB.java     |    19 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |    17 +
 .../ClientNamenodeProtocolTranslatorPB.java     |    15 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |     8 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |     3 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |    20 +-
 .../token/block/BlockTokenSecretManager.java    |    40 +-
 .../hadoop/hdfs/server/balancer/Balancer.java   |   131 +-
 .../hadoop/hdfs/server/balancer/Dispatcher.java |   155 +-
 .../hadoop/hdfs/server/balancer/ExitStatus.java |     3 +-
 .../hdfs/server/balancer/MovedBlocks.java       |     5 +
 .../hdfs/server/balancer/NameNodeConnector.java |    16 +
 .../hdfs/server/blockmanagement/BlockInfo.java  |    33 +-
 .../blockmanagement/BlockInfoContiguous.java    |    10 +-
 .../BlockInfoContiguousUnderConstruction.java   |   281 +
 .../blockmanagement/BlockInfoStriped.java       |     8 +-
 .../BlockInfoStripedUnderConstruction.java      |   297 +
 .../BlockInfoUnderConstructionContiguous.java   |   281 -
 .../BlockInfoUnderConstructionStriped.java      |   297 -
 .../server/blockmanagement/BlockManager.java    |   174 +-
 .../BlockManagerFaultInjector.java              |    52 +
 .../BlockPlacementPolicyDefault.java            |     3 +-
 .../BlockPlacementPolicyRackFaultTolarent.java  |   154 -
 .../BlockPlacementPolicyRackFaultTolerant.java  |   154 +
 .../BlockReportLeaseManager.java                |   354 +
 .../blockmanagement/BlockStatsMXBean.java       |    36 +
 .../BlockStoragePolicySuite.java                |     6 +-
 .../hdfs/server/blockmanagement/BlocksMap.java  |     8 +-
 .../CacheReplicationMonitor.java                |    10 +-
 .../blockmanagement/CorruptReplicasMap.java     |    23 +-
 .../blockmanagement/DatanodeDescriptor.java     |    58 +-
 .../server/blockmanagement/DatanodeManager.java |    22 +-
 .../blockmanagement/DatanodeStatistics.java     |     6 +
 .../blockmanagement/DatanodeStorageInfo.java    |     7 +-
 .../blockmanagement/DecommissionManager.java    |    42 +-
 .../blockmanagement/HeartbeatManager.java       |    96 +-
 .../blockmanagement/InvalidateBlocks.java       |     7 +-
 .../blockmanagement/StorageTypeStats.java       |   115 +
 .../blockmanagement/UnderReplicatedBlocks.java  |     9 +-
 .../hdfs/server/common/HdfsServerConstants.java |     3 -
 .../hadoop/hdfs/server/common/Storage.java      |    24 +-
 .../hdfs/server/datanode/BPServiceActor.java    |    76 +-
 .../hdfs/server/datanode/BlockReceiver.java     |   160 +-
 .../hdfs/server/datanode/BlockScanner.java      |    38 +-
 .../hdfs/server/datanode/BlockSender.java       |    11 +-
 .../hadoop/hdfs/server/datanode/DNConf.java     |    17 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |    81 +-
 .../hdfs/server/datanode/DataStorage.java       |    47 +-
 .../hdfs/server/datanode/DataXceiver.java       |    57 +-
 .../hdfs/server/datanode/ReplicaInPipeline.java |    15 +-
 .../datanode/ReplicaInPipelineInterface.java    |    10 +
 .../hdfs/server/datanode/StorageLocation.java   |     4 +-
 .../hdfs/server/datanode/VolumeScanner.java     |    20 +-
 .../server/datanode/fsdataset/FsDatasetSpi.java |   149 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  |    47 +-
 .../datanode/fsdataset/impl/BlockPoolSlice.java |    31 +-
 .../datanode/fsdataset/impl/FsDatasetCache.java |     7 +
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   103 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |     7 +-
 .../server/datanode/web/DatanodeHttpServer.java |    53 +-
 .../datanode/web/webhdfs/ExceptionHandler.java  |     2 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |   113 +-
 .../hdfs/server/namenode/AclTransformation.java |    30 +-
 .../hdfs/server/namenode/BackupImage.java       |     2 +-
 .../hdfs/server/namenode/CacheManager.java      |    24 +-
 .../hdfs/server/namenode/CheckpointConf.java    |    14 +-
 .../hdfs/server/namenode/Checkpointer.java      |     8 +-
 .../server/namenode/EncryptionZoneManager.java  |     7 +
 .../hdfs/server/namenode/FSDirAppendOp.java     |   268 +
 .../hdfs/server/namenode/FSDirAttrOp.java       |    26 +-
 .../hdfs/server/namenode/FSDirConcatOp.java     |     5 +-
 .../server/namenode/FSDirStatAndListingOp.java  |   106 +-
 .../hdfs/server/namenode/FSDirTruncateOp.java   |   370 +
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |    17 +-
 .../hdfs/server/namenode/FSDirectory.java       |   114 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |    54 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |    22 +-
 .../hdfs/server/namenode/FSEditLogOp.java       |     4 +-
 .../hadoop/hdfs/server/namenode/FSImage.java    |    27 +-
 .../hdfs/server/namenode/FSImageFormat.java     |     8 +-
 .../server/namenode/FSImageFormatPBINode.java   |    23 +-
 .../server/namenode/FSImageFormatProtobuf.java  |    33 +-
 .../server/namenode/FSImageSerialization.java   |     4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   924 +-
 .../hdfs/server/namenode/FSNamesystemLock.java  |    11 +
 .../hadoop/hdfs/server/namenode/INode.java      |     2 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |    11 +-
 .../hdfs/server/namenode/ImageServlet.java      |    88 +-
 .../namenode/InotifyFSEditLogOpTranslator.java  |     4 +
 .../hdfs/server/namenode/LeaseManager.java      |    17 +-
 .../hdfs/server/namenode/NNUpgradeUtil.java     |    18 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |     5 +-
 .../server/namenode/NameNodeHttpServer.java     |     7 +-
 .../server/namenode/NameNodeLayoutVersion.java  |    48 +-
 .../hdfs/server/namenode/NameNodeMXBean.java    |     5 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |    40 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |     7 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |    20 +-
 .../hadoop/hdfs/server/namenode/SafeMode.java   |     1 -
 .../hdfs/server/namenode/SafeModeException.java |    39 -
 .../hdfs/server/namenode/SecondaryNameNode.java |     4 +-
 .../hdfs/server/namenode/TransferFsImage.java   |    47 +-
 .../server/namenode/ha/BootstrapStandby.java    |    94 +-
 .../ha/ConfiguredFailoverProxyProvider.java     |    52 +-
 .../hdfs/server/namenode/ha/EditLogTailer.java  |   162 +-
 .../server/namenode/ha/RemoteNameNodeInfo.java  |   133 +
 .../ha/RequestHedgingProxyProvider.java         |   186 +
 .../server/namenode/ha/StandbyCheckpointer.java |   182 +-
 .../hdfs/server/namenode/ha/StandbyState.java   |     3 +-
 .../namenode/metrics/FSNamesystemMBean.java     |    14 +
 .../snapshot/FSImageFormatPBSnapshot.java       |     6 +-
 .../hdfs/server/namenode/snapshot/FileDiff.java |     1 -
 .../server/namenode/snapshot/FileDiffList.java  |     4 +-
 .../top/window/RollingWindowManager.java        |     8 +-
 .../web/resources/NamenodeWebHdfsMethods.java   |    12 +-
 .../server/protocol/BlockReportContext.java     |    25 +-
 .../hdfs/server/protocol/DatanodeProtocol.java  |     5 +-
 .../hdfs/server/protocol/HeartbeatResponse.java |    10 +-
 .../hdfs/server/protocol/RegisterCommand.java   |     2 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |    75 +-
 .../hdfs/tools/DFSZKFailoverController.java     |    13 +
 .../org/apache/hadoop/hdfs/tools/DFSck.java     |    10 +-
 .../apache/hadoop/hdfs/tools/DebugAdmin.java    |    10 +-
 .../hdfs/tools/DelegationTokenFetcher.java      |    19 +-
 .../offlineImageViewer/FSImageHandler.java      |    60 +-
 .../tools/offlineImageViewer/FSImageLoader.java |   164 +
 .../offlineImageViewer/PBImageXmlWriter.java    |    20 +-
 .../src/main/native/fuse-dfs/CMakeLists.txt     |    91 +-
 .../src/main/native/fuse-dfs/fuse_impls_open.c  |     4 -
 .../src/main/native/libhdfs/expect.h            |    18 +
 .../hadoop-hdfs/src/main/native/libhdfs/hdfs.c  |     1 +
 .../hadoop-hdfs/src/main/native/libhdfs/hdfs.h  |     3 +-
 .../main/native/libhdfs/test/test_libhdfs_ops.c |     9 +
 .../main/native/libhdfs/test_libhdfs_threaded.c |    17 +-
 .../src/main/proto/ClientDatanodeProtocol.proto |   235 -
 .../src/main/proto/ClientNamenodeProtocol.proto |   870 -
 .../src/main/proto/DatanodeProtocol.proto       |     6 +
 .../hadoop-hdfs/src/main/proto/acl.proto        |   113 -
 .../src/main/proto/datatransfer.proto           |   304 -
 .../hadoop-hdfs/src/main/proto/editlog.proto    |    35 +
 .../hadoop-hdfs/src/main/proto/encryption.proto |    67 -
 .../src/main/proto/erasurecoding.proto          |    68 -
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |   648 -
 .../hadoop-hdfs/src/main/proto/inotify.proto    |   126 -
 .../hadoop-hdfs/src/main/proto/xattr.proto      |    80 -
 .../src/main/resources/hdfs-default.xml         |    68 +-
 .../src/main/webapps/hdfs/dfshealth.html        |    87 +-
 .../src/main/webapps/hdfs/dfshealth.js          |    71 +-
 .../webapps/static/dataTables.bootstrap.css     |   371 +
 .../main/webapps/static/dataTables.bootstrap.js |   205 +
 .../src/main/webapps/static/hadoop.css          |    51 +
 .../webapps/static/jquery.dataTables.min.js     |   160 +
 .../src/main/webapps/static/moment.min.js       |     7 +
 .../site/markdown/CentralizedCacheManagement.md |     2 +
 .../src/site/markdown/HDFSCommands.md           |    34 +-
 .../markdown/HDFSHighAvailabilityWithNFS.md     |    49 +-
 .../markdown/HDFSHighAvailabilityWithQJM.md     |    42 +-
 .../src/site/markdown/HdfsImageViewer.md        |     3 +
 .../src/site/markdown/HdfsMultihoming.md        |     2 +
 .../src/site/markdown/HdfsNfsGateway.md         |     2 +-
 .../src/site/markdown/MemoryStorage.md          |   130 +
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    |    14 +-
 .../site/resources/images/LazyPersistWrites.png |   Bin 0 -> 107161 bytes
 .../org/apache/hadoop/fs/TestGlobPaths.java     |    25 +
 .../TestSWebHdfsFileContextMainOperations.java  |   110 +
 .../TestWebHdfsFileContextMainOperations.java   |   157 +
 .../contract/hdfs/TestHDFSContractAppend.java   |    15 -
 .../hdfs/TestHDFSContractGetFileStatus.java     |    46 +
 .../contract/hdfs/TestHDFSContractSetTimes.java |    45 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |    39 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |   494 +-
 .../apache/hadoop/hdfs/MiniDFSNNTopology.java   |    18 +-
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |   227 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |    66 +-
 .../hdfs/TestDFSInotifyEventInputStream.java    |    47 +-
 .../apache/hadoop/hdfs/TestDFSInputStream.java  |    25 +
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |   323 +-
 .../org/apache/hadoop/hdfs/TestDFSUpgrade.java  |    58 +-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |     6 +-
 .../hadoop/hdfs/TestDatanodeLayoutUpgrade.java  |     8 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |   124 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |   108 +-
 .../apache/hadoop/hdfs/TestEncryptionZones.java |     6 +
 .../apache/hadoop/hdfs/TestFileCorruption.java  |    19 +-
 .../apache/hadoop/hdfs/TestFileCreation.java    |     6 -
 .../apache/hadoop/hdfs/TestLeaseRecovery.java   |    46 +
 .../java/org/apache/hadoop/hdfs/TestQuota.java  |    21 +
 .../org/apache/hadoop/hdfs/TestReplication.java |    26 +-
 .../apache/hadoop/hdfs/TestRollingUpgrade.java  |   170 +-
 .../org/apache/hadoop/hdfs/TestSafeMode.java    |     4 +-
 .../hdfs/protocol/TestBlockListAsLongs.java     |     4 +-
 .../hadoop/hdfs/protocol/TestLayoutVersion.java |   102 +-
 .../hadoop/hdfs/qjournal/MiniQJMHACluster.java  |    94 +-
 .../security/token/block/TestBlockToken.java    |    10 +-
 .../hdfs/server/balancer/TestBalancer.java      |   116 +-
 .../server/blockmanagement/TestBlockInfo.java   |    14 +-
 .../TestBlockInfoUnderConstruction.java         |     2 +-
 .../blockmanagement/TestBlockManager.java       |    40 +-
 .../TestBlockReportRateLimiting.java            |   220 +
 .../blockmanagement/TestBlockStatsMXBean.java   |   145 +
 .../blockmanagement/TestCorruptReplicaInfo.java |    12 +-
 .../blockmanagement/TestDatanodeDescriptor.java |     4 +-
 .../blockmanagement/TestDatanodeManager.java    |    21 +-
 .../blockmanagement/TestHeartbeatHandling.java  |     6 +-
 .../TestNameNodePrunesMissingStorages.java      |     2 +-
 .../blockmanagement/TestPendingReplication.java |     4 +-
 .../blockmanagement/TestReplicationPolicy.java  |    10 +-
 .../hdfs/server/datanode/DataNodeTestUtils.java |    11 +
 .../server/datanode/SimulatedFSDataset.java     |     5 +
 .../server/datanode/TestBPOfferService.java     |     7 +-
 .../TestBlockHasMultipleReplicasOnSameDN.java   |     2 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |     6 +-
 .../datanode/TestBpServiceActorScheduler.java   |     2 +-
 .../server/datanode/TestCachingStrategy.java    |     5 +-
 .../TestDataXceiverLazyPersistHint.java         |   178 +
 .../TestDatanodeProtocolRetryPolicy.java        |     8 +-
 .../server/datanode/TestFsDatasetCache.java     |     9 +-
 .../TestNNHandlesBlockReportPerStorage.java     |     2 +-
 .../TestNNHandlesCombinedBlockReport.java       |     2 +-
 .../datanode/TestReadOnlySharedStorage.java     |     2 +-
 .../hdfs/server/datanode/TestStorageReport.java |     2 +-
 .../extdataset/ExternalReplicaInPipeline.java   |     7 +
 .../fsdataset/impl/LazyPersistTestCase.java     |    42 +-
 .../fsdataset/impl/TestFsDatasetImpl.java       |    35 +
 .../impl/TestLazyPersistLockedMemory.java       |    25 +-
 .../impl/TestLazyPersistReplicaPlacement.java   |    36 +-
 .../datanode/fsdataset/impl/TestLazyWriter.java |    62 +-
 .../fsdataset/impl/TestRbwSpaceReservation.java |    73 +
 .../fsdataset/impl/TestScrLazyPersistFiles.java |   193 +-
 .../hadoop/hdfs/server/mover/TestMover.java     |    55 +-
 .../hdfs/server/mover/TestStorageMover.java     |    26 +-
 .../hdfs/server/namenode/CreateEditsLog.java    |     4 +-
 .../hdfs/server/namenode/FSImageTestUtil.java   |     2 +-
 .../hdfs/server/namenode/FSXAttrBaseTest.java   |     5 +-
 .../server/namenode/NNThroughputBenchmark.java  |     8 +-
 .../hdfs/server/namenode/NameNodeAdapter.java   |     2 +-
 .../server/namenode/TestAclTransformation.java  |    55 +-
 .../server/namenode/TestAddStripedBlocks.java   |    20 +-
 .../hdfs/server/namenode/TestBackupNode.java    |     2 +-
 ...stBlockPlacementPolicyRackFaultTolarent.java |   209 -
 ...stBlockPlacementPolicyRackFaultTolerant.java |   209 +
 .../namenode/TestBlockUnderConstruction.java    |     4 +-
 .../server/namenode/TestCacheDirectives.java    |    25 +
 .../hdfs/server/namenode/TestCheckpoint.java    |     3 +-
 .../hdfs/server/namenode/TestClusterId.java     |     6 +-
 .../TestCommitBlockSynchronization.java         |    12 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |    49 +-
 .../hdfs/server/namenode/TestEditLog.java       |    16 +-
 .../server/namenode/TestFSEditLogLoader.java    |     2 +-
 .../hdfs/server/namenode/TestFSImage.java       |    50 +-
 .../hdfs/server/namenode/TestFSNamesystem.java  |    48 +
 .../server/namenode/TestFSNamesystemMBean.java  |     5 +
 .../hdfs/server/namenode/TestFileTruncate.java  |    35 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |    16 +-
 .../server/namenode/TestGetBlockLocations.java  |     4 +-
 .../hdfs/server/namenode/TestHDFSConcat.java    |    24 +-
 .../hdfs/server/namenode/TestINodeFile.java     |    12 +-
 .../hdfs/server/namenode/TestLeaseManager.java  |    65 +-
 .../server/namenode/TestNameNodeMXBean.java     |    28 +-
 .../server/namenode/TestQuotaByStorageType.java |    14 +-
 .../namenode/TestQuotaWithStripedBlocks.java    |     2 +-
 .../server/namenode/TestStripedINodeFile.java   |    16 +-
 .../namenode/TestTruncateQuotaUpdate.java       |    10 +-
 .../server/namenode/ha/HAStressTestHarness.java |    46 +-
 .../hdfs/server/namenode/ha/HATestUtil.java     |    60 +-
 .../namenode/ha/TestBootstrapStandby.java       |   176 +-
 .../ha/TestBootstrapStandbyWithQJM.java         |    47 +-
 .../ha/TestDNFencingWithReplication.java        |     1 +
 .../server/namenode/ha/TestEditLogTailer.java   |    12 +-
 .../ha/TestFailoverWithBlockTokensEnabled.java  |    55 +-
 .../server/namenode/ha/TestHAConfiguration.java |    49 +-
 .../namenode/ha/TestPipelinesFailover.java      |   110 +-
 .../namenode/ha/TestRemoteNameNodeInfo.java     |    61 +
 .../ha/TestRequestHedgingProxyProvider.java     |   350 +
 .../namenode/ha/TestRetryCacheWithHA.java       |    43 +-
 .../namenode/ha/TestSeveralNameNodes.java       |   179 +
 .../namenode/ha/TestStandbyCheckpoints.java     |   106 +-
 .../server/namenode/ha/TestStandbyIsHot.java    |     2 +
 .../namenode/snapshot/SnapshotTestHelper.java   |     9 +-
 .../snapshot/TestFileWithSnapshotFeature.java   |     3 +-
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  |   117 +-
 .../hadoop/hdfs/tools/TestDebugAdmin.java       |     8 +
 .../hdfs/tools/TestDelegationTokenFetcher.java  |    54 +
 .../TestOfflineImageViewerForAcl.java           |    24 +
 ...TestOfflineImageViewerForContentSummary.java |   248 +
 .../TestOfflineImageViewerForXAttr.java         |   265 +
 .../hdfs/web/TestByteRangeInputStream.java      |    35 +-
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |   100 +
 .../hdfs/web/TestWebHdfsFileSystemContract.java |     2 +-
 .../apache/hadoop/net/TestNetworkTopology.java  |     1 +
 .../apache/hadoop/security/TestPermission.java  |    26 +
 .../hadoop/test/MiniDFSClusterManager.java      |     4 +
 .../org/apache/hadoop/tracing/TestTracing.java  |   117 +-
 .../TestTracingShortCircuitLocalRead.java       |     4 +-
 .../src/test/resources/contract/hdfs.xml        |    10 +
 .../src/test/resources/hadoop-0.23-reserved.tgz |   Bin 4558 -> 5590 bytes
 .../src/test/resources/hadoop-1-reserved.tgz    |   Bin 2572 -> 3348 bytes
 .../src/test/resources/hadoop-2-reserved.tgz    |   Bin 2838 -> 3465 bytes
 .../src/test/resources/hadoop-22-dfs-dir.tgz    |   Bin 318180 -> 413239 bytes
 .../src/test/resources/hadoop1-bbw.tgz          |   Bin 40234 -> 43294 bytes
 .../src/test/resources/log4j.properties         |     2 +-
 .../src/test/resources/testHDFSConf.xml         |     2 +-
 hadoop-mapreduce-project/CHANGES.txt            |   153 +-
 hadoop-mapreduce-project/bin/mapred             |    33 +-
 .../dev-support/findbugs-exclude.xml            |     1 +
 .../hadoop/mapred/LocalContainerLauncher.java   |     2 +-
 .../jobhistory/JobHistoryEventHandler.java      |    27 +-
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |    30 +-
 .../v2/app/job/event/JobStartEvent.java         |     2 +-
 .../event/JobTaskAttemptFetchFailureEvent.java  |     9 +-
 .../TaskAttemptTooManyFetchFailureEvent.java    |    50 +
 .../mapreduce/v2/app/job/impl/JobImpl.java      |    53 +-
 .../v2/app/job/impl/TaskAttemptImpl.java        |    25 +-
 .../mapreduce/v2/app/job/impl/TaskImpl.java     |     4 +-
 .../mapreduce/v2/app/rm/RMCommunicator.java     |    51 +-
 .../app/rm/RMContainerAllocationException.java  |    31 +
 .../v2/app/rm/RMContainerAllocator.java         |    23 +-
 .../v2/app/rm/RMContainerRequestor.java         |    32 +-
 .../mapreduce/v2/app/webapp/AttemptsPage.java   |     8 +
 .../mapreduce/v2/app/webapp/JobBlock.java       |     2 +
 .../mapreduce/v2/app/webapp/TaskPage.java       |    93 +-
 .../mapreduce/v2/app/webapp/dao/JobInfo.java    |     6 +
 .../hadoop/mapreduce/jobhistory/TestEvents.java |   151 +-
 .../mapreduce/jobhistory/TestJobSummary.java    |    10 +-
 .../v2/api/records/TestTaskAttemptReport.java   |   131 +
 .../v2/api/records/TestTaskReport.java          |   139 +
 .../apache/hadoop/mapreduce/v2/app/MRApp.java   |    15 +
 .../mapreduce/v2/app/TestFetchFailure.java      |    31 +-
 .../mapreduce/v2/app/TestMRAppMaster.java       |    88 +-
 .../mapreduce/v2/app/job/impl/TestJobImpl.java  |   130 +-
 .../v2/app/job/impl/TestTaskAttempt.java        |   143 +-
 .../mapreduce/v2/app/rm/TestRMCommunicator.java |    99 +
 .../v2/app/rm/TestRMContainerAllocator.java     |   137 +-
 .../v2/app/webapp/TestAMWebServicesJobs.java    |     2 +-
 .../mapreduce/v2/app/webapp/TestBlocks.java     |    92 +-
 .../v2/api/records/TaskAttemptReport.java       |     3 +
 .../mapreduce/v2/api/records/TaskReport.java    |     5 +-
 .../impl/pb/TaskAttemptReportPBImpl.java        |    38 +-
 .../api/records/impl/pb/TaskReportPBImpl.java   |    38 +-
 .../v2/jobhistory/FileNameIndexUtils.java       |    17 +-
 .../mapreduce/v2/jobhistory/JHAdminConfig.java  |    14 +
 .../apache/hadoop/mapreduce/v2/util/MRApps.java |     5 -
 .../v2/jobhistory/TestFileNameIndexUtils.java   |    26 +
 .../apache/hadoop/mapred/FileInputFormat.java   |     2 +-
 .../apache/hadoop/mapred/LineRecordReader.java  |     4 +-
 .../java/org/apache/hadoop/mapred/Task.java     |     4 +-
 .../org/apache/hadoop/mapred/lib/Chain.java     |     2 +-
 .../hadoop/mapred/pipes/PipesPartitioner.java   |     6 +-
 .../hadoop/mapreduce/JobResourceUploader.java   |    38 +-
 .../apache/hadoop/mapreduce/MRJobConfig.java    |    20 +
 .../ClientDistributedCacheManager.java          |     1 +
 .../mapreduce/jobhistory/AMStartedEvent.java    |    24 +-
 .../mapreduce/jobhistory/AvroArrayUtils.java    |     2 +-
 .../mapreduce/jobhistory/EventReader.java       |    34 +-
 .../mapreduce/jobhistory/EventWriter.java       |    70 +-
 .../mapreduce/jobhistory/JobFinishedEvent.java  |    39 +-
 .../mapreduce/jobhistory/JobHistoryParser.java  |     2 +-
 .../jobhistory/JobInfoChangeEvent.java          |    12 +-
 .../mapreduce/jobhistory/JobInitedEvent.java    |    24 +-
 .../jobhistory/JobPriorityChangeEvent.java      |    10 +-
 .../jobhistory/JobStatusChangedEvent.java       |     8 +-
 .../mapreduce/jobhistory/JobSubmittedEvent.java |    62 +-
 .../JobUnsuccessfulCompletionEvent.java         |     4 +-
 .../jobhistory/MapAttemptFinishedEvent.java     |    66 +-
 .../jobhistory/ReduceAttemptFinishedEvent.java  |    70 +-
 .../jobhistory/TaskAttemptFinishedEvent.java    |    34 +-
 .../jobhistory/TaskAttemptStartedEvent.java     |    49 +-
 .../TaskAttemptUnsuccessfulCompletionEvent.java |    66 +-
 .../mapreduce/jobhistory/TaskFailedEvent.java   |    32 +-
 .../mapreduce/jobhistory/TaskFinishedEvent.java |    26 +-
 .../mapreduce/jobhistory/TaskStartedEvent.java  |    20 +-
 .../mapreduce/jobhistory/TaskUpdatedEvent.java  |    10 +-
 .../mapreduce/lib/input/FileInputFormat.java    |     3 +-
 .../mapreduce/lib/input/LineRecordReader.java   |     3 +-
 .../lib/input/UncompressedSplitLineReader.java  |   125 +
 .../lib/output/FileOutputCommitter.java         |     2 +-
 .../hadoop/mapreduce/task/reduce/Fetcher.java   |     1 +
 .../task/reduce/IFileWrappedMapOutput.java      |    66 +
 .../task/reduce/InMemoryMapOutput.java          |    26 +-
 .../mapreduce/task/reduce/LocalFetcher.java     |    15 +-
 .../mapreduce/task/reduce/MergeManagerImpl.java |     5 +-
 .../mapreduce/task/reduce/OnDiskMapOutput.java  |    33 +-
 .../task/reduce/ShuffleSchedulerImpl.java       |     7 +-
 .../src/main/resources/mapred-default.xml       |    76 +-
 .../src/site/markdown/EncryptedShuffle.md       |     8 +-
 .../src/site/markdown/MapReduceTutorial.md      |     2 +-
 .../src/site/markdown/MapredCommands.md         |    14 +-
 .../hadoop/mapred/TestLineRecordReader.java     |    77 +-
 .../TestClientDistributedCacheManager.java      |    28 +
 .../lib/input/TestLineRecordReader.java         |    79 +-
 .../mapreduce/task/reduce/TestFetcher.java      |    27 +-
 .../hadoop/mapreduce/v2/hs/CompletedTask.java   |     2 +-
 .../mapreduce/v2/hs/CompletedTaskAttempt.java   |     2 +-
 .../mapreduce/v2/hs/JobHistoryServer.java       |    14 +-
 .../mapreduce/v2/hs/server/HSAdminServer.java   |    43 +-
 .../mapreduce/v2/hs/webapp/HsAttemptsPage.java  |     7 +
 .../mapreduce/v2/hs/webapp/HsController.java    |    28 +-
 .../mapreduce/v2/hs/webapp/HsCountersPage.java  |     8 +-
 .../mapreduce/v2/hs/webapp/HsJobBlock.java      |     2 +-
 .../mapreduce/v2/hs/webapp/HsLogsPage.java      |    11 +-
 .../v2/hs/webapp/HsSingleCounterPage.java       |     8 +-
 .../mapreduce/v2/hs/webapp/HsTaskPage.java      |    11 +-
 .../hadoop/mapreduce/v2/hs/webapp/HsView.java   |    18 +-
 .../v2/hs/server/TestHSAdminServer.java         |    55 +
 .../mapreduce/v2/hs/webapp/TestBlocks.java      |     2 +-
 .../org/apache/hadoop/mapred/YARNRunner.java    |    30 +-
 .../java/org/apache/hadoop/hdfs/NNBench.java    |    16 +-
 .../hadoop/mapred/TestLocalJobSubmission.java   |     4 +-
 .../apache/hadoop/mapred/TestYARNRunner.java    |    16 +
 .../mapred/pipes/TestPipeApplication.java       |     5 +-
 .../hadoop-mapreduce-client-nativetask/pom.xml  |     2 +-
 .../src/CMakeLists.txt                          |   337 +-
 .../src/JNIFlags.cmake                          |   118 -
 .../hadoop/mapred/FadvisedChunkedFile.java      |     5 +-
 .../hadoop/mapred/FadvisedFileRegion.java       |     6 +-
 .../apache/hadoop/mapred/ShuffleHandler.java    |    18 +-
 .../hadoop/mapred/TestShuffleHandler.java       |   105 +
 .../hadoop/examples/BaileyBorweinPlouffe.java   |     2 +-
 .../apache/hadoop/examples/QuasiMonteCarlo.java |     4 +
 hadoop-mapreduce-project/pom.xml                |     4 +-
 hadoop-maven-plugins/pom.xml                    |     8 +
 .../hadoop/maven/plugin/protoc/ProtocMojo.java  |   188 +-
 hadoop-project-dist/pom.xml                     |    20 +-
 hadoop-project/pom.xml                          |    15 +-
 hadoop-project/src/site/site.xml                |     3 +-
 .../org/apache/hadoop/tools/HadoopArchives.java |   108 +-
 .../src/site/markdown/HadoopArchives.md.vm      |     2 +-
 .../apache/hadoop/tools/TestHadoopArchives.java |    26 +-
 hadoop-tools/hadoop-aws/pom.xml                 |     2 +-
 .../org/apache/hadoop/fs/s3/S3Credentials.java  |    10 +-
 .../org/apache/hadoop/fs/s3a/Constants.java     |     9 +-
 .../hadoop/fs/s3a/S3AFastOutputStream.java      |     2 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |    26 +-
 .../apache/hadoop/fs/s3a/S3AOutputStream.java   |    15 +-
 .../src/site/markdown/tools/hadoop-aws/index.md |     6 +
 .../apache/hadoop/fs/s3/TestS3Credentials.java  |   107 +-
 .../fs/azure/AzureNativeFileSystemStore.java    |    15 +-
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |    13 +-
 .../hadoop/fs/azure/PageBlobInputStream.java    |    32 +-
 .../hadoop/fs/azure/PageBlobOutputStream.java   |    10 +-
 .../hadoop/fs/azure/StorageInterface.java       |     6 +-
 .../hadoop/fs/azure/StorageInterfaceImpl.java   |     4 +-
 .../hadoop/fs/azure/MockStorageInterface.java   |     4 +-
 .../fs/azure/NativeAzureFileSystemBaseTest.java |    79 +-
 .../TestAzureFileSystemErrorConditions.java     |     1 +
 .../hadoop/fs/azure/TestBlobDataValidation.java |     1 +
 ...tiveAzureFileSystemContractPageBlobLive.java |    90 +
 .../fs/azure/TestWasbUriAndConfiguration.java   |    30 +-
 .../src/test/resources/azure-test.xml           |     4 +
 .../java/org/apache/hadoop/tools/DistCp.java    |     3 +-
 .../apache/hadoop/tools/DistCpConstants.java    |     2 +-
 .../apache/hadoop/tools/DistCpOptionSwitch.java |     5 +-
 .../org/apache/hadoop/tools/DistCpOptions.java  |     6 +-
 .../org/apache/hadoop/tools/OptionsParser.java  |     2 +-
 .../apache/hadoop/tools/mapred/CopyMapper.java  |    17 +-
 .../tools/mapred/RetriableFileCopyCommand.java  |     2 +-
 .../apache/hadoop/tools/util/DistCpUtils.java   |    41 +-
 .../hadoop/tools/util/ThrottledInputStream.java |     6 +-
 .../apache/hadoop/tools/TestOptionsParser.java  |    16 +-
 .../gridmix/DummyResourceCalculatorPlugin.java  |    42 +
 hadoop-tools/hadoop-openstack/pom.xml           |     4 +-
 .../fs/swift/snative/SwiftNativeFileSystem.java |     9 +
 .../fs/swift/TestSwiftFileSystemBasicOps.java   |     7 +
 hadoop-tools/hadoop-pipes/src/CMakeLists.txt    |    49 +-
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |     8 +-
 .../yarn/sls/scheduler/RMNodeWrapper.java       |     5 +
 .../sls/scheduler/ResourceSchedulerWrapper.java |    10 +
 .../apache/hadoop/record/BinaryRecordInput.java |     7 +-
 .../hadoop/record/BinaryRecordOutput.java       |    17 +-
 .../org/apache/hadoop/streaming/PipeMapRed.java |    19 +-
 .../org/apache/hadoop/streaming/StreamJob.java  |     5 +-
 .../hadoop/typedbytes/TypedBytesInput.java      |     8 +-
 .../hadoop/typedbytes/TypedBytesOutput.java     |     8 +-
 .../typedbytes/TypedBytesRecordInput.java       |     8 +-
 .../typedbytes/TypedBytesRecordOutput.java      |     8 +-
 .../typedbytes/TypedBytesWritableInput.java     |     8 +-
 .../typedbytes/TypedBytesWritableOutput.java    |     8 +-
 .../src/site/markdown/HadoopStreaming.md.vm     |     9 +
 hadoop-yarn-project/CHANGES.txt                 |   444 +-
 .../hadoop-yarn/bin/start-yarn.sh               |     7 +-
 .../hadoop-yarn/bin/stop-yarn.sh                |     7 +-
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |    61 +-
 .../hadoop-yarn/hadoop-yarn-api/pom.xml         |    34 +
 .../hadoop/yarn/api/ApplicationConstants.java   |    58 +-
 .../yarn/api/records/ApplicationReport.java     |    48 +
 .../hadoop/yarn/api/records/NodeState.java      |    10 +-
 .../yarn/api/records/QueueStatistics.java       |    36 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   112 +-
 .../yarn/server/api/ContainerContext.java       |    19 +
 .../api/ContainerInitializationContext.java     |     7 +
 .../server/api/ContainerTerminationContext.java |     7 +
 .../hadoop/yarn/server/api/ContainerType.java   |    34 +
 .../apache/hadoop/yarn/util/package-info.java   |     2 -
 .../src/main/proto/yarn_protos.proto            |    11 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   136 +
 .../distributedshell/ApplicationMaster.java     |    31 +-
 .../distributedshell/TestDSAppMaster.java       |    11 +-
 .../org/apache/hadoop/yarn/client/SCMAdmin.java |     2 +-
 .../api/async/impl/NMClientAsyncImpl.java       |     4 +-
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |     6 +-
 .../hadoop/yarn/client/cli/ClusterCLI.java      |     2 +-
 .../hadoop/yarn/client/cli/RMAdminCLI.java      |    32 +-
 .../apache/hadoop/yarn/client/cli/TopCLI.java   |    24 +-
 .../hadoop/yarn/client/ProtocolHATestBase.java  |    26 +-
 ...estApplicationMasterServiceProtocolOnHA.java |    10 +-
 .../hadoop/yarn/client/TestRMFailover.java      |    26 +-
 .../yarn/client/TestResourceTrackerOnHA.java    |     2 +-
 .../yarn/client/api/impl/TestYarnClient.java    |    10 +-
 .../hadoop/yarn/client/cli/TestClusterCLI.java  |     4 +-
 .../hadoop/yarn/client/cli/TestRMAdminCLI.java  |    17 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |     5 +-
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |     2 +-
 .../impl/pb/ApplicationReportPBImpl.java        |    49 +
 .../yarn/api/records/impl/pb/ProtoUtils.java    |    12 +
 .../api/records/impl/pb/QueueInfoPBImpl.java    |     2 +-
 .../records/impl/pb/QueueStatisticsPBImpl.java  |    36 +
 .../records/impl/pb/ResourceRequestPBImpl.java  |     5 +-
 .../impl/pb/SerializedExceptionPBImpl.java      |    28 +-
 .../apache/hadoop/yarn/client/ServerProxy.java  |    23 +-
 .../client/api/impl/TimelineClientImpl.java     |    28 +-
 .../yarn/client/api/impl/package-info.java      |     4 -
 .../hadoop/yarn/client/api/package-info.java    |     4 -
 .../hadoop/yarn/event/AsyncDispatcher.java      |    17 +-
 .../hadoop/yarn/factories/package-info.java     |     2 -
 .../yarn/factory/providers/package-info.java    |     2 -
 .../logaggregation/AggregatedLogFormat.java     |    83 +-
 .../nodelabels/CommonNodeLabelsManager.java     |    37 +-
 .../nodelabels/FileSystemNodeLabelsStore.java   |    47 +-
 .../yarn/security/ContainerTokenIdentifier.java |    43 +-
 .../state/InvalidStateTransitionException.java  |    51 +
 .../state/InvalidStateTransitonException.java   |    21 +-
 .../apache/hadoop/yarn/state/StateMachine.java  |     2 +-
 .../hadoop/yarn/state/StateMachineFactory.java  |    10 +-
 .../java/org/apache/hadoop/yarn/util/Apps.java  |    19 +-
 .../apache/hadoop/yarn/util/CpuTimeTracker.java |   100 -
 .../util/LinuxResourceCalculatorPlugin.java     |   352 +-
 .../yarn/util/ProcfsBasedProcessTree.java       |    34 +-
 .../org/apache/hadoop/yarn/util/RMHAUtils.java  |    33 +-
 .../yarn/util/ResourceCalculatorPlugin.java     |   116 +-
 .../yarn/util/WindowsBasedProcessTree.java      |     2 +-
 .../util/WindowsResourceCalculatorPlugin.java   |   152 +-
 .../resource/DominantResourceCalculator.java    |    15 +
 .../hadoop/yarn/util/resource/Resources.java    |    14 +-
 .../hadoop/yarn/webapp/YarnWebParams.java       |     3 +
 .../yarn/webapp/view/TwoColumnLayout.java       |     2 +-
 .../main/proto/server/yarn_security_token.proto |    69 -
 .../src/main/proto/yarn_security_token.proto    |    70 +
 .../resources/webapps/static/yarn.dt.plugins.js |    16 +-
 .../src/main/resources/yarn-default.xml         |   600 +-
 .../hadoop/yarn/api/TestApplicatonReport.java   |     4 +-
 .../impl/pb/TestSerializedExceptionPBImpl.java  |    23 +-
 .../hadoop/yarn/conf/TestYarnConfiguration.java |    22 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   106 -
 .../hadoop/yarn/event/DrainDispatcher.java      |    11 +-
 .../hadoop/yarn/event/TestAsyncDispatcher.java  |    79 +
 .../nodelabels/TestCommonNodeLabelsManager.java |    17 +-
 .../yarn/security/TestYARNTokenIdentifier.java  |    53 +
 .../org/apache/hadoop/yarn/util/TestApps.java   |    61 +
 .../util/TestLinuxResourceCalculatorPlugin.java |   235 -
 .../util/TestResourceCalculatorProcessTree.java |     2 +-
 .../hadoop/yarn/util/TestWebAppUtils.java       |    81 -
 .../TestWindowsResourceCalculatorPlugin.java    |    86 -
 .../util/resource/TestResourceCalculator.java   |   125 +
 .../yarn/webapp/util/TestWebAppUtils.java       |    52 +
 .../pom.xml                                     |     2 +-
 .../ApplicationHistoryClientService.java        |     9 +-
 .../ApplicationHistoryManager.java              |    16 +-
 .../ApplicationHistoryManagerImpl.java          |     4 +-
 ...pplicationHistoryManagerOnTimelineStore.java |    63 +-
 .../ApplicationHistoryServer.java               |    14 +-
 .../webapp/AHSView.java                         |     2 +-
 .../server/timeline/TimelineDataManager.java    |   123 +
 .../timeline/TimelineDataManagerMetrics.java    |   174 +
 ...TimelineAuthenticationFilterInitializer.java |     5 +-
 .../TestApplicationHistoryClientService.java    |    97 +-
 ...pplicationHistoryManagerOnTimelineStore.java |    34 +-
 .../webapp/TestAHSWebServices.java              |    64 +-
 .../timeline/TestTimelineDataManager.java       |     1 +
 .../TestTimelineAuthenticationFilter.java       |    11 +
 .../hadoop/yarn/server/api/ResourceTracker.java |    16 +-
 .../pb/client/ResourceTrackerPBClientImpl.java  |    18 +
 .../service/ResourceTrackerPBServiceImpl.java   |    27 +-
 .../UnRegisterNodeManagerRequest.java           |    38 +
 .../UnRegisterNodeManagerResponse.java          |    30 +
 .../pb/UnRegisterNodeManagerRequestPBImpl.java  |   108 +
 .../pb/UnRegisterNodeManagerResponsePBImpl.java |    70 +
 .../yarn/server/api/records/NodeStatus.java     |    59 +-
 .../server/api/records/ResourceUtilization.java |   133 +
 .../api/records/impl/pb/NodeStatusPBImpl.java   |    56 +-
 .../impl/pb/ResourceUtilizationPBImpl.java      |   104 +
 .../yarn/server/api/records/package-info.java   |    19 +
 .../metrics/ApplicationMetricsConstants.java    |     7 +
 .../hadoop/yarn/server/utils/BuilderUtils.java  |     4 +-
 .../server/utils/YarnServerBuilderUtils.java    |    11 +-
 .../hadoop/yarn/server/webapp/AppBlock.java     |    24 +-
 .../hadoop/yarn/server/webapp/AppsBlock.java    |    49 +-
 .../hadoop/yarn/server/webapp/WebPageUtils.java |    10 +-
 .../hadoop/yarn/server/webapp/WebServices.java  |    53 +-
 .../hadoop/yarn/server/webapp/dao/AppInfo.java  |    35 +-
 .../src/main/proto/ResourceTracker.proto        |     1 +
 .../main/proto/yarn_server_common_protos.proto  |     8 +
 .../yarn_server_common_service_protos.proto     |     7 +
 .../yarn/TestResourceTrackerPBClientImpl.java   |    34 +-
 .../apache/hadoop/yarn/TestYSCRPCFactories.java |    10 +-
 .../hadoop/yarn/TestYarnServerApiClasses.java   |    12 +
 .../hadoop-yarn-server-nodemanager/pom.xml      |     2 +-
 .../src/CMakeLists.txt                          |    23 +-
 .../server/nodemanager/ContainerExecutor.java   |    54 +-
 .../hadoop/yarn/server/nodemanager/Context.java |     2 +
 .../nodemanager/DefaultContainerExecutor.java   |     2 +-
 .../nodemanager/DockerContainerExecutor.java    |     2 +-
 .../nodemanager/LinuxContainerExecutor.java     |   226 +-
 .../nodemanager/LocalDirsHandlerService.java    |    59 +-
 .../yarn/server/nodemanager/NodeManager.java    |    51 +-
 .../server/nodemanager/NodeResourceMonitor.java |    10 +-
 .../nodemanager/NodeResourceMonitorImpl.java    |   140 +
 .../nodemanager/NodeStatusUpdaterImpl.java      |    83 +-
 .../containermanager/AuxServices.java           |     6 +-
 .../application/ApplicationImpl.java            |     4 +-
 .../container/ContainerImpl.java                |    11 +-
 .../launcher/ContainerLaunch.java               |    24 +-
 .../launcher/RecoveredContainerLaunch.java      |     3 +-
 .../linux/privileged/PrivilegedOperation.java   |    47 +-
 .../PrivilegedOperationException.java           |    30 +-
 .../privileged/PrivilegedOperationExecutor.java |    32 +-
 .../linux/resources/CGroupsHandler.java         |     8 +
 .../linux/resources/CGroupsHandlerImpl.java     |    12 +-
 .../runtime/DefaultLinuxContainerRuntime.java   |   148 +
 .../DelegatingLinuxContainerRuntime.java        |   110 +
 .../runtime/DockerLinuxContainerRuntime.java    |   273 +
 .../linux/runtime/LinuxContainerRuntime.java    |    38 +
 .../runtime/LinuxContainerRuntimeConstants.java |    69 +
 .../linux/runtime/docker/DockerClient.java      |    82 +
 .../linux/runtime/docker/DockerCommand.java     |    66 +
 .../linux/runtime/docker/DockerLoadCommand.java |    30 +
 .../linux/runtime/docker/DockerRunCommand.java  |   107 +
 .../localizer/LocalizedResource.java            |     4 +-
 .../localizer/ResourceLocalizationService.java  |    31 +-
 .../logaggregation/AppLogAggregatorImpl.java    |    10 +-
 .../monitor/ContainersMonitor.java              |     3 +-
 .../monitor/ContainersMonitorImpl.java          |    47 +-
 .../runtime/ContainerExecutionException.java    |    85 +
 .../runtime/ContainerRuntime.java               |    50 +
 .../runtime/ContainerRuntimeConstants.java      |    33 +
 .../runtime/ContainerRuntimeContext.java        |   105 +
 .../executor/ContainerLivenessContext.java      |    13 +
 .../executor/ContainerReacquisitionContext.java |    13 +
 .../executor/ContainerSignalContext.java        |    13 +
 .../executor/ContainerStartContext.java         |    23 +-
 .../nodemanager/metrics/NodeManagerMetrics.java |    13 +-
 .../util/CgroupsLCEResourcesHandler.java        |    19 +-
 .../util/NodeManagerHardwareUtils.java          |   238 +-
 .../nodemanager/webapp/ContainerLogsUtils.java  |     2 +-
 .../server/nodemanager/webapp/NodePage.java     |     8 +-
 .../server/nodemanager/webapp/dao/NodeInfo.java |     6 +
 .../container-executor/impl/configuration.c     |    30 +-
 .../container-executor/impl/configuration.h     |     2 +
 .../impl/container-executor.c                   |   466 +-
 .../impl/container-executor.h                   |    29 +-
 .../main/native/container-executor/impl/main.c  |    97 +-
 .../test/test-container-executor.c              |    56 +-
 .../server/nodemanager/LocalRMInterface.java    |    10 +
 .../nodemanager/MockNodeStatusUpdater.java      |     9 +
 .../nodemanager/TestContainerExecutor.java      |    56 +-
 .../TestLinuxContainerExecutorWithMocks.java    |   131 +-
 .../TestLocalDirsHandlerService.java            |     2 +-
 .../nodemanager/TestNodeResourceMonitor.java    |    35 +
 .../nodemanager/TestNodeStatusUpdater.java      |    78 +
 .../TestNodeStatusUpdaterForLabels.java         |     8 +
 .../containermanager/TestNMProxy.java           |    99 +-
 .../launcher/TestContainerLaunch.java           |    10 +-
 .../TestPrivilegedOperationExecutor.java        |    14 +-
 .../runtime/TestDockerContainerRuntime.java     |   219 +
 .../TestResourceLocalizationService.java        |    10 +-
 .../TestLogAggregationService.java              |    90 +-
 .../monitor/TestContainersMonitor.java          |     2 +-
 .../util/TestCgroupsLCEResourcesHandler.java    |     2 +
 .../util/TestNodeManagerHardwareUtils.java      |   148 +-
 .../webapp/TestContainerLogsPage.java           |    70 +-
 .../nodemanager/webapp/TestNMWebServices.java   |     2 +-
 .../hadoop-yarn-server-resourcemanager/pom.xml  |    14 +-
 .../server/resourcemanager/AdminService.java    |    19 +-
 .../ApplicationMasterService.java               |    19 +-
 .../server/resourcemanager/ClientRMService.java |    11 +-
 .../server/resourcemanager/ClusterMetrics.java  |    14 +
 .../resourcemanager/NodesListManager.java       |    28 +-
 .../resourcemanager/RMActiveServiceContext.java |    30 -
 .../server/resourcemanager/RMAppManager.java    |    20 +-
 .../server/resourcemanager/RMContextImpl.java   |    26 +-
 .../server/resourcemanager/ResourceManager.java |    79 +-
 .../resourcemanager/ResourceTrackerService.java |    46 +-
 .../resourcemanager/amlauncher/AMLauncher.java  |    29 +-
 .../amlauncher/ApplicationMasterLauncher.java   |    30 +-
 .../metrics/ApplicationCreatedEvent.java        |    25 +-
 .../metrics/SystemMetricsPublisher.java         |    11 +-
 .../monitor/SchedulingEditPolicy.java           |     6 +-
 .../monitor/SchedulingMonitor.java              |     3 +-
 .../ProportionalCapacityPreemptionPolicy.java   |    63 +-
 .../nodelabels/RMNodeLabelsManager.java         |    13 +-
 .../recovery/FileSystemRMStateStore.java        |   229 +-
 .../recovery/LeveldbRMStateStore.java           |   114 +-
 .../recovery/MemoryRMStateStore.java            |    57 +
 .../recovery/NullRMStateStore.java              |    23 +
 .../resourcemanager/recovery/RMStateStore.java  |   171 +-
 .../recovery/RMStateStoreEventType.java         |     5 +-
 .../RMStateStoreStoreReservationEvent.java      |    56 +
 .../recovery/ZKRMStateStore.java                |   881 +-
 .../reservation/AbstractReservationSystem.java  |     2 +
 .../reservation/GreedyReservationAgent.java     |   385 -
 .../reservation/InMemoryPlan.java               |    22 +-
 .../InMemoryReservationAllocation.java          |    30 +-
 .../resourcemanager/reservation/Plan.java       |     2 +-
 .../reservation/PlanContext.java                |     2 +
 .../resourcemanager/reservation/PlanView.java   |    40 +-
 .../resourcemanager/reservation/Planner.java    |    47 -
 .../RLESparseResourceAllocation.java            |    98 +-
 .../reservation/ReservationAgent.java           |    72 -
 .../reservation/ReservationAllocation.java      |     3 +-
 .../ReservationSchedulerConfiguration.java      |    16 +-
 .../reservation/ReservationSystem.java          |    14 +-
 .../reservation/ReservationSystemUtil.java      |   155 +
 .../reservation/SimpleCapacityReplanner.java    |   113 -
 .../planning/AlignedPlannerWithGreedy.java      |   123 +
 .../planning/GreedyReservationAgent.java        |    97 +
 .../reservation/planning/IterativePlanner.java  |   338 +
 .../reservation/planning/Planner.java           |    49 +
 .../reservation/planning/PlanningAlgorithm.java |   207 +
 .../reservation/planning/ReservationAgent.java  |    73 +
 .../planning/SimpleCapacityReplanner.java       |   118 +
 .../reservation/planning/StageAllocator.java    |    55 +
 .../planning/StageAllocatorGreedy.java          |   152 +
 .../planning/StageAllocatorLowCostAligned.java  |   360 +
 .../planning/StageEarliestStart.java            |    46 +
 .../planning/StageEarliestStartByDemand.java    |   106 +
 .../StageEarliestStartByJobArrival.java         |    39 +
 .../planning/TryManyReservationAgents.java      |   114 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |    44 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |    22 +-
 .../rmcontainer/RMContainerImpl.java            |    76 +-
 .../server/resourcemanager/rmnode/RMNode.java   |     2 +
 .../resourcemanager/rmnode/RMNodeEventType.java |     1 +
 .../resourcemanager/rmnode/RMNodeImpl.java      |    86 +-
 .../scheduler/AbstractYarnScheduler.java        |    61 +-
 .../scheduler/AppSchedulingInfo.java            |     7 +-
 .../scheduler/ContainerPreemptEvent.java        |     8 +-
 .../scheduler/ContainerPreemptEventType.java    |    26 -
 .../server/resourcemanager/scheduler/Queue.java |     8 +
 .../scheduler/ResourceLimits.java               |    19 +-
 .../scheduler/SchedulerApplication.java         |    22 +
 .../scheduler/SchedulerApplicationAttempt.java  |    39 +-
 .../scheduler/YarnScheduler.java                |    31 +
 .../scheduler/capacity/AbstractCSQueue.java     |    45 +-
 .../scheduler/capacity/CSAssignment.java        |    23 +-
 .../capacity/CapacityHeadroomProvider.java      |    16 +-
 .../scheduler/capacity/CapacityScheduler.java   |   246 +-
 .../CapacitySchedulerConfiguration.java         |    20 +-
 .../capacity/CapacitySchedulerContext.java      |     2 -
 .../scheduler/capacity/LeafQueue.java           |   959 +-
 .../scheduler/capacity/ParentQueue.java         |    36 +-
 .../scheduler/capacity/ReservationQueue.java    |     4 -
 .../capacity/allocator/AllocationState.java     |    28 +
 .../capacity/allocator/ContainerAllocation.java |    89 +
 .../capacity/allocator/ContainerAllocator.java  |   150 +
 .../allocator/RegularContainerAllocator.java    |   698 +
 .../scheduler/common/fica/FiCaSchedulerApp.java |   132 +-
 .../scheduler/event/AppAddedSchedulerEvent.java |    28 +-
 .../event/ContainerRescheduledEvent.java        |    35 +
 .../scheduler/event/SchedulerEventType.java     |    10 +-
 .../scheduler/fair/FSAppAttempt.java            |   132 +-
 .../scheduler/fair/FSLeafQueue.java             |    52 +-
 .../scheduler/fair/FSOpDurations.java           |     6 +
 .../scheduler/fair/FSParentQueue.java           |   219 +-
 .../resourcemanager/scheduler/fair/FSQueue.java |    21 +
 .../scheduler/fair/FSSchedulerNode.java         |    11 +-
 .../scheduler/fair/FairScheduler.java           |   127 +-
 .../scheduler/fair/QueueManager.java            |     3 +-
 .../scheduler/fair/SchedulingPolicy.java        |    11 +
 .../DominantResourceFairnessPolicy.java         |    18 +-
 .../fair/policies/FairSharePolicy.java          |    11 +-
 .../scheduler/fair/policies/FifoPolicy.java     |    15 +-
 .../scheduler/fifo/FifoScheduler.java           |    15 +
 .../AbstractComparatorOrderingPolicy.java       |    31 +
 .../scheduler/policy/FairOrderingPolicy.java    |    11 +-
 .../scheduler/policy/FifoComparator.java        |    11 +-
 .../scheduler/policy/FifoOrderingPolicy.java    |     6 +-
 .../scheduler/policy/OrderingPolicy.java        |     5 +
 .../scheduler/policy/SchedulableEntity.java     |     5 +
 .../security/RMContainerTokenSecretManager.java |    13 +-
 .../webapp/CapacitySchedulerPage.java           |    12 +-
 .../webapp/DefaultSchedulerPage.java            |     4 +-
 .../webapp/FairSchedulerAppsBlock.java          |     6 +
 .../webapp/FairSchedulerPage.java               |    10 +-
 .../webapp/MetricsOverviewTable.java            |     2 +
 .../resourcemanager/webapp/NodesPage.java       |     1 +
 .../resourcemanager/webapp/RMAppsBlock.java     |    25 +-
 .../resourcemanager/webapp/RMWebServices.java   |    27 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |    19 +-
 .../webapp/dao/CapacitySchedulerQueueInfo.java  |    18 +
 .../webapp/dao/ClusterMetricsInfo.java          |     8 +-
 .../webapp/dao/FairSchedulerQueueInfo.java      |    28 +-
 .../yarn_server_resourcemanager_recovery.proto  |    18 +-
 .../server/resourcemanager/Application.java     |    12 +
 .../yarn/server/resourcemanager/HATestUtil.java |    38 +
 .../yarn/server/resourcemanager/MockAM.java     |    14 +-
 .../yarn/server/resourcemanager/MockNodes.java  |     5 +
 .../yarn/server/resourcemanager/MockRM.java     |    57 +-
 .../server/resourcemanager/TestAppManager.java  |     6 +-
 .../TestApplicationMasterLauncher.java          |     7 +-
 .../resourcemanager/TestClientRMService.java    |     1 +
 .../resourcemanager/TestRMAdminService.java     |    49 +-
 .../resourcemanager/TestRMDispatcher.java       |    79 +
 .../resourcemanager/TestRMEmbeddedElector.java  |    28 +-
 .../resourcemanager/TestRMNodeTransitions.java  |   112 +-
 .../server/resourcemanager/TestRMRestart.java   |     8 +-
 .../resourcemanager/TestResourceManager.java    |     1 +
 .../TestResourceTrackerService.java             |   123 +-
 .../TestWorkPreservingRMRestart.java            |    10 +-
 .../applicationsmanager/TestAMRestart.java      |     2 +-
 .../TestRMAppLogAggregationStatus.java          |     4 +-
 .../metrics/TestSystemMetricsPublisher.java     |    53 +
 ...estProportionalCapacityPreemptionPolicy.java |   308 +-
 ...pacityPreemptionPolicyForNodePartitions.java |   135 +-
 .../nodelabels/TestRMNodeLabelsManager.java     |    48 +-
 .../recovery/RMStateStoreTestBase.java          |   189 +-
 .../recovery/TestFSRMStateStore.java            |    22 +-
 .../recovery/TestLeveldbRMStateStore.java       |     6 +
 .../resourcemanager/recovery/TestProtos.java    |    36 +
 .../recovery/TestZKRMStateStore.java            |    84 +-
 .../recovery/TestZKRMStateStorePerf.java        |    12 +-
 .../TestZKRMStateStoreZKClientConnections.java  |   181 +-
 .../reservation/ReservationSystemTestUtil.java  |   197 +-
 .../reservation/TestCapacityOverTimePolicy.java |    18 +-
 .../TestCapacityReservationSystem.java          |    94 -
 .../TestCapacitySchedulerPlanFollower.java      |    12 +-
 .../reservation/TestFairReservationSystem.java  |   128 -
 .../TestFairSchedulerPlanFollower.java          |    72 +-
 .../reservation/TestGreedyReservationAgent.java |   604 -
 .../reservation/TestInMemoryPlan.java           |    39 +-
 .../TestInMemoryReservationAllocation.java      |    65 +-
 .../reservation/TestNoOverCommitPolicy.java     |     1 +
 .../TestRLESparseResourceAllocation.java        |    84 +-
 .../reservation/TestReservationSystem.java      |   213 +
 .../TestSchedulerPlanFollowerBase.java          |     1 +
 .../TestSimpleCapacityReplanner.java            |   161 -
 .../planning/TestAlignedPlanner.java            |   820 +
 .../planning/TestGreedyReservationAgent.java    |   611 +
 .../planning/TestSimpleCapacityReplanner.java   |   170 +
 .../resourcetracker/TestNMReconnect.java        |    67 +-
 .../rmapp/TestNodesListManager.java             |   162 +
 .../rmapp/TestRMAppTransitions.java             |     6 +-
 .../attempt/TestRMAppAttemptTransitions.java    |     8 +-
 .../rmcontainer/TestRMContainerImpl.java        |    79 +-
 .../scheduler/TestAbstractYarnScheduler.java    |   164 +-
 .../scheduler/TestSchedulerUtils.java           |    47 +
 .../capacity/TestApplicationLimits.java         |    34 +-
 .../capacity/TestApplicationPriority.java       |   570 +
 .../capacity/TestCapacityScheduler.java         |   276 +-
 .../TestCapacitySchedulerNodeLabelUpdate.java   |    64 +
 .../scheduler/capacity/TestChildQueueOrder.java |     3 +-
 .../capacity/TestContainerAllocation.java       |    90 +-
 .../scheduler/capacity/TestLeafQueue.java       |   547 +-
 .../scheduler/capacity/TestParentQueue.java     |     2 -
 .../scheduler/capacity/TestQueueParsing.java    |    71 +
 .../capacity/TestReservationQueue.java          |    26 +-
 .../scheduler/capacity/TestReservations.java    |   119 +-
 .../scheduler/capacity/TestUtils.java           |    30 +-
 .../scheduler/fair/FairSchedulerTestBase.java   |    31 +-
 .../scheduler/fair/TestFSLeafQueue.java         |    64 +
 .../scheduler/fair/TestFairScheduler.java       |   493 +-
 .../scheduler/fair/TestSchedulingUpdate.java    |   135 +
 .../scheduler/fifo/TestFifoScheduler.java       |    23 +-
 .../scheduler/policy/MockSchedulableEntity.java |    13 +-
 .../security/TestDelegationTokenRenewer.java    |    29 +-
 .../resourcemanager/webapp/TestNodesPage.java   |     4 +-
 .../webapp/TestRMWebServices.java               |    21 +-
 .../webapp/TestRMWebServicesApps.java           |    36 +-
 .../TestRMWebServicesAppsModification.java      |     2 +-
 .../webapp/TestRMWebServicesCapacitySched.java  |     4 +-
 .../webapp/TestRMWebServicesNodeLabels.java     |    51 +-
 .../webapp/dao/TestFairSchedulerQueueInfo.java  |    59 +
 .../hadoop/yarn/server/MiniYARNCluster.java     |    32 +-
 .../hadoop/yarn/server/TestMiniYarnCluster.java |    37 +
 .../yarn/server/webproxy/AppReportFetcher.java  |    79 +-
 .../yarn/server/webproxy/WebAppProxyServer.java |    32 +-
 .../server/webproxy/WebAppProxyServlet.java     |     5 +-
 .../server/webproxy/TestAppReportFetcher.java   |   117 +
 .../server/webproxy/TestWebAppProxyServlet.java |    57 +
 .../amfilter/TestAmFilterInitializer.java       |    47 +
 .../src/site/markdown/NodeManager.md            |    49 +-
 .../src/site/markdown/NodeManagerRestart.md     |    53 -
 .../src/site/markdown/ResourceManagerHA.md      |    16 +-
 .../src/site/markdown/ResourceManagerRest.md    |    20 +-
 .../src/site/markdown/ResourceManagerRestart.md |    32 +-
 .../src/site/markdown/TimelineServer.md         |    77 +-
 .../hadoop-yarn-site/src/site/markdown/YARN.md  |    22 +-
 .../src/site/markdown/YarnCommands.md           |    10 +-
 hadoop-yarn-project/pom.xml                     |     2 +-
 pom.xml                                         |     2 +
 1193 files changed, 53822 insertions(+), 50194 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
index 33c2ed9,aeaa980..03b4a1c
--- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
@@@ -53,6 -63,48 +63,49 @@@ http://maven.apache.org/xsd/maven-4.0.0
            </excludes>
          </configuration>
        </plugin>
+       <plugin>
+         <groupId>org.apache.hadoop</groupId>
+         <artifactId>hadoop-maven-plugins</artifactId>
+         <executions>
+           <execution>
+             <id>compile-protoc</id>
+             <phase>generate-sources</phase>
+             <goals>
+               <goal>protoc</goal>
+             </goals>
+             <configuration>
+               <protocVersion>${protobuf.version}</protocVersion>
+               <protocCommand>${protoc.path}</protocCommand>
+               <imports>
+                 <param>${basedir}/../../hadoop-common-project/hadoop-common/src/main/proto</param>
+                 <param>${basedir}/src/main/proto</param>
+               </imports>
+               <source>
+                 <directory>${basedir}/src/main/proto</directory>
+                 <includes>
+                   <include>ClientDatanodeProtocol.proto</include>
+                   <include>ClientNamenodeProtocol.proto</include>
+                   <include>acl.proto</include>
+                   <include>xattr.proto</include>
+                   <include>datatransfer.proto</include>
+                   <include>hdfs.proto</include>
+                   <include>encryption.proto</include>
+                   <include>inotify.proto</include>
++                  <include>erasurecoding.proto</include>
+                 </includes>
+               </source>
+               <output>${project.build.directory}/generated-sources/java</output>
+             </configuration>
+           </execution>
+         </executions>
+       </plugin>
+       <plugin>
+         <groupId>org.apache.maven.plugins</groupId>
+         <artifactId>maven-javadoc-plugin</artifactId>
+         <configuration>
+           <excludePackageNames>org.apache.hadoop.hdfs.protocol.proto</excludePackageNames>
+         </configuration>
+       </plugin>
      </plugins>
    </build>
  </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------


[22/50] [abbrv] hadoop git commit: HDFS-8975. Erasure coding : Fix random failure in TestSafeModeWithStripedFile (Contributed by J.Andreina)

Posted by wa...@apache.org.
HDFS-8975. Erasure coding : Fix random failure in TestSafeModeWithStripedFile (Contributed by J.Andreina)


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

Branch: refs/heads/trunk
Commit: ce02b5532c3d506f8eee7af268216804815fa055
Parents: 96d6b51
Author: Vinayakumar B <vi...@apache.org>
Authored: Sat Sep 12 13:29:25 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Sat Sep 12 13:29:25 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        | 4 ++++
 .../org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java     | 5 +++++
 2 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce02b553/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index f49a974..47bab0b 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -418,3 +418,7 @@
 
     HDFS-8853. Erasure Coding: Provide ECSchema validation when setting EC
     policy. (andreina via zhz)
+
+    HDFS-8975. Erasure coding : Fix random failure in TestSafeModeWithStripedFile
+    (J.Andreina via vinayakumarb)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce02b553/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
index 318eb9f..9ab0834 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
@@ -53,6 +53,7 @@ public class TestSafeModeWithStripedFile {
   public void setup() throws IOException {
     conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 100);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
     cluster.waitActive();
@@ -124,6 +125,7 @@ public class TestSafeModeWithStripedFile {
     // so the safe blocks count doesn't increment.
     for (int i = 0; i < minStorages - 1; i++) {
       cluster.restartDataNode(dnprops.remove(0));
+      cluster.waitActive();
       cluster.triggerBlockReports();
       assertEquals(0, NameNodeAdapter.getSafeModeSafeBlocks(nn));
     }
@@ -131,17 +133,20 @@ public class TestSafeModeWithStripedFile {
     // the block of smallFile reaches minStorages,
     // so the safe blocks count increment.
     cluster.restartDataNode(dnprops.remove(0));
+    cluster.waitActive();
     cluster.triggerBlockReports();
     assertEquals(1, NameNodeAdapter.getSafeModeSafeBlocks(nn));
 
     // the 2 blocks of bigFile need DATA_BLK_NUM storages to be safe
     for (int i = minStorages; i < DATA_BLK_NUM - 1; i++) {
       cluster.restartDataNode(dnprops.remove(0));
+      cluster.waitActive();
       cluster.triggerBlockReports();
       assertTrue(nn.isInSafeMode());
     }
 
     cluster.restartDataNode(dnprops.remove(0));
+    cluster.waitActive();
     cluster.triggerBlockReports();
     assertFalse(nn.isInSafeMode());
   }