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 zh...@apache.org on 2015/05/11 21:23:01 UTC

[01/50] hadoop git commit: HDFS-8027. Erasure Coding: Update CHANGES-HDFS-7285.txt with branch commits (Vinayakumar B)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 d96c64cb3 -> 10e2d80c8 (forced update)


HDFS-8027. Erasure Coding: Update CHANGES-HDFS-7285.txt with branch commits (Vinayakumar B)


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

Branch: refs/heads/HDFS-7285
Commit: e594f848cfd1a8ac49af4ab941b2cb6577f23e33
Parents: 11fef75
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Apr 15 12:23:07 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:13 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt             | 15 +++++++++++++++
 1 file changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e594f848/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 07bbd4a..9fdac98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -51,11 +51,20 @@
     HDFS-7839. Erasure coding: implement facilities in NameNode to create and
     manage EC zones (Zhe Zhang)
 
+    HDFS-7969. Erasure coding: NameNode support for lease recovery of striped
+    block groups. (Zhe Zhang)
+
+    HDFS-7782. Erasure coding: pread from files in striped layout.
+    (Zhe Zhang and Jing Zhao via Zhe Zhang)
+
     HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from
     NameNode (vinayakumarb)
     
     HDFS-8074. Define a system-wide default EC schema. (Kai Zheng)
 
+    HDFS-8077. Erasure coding: fix bugs in EC zone and symlinks.
+    (Jing Zhao and Zhe Zhang via Jing Zhao)
+
     HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng)
 
     HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng)
@@ -63,5 +72,11 @@
     HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all
     ECSchemas loaded in Namenode. (vinayakumarb)
 
+    HDFS-8122. Erasure Coding: Support specifying ECSchema during creation of ECZone.
+    (Vinayakumar B via Zhe Zhang)
+
+    HDFS-8114. Erasure coding: Add auditlog FSNamesystem#createErasureCodingZone if this
+    operation fails. (Rakesh R via Zhe Zhang)
+
     HDFS-8123. Erasure Coding: Better to move EC related proto messages to a
     separate erasurecoding proto file (Rakesh R via vinayakumarb)
\ No newline at end of file


[34/50] hadoop git commit: HDFS-7949. WebImageViewer need support file size calculation with striped blocks. Contributed by Rakesh R.

Posted by zh...@apache.org.
HDFS-7949. WebImageViewer need support file size calculation with striped blocks. Contributed by Rakesh R.


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

Branch: refs/heads/HDFS-7285
Commit: 74dfef35852903f4770b0cdcab9c94dcaa801e18
Parents: 2bd2dd7
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri May 1 15:59:58 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:23 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../blockmanagement/BlockInfoStriped.java       |  27 +--
 .../tools/offlineImageViewer/FSImageLoader.java |  21 ++-
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  22 +++
 ...TestOfflineImageViewerWithStripedBlocks.java | 166 +++++++++++++++++++
 5 files changed, 212 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/74dfef35/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 596bbcf..145494f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -155,3 +155,6 @@
 
     HDFS-8308. Erasure Coding: NameNode may get blocked in waitForLoadingFSImage()
     when loading editlog. (jing9)
+
+    HDFS-7949. WebImageViewer need support file size calculation with striped 
+    blocks. (Rakesh R via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74dfef35/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 23e3153..f0e52e3 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
@@ -19,9 +19,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-
-import java.io.DataOutput;
-import java.io.IOException;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 
@@ -203,28 +201,9 @@ public class BlockInfoStriped extends BlockInfo {
     // 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.
-
-    // 0. Calculate the total bytes per stripes <Num Bytes per Stripes>
-    long numBytesPerStripe = dataBlockNum * BLOCK_STRIPED_CELL_SIZE;
-    if (getNumBytes() % numBytesPerStripe == 0) {
-      return getNumBytes() / dataBlockNum * getTotalBlockNum();
+    return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(),
+        dataBlockNum, parityBlockNum, BLOCK_STRIPED_CELL_SIZE);
     }
-    // 1. Calculate the number of stripes in this block group. <Num Stripes>
-    long numStripes = (getNumBytes() - 1) / numBytesPerStripe + 1;
-    // 2. Calculate the parity cell length in the last stripe. Note that the
-    //    size of parity cells should equal the size of the first cell, if it
-    //    is not full. <Last Stripe Parity Cell Length>
-    long lastStripeParityCellLen = Math.min(getNumBytes() % numBytesPerStripe,
-        BLOCK_STRIPED_CELL_SIZE);
-    // 3. Total consumed space is the total of
-    //     - The total of the full cells of data blocks and parity blocks.
-    //     - The remaining of data block which does not make a stripe.
-    //     - The last parity block cells. These size should be same
-    //       to the first cell in this stripe.
-    return getTotalBlockNum() * (BLOCK_STRIPED_CELL_SIZE * (numStripes - 1))
-        + getNumBytes() % numBytesPerStripe
-        + lastStripeParityCellLen * parityBlockNum;
-  }
 
   @Override
   public final boolean isStriped() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74dfef35/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
index 351ff03..42f6c0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 
 import java.io.BufferedInputStream;
-import java.io.EOFException;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -42,12 +41,15 @@ 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.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StripedBlockProto;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf;
 import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.web.JsonUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.LimitInputStream;
@@ -483,8 +485,21 @@ class FSImageLoader {
 
   static long getFileSize(FsImageProto.INodeSection.INodeFile f) {
     long size = 0;
-    for (HdfsProtos.BlockProto p : f.getBlocksList()) {
-      size += p.getNumBytes();
+    if (f.hasStripedBlocks()) {
+      List<StripedBlockProto> blocksList = f.getStripedBlocks().getBlocksList();
+      // Get total of actual data block size
+      for (StripedBlockProto p : blocksList) {
+        // Total usage by this striped blocks should be the total of data
+        // blocks and parity blocks
+        size += StripedBlockUtil.spaceConsumedByStripedBlock(p.getBlock()
+            .getNumBytes(), p.getDataBlockNum(), p.getParityBlockNum(),
+            HdfsConstants.BLOCK_STRIPED_CELL_SIZE);
+      }
+    } else {
+      for (HdfsProtos.BlockProto p : f.getBlocksList()) {
+        size += p.getNumBytes();
+      }
+
     }
     return size;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74dfef35/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 cb6d39a..b18e36f 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
@@ -232,6 +232,28 @@ public class StripedBlockUtil {
   }
 
   /**
+   * 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;
+  }
+
+  /**
    * This class represents the portion of I/O associated with each block in the
    * striped block group.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74dfef35/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
new file mode 100644
index 0000000..f3ef39a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
@@ -0,0 +1,166 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.tools.offlineImageViewer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+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.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.junit.AfterClass;
+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 MiniDFSCluster cluster;
+  private static DistributedFileSystem fs;
+  private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private static final int stripesPerBlock = 3;
+  private static final int blockSize = cellSize * stripesPerBlock;
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    int numDNs = dataBlocks + parityBlocks + 2;
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.waitActive();
+    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    fs = cluster.getFileSystem();
+    Path eczone = new Path("/eczone");
+    fs.mkdirs(eczone);
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testFileEqualToOneStripe() throws Exception {
+    int numBytes = cellSize;
+    testFileSize(numBytes);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileLessThanOneStripe() throws Exception {
+    int numBytes = cellSize - 100;
+    testFileSize(numBytes);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileHavingMultipleBlocks() throws Exception {
+    int numBytes = blockSize * 3;
+    testFileSize(numBytes);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileLargerThanABlockGroup1() throws IOException {
+    testFileSize(blockSize * dataBlocks + cellSize + 123);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileLargerThanABlockGroup2() throws IOException {
+    testFileSize(blockSize * dataBlocks * 3 + cellSize * dataBlocks + cellSize
+        + 123);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileFullBlockGroup() throws IOException {
+    testFileSize(blockSize * dataBlocks);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileMoreThanOneStripe() throws Exception {
+    int numBytes = blockSize + blockSize / 2;
+    testFileSize(numBytes);
+  }
+
+  private void testFileSize(int numBytes) throws IOException,
+      UnresolvedLinkException, SnapshotAccessControlException {
+    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    File orgFsimage = null;
+    Path file = new Path("/eczone/striped");
+    FSDataOutputStream out = fs.create(file, true);
+    byte[] bytes = DFSTestUtil.generateSequentialBytes(0, numBytes);
+    out.write(bytes);
+    out.close();
+
+    // Write results to the fsimage file
+    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
+    fs.saveNamespace();
+
+    // Determine location of fsimage file
+    orgFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil
+        .getFSImage(cluster.getNameNode()).getStorage().getStorageDir(0));
+    if (orgFsimage == null) {
+      throw new RuntimeException("Didn't generate or can't find fsimage");
+    }
+    FSImageLoader loader = FSImageLoader.load(orgFsimage.getAbsolutePath());
+    String fileStatus = loader.getFileStatus("/eczone/striped");
+    long expectedSpaceConsumed = StripedBlockUtil.spaceConsumedByStripedBlock(
+        bytes.length, HdfsConstants.NUM_DATA_BLOCKS,
+        HdfsConstants.NUM_PARITY_BLOCKS, HdfsConstants.BLOCK_STRIPED_CELL_SIZE);
+
+    // Verify space consumed present in BlockInfoStriped
+    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+    INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+    assertTrue("Invalid block size", fileNode.getBlocks().length > 0);
+    long actualSpaceConsumed = 0;
+    for (BlockInfo blockInfo : fileNode.getBlocks()) {
+      assertTrue("Didn't find block striped information",
+          blockInfo instanceof BlockInfoStriped);
+      BlockInfoStriped b = (BlockInfoStriped) blockInfo;
+      actualSpaceConsumed += b.spaceConsumed();
+    }
+
+    assertEquals("Wrongly computed file size contains striped blocks",
+        expectedSpaceConsumed, actualSpaceConsumed);
+
+    // Verify space consumed present in filestatus
+    String EXPECTED_FILE_SIZE = "\"length\":"
+        + String.valueOf(expectedSpaceConsumed);
+    assertTrue(
+        "Wrongly computed file size contains striped blocks, file status:"
+            + fileStatus + ". Expected file size is : " + EXPECTED_FILE_SIZE,
+        fileStatus.contains(EXPECTED_FILE_SIZE));
+  }
+}


[30/50] hadoop git commit: HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil. Contributed by Zhe Zhang.


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

Branch: refs/heads/HDFS-7285
Commit: d2e70cb996e09488c7497bd3fd0548c7211ab592
Parents: a71bfe0
Author: Zhe Zhang <zh...@apache.org>
Authored: Wed Apr 29 23:49:52 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:22 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../hadoop/hdfs/DFSStripedInputStream.java      | 111 +-----------
 .../hadoop/hdfs/util/StripedBlockUtil.java      | 174 +++++++++++++++++++
 .../hadoop/hdfs/TestPlanReadPortions.java       |  11 +-
 4 files changed, 186 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2e70cb9/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 6a9bdee..ca60487 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -146,3 +146,6 @@
 
     HDFS-8272. Erasure Coding: simplify the retry logic in DFSStripedInputStream 
     (stateful read). (Jing Zhao via Zhe Zhang)
+
+    HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil.
+    (Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2e70cb9/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 3da7306..0dc98fd 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
@@ -17,12 +17,14 @@
  */
 package org.apache.hadoop.hdfs;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.planReadPortions;
+
 import org.apache.hadoop.net.NetUtils;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
@@ -31,8 +33,6 @@ import org.apache.htrace.TraceScope;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Set;
 import java.util.Map;
 import java.util.HashMap;
@@ -69,59 +69,6 @@ import java.util.concurrent.Future;
  *   3. pread with decode support: TODO: will be supported after HDFS-7678
  *****************************************************************************/
 public class DFSStripedInputStream extends DFSInputStream {
-  /**
-   * This method plans the read portion from each block in the stripe
-   * @param dataBlkNum The number of data blocks in the striping group
-   * @param cellSize The size of each striping cell
-   * @param startInBlk Starting offset in the striped block
-   * @param len Length of the read request
-   * @param bufOffset  Initial offset in the result buffer
-   * @return array of {@link ReadPortion}, each representing the portion of I/O
-   *         for an individual block in the group
-   */
-  @VisibleForTesting
-  static ReadPortion[] planReadPortions(final int dataBlkNum,
-      final int cellSize, final long startInBlk, final int len, int bufOffset) {
-    ReadPortion[] results = new ReadPortion[dataBlkNum];
-    for (int i = 0; i < dataBlkNum; i++) {
-      results[i] = new ReadPortion();
-    }
-
-    // cellIdxInBlk is the index of the cell in the block
-    // E.g., cell_3 is the 2nd cell in blk_0
-    int cellIdxInBlk = (int) (startInBlk / (cellSize * dataBlkNum));
-
-    // blkIdxInGroup is the index of the block in the striped block group
-    // E.g., blk_2 is the 3rd block in the group
-    final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum);
-    results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk +
-        startInBlk % cellSize;
-    boolean crossStripe = false;
-    for (int i = 1; i < dataBlkNum; i++) {
-      if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) {
-        cellIdxInBlk++;
-        crossStripe = true;
-      }
-      results[(blkIdxInGroup + i) % dataBlkNum].startOffsetInBlock =
-          cellSize * cellIdxInBlk;
-    }
-
-    int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len);
-    results[blkIdxInGroup].offsetsInBuf.add(bufOffset);
-    results[blkIdxInGroup].lengths.add(firstCellLen);
-    results[blkIdxInGroup].readLength += firstCellLen;
-
-    int i = (blkIdxInGroup + 1) % dataBlkNum;
-    for (int done = firstCellLen; done < len; done += cellSize) {
-      ReadPortion rp = results[i];
-      rp.offsetsInBuf.add(done + bufOffset);
-      final int readLen = Math.min(len - done, cellSize);
-      rp.lengths.add(readLen);
-      rp.readLength += readLen;
-      i = (i + 1) % dataBlkNum;
-    }
-    return results;
-  }
 
   private static class ReaderRetryPolicy {
     private int fetchEncryptionKeyTimes = 1;
@@ -520,56 +467,4 @@ public class DFSStripedInputStream extends DFSInputStream {
     }
     throw new InterruptedException("let's retry");
   }
-
-
-  /**
-   * This class represents the portion of I/O associated with each block in the
-   * striped block group.
-   */
-  static class ReadPortion {
-    /**
-     * startOffsetInBlock
-     *     |
-     *     v
-     *     |<-lengths[0]->|<-  lengths[1]  ->|<-lengths[2]->|
-     * +------------------+------------------+----------------+
-     * |      cell_0      |      cell_3      |     cell_6     |  <- blk_0
-     * +------------------+------------------+----------------+
-     *   _/                \_______________________
-     *  |                                          |
-     *  v offsetsInBuf[0]                          v offsetsInBuf[1]
-     * +------------------------------------------------------+
-     * |  cell_0     |      cell_1 and cell_2      |cell_3 ...|   <- buf
-     * |  (partial)  |    (from blk_1 and blk_2)   |          |
-     * +------------------------------------------------------+
-     */
-    private long startOffsetInBlock = 0;
-    private int readLength = 0;
-    private final List<Integer> offsetsInBuf = new ArrayList<>();
-    private final List<Integer> lengths = new ArrayList<>();
-
-    int[] getOffsets() {
-      int[] offsets = new int[offsetsInBuf.size()];
-      for (int i = 0; i < offsets.length; i++) {
-        offsets[i] = offsetsInBuf.get(i);
-      }
-      return offsets;
-    }
-
-    int[] getLengths() {
-      int[] lens = new int[this.lengths.size()];
-      for (int i = 0; i < lens.length; i++) {
-        lens[i] = this.lengths.get(i);
-      }
-      return lens;
-    }
-
-    int getReadLength() {
-      return readLength;
-    }
-
-    long getStartOffsetInBlock() {
-      return startOffsetInBlock;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2e70cb9/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 d622d4d..cb6d39a 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
@@ -18,6 +18,7 @@
 
 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.protocol.DatanodeInfo;
@@ -27,6 +28,15 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 
 import com.google.common.base.Preconditions;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+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;
+
 /**
  * Utility class for analyzing striped block groups
  */
@@ -134,4 +144,168 @@ public class StripedBlockUtil {
         + offsetInBlk % cellSize; // partial cell
   }
 
+  /**
+   * This method plans the read portion from each block in the stripe
+   * @param dataBlkNum The number of data blocks in the striping group
+   * @param cellSize The size of each striping cell
+   * @param startInBlk Starting offset in the striped block
+   * @param len Length of the read request
+   * @param bufOffset  Initial offset in the result buffer
+   * @return array of {@link ReadPortion}, each representing the portion of I/O
+   *         for an individual block in the group
+   */
+  @VisibleForTesting
+  public static ReadPortion[] planReadPortions(final int dataBlkNum,
+      final int cellSize, final long startInBlk, final int len, int bufOffset) {
+    ReadPortion[] results = new ReadPortion[dataBlkNum];
+    for (int i = 0; i < dataBlkNum; i++) {
+      results[i] = new ReadPortion();
+    }
+
+    // cellIdxInBlk is the index of the cell in the block
+    // E.g., cell_3 is the 2nd cell in blk_0
+    int cellIdxInBlk = (int) (startInBlk / (cellSize * dataBlkNum));
+
+    // blkIdxInGroup is the index of the block in the striped block group
+    // E.g., blk_2 is the 3rd block in the group
+    final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum);
+    results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk +
+        startInBlk % cellSize;
+    boolean crossStripe = false;
+    for (int i = 1; i < dataBlkNum; i++) {
+      if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) {
+        cellIdxInBlk++;
+        crossStripe = true;
+      }
+      results[(blkIdxInGroup + i) % dataBlkNum].startOffsetInBlock =
+          cellSize * cellIdxInBlk;
+    }
+
+    int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len);
+    results[blkIdxInGroup].offsetsInBuf.add(bufOffset);
+    results[blkIdxInGroup].lengths.add(firstCellLen);
+    results[blkIdxInGroup].readLength += firstCellLen;
+
+    int i = (blkIdxInGroup + 1) % dataBlkNum;
+    for (int done = firstCellLen; done < len; done += cellSize) {
+      ReadPortion rp = results[i];
+      rp.offsetsInBuf.add(done + bufOffset);
+      final int readLen = Math.min(len - done, cellSize);
+      rp.lengths.add(readLen);
+      rp.readLength += readLen;
+      i = (i + 1) % dataBlkNum;
+    }
+    return results;
+  }
+
+  /**
+   * Get the next completed striped read task
+   *
+   * @return {@link StripedReadResult} 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 StripedReadResult getNextCompletedStripedRead(
+      CompletionService<Void> readService, Map<Future<Void>,
+      Integer> futures, final long threshold) throws InterruptedException {
+    Preconditions.checkArgument(!futures.isEmpty());
+    Preconditions.checkArgument(threshold > 0);
+    Future<Void> future = null;
+    try {
+      future = readService.poll(threshold, TimeUnit.MILLISECONDS);
+      if (future != null) {
+        future.get();
+        return new StripedReadResult(futures.remove(future),
+            StripedReadResult.SUCCESSFUL);
+      } else {
+        return new StripedReadResult(StripedReadResult.TIMEOUT);
+      }
+    } catch (ExecutionException e) {
+      return new StripedReadResult(futures.remove(future),
+          StripedReadResult.FAILED);
+    } catch (CancellationException e) {
+      return new StripedReadResult(futures.remove(future),
+          StripedReadResult.CANCELLED);
+    }
+  }
+
+  /**
+   * This class represents the portion of I/O associated with each block in the
+   * striped block group.
+   */
+  public static class ReadPortion {
+    /**
+     * startOffsetInBlock
+     *     |
+     *     v
+     *     |<-lengths[0]->|<-  lengths[1]  ->|<-lengths[2]->|
+     * +------------------+------------------+----------------+
+     * |      cell_0      |      cell_3      |     cell_6     |  <- blk_0
+     * +------------------+------------------+----------------+
+     *   _/                \_______________________
+     *  |                                          |
+     *  v offsetsInBuf[0]                          v offsetsInBuf[1]
+     * +------------------------------------------------------+
+     * |  cell_0     |      cell_1 and cell_2      |cell_3 ...|   <- buf
+     * |  (partial)  |    (from blk_1 and blk_2)   |          |
+     * +------------------------------------------------------+
+     */
+    public long startOffsetInBlock = 0;
+    public int readLength = 0;
+    public final List<Integer> offsetsInBuf = new ArrayList<>();
+    public final List<Integer> lengths = 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.lengths.size()];
+      for (int i = 0; i < lens.length; i++) {
+        lens[i] = this.lengths.get(i);
+      }
+      return lens;
+    }
+
+    public boolean containsReadPortion(ReadPortion rp) {
+      long end = startOffsetInBlock + readLength;
+      return startOffsetInBlock <= rp.startOffsetInBlock && end >=
+          rp.startOffsetInBlock + rp.readLength;
+    }
+  }
+
+  /**
+   * 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 StripedReadResult {
+    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 StripedReadResult(int state) {
+      Preconditions.checkArgument(state == TIMEOUT,
+          "Only timeout result should return negative index.");
+      this.index = -1;
+      this.state = state;
+    }
+
+    public StripedReadResult(int index, int state) {
+      Preconditions.checkArgument(state != TIMEOUT,
+          "Timeout result should return negative index.");
+      this.index = index;
+      this.state = state;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2e70cb9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
index cf84b30..3b5787a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
@@ -19,7 +19,8 @@ package org.apache.hadoop.hdfs;
 
 import org.junit.Test;
 
-import static org.apache.hadoop.hdfs.DFSStripedInputStream.ReadPortion;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion;
 import static org.junit.Assert.*;
 
 public class TestPlanReadPortions {
@@ -32,13 +33,13 @@ public class TestPlanReadPortions {
   private void testPlanReadPortions(int startInBlk, int length,
       int bufferOffset, int[] readLengths, int[] offsetsInBlock,
       int[][] bufferOffsets, int[][] bufferLengths) {
-    ReadPortion[] results = DFSStripedInputStream.planReadPortions(GROUP_SIZE,
+    ReadPortion[] results = StripedBlockUtil.planReadPortions(GROUP_SIZE,
         CELLSIZE, startInBlk, length, bufferOffset);
     assertEquals(GROUP_SIZE, results.length);
 
     for (int i = 0; i < GROUP_SIZE; i++) {
-      assertEquals(readLengths[i], results[i].getReadLength());
-      assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock());
+      assertEquals(readLengths[i], results[i].readLength);
+      assertEquals(offsetsInBlock[i], results[i].startOffsetInBlock);
       final int[] bOffsets = results[i].getOffsets();
       assertArrayEquals(bufferOffsets[i], bOffsets);
       final int[] bLengths = results[i].getLengths();
@@ -47,7 +48,7 @@ public class TestPlanReadPortions {
   }
 
   /**
-   * Test {@link DFSStripedInputStream#planReadPortions}
+   * Test {@link StripedBlockUtil#planReadPortions}
    */
   @Test
   public void testPlanReadPortions() {


[08/50] hadoop git commit: HADOOP-11841. Remove unused ecschema-def.xml files.

Posted by zh...@apache.org.
HADOOP-11841. Remove unused ecschema-def.xml files.


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

Branch: refs/heads/HDFS-7285
Commit: 44a30f92ef20eb6cb7b567dfc53f03b8db1b78e7
Parents: c1aa8f5
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Apr 17 16:07:07 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:15 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  2 ++
 .../src/main/conf/ecschema-def.xml              | 35 -------------------
 .../hadoop/fs/CommonConfigurationKeys.java      |  5 ---
 .../hadoop/io/erasurecode/SchemaLoader.java     | 36 +++++++++++---------
 .../hadoop/io/erasurecode/TestSchemaLoader.java | 12 ++-----
 5 files changed, 25 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/44a30f92/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index b850e11..9749270 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -42,3 +42,5 @@
     ( Kai Zheng via vinayakumarb )
   
     HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng)
+
+    HADOOP-11841. Remove unused ecschema-def.xml files.  (szetszwo)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44a30f92/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
deleted file mode 100644
index e36d386..0000000
--- a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
+++ /dev/null
@@ -1,35 +0,0 @@
-<?xml version="1.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.
-
--->
-
-<!--
-Please define your EC schemas here. Note, once these schemas are loaded
-and referenced by EC storage policies, any change to them will be ignored.
-You can modify and remove those not used yet, or add new ones.
--->
-
-<schemas>
-  <schema name="RS-10-4">
-    <k>10</k>
-    <m>4</m>
-    <codec>RS</codec>
-  </schema>
-</schemas>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44a30f92/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 8a5211a..bd2a24b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -143,11 +143,6 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   /** Supported erasure codec classes */
   public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs";
 
-  public static final String IO_ERASURECODE_SCHEMA_FILE_KEY =
-      "io.erasurecode.schema.file";
-  public static final String IO_ERASURECODE_SCHEMA_FILE_DEFAULT =
-      "ecschema-def.xml";
-
   /** Use XOR raw coder when possible for the RS codec */
   public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY =
       "io.erasurecode.codec.rs.usexor";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44a30f92/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
index 75dd03a..9b10c78 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
@@ -17,20 +17,27 @@
  */
 package org.apache.hadoop.io.erasurecode;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.w3c.dom.*;
-import org.xml.sax.SAXException;
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.util.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.w3c.dom.Text;
+import org.xml.sax.SAXException;
 
 /**
  * A EC schema loading utility that loads predefined EC schemas from XML file
@@ -42,8 +49,8 @@ public class SchemaLoader {
    * Load predefined ec schemas from configuration file. This file is
    * expected to be in the XML format.
    */
-  public List<ECSchema> loadSchema(Configuration conf) {
-    File confFile = getSchemaFile(conf);
+  public List<ECSchema> loadSchema(String schemaFilePath) {
+    File confFile = getSchemaFile(schemaFilePath);
     if (confFile == null) {
       LOG.warn("Not found any predefined EC schema file");
       return Collections.emptyList();
@@ -100,10 +107,7 @@ public class SchemaLoader {
    * Path to the XML file containing predefined ec schemas. If the path is
    * relative, it is searched for in the classpath.
    */
-  private File getSchemaFile(Configuration conf) {
-    String schemaFilePath = conf.get(
-        CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_KEY,
-        CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_DEFAULT);
+  private File getSchemaFile(String schemaFilePath) {
     File schemaFile = new File(schemaFilePath);
     if (! schemaFile.isAbsolute()) {
       URL url = Thread.currentThread().getContextClassLoader()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44a30f92/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
index 7bb0a9a..6caeedb 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
@@ -17,16 +17,14 @@
  */
 package org.apache.hadoop.io.erasurecode;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 import java.io.File;
 import java.io.FileWriter;
 import java.io.PrintWriter;
 import java.util.List;
 
-import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 public class TestSchemaLoader {
 
@@ -54,12 +52,8 @@ public class TestSchemaLoader {
     out.println("</schemas>");
     out.close();
 
-    Configuration conf = new Configuration();
-    conf.set(CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_KEY,
-        SCHEMA_FILE);
-
     SchemaLoader schemaLoader = new SchemaLoader();
-    List<ECSchema> schemas = schemaLoader.loadSchema(conf);
+    List<ECSchema> schemas = schemaLoader.loadSchema(SCHEMA_FILE);
 
     assertEquals(2, schemas.size());
 


[16/50] hadoop git commit: HDFS-8188. Erasure coding: refactor client-related code to sync with HDFS-8082 and HDFS-8169. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-8188. Erasure coding: refactor client-related code to sync with HDFS-8082 and HDFS-8169. Contributed by Zhe Zhang.


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

Branch: refs/heads/HDFS-7285
Commit: 166e565becc08feeee2806617e69ce100a85b156
Parents: c3a866e
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 20 14:19:12 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:17 2015 -0700

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       | 12 ++++
 .../hdfs/protocol/LocatedStripedBlock.java      | 64 +++++++++++++++++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 21 ++----
 .../hadoop/hdfs/client/impl/DfsClientConf.java  | 21 +++++-
 .../hdfs/protocol/LocatedStripedBlock.java      | 73 --------------------
 .../server/blockmanagement/BlockManager.java    | 25 ++++---
 .../hdfs/server/namenode/FSNamesystem.java      |  2 +-
 .../server/namenode/TestStripedINodeFile.java   |  3 +-
 8 files changed, 120 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/166e565b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 26283aa..6006d71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -177,6 +177,18 @@ public interface HdfsClientConfigKeys {
     int     THREADPOOL_SIZE_DEFAULT = 0;
   }
 
+  /** dfs.client.read.striped configuration properties */
+  interface StripedRead {
+    String PREFIX = Read.PREFIX + "striped.";
+
+    String  THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size";
+    /**
+     * With default 6+3 schema, each normal read could span 6 DNs. So this
+     * default value accommodates 3 read streams
+     */
+    int     THREADPOOL_SIZE_DEFAULT = 18;
+  }
+
   /** dfs.http.client configuration properties */
   interface HttpClient {
     String  PREFIX = "dfs.http.client.";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/166e565b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
new file mode 100644
index 0000000..93a5948
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageType;
+
+import java.util.Arrays;
+
+/**
+ * {@link LocatedBlock} with striped block support. For a striped block, each
+ * datanode storage is associated with a block in the block group. We need to
+ * record the index (in the striped block group) for each of them.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class LocatedStripedBlock extends LocatedBlock {
+  private int[] blockIndices;
+
+  public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs,
+      String[] storageIDs, StorageType[] storageTypes, int[] indices,
+      long startOffset, boolean corrupt, DatanodeInfo[] cachedLocs) {
+    super(b, locs, storageIDs, storageTypes, startOffset, corrupt, cachedLocs);
+    assert indices != null && indices.length == locs.length;
+    this.blockIndices = new int[indices.length];
+    System.arraycopy(indices, 0, blockIndices, 0, indices.length);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "{" + getBlock()
+        + "; getBlockSize()=" + getBlockSize()
+        + "; corrupt=" + isCorrupt()
+        + "; offset=" + getStartOffset()
+        + "; locs=" + Arrays.asList(getLocations())
+        + "; indices=" + Arrays.asList(blockIndices)
+        + "}";
+  }
+
+  public int[] getBlockIndices() {
+    return this.blockIndices;
+  }
+
+  @Override
+  public boolean isStriped() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/166e565b/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 b1dab46..111398f 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
@@ -377,21 +377,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
         dfsClientConf);
 
     if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) {
-      this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize());
-    }
-    numThreads = conf.getInt(
-        DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE,
-        DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE);
-    if (numThreads <= 0) {
-      LOG.warn("The value of "
-          + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE
-          + " must be greater than 0. The current setting is " + numThreads
-          + ". Reset it to the default value "
-          + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE);
-      numThreads =
-          DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE;
-    }
-    this.initThreadsNumForStripedReads(numThreads);
+      this.initThreadsNumForHedgedReads(dfsClientConf.
+          getHedgedReadThreadpoolSize());
+    }
+
+    this.initThreadsNumForStripedReads(dfsClientConf.
+        getStripedReadThreadpoolSize());
     this.saslClient = new SaslDataTransferClient(
       conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf),
       TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/166e565b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
index a257e32..32a3da0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
@@ -38,6 +38,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIM
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -101,6 +102,8 @@ public class DfsClientConf {
   private final long hedgedReadThresholdMillis;
   private final int hedgedReadThreadpoolSize;
 
+  private final int stripedReadThreadpoolSize;
+
   public DfsClientConf(Configuration conf) {
     // The hdfsTimeout is currently the same as the ipc timeout 
     hdfsTimeout = Client.getTimeout(conf);
@@ -191,7 +194,7 @@ public class DfsClientConf {
     connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
         DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
     hdfsBlocksMetadataEnabled = conf.getBoolean(
-        DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED, 
+        DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED,
         DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT);
     fileBlockStorageLocationsNumThreads = conf.getInt(
         DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS,
@@ -215,6 +218,13 @@ public class DfsClientConf {
     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.");
   }
 
   private DataChecksum.Type getChecksumType(Configuration conf) {
@@ -492,6 +502,13 @@ public class DfsClientConf {
   }
 
   /**
+   * @return the stripedReadThreadpoolSize
+   */
+  public int getStripedReadThreadpoolSize() {
+    return stripedReadThreadpoolSize;
+  }
+
+  /**
    * @return the shortCircuitConf
    */
   public ShortCircuitConf getShortCircuitConf() {
@@ -744,4 +761,4 @@ public class DfsClientConf {
       return builder.toString();
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/166e565b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
deleted file mode 100644
index 98614db..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
+++ /dev/null
@@ -1,73 +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;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-
-import java.util.Arrays;
-
-/**
- * {@link LocatedBlock} with striped block support. For a striped block, each
- * datanode storage is associated with a block in the block group. We need to
- * record the index (in the striped block group) for each of them.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class LocatedStripedBlock extends LocatedBlock {
-  private int[] blockIndices;
-
-  public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs,
-      String[] storageIDs, StorageType[] storageTypes, int[] indices,
-      long startOffset, boolean corrupt, DatanodeInfo[] cachedLocs) {
-    super(b, locs, storageIDs, storageTypes, startOffset, corrupt, cachedLocs);
-    assert indices != null && indices.length == locs.length;
-    this.blockIndices = new int[indices.length];
-    System.arraycopy(indices, 0, blockIndices, 0, indices.length);
-  }
-
-  public LocatedStripedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages,
-      int[] indices, long startOffset, boolean corrupt) {
-    this(b, DatanodeStorageInfo.toDatanodeInfos(storages),
-        DatanodeStorageInfo.toStorageIDs(storages),
-        DatanodeStorageInfo.toStorageTypes(storages), indices,
-        startOffset, corrupt, EMPTY_LOCS);
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "{" + getBlock()
-        + "; getBlockSize()=" + getBlockSize()
-        + "; corrupt=" + isCorrupt()
-        + "; offset=" + getStartOffset()
-        + "; locs=" + Arrays.asList(getLocations())
-        + "; indices=" + Arrays.asList(blockIndices)
-        + "}";
-  }
-
-  public int[] getBlockIndices() {
-    return this.blockIndices;
-  }
-
-  @Override
-  public boolean isStriped() {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/166e565b/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 712c665..5d998c8 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
@@ -873,7 +873,7 @@ public class BlockManager {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
-        return new LocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
+        return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
             false);
       } else {
         assert blk instanceof BlockInfoContiguousUnderConstruction;
@@ -882,13 +882,8 @@ public class BlockManager {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
-        return new LocatedBlock(eb, storages, pos, false);
+        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
@@ -931,7 +926,7 @@ public class BlockManager {
     final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
     return blockIndices == null ?
         newLocatedBlock(eb, machines, pos, isCorrupt) :
-        new LocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
+        newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
   }
 
   /** Create a LocatedBlocks. */
@@ -3499,7 +3494,7 @@ public class BlockManager {
     if (pendingReplicationBlocksCount == 0 &&
         underReplicatedBlocksCount == 0) {
       LOG.info("Node {} is dead and there are no under-replicated" +
-          " blocks or blocks pending replication. Safe to decommission.", 
+          " blocks or blocks pending replication. Safe to decommission.",
           node);
       return true;
     }
@@ -3905,6 +3900,18 @@ public class BlockManager {
         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);
+  }
+
   /**
    * This class is used internally by {@link this#computeRecoveryWorkForBlocks}
    * to represent a task to recover a block through replication or erasure

http://git-wip-us.apache.org/repos/asf/hadoop/blob/166e565b/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 f176128..c77b193 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
@@ -3359,7 +3359,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final LocatedBlock lBlk;
     if (blk.isStriped()) {
       assert blk instanceof BlockInfoStripedUnderConstruction;
-      lBlk = new LocatedStripedBlock(getExtendedBlock(blk), locs,
+      lBlk = BlockManager.newLocatedStripedBlock(getExtendedBlock(blk), locs,
           ((BlockInfoStripedUnderConstruction) blk).getBlockIndices(),
           offset, false);
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/166e565b/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 d251c30..4a6d6cc 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
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -45,7 +46,7 @@ public class TestStripedINodeFile {
       "userName", null, FsPermission.getDefault());
 
   private static INodeFile createStripedINodeFile() {
-    return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+    return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID);
   }
 


[43/50] hadoop git commit: HADOOP-11921. Enhance tests for erasure coders. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11921. Enhance tests for erasure coders. Contributed by Kai Zheng


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

Branch: refs/heads/HDFS-7285
Commit: 9f929d37dcf1d78fc74580044096fe592c6b864b
Parents: e367ca4
Author: Kai Zheng <ka...@intel.com>
Authored: Thu May 7 17:05:04 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:40:58 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  2 +
 .../hadoop/io/erasurecode/TestCoderBase.java    | 50 ++++++-----
 .../erasurecode/coder/TestErasureCoderBase.java | 89 +++++++++++---------
 .../erasurecode/coder/TestRSErasureCoder.java   | 64 ++++++++++----
 .../io/erasurecode/coder/TestXORCoder.java      | 24 ++++--
 .../io/erasurecode/rawcoder/TestRSRawCoder.java | 76 +++++++++--------
 .../rawcoder/TestRSRawCoderBase.java            | 51 +++++++++++
 .../erasurecode/rawcoder/TestRawCoderBase.java  | 45 +++++-----
 .../erasurecode/rawcoder/TestXORRawCoder.java   | 24 ++++--
 9 files changed, 274 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f929d37/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index 9749270..7a344a8 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -44,3 +44,5 @@
     HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng)
 
     HADOOP-11841. Remove unused ecschema-def.xml files.  (szetszwo)
+
+    HADOOP-11921 Enhance tests for erasure coders. (Kai Zheng)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f929d37/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
index 22fd98d..be1924c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -49,15 +49,15 @@ public abstract class TestCoderBase {
    * Prepare before running the case.
    * @param numDataUnits
    * @param numParityUnits
-   * @param erasedIndexes
+   * @param erasedDataIndexes
    */
   protected void prepare(Configuration conf, int numDataUnits,
-                         int numParityUnits, int[] erasedIndexes) {
+                         int numParityUnits, int[] erasedDataIndexes) {
     this.conf = conf;
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
-    this.erasedDataIndexes = erasedIndexes != null ?
-        erasedIndexes : new int[] {0};
+    this.erasedDataIndexes = erasedDataIndexes != null ?
+        erasedDataIndexes : new int[] {0};
   }
 
   /**
@@ -82,15 +82,19 @@ public abstract class TestCoderBase {
   }
 
   /**
-   * Adjust and return erased indexes based on the array of the input chunks (
-   * parity chunks + data chunks).
-   * @return
+   * Adjust and return erased indexes altogether, including erased data indexes
+   * and parity indexes.
+   * @return erased indexes altogether
    */
   protected int[] getErasedIndexesForDecoding() {
     int[] erasedIndexesForDecoding = new int[erasedDataIndexes.length];
+
+    int idx = 0;
+
     for (int i = 0; i < erasedDataIndexes.length; i++) {
-      erasedIndexesForDecoding[i] = erasedDataIndexes[i] + numParityUnits;
+      erasedIndexesForDecoding[idx ++] = erasedDataIndexes[i] + numParityUnits;
     }
+
     return erasedIndexesForDecoding;
   }
 
@@ -116,30 +120,23 @@ public abstract class TestCoderBase {
   }
 
   /**
-   * Have a copy of the data chunks that's to be erased thereafter. The copy
-   * will be used to compare and verify with the to be recovered chunks.
+   * Erase chunks to test the recovering of them. Before erasure clone them
+   * first so could return them.
    * @param dataChunks
-   * @return
+   * @return clone of erased chunks
    */
-  protected ECChunk[] copyDataChunksToErase(ECChunk[] dataChunks) {
-    ECChunk[] copiedChunks = new ECChunk[erasedDataIndexes.length];
-
-    int j = 0;
-    for (int i = 0; i < erasedDataIndexes.length; i++) {
-      copiedChunks[j ++] = cloneChunkWithData(dataChunks[erasedDataIndexes[i]]);
-    }
+  protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks) {
+    ECChunk[] toEraseChunks = new ECChunk[erasedDataIndexes.length];
 
-    return copiedChunks;
-  }
+    int idx = 0;
 
-  /**
-   * Erase some data chunks to test the recovering of them
-   * @param dataChunks
-   */
-  protected void eraseSomeDataBlocks(ECChunk[] dataChunks) {
     for (int i = 0; i < erasedDataIndexes.length; i++) {
-      eraseDataFromChunk(dataChunks[erasedDataIndexes[i]]);
+      ECChunk chunk = dataChunks[erasedDataIndexes[i]];
+      toEraseChunks[idx ++] = cloneChunkWithData(chunk);
+      eraseDataFromChunk(chunk);
     }
+
+    return toEraseChunks;
   }
 
   /**
@@ -277,6 +274,7 @@ public abstract class TestCoderBase {
    */
   protected ECChunk[] prepareOutputChunksForDecoding() {
     ECChunk[] chunks = new ECChunk[erasedDataIndexes.length];
+
     for (int i = 0; i < chunks.length; i++) {
       chunks[i] = allocateOutputChunk();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f929d37/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
index 05a62a7..fdd0b50 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
@@ -29,6 +29,9 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   protected Class<? extends ErasureCoder> encoderClass;
   protected Class<? extends ErasureCoder> decoderClass;
 
+  private ErasureCoder encoder;
+  private ErasureCoder decoder;
+
   protected int numChunksInBlock = 16;
 
   /**
@@ -54,39 +57,27 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
    */
   protected void testCoding(boolean usingDirectBuffer) {
     this.usingDirectBuffer = usingDirectBuffer;
-
-    ErasureCoder encoder = createEncoder();
+    prepareCoders();
 
     // Generate data and encode
     ECBlockGroup blockGroup = prepareBlockGroupForEncoding();
     // Backup all the source chunks for later recovering because some coders
     // may affect the source data.
-    TestBlock[] clonedDataBlocks = cloneBlocksWithData((TestBlock[])
-        blockGroup.getDataBlocks());
-    // Make a copy of a strip for later comparing
-    TestBlock[] toEraseBlocks = copyDataBlocksToErase(clonedDataBlocks);
+    TestBlock[] clonedDataBlocks = cloneBlocksWithData((TestBlock[]) blockGroup.getDataBlocks());
 
     ErasureCodingStep codingStep;
-    try {
-      codingStep = encoder.calculateCoding(blockGroup);
-      performCodingStep(codingStep);
-    } finally {
-      encoder.release();
-    }
-    // Erase the copied sources
-    eraseSomeDataBlocks(clonedDataBlocks);
+    codingStep = encoder.calculateCoding(blockGroup);
+    performCodingStep(codingStep);
+    // Erase specified sources but return copies of them for later comparing
+    TestBlock[] backupBlocks = backupAndEraseBlocks(clonedDataBlocks);
 
-    //Decode
+    // Decode
     blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
-    ErasureCoder decoder = createDecoder();
-    try {
-      codingStep = decoder.calculateCoding(blockGroup);
-      performCodingStep(codingStep);
-    } finally {
-      decoder.release();
-    }
-    //Compare
-    compareAndVerify(toEraseBlocks, codingStep.getOutputBlocks());
+    codingStep = decoder.calculateCoding(blockGroup);
+    performCodingStep(codingStep);
+
+    // Compare
+    compareAndVerify(backupBlocks, codingStep.getOutputBlocks());
   }
 
   /**
@@ -129,8 +120,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   protected void compareAndVerify(ECBlock[] erasedBlocks,
                                   ECBlock[] recoveredBlocks) {
     for (int i = 0; i < erasedBlocks.length; ++i) {
-      compareAndVerify(((TestBlock) erasedBlocks[i]).chunks,
-          ((TestBlock) recoveredBlocks[i]).chunks);
+      compareAndVerify(((TestBlock) erasedBlocks[i]).chunks, ((TestBlock) recoveredBlocks[i]).chunks);
     }
   }
 
@@ -151,6 +141,16 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     return encoder;
   }
 
+  private void prepareCoders() {
+    if (encoder == null) {
+      encoder = createEncoder();
+    }
+
+    if (decoder == null) {
+      decoder = createDecoder();
+    }
+  }
+
   /**
    * Create the erasure decoder for the test.
    * @return
@@ -202,6 +202,26 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   }
 
   /**
+   * Erase blocks to test the recovering of them. Before erasure clone them
+   * first so could return themselves.
+   * @param dataBlocks
+   * @return clone of erased dataBlocks
+   */
+  protected TestBlock[] backupAndEraseBlocks(TestBlock[] dataBlocks) {
+    TestBlock[] toEraseBlocks = new TestBlock[erasedDataIndexes.length];
+
+    int idx = 0;
+
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
+      TestBlock block = dataBlocks[erasedDataIndexes[i]];
+      toEraseBlocks[idx ++] = cloneBlockWithData(block);
+      eraseDataFromBlock(block);
+    }
+
+    return toEraseBlocks;
+  }
+
+  /**
    * Copy those data blocks that's to be erased for later comparing and
    * verifying.
    * @param dataBlocks
@@ -255,22 +275,9 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   }
 
   /**
-   * Erase some data blocks specified by the indexes from the data blocks.
-   * @param dataBlocks
-   */
-  protected void eraseSomeDataBlocks(TestBlock[] dataBlocks) {
-    for (int i = 0; i < erasedDataIndexes.length; ++i) {
-      eraseDataFromBlock(dataBlocks, erasedDataIndexes[i]);
-    }
-  }
-
-  /**
-   * Erase data from a block specified by erased index.
-   * @param blocks
-   * @param erasedIndex
+   * Erase data from a block.
    */
-  protected void eraseDataFromBlock(TestBlock[] blocks, int erasedIndex) {
-    TestBlock theBlock = blocks[erasedIndex];
+  protected void eraseDataFromBlock(TestBlock theBlock) {
     eraseDataFromChunks(theBlock.chunks);
     theBlock.setErased(true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f929d37/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
index 3507dd2..7d9d37a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
@@ -40,19 +40,18 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
   }
 
   @Test
-  public void testCodingNoDirectBuffer_10x4() {
-    prepare(null, 10, 4, null);
+  public void testCodingNoDirectBuffer_10x4_erasing_d0() {
+    prepare(null, 10, 4, new int[] {0});
+    /**
+     * Doing twice to test if the coders can be repeatedly reused. This matters
+     * as the underlying coding buffers are shared, which may have bugs.
+     */
+    testCoding(false);
     testCoding(false);
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4() {
-    prepare(null, 10, 4, null);
-    testCoding(true);
-  }
-
-  @Test
-  public void testCodingDirectBufferWithConf_10x4() {
+  public void testCodingDirectBufferWithConf_10x4_erasing_d0() {
     /**
      * This tests if the two configuration items work or not.
      */
@@ -61,31 +60,62 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
         RSRawErasureCoderFactory.class.getCanonicalName());
     conf.setBoolean(
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false);
-    prepare(conf, 10, 4, null);
+
+    prepare(conf, 10, 4, new int[]{0});
+
     testCoding(true);
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
+  public void testCodingDirectBuffer_10x4_erasing_d2() {
+    prepare(null, 10, 4, new int[] {2});
+    testCoding(true);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_d0() {
+    prepare(null, 10, 4, new int[] {0});
+    testCoding(true);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingBothBuffers_10x4_erasing_d0() {
+    prepare(null, 10, 4, new int[] {0});
+
+    /**
+     * Doing in mixed buffer usage model to test if the coders can be repeatedly
+     * reused with different buffer usage model. This matters as the underlying
+     * coding buffers are shared, which may have bugs.
+     */
+    testCoding(true);
+    testCoding(false);
+    testCoding(true);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasure_of_d2_d4() {
     prepare(null, 10, 4, new int[] {2, 4});
     testCoding(true);
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasing_all() {
-    prepare(null, 10, 4, new int[] {0, 1, 2, 3});
+  public void testCodingDirectBuffer_10x4_erasing_d0_d1() {
+    prepare(null, 10, 4, new int[] {0, 1});
     testCoding(true);
   }
 
   @Test
-  public void testCodingNoDirectBuffer_3x3() {
-    prepare(null, 3, 3, null);
+  public void testCodingNoDirectBuffer_3x3_erasing_d0() {
+    prepare(null, 3, 3, new int[] {0});
     testCoding(false);
   }
 
   @Test
-  public void testCodingDirectBuffer_3x3() {
-    prepare(null, 3, 3, null);
+  public void testCodingDirectBuffer_3x3_erasing_d0() {
+    prepare(null, 3, 3, new int[] {0});
     testCoding(true);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f929d37/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java
index 109e46e..87aa656 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java
@@ -32,19 +32,33 @@ public class TestXORCoder extends TestErasureCoderBase {
 
     this.numDataUnits = 10;
     this.numParityUnits = 1;
-    this.erasedDataIndexes = new int[] {0};
-
     this.numChunksInBlock = 10;
   }
 
   @Test
-  public void testCodingNoDirectBuffer() {
+  public void testCodingNoDirectBuffer_erasing_d0() {
+    prepare(null, 10, 1, new int[] {0});
+
+    /**
+     * Doing twice to test if the coders can be repeatedly reused. This matters
+     * as the underlying coding buffers are shared, which may have bugs.
+     */
+    testCoding(false);
     testCoding(false);
   }
 
   @Test
-  public void testCodingDirectBuffer() {
+  public void testCodingBothBuffers_erasing_d5() {
+    prepare(null, 10, 1, new int[]{5});
+
+    /**
+     * Doing in mixed buffer usage model to test if the coders can be repeatedly
+     * reused with different buffer usage model. This matters as the underlying
+     * coding buffers are shared, which may have bugs.
+     */
     testCoding(true);
+    testCoding(false);
+    testCoding(true);
+    testCoding(false);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f929d37/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
index 8bb5d0f..9ba3e88 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
@@ -17,26 +17,13 @@
  */
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
-import org.apache.hadoop.io.erasurecode.ECChunk;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.nio.ByteBuffer;
-
 /**
- * Test raw Reed-solomon encoding and decoding.
+ * Test raw Reed-solomon coder implemented in Java.
  */
-public class TestRSRawCoder extends TestRawCoderBase {
-
-  private static int symbolSize = 0;
-  private static int symbolMax = 0;
-
-  static {
-    symbolSize = (int) Math.round(Math.log(
-        RSUtil.GF.getFieldSize()) / Math.log(2));
-    symbolMax = (int) Math.pow(2, symbolSize);
-  }
+public class TestRSRawCoder extends TestRSRawCoderBase {
 
   @Before
   public void setup() {
@@ -45,49 +32,66 @@ public class TestRSRawCoder extends TestRawCoderBase {
   }
 
   @Test
-  public void testCodingNoDirectBuffer_10x4() {
-    prepare(null, 10, 4, null);
+  public void testCodingNoDirectBuffer_10x4_erasing_d0() {
+    prepare(null, 10, 4, new int[] {0});
+    /**
+     * Doing twice to test if the coders can be repeatedly reused. This matters
+     * as the underlying coding buffers are shared, which may have bugs.
+     */
+    testCoding(false);
     testCoding(false);
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4() {
-    prepare(null, 10, 4, null);
+  public void testCodingDirectBuffer_10x4_erasing_d2() {
+    prepare(null, 10, 4, new int[] {2});
+    testCoding(true);
     testCoding(true);
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
-    prepare(null, 10, 4, new int[] {2, 4});
+  public void testCodingDirectBuffer_10x4_erasing_d0() {
+    prepare(null, 10, 4, new int[] {0});
+    testCoding(true);
     testCoding(true);
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasing_all() {
-    prepare(null, 10, 4, new int[] {0, 1, 2, 3});
+  public void testCodingBothBuffers_10x4_erasing_d0() {
+    prepare(null, 10, 4, new int[] {0});
+
+    /**
+     * Doing in mixed buffer usage model to test if the coders can be repeatedly
+     * reused with different buffer usage model. This matters as the underlying
+     * coding buffers are shared, which may have bugs.
+     */
+    testCoding(true);
+    testCoding(false);
     testCoding(true);
+    testCoding(false);
   }
 
   @Test
-  public void testCodingNoDirectBuffer_3x3() {
-    prepare(null, 3, 3, null);
-    testCoding(false);
+  public void testCodingDirectBuffer_10x4_erasure_of_d2_d4() {
+    prepare(null, 10, 4, new int[] {2, 4});
+    testCoding(true);
   }
 
   @Test
-  public void testCodingDirectBuffer_3x3() {
-    prepare(null, 3, 3, null);
+  public void testCodingDirectBuffer_10x4_erasing_d0_d1() {
+    prepare(null, 10, 4, new int[] {0, 1});
     testCoding(true);
   }
 
-  @Override
-  protected ECChunk generateDataChunk() {
-    ByteBuffer buffer = allocateOutputBuffer();
-    for (int i = 0; i < chunkSize; i++) {
-      buffer.put((byte) RAND.nextInt(symbolMax));
-    }
-    buffer.flip();
+  @Test
+  public void testCodingNoDirectBuffer_3x3_erasing_d0() {
+    prepare(null, 3, 3, new int[] {0});
+    testCoding(false);
+  }
 
-    return new ECChunk(buffer);
+  @Test
+  public void testCodingDirectBuffer_3x3_erasing_d0() {
+    prepare(null, 3, 3, new int[] {0});
+    testCoding(true);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f929d37/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java
new file mode 100644
index 0000000..f9e8a6b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java
@@ -0,0 +1,51 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Test base for raw Reed-solomon coders.
+ */
+public abstract class TestRSRawCoderBase extends TestRawCoderBase {
+
+  private static int symbolSize = 0;
+  private static int symbolMax = 0;
+
+  static {
+    symbolSize = (int) Math.round(Math.log(
+        RSUtil.GF.getFieldSize()) / Math.log(2));
+    symbolMax = (int) Math.pow(2, symbolSize);
+  }
+
+  @Override
+  protected ECChunk generateDataChunk() {
+    ByteBuffer buffer = allocateOutputBuffer();
+    for (int i = 0; i < chunkSize; i++) {
+      buffer.put((byte) RAND.nextInt(symbolMax));
+    }
+    buffer.flip();
+
+    return new ECChunk(buffer);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f929d37/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
index b036eed..7ba320a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.io.erasurecode.TestCoderBase;
 public abstract class TestRawCoderBase extends TestCoderBase {
   protected Class<? extends RawErasureEncoder> encoderClass;
   protected Class<? extends RawErasureDecoder> decoderClass;
+  private RawErasureEncoder encoder;
+  private RawErasureDecoder decoder;
 
   /**
    * Generating source data, encoding, recovering and then verifying.
@@ -37,40 +39,41 @@ public abstract class TestRawCoderBase extends TestCoderBase {
    */
   protected void testCoding(boolean usingDirectBuffer) {
     this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders();
 
     // Generate data and encode
     ECChunk[] dataChunks = prepareDataChunksForEncoding();
     ECChunk[] parityChunks = prepareParityChunksForEncoding();
-    RawErasureEncoder encoder = createEncoder();
 
     // Backup all the source chunks for later recovering because some coders
     // may affect the source data.
     ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks);
-    // Make a copy of a strip for later comparing
-    ECChunk[] toEraseDataChunks = copyDataChunksToErase(clonedDataChunks);
 
-    try {
-      encoder.encode(dataChunks, parityChunks);
-    } finally {
-      encoder.release();
-    }
-    // Erase the copied sources
-    eraseSomeDataBlocks(clonedDataChunks);
+    encoder.encode(dataChunks, parityChunks);
+
+    // Backup and erase some chunks
+    ECChunk[] backupChunks = backupAndEraseChunks(clonedDataChunks);
+
+    // Decode
+    ECChunk[] inputChunks = prepareInputChunksForDecoding(
+        clonedDataChunks, parityChunks);
 
-    //Decode
-    ECChunk[] inputChunks = prepareInputChunksForDecoding(clonedDataChunks,
-        parityChunks);
     ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
-    RawErasureDecoder decoder = createDecoder();
-    try {
-      decoder.decode(inputChunks,
-          getErasedIndexesForDecoding(), recoveredChunks);
-    } finally {
-      decoder.release();
+
+    decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks);
+
+    // Compare
+    compareAndVerify(backupChunks, recoveredChunks);
+  }
+
+  private void prepareCoders() {
+    if (encoder == null) {
+      encoder = createEncoder();
     }
 
-    //Compare
-    compareAndVerify(toEraseDataChunks, recoveredChunks);
+    if (decoder == null) {
+      decoder = createDecoder();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f929d37/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java
index e66e48d..62ce4fb 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java
@@ -32,18 +32,32 @@ public class TestXORRawCoder extends TestRawCoderBase {
 
     this.numDataUnits = 10;
     this.numParityUnits = 1;
-
-    this.erasedDataIndexes = new int[] {0};
   }
 
   @Test
-  public void testCodingNoDirectBuffer() {
+  public void testCodingNoDirectBuffer_erasing_d0() {
+    prepare(null, 10, 1, new int[] {0});
+
+    /**
+     * Doing twice to test if the coders can be repeatedly reused. This matters
+     * as the underlying coding buffers are shared, which may have bugs.
+     */
+    testCoding(false);
     testCoding(false);
   }
 
   @Test
-  public void testCodingDirectBuffer() {
+  public void testCodingBothBuffers_erasing_d5() {
+    prepare(null, 10, 1, new int[]{5});
+
+    /**
+     * Doing in mixed buffer usage model to test if the coders can be repeatedly
+     * reused with different buffer usage model. This matters as the underlying
+     * coding buffers are shared, which may have bugs.
+     */
     testCoding(true);
+    testCoding(false);
+    testCoding(true);
+    testCoding(false);
   }
-
 }


[03/50] hadoop git commit: HDFS-8114. Erasure coding: Add auditlog FSNamesystem#createErasureCodingZone if this operation fails. Contributed by Rakesh R.

Posted by zh...@apache.org.
HDFS-8114. Erasure coding: Add auditlog FSNamesystem#createErasureCodingZone if this operation fails. Contributed by Rakesh R.


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

Branch: refs/heads/HDFS-7285
Commit: d7eacf5a984c73dde0eed6e076def1dca9e5d036
Parents: 7aa3919
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 13 11:15:02 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:13 2015 -0700

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      | 21 ++++++++++++++------
 1 file changed, 15 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7eacf5a/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 07b5bcf..9dee676 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
@@ -8122,11 +8122,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       SafeModeException, AccessControlException {
     String src = srcArg;
     HdfsFileStatus resultingStat = null;
-    checkSuperuserPrivilege();
-    checkOperation(OperationCategory.WRITE);
-    final byte[][] pathComponents =
-        FSDirectory.getPathComponentsForReservedPath(src);
-    FSPermissionChecker pc = getPermissionChecker();
+    FSPermissionChecker pc = null;
+    byte[][] pathComponents = null;
+    boolean success = false;
+    try {
+      checkSuperuserPrivilege();
+      checkOperation(OperationCategory.WRITE);
+      pathComponents =
+          FSDirectory.getPathComponentsForReservedPath(src);
+      pc = getPermissionChecker();
+    } catch (Throwable e) {
+      logAuditEvent(success, "createErasureCodingZone", srcArg);
+      throw e;
+    }
     writeLock();
     try {
       checkSuperuserPrivilege();
@@ -8140,11 +8148,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
       final INodesInPath iip = dir.getINodesInPath4Write(src, false);
       resultingStat = dir.getAuditFileInfo(iip);
+      success = true;
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    logAuditEvent(true, "createErasureCodingZone", srcArg, null, resultingStat);
+    logAuditEvent(success, "createErasureCodingZone", srcArg, null, resultingStat);
   }
 
   /**


[24/50] hadoop git commit: HDFS-8230. Erasure Coding: Ignore DatanodeProtocol#DNA_ERASURE_CODING_RECOVERY commands from standbynode if any (Contributed by Vinayakumar B)

Posted by zh...@apache.org.
HDFS-8230. Erasure Coding: Ignore DatanodeProtocol#DNA_ERASURE_CODING_RECOVERY commands from standbynode if any (Contributed by Vinayakumar B)


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

Branch: refs/heads/HDFS-7285
Commit: e4a4fd0f102b686e7b7868ca79e0fba8e1b3a85e
Parents: fc1609d
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 28 14:14:33 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:20 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt          | 3 +++
 .../org/apache/hadoop/hdfs/server/datanode/BPOfferService.java    | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4a4fd0f/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 e8db485..c28473b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -134,3 +134,6 @@
 
     HDFS-8033. Erasure coding: stateful (non-positional) read from files in 
     striped layout (Zhe Zhang)
+
+    HDFS-8230. Erasure Coding: Ignore DatanodeProtocol#DNA_ERASURE_CODING_RECOVERY 
+    commands from standbynode if any (vinayakumarb)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e4a4fd0f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index 69baac7..6606d0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -757,6 +757,7 @@ class BPOfferService {
     case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
     case DatanodeProtocol.DNA_CACHE:
     case DatanodeProtocol.DNA_UNCACHE:
+    case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY:
       LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
       break;
     default:


[40/50] hadoop git commit: HDFS-7348. Erasure Coding: DataNode reconstruct striped blocks. Contributed by Yi Liu.

Posted by zh...@apache.org.
HDFS-7348. Erasure Coding: DataNode reconstruct striped blocks. Contributed by Yi Liu.


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

Branch: refs/heads/HDFS-7285
Commit: bdb8947619a8b94a12bf65b4b72a4c8a2f4896d2
Parents: 8eeea75
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue May 5 16:33:56 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:40:46 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../org/apache/hadoop/hdfs/BlockReader.java     |   6 +
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |   5 +
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |   5 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   6 +
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  10 +-
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |   5 +
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |   5 +
 .../hadoop/hdfs/server/datanode/DNConf.java     |  27 +
 .../hadoop/hdfs/server/datanode/DataNode.java   |  31 +-
 .../erasurecode/ErasureCodingWorker.java        | 893 ++++++++++++++++++-
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  49 +-
 .../src/main/resources/hdfs-default.xml         |  33 +-
 .../hadoop/hdfs/TestRecoverStripedFile.java     | 356 ++++++++
 14 files changed, 1378 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/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 7efaa5a..0d2d448 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -175,3 +175,6 @@
     
     HDFS-7672. Handle write failure for stripping blocks and refactor the
     existing code in DFSStripedOutputStream and StripedDataStreamer.  (szetszwo)
+
+    HDFS-7348. Erasure Coding: DataNode reconstruct striped blocks. 
+    (Yi Liu via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
index aa3e8ba..0a5511e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
@@ -24,6 +24,7 @@ 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
@@ -99,4 +100,9 @@ public interface BlockReader extends ByteBufferReadable {
    *                      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/bdb89476/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
index d913f3a..0b2420d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
@@ -738,4 +738,9 @@ class BlockReaderLocal implements BlockReader {
   void forceUnanchorable() {
     replica.getSlot().makeUnanchorable();
   }
+
+  @Override
+  public DataChecksum getDataChecksum() {
+    return checksum;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
index c16ffdf..04cf733 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
@@ -732,4 +732,9 @@ class BlockReaderLocalLegacy implements BlockReader {
   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
     return null;
   }
+
+  @Override
+  public DataChecksum getDataChecksum() {
+    return checksum;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/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 42c5f94..3e2ba4a 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
@@ -365,6 +365,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   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 = 256 * 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_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/bdb89476/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
index 9cd1ec1..a26e35e 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
@@ -37,7 +37,7 @@ import org.apache.htrace.Span;
  ****************************************************************/
 
 @InterfaceAudience.Private
-class DFSPacket {
+public class DFSPacket {
   public static final long HEART_BEAT_SEQNO = -1L;
   private static long[] EMPTY = new long[0];
   private final long seqno; // sequence number of buffer in block
@@ -80,7 +80,7 @@ class DFSPacket {
    * @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;
@@ -114,7 +114,7 @@ class DFSPacket {
     dataPos += len;
   }
 
-  synchronized void writeData(ByteBuffer inBuffer, int len)
+  public synchronized void writeData(ByteBuffer inBuffer, int len)
       throws ClosedChannelException {
     checkBuffer();
     len =  len > inBuffer.remaining() ? inBuffer.remaining() : len;
@@ -135,7 +135,7 @@ class DFSPacket {
    * @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) {
@@ -154,7 +154,7 @@ class DFSPacket {
    * @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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
index d70f419..70cce7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
@@ -505,4 +505,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
     return null;
   }
+
+  @Override
+  public DataChecksum getDataChecksum() {
+    return checksum;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index c368d65..cce44b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@ -474,4 +474,9 @@ public class RemoteBlockReader2  implements BlockReader {
   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
     return null;
   }
+
+  @Override
+  public DataChecksum getDataChecksum() {
+    return checksum;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
index 4b7fbc3..d25642f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
@@ -236,6 +236,33 @@ public class DNConf {
   }
 
   /**
+   * Returns true if connect to datanode via hostname
+   * 
+   * @return boolean true if connect to datanode via hostname
+   */
+  public boolean getConnectToDnViaHostname() {
+    return connectToDnViaHostname;
+  }
+
+  /**
+   * Returns socket timeout
+   * 
+   * @return int socket timeout
+   */
+  public int getSocketTimeout() {
+    return socketTimeout;
+  }
+
+  /**
+   * Returns socket write timeout
+   * 
+   * @return int socket write timeout
+   */
+  public int getSocketWriteTimeout() {
+    return socketWriteTimeout;
+  }
+
+  /**
    * Returns the SaslPropertiesResolver configured for use with
    * DataTransferProtocol, or null if not configured.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 221ba38..5eca2c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -1161,7 +1161,8 @@ public class DataNode extends ReconfigurableBase
     saslClient = new SaslDataTransferClient(dnConf.conf, 
         dnConf.saslPropsResolver, dnConf.trustedChannelResolver);
     saslServer = new SaslDataTransferServer(dnConf, blockPoolTokenSecretManager);
-    ecWorker = new ErasureCodingWorker(conf); // Initialize ErasureCoding worker
+    // Initialize ErasureCoding worker
+    ecWorker = new ErasureCodingWorker(conf, this);
   }
 
   /**
@@ -1226,6 +1227,10 @@ public class DataNode extends ReconfigurableBase
     return UUID.randomUUID().toString();
   }
 
+  public SaslDataTransferClient getSaslClient() {
+    return saslClient;
+  }
+
   /**
    * Verify that the DatanodeUuid has been initialized. If this is a new
    * datanode then we generate a new Datanode Uuid and persist it to disk.
@@ -1488,7 +1493,7 @@ public class DataNode extends ReconfigurableBase
   /**
    * Creates either NIO or regular depending on socketWriteTimeout.
    */
-  protected Socket newSocket() throws IOException {
+  public Socket newSocket() throws IOException {
     return (dnConf.socketWriteTimeout > 0) ? 
            SocketChannel.open().socket() : new Socket();                                   
   }
@@ -2143,11 +2148,8 @@ public class DataNode extends ReconfigurableBase
         //
         // 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);
@@ -2214,6 +2216,19 @@ public class DataNode extends ReconfigurableBase
     }
   }
 
+  /***
+   * Use BlockTokenSecretManager to generate block token for current user.
+   */
+  public Token<BlockTokenIdentifier> getBlockAccessToken(ExtendedBlock b,
+      EnumSet<AccessMode> mode) throws IOException {
+    Token<BlockTokenIdentifier> accessToken = 
+        BlockTokenSecretManager.DUMMY_TOKEN;
+    if (isBlockTokenEnabled) {
+      accessToken = blockPoolTokenSecretManager.generateToken(b, mode);
+    }
+    return accessToken;
+  }
+
   /**
    * Returns a new DataEncryptionKeyFactory that generates a key from the
    * BlockPoolTokenSecretManager, using the block pool ID of the given block.
@@ -2221,7 +2236,7 @@ public class DataNode extends ReconfigurableBase
    * @param block for which the factory needs to create a key
    * @return DataEncryptionKeyFactory for block's block pool ID
    */
-  DataEncryptionKeyFactory getDataEncryptionKeyFactoryForBlock(
+  public DataEncryptionKeyFactory getDataEncryptionKeyFactoryForBlock(
       final ExtendedBlock block) {
     return new DataEncryptionKeyFactory() {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/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 6430308..c4e568f 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
@@ -17,15 +17,68 @@
  */
 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.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.CommonConfigurationKeys;
+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.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.common.HdfsServerConstants;
+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.io.erasurecode.coder.AbstractErasureCoder;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripedReadResult;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
-import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+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;
 
 /**
  * ErasureCodingWorker handles the erasure coding recovery work commands. These
@@ -34,41 +87,60 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
  * commands.
  */
 public final class ErasureCodingWorker {
-
+  private final Log LOG = DataNode.LOG;
+  
+  private final DataNode datanode; 
   private Configuration conf;
-  RawErasureCoder rawEncoder = null;
-  RawErasureCoder rawDecoder = null;
 
-  public ErasureCodingWorker(Configuration conf) {
+  private ThreadPoolExecutor STRIPED_READ_TRHEAD_POOL;
+  private final int STRIPED_READ_THRESHOLD_MILLIS;
+  private final int STRIPED_READ_BUFFER_SIZE;
+
+  public ErasureCodingWorker(Configuration conf, DataNode datanode) {
+    this.datanode = datanode;
     this.conf = conf;
-    initialize();
-  }
 
-  /**
-   * Initializes the required resources for handling the erasure coding recovery
-   * work.
-   */
-  public void initialize() {
-    // Right now directly used RS coder. Once other coders integration ready, we
-    // can load preferred codec here.
-    initializeErasureEncoder();
-    initializeErasureDecoder();
+    STRIPED_READ_THRESHOLD_MILLIS = conf.getInt(
+        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THRESHOLD_MILLIS_KEY,
+        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THRESHOLD_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);
   }
 
-  private void initializeErasureDecoder() {
-    rawDecoder = AbstractErasureCoder.createRawCoder(conf,
-        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, false);
-    if (rawDecoder == null) {
-      rawDecoder = new RSRawDecoder();
-    }
+  private RawErasureEncoder newEncoder() {
+    return new RSRawEncoder();
+  }
+  
+  private RawErasureDecoder newDecoder() {
+    return new RSRawDecoder();
   }
 
-  private void initializeErasureEncoder() {
-    rawEncoder = AbstractErasureCoder.createRawCoder(conf,
-        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, true);
-    if (rawEncoder == null) {
-      rawEncoder = new RSRawEncoder();
-    }
+  private void initializeStripedReadThreadPool(int num) {
+    STRIPED_READ_TRHEAD_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_TRHEAD_POOL.allowCoreThreadTimeOut(true);
   }
 
   /**
@@ -78,6 +150,765 @@ public final class ErasureCodingWorker {
    *          BlockECRecoveryInfo
    */
   public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
-    // HDFS-7348 : Implement the actual recovery process
+    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);
+      }
+    }
+  }
+
+  /**
+   * 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 RawErasureEncoder encoder;
+    private RawErasureDecoder decoder;
+
+    // Striped read buffer size
+    private int bufferSize;
+
+    private final ExtendedBlock blockGroup;
+    // position in striped block
+    private long positionInBlock;
+
+    // sources
+    private final short[] liveIndices;
+    private DatanodeInfo[] sources;
+
+    private List<StripedReader> stripedReaders;
+
+    // targets
+    private DatanodeInfo[] targets;
+    private StorageType[] targetStorageTypes;
+
+    private short[] targetIndices;
+    private ByteBuffer[] targetBuffers;
+
+    private Socket[] targetSockets;
+    private DataOutputStream[] targetOutputStreams;
+    private DataInputStream[] targetInputStreams;
+
+    private long[] blockOffset4Targets;
+    private long[] seqNo4Targets;
+
+    private final 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 CachingStrategy cachingStrategy;
+
+    private Map<Future<Void>, Integer> futures = new HashMap<>();
+    private CompletionService<Void> readService =
+        new ExecutorCompletionService<>(STRIPED_READ_TRHEAD_POOL);
+
+    ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) {
+      ECSchema schema = recoveryInfo.getECSchema();
+      dataBlkNum = schema.getNumDataUnits();
+      parityBlkNum = schema.getNumParityUnits();
+      cellSize = schema.getChunkSize();
+
+      blockGroup = recoveryInfo.getExtendedBlock();
+
+      liveIndices = recoveryInfo.getLiveBlockIndices();
+      sources = recoveryInfo.getSourceDnInfos();
+      stripedReaders = new ArrayList<>(sources.length);
+
+      Preconditions.checkArgument(liveIndices.length >= dataBlkNum,
+          "No enough live striped blocks.");
+      Preconditions.checkArgument(liveIndices.length == sources.length);
+
+      targets = recoveryInfo.getTargetDnInfos();
+      targetStorageTypes = recoveryInfo.getTargetStorageTypes();
+      targetIndices = new short[targets.length];
+      targetBuffers = new ByteBuffer[targets.length];
+
+      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 ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) {
+      return StripedBlockUtil.constructStripedBlock(blockGroup, cellSize,
+          dataBlkNum, i);
+    }
+
+    private long getBlockLen(ExtendedBlock blockGroup, int i) { 
+      return StripedBlockUtil.getStripedBlockLength(blockGroup.getNumBytes(),
+          cellSize, dataBlkNum, i);
+    }
+
+    @Override
+    public void run() {
+      try {
+        // Store the indices of successfully read source
+        // This will be updated after doing real read.
+        int[] success = new int[dataBlkNum];
+
+        int nsuccess = 0;
+        for (int i = 0; i < sources.length && nsuccess < dataBlkNum; i++) {
+          StripedReader reader = new StripedReader(liveIndices[i]);
+          stripedReaders.add(reader);
+
+          BlockReader blockReader = newBlockReader(
+              getBlock(blockGroup, liveIndices[i]), 0, sources[i]);
+          if (blockReader != null) {
+            initChecksumAndBufferSizeIfNeeded(blockReader);
+            reader.blockReader = blockReader;
+            reader.buffer = ByteBuffer.allocate(bufferSize);
+            success[nsuccess++] = i;
+          }
+        }
+
+        if (nsuccess < dataBlkNum) {
+          String error = "Can't find minimum sources required by "
+              + "recovery, block id: " + blockGroup.getBlockId();
+          LOG.warn(error);
+          throw new IOException(error);
+        }
+
+        for (int i = 0; i < targets.length; i++) {
+          targetBuffers[i] = ByteBuffer.allocate(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)];
+
+        // Store whether the target is success
+        boolean[] targetsStatus = new boolean[targets.length];
+        if (initTargetStreams(targetsStatus) == 0) {
+          String error = "All targets are failed.";
+          LOG.warn(error);
+          throw new IOException(error);
+        }
+
+        long firstStripedBlockLength = getBlockLen(blockGroup, 0);
+        while (positionInBlock < firstStripedBlockLength) {
+          int toRead = Math.min(
+              bufferSize, (int)(firstStripedBlockLength - positionInBlock));
+          // step1: read minimum striped buffer size data required by recovery.
+          nsuccess = readMinimumStripedData4Recovery(success);
+
+          if (nsuccess < dataBlkNum) {
+            String error = "Can't read data from minimum number of sources "
+                + "required by recovery, block id: " + blockGroup.getBlockId();
+            LOG.warn(error);
+            throw new IOException(error);
+          }
+
+          // step2: encode/decode to recover 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.";
+            LOG.warn(error);
+            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);
+      } finally {
+        // 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);
+      }
+    }
+
+    // assume liveIndices is not ordered.
+    private void getTargetIndices() {
+      BitSet bitset = new BitSet(dataBlkNum + parityBlkNum);
+      for (int i = 0; i < sources.length; i++) {
+        bitset.set(liveIndices[i]);
+      }
+      int m = 0;
+      for (int i = 0; i < dataBlkNum + parityBlkNum && m < targets.length; i++) {
+        if (!bitset.get(i)) {
+          targetIndices[m++] = (short)i;
+        }
+      }
+    }
+
+    /**
+     * Read minimum striped buffer size data required by recovery.
+     * <code>success</code> list will be updated after read.
+     * 
+     * Initially we only read from <code>dataBlkNum</code> sources, 
+     * if timeout or failure for some source, we will try to schedule 
+     * read from a new source. 
+     */
+    private int readMinimumStripedData4Recovery(int[] success) {
+
+      BitSet used = new BitSet(sources.length);
+      for (int i = 0; i < dataBlkNum; i++) {
+        StripedReader reader = stripedReaders.get(success[i]);
+        Callable<Void> readCallable = readFromBlock(
+            reader.blockReader, reader.buffer);
+        Future<Void> f = readService.submit(readCallable);
+        futures.put(f, success[i]);
+        used.set(success[i]);
+      }
+
+      int nsuccess = 0;
+      while (!futures.isEmpty()) {
+        try {
+          StripedReadResult result = 
+              StripedBlockUtil.getNextCompletedStripedRead(
+                  readService, futures, STRIPED_READ_THRESHOLD_MILLIS);
+          if (result.state == StripedReadResult.SUCCESSFUL) {
+            success[nsuccess++] = result.index;
+            if (nsuccess >= dataBlkNum) {
+              // cancel remaining reads if we read successfully from minimum
+              // number of sources required for recovery.
+              cancelReads(futures.keySet());
+              futures.clear();
+              break;
+            }
+          } else if (result.state == StripedReadResult.FAILED) {
+            // If read failed for some source, we should not use it anymore 
+            // and schedule read from a new source.
+            StripedReader failedReader = stripedReaders.get(result.index);
+            closeBlockReader(failedReader.blockReader);
+            failedReader.blockReader = null;
+            scheduleNewRead(used);
+          } else if (result.state == StripedReadResult.TIMEOUT) {
+            // If timeout, we also schedule a new read.
+            scheduleNewRead(used);
+          }
+        } catch (InterruptedException e) {
+          LOG.info("Read data interrupted.", e);
+          break;
+        }
+      }
+
+      return nsuccess;
+    }
+
+    /**
+     * Return true if need to do encoding to recovery missed striped block.
+     */
+    private boolean shouldEncode(int[] success) {
+      for (int i = 0; i < success.length; i++) {
+        if (stripedReaders.get(success[i]).index >= dataBlkNum) {
+          return false;
+        }
+      }
+      return true;
+    }
+    
+    private void paddingBufferToLen(ByteBuffer buffer, int len) {
+      int toPadding = len - buffer.position();
+      for (int i = 0; i < toPadding; i++) {
+        buffer.put((byte) 0);
+      }
+    }
+    
+    // Initialize encoder
+    private void initEncoderIfNecessary() {
+      if (encoder == null) {
+        encoder = newEncoder();
+        encoder.initialize(dataBlkNum, parityBlkNum, bufferSize);
+      }
+    }
+    
+    // Initialize decoder
+    private void initDecoderIfNecessary() {
+      if (decoder == null) {
+        decoder = newDecoder();
+        decoder.initialize(dataBlkNum, parityBlkNum, bufferSize);
+      }
+    }
+
+    private void recoverTargets(int[] success, boolean[] targetsStatus,
+        int toRecoverLen) {
+      if (shouldEncode(success)) {
+        initEncoderIfNecessary();
+        ByteBuffer[] dataBuffers = new ByteBuffer[dataBlkNum];
+        ByteBuffer[] parityBuffers = new ByteBuffer[parityBlkNum];
+        for (int i = 0; i < dataBlkNum; i++) {
+          StripedReader reader = stripedReaders.get(i);
+          ByteBuffer buffer = reader.buffer;
+          paddingBufferToLen(buffer, toRecoverLen);
+          dataBuffers[i] = (ByteBuffer)buffer.flip();
+        }
+        for (int i = dataBlkNum; i < stripedReaders.size(); i++) {
+          StripedReader reader = stripedReaders.get(i);
+          parityBuffers[reader.index - dataBlkNum] = cleanBuffer(reader.buffer);
+        }
+        for (int i = 0; i < targets.length; i++) {
+          parityBuffers[targetIndices[i] - dataBlkNum] = targetBuffers[i];
+        }
+        for (int i = 0; i < parityBlkNum; i++) {
+          if (parityBuffers[i] == null) {
+            parityBuffers[i] = ByteBuffer.allocate(toRecoverLen);
+          } else {
+            parityBuffers[i].limit(toRecoverLen);
+          }
+        }
+        encoder.encode(dataBuffers, parityBuffers);
+      } else {
+        /////////// TODO: wait for HADOOP-11847 /////////////
+        ////////// The current decode method always try to decode parityBlkNum number of data blocks. ////////////
+        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);
+          int index = reader.index < dataBlkNum ? 
+              reader.index + parityBlkNum : reader.index - dataBlkNum;
+          inputs[index] = (ByteBuffer)buffer.flip();
+        }
+        int[] indices4Decode = new int[parityBlkNum];
+        int m = 0;
+        for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {
+          if (inputs[i] == null) {
+            inputs[i] = ByteBuffer.allocate(toRecoverLen);
+            indices4Decode[m++] = i;
+          }
+        }
+        ByteBuffer[] outputs = new ByteBuffer[parityBlkNum];
+        m = 0;
+        // targetIndices is subset of indices4Decode
+        for (int i = 0; i < parityBlkNum; i++) {
+          if (m < targetIndices.length && 
+              (indices4Decode[i] - parityBlkNum) == targetIndices[m]) {
+            outputs[i] = targetBuffers[m++];
+            outputs[i].limit(toRecoverLen);
+          } else {
+            outputs[i] = ByteBuffer.allocate(toRecoverLen);
+          }
+        }
+        
+        decoder.decode(inputs, indices4Decode, 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 read from a new source, we first try un-initial source, 
+     * then try un-used source in this round and bypass failed source.
+     */
+    private void scheduleNewRead(BitSet used) {
+      StripedReader reader = null;
+      int m = stripedReaders.size();
+      while (m < sources.length && reader == null) {
+        reader = new StripedReader(liveIndices[m]);
+        BlockReader blockReader = newBlockReader(
+            getBlock(blockGroup, liveIndices[m]), positionInBlock, sources[m]);
+        stripedReaders.add(reader);
+        if (blockReader != null) {
+          assert blockReader.getDataChecksum().equals(checksum);
+          reader.blockReader = blockReader;
+          reader.buffer = ByteBuffer.allocate(bufferSize);
+        } else {
+          m++;
+          reader = null;
+        }
+      }
+
+      for (int i = 0; reader == null && i < stripedReaders.size(); i++) {
+        StripedReader r = stripedReaders.get(i);
+        if (r.blockReader != null && !used.get(i)) {
+          closeBlockReader(r.blockReader);
+          r.blockReader = newBlockReader(
+              getBlock(blockGroup, liveIndices[i]), positionInBlock,
+              sources[i]);
+          if (r.blockReader != null) {
+            m = i;
+            reader = r;
+          }
+        }
+      }
+
+      if (reader != null) {
+        Callable<Void> readCallable = readFromBlock(
+            reader.blockReader, reader.buffer);
+        Future<Void> f = readService.submit(readCallable);
+        futures.put(f, m);
+        used.set(m);
+      }
+    }
+
+    // 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 startOffset, DatanodeInfo dnInfo) {
+      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, startOffset, block.getNumBytes(), 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(), 
+            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();
+        }
+      }
+
+      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,
+              HdfsServerConstants.SMALL_BUFFER_SIZE));
+          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 class StripedReader {
+    short index;
+    BlockReader blockReader;
+    ByteBuffer buffer;
+
+    public StripedReader(short index) {
+      this.index = index;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/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 24d4bfb..45bbf6b 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
@@ -19,6 +19,7 @@
 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.protocol.DatanodeInfo;
@@ -77,10 +78,8 @@ public class StripedBlockUtil {
   public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg,
       int idxInReturnedLocs, int cellSize, int dataBlkNum,
       int idxInBlockGroup) {
-    final ExtendedBlock blk = new ExtendedBlock(bg.getBlock());
-    blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup);
-    blk.setNumBytes(getInternalBlockLength(bg.getBlockSize(),
-        cellSize, dataBlkNum, idxInBlockGroup));
+    final ExtendedBlock blk = constructInternalBlock(
+        bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup);
 
     return new LocatedBlock(blk,
         new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
@@ -91,6 +90,44 @@ public class StripedBlockUtil {
   }
 
   /**
+   * 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;
+  }
+  
+  /**
+   * This method creates an internal {@link ExtendedBlock} at the given index
+   * of a block group, for both data and parity block.
+   */
+  public static ExtendedBlock constructStripedBlock(ExtendedBlock blockGroup,
+      int cellSize, int dataBlkNum, int idxInBlockGroup) {
+    ExtendedBlock block = new ExtendedBlock(blockGroup);
+    block.setBlockId(blockGroup.getBlockId() + idxInBlockGroup);
+    block.setNumBytes(getStripedBlockLength(blockGroup.getNumBytes(), cellSize,
+        dataBlkNum, idxInBlockGroup));
+    return block;
+  }
+
+  /**
+   * Returns an internal block length at the given index of a block group,
+   * for both data and parity block.
+   */
+  public static long getStripedBlockLength(long numBytes, int cellSize,
+      int dataBlkNum, int idxInBlockGroup) {
+    // parity block length is the same as the first striped block length. 
+    return StripedBlockUtil.getInternalBlockLength(
+        numBytes, cellSize, dataBlkNum, 
+        idxInBlockGroup < dataBlkNum ? idxInBlockGroup : 0);
+  }
+
+  /**
    * 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
@@ -208,8 +245,8 @@ public class StripedBlockUtil {
    * @throws InterruptedException
    */
   public static StripedReadResult getNextCompletedStripedRead(
-      CompletionService<Void> readService, Map<Future<Void>,
-      Integer> futures, final long threshold) throws InterruptedException {
+      CompletionService<Void> readService, Map<Future<Void>, Integer> futures,
+      final long threshold) throws InterruptedException {
     Preconditions.checkArgument(!futures.isEmpty());
     Preconditions.checkArgument(threshold > 0);
     Future<Void> future = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/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 fe1d1de..9e00e85 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
@@ -1,4 +1,4 @@
-<?xml version="1.0"?>
+><?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 
 <!--
@@ -2303,11 +2303,11 @@
   </description>
 </property>
 
-  <property>
-    <name>dfs.datanode.block-pinning.enabled</name>
-    <value>false</value>
-    <description>Whether pin blocks on favored DataNode.</description>
-  </property>
+<property>
+  <name>dfs.datanode.block-pinning.enabled</name>
+  <value>false</value>
+  <description>Whether pin blocks on favored DataNode.</description>
+</property>
 
 <property>
   <name>dfs.client.block.write.locateFollowingBlock.initial.delay.ms</name>
@@ -2334,4 +2334,25 @@
   </description>
 </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>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdb89476/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
new file mode 100644
index 0000000..b4f05d4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
@@ -0,0 +1,356 @@
+/**
+ * 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.Path;
+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.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.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+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;
+
+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 blockSize = cellSize * 3;
+  private static final int groupSize = dataBlkNum + parityBlkNum;
+  private static final int dnNum = groupSize + parityBlkNum;
+  
+  private MiniDFSCluster cluster;
+  private Configuration conf;
+  private DistributedFileSystem fs;
+  // Map: DatanodeID -> datanode index in cluster
+  private Map<DatanodeID, Integer> dnMap = new HashMap<DatanodeID, Integer>();
+
+  @Before
+  public void setup() throws IOException {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY, cellSize - 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dnNum).build();;
+    cluster.waitActive();
+    
+    fs = cluster.getFileSystem();
+    fs.getClient().createErasureCodingZone("/", null);
+
+    List<DataNode> datanodes = cluster.getDataNodes();
+    for (int i = 0; i < dnNum; i++) {
+      dnMap.put(datanodes.get(i).getDatanodeId(), i);
+    }
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverOneParityBlock() throws Exception {
+    int fileLen = 10 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverOneParityBlock", fileLen, 0, 1);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverThreeParityBlocks() throws Exception {
+    int fileLen = 3 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverThreeParityBlocks", fileLen, 0, 3);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverThreeDataBlocks() throws Exception {
+    int fileLen = 3 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverThreeDataBlocks", fileLen, 1, 3);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverOneDataBlock() throws Exception {
+    ////TODO: TODO: wait for HADOOP-11847
+    //int fileLen = 10 * blockSize + blockSize/10;
+    //assertFileBlocksRecovery("/testRecoverOneDataBlock", fileLen, 1, 1);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverAnyBlocks() throws Exception {
+    ////TODO: TODO: wait for HADOOP-11847
+    //int fileLen = 3 * blockSize + blockSize/10;
+    //assertFileBlocksRecovery("/testRecoverAnyBlocks", fileLen, 2, 2);
+  }
+  
+  /**
+   * Test the file blocks recovery.
+   * 1. Check the replica is recovered in the target datanode, 
+   *    and verify the block replica length, generationStamp and content.
+   * 2. Read the file and verify content. 
+   */
+  private void assertFileBlocksRecovery(String fileName, int fileLen,
+      int recovery, int toRecoverBlockNum) throws Exception {
+    if (recovery != 0 && recovery != 1 && recovery != 2) {
+      Assert.fail("Invalid recovery: 0 is to recovery parity blocks,"
+          + "1 is to recovery data blocks, 2 is any.");
+    }
+    if (toRecoverBlockNum < 1 || toRecoverBlockNum > parityBlkNum) {
+      Assert.fail("toRecoverBlockNum should be between 1 ~ " + parityBlkNum);
+    }
+    
+    Path file = new Path(fileName);
+    
+    testCreateStripedFile(file, fileLen);
+    
+    LocatedBlocks locatedBlocks = getLocatedBlocks(file);
+    assertEquals(locatedBlocks.getFileLength(), fileLen);
+    
+    LocatedStripedBlock lastBlock = 
+        (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
+    
+    DatanodeInfo[] storageInfos = lastBlock.getLocations();
+    int[] indices = lastBlock.getBlockIndices();
+    
+    BitSet bitset = new BitSet(dnNum);
+    for (DatanodeInfo storageInfo : storageInfos) {
+      bitset.set(dnMap.get(storageInfo));
+    }
+    
+    int[] toDead = new int[toRecoverBlockNum];
+    int n = 0;
+    for (int i = 0; i < indices.length; i++) {
+      if (n < toRecoverBlockNum) {
+        if (recovery == 0) {
+          if (indices[i] >= dataBlkNum) {
+            toDead[n++] = i;
+          }
+        } else if (recovery == 1) {
+          if (indices[i] < dataBlkNum) {
+            toDead[n++] = i;
+          }
+        } else {
+          toDead[n++] = i;
+        }
+      } else {
+        break;
+      }
+    }
+    
+    DatanodeInfo[] dataDNs = new DatanodeInfo[toRecoverBlockNum];
+    int[] deadDnIndices = new int[toRecoverBlockNum];
+    ExtendedBlock[] blocks = new ExtendedBlock[toRecoverBlockNum];
+    File[] replicas = new File[toRecoverBlockNum];
+    File[] metadatas = new File[toRecoverBlockNum];
+    byte[][] replicaContents = new byte[toRecoverBlockNum][];
+    for (int i = 0; i < toRecoverBlockNum; i++) {
+      dataDNs[i] = storageInfos[toDead[i]];
+      deadDnIndices[i] = dnMap.get(dataDNs[i]);
+      
+      // Check the block replica file on deadDn before it dead.
+      blocks[i] = StripedBlockUtil.constructStripedBlock(
+          lastBlock.getBlock(), cellSize, dataBlkNum, indices[toDead[i]]);
+      replicas[i] = cluster.getBlockFile(deadDnIndices[i], blocks[i]);
+      metadatas[i] = cluster.getBlockMetadataFile(deadDnIndices[i], blocks[i]);
+      // the block replica on the datanode should be the same as expected
+      assertEquals(replicas[i].length(), 
+          StripedBlockUtil.getStripedBlockLength(
+          lastBlock.getBlockSize(), cellSize, dataBlkNum, indices[toDead[i]]));
+      assertTrue(metadatas[i].getName().
+          endsWith(blocks[i].getGenerationStamp() + ".meta"));
+      replicaContents[i] = readReplica(replicas[i]);
+    }
+    
+    try {
+      DatanodeID[] dnIDs = new DatanodeID[toRecoverBlockNum];
+      for (int i = 0; i < toRecoverBlockNum; i++) {
+        /*
+         * Kill the datanode which contains one replica
+         * We need to make sure it dead in namenode: clear its update time and 
+         * trigger NN to check heartbeat.
+         */
+        DataNode dn = cluster.getDataNodes().get(deadDnIndices[i]);
+        dn.shutdown();
+        dnIDs[i] = dn.getDatanodeId();
+      }
+      setDataNodesDead(dnIDs);
+       
+      
+      // Check the locatedBlocks of the file again
+      locatedBlocks = getLocatedBlocks(file);
+      lastBlock = (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
+      storageInfos = lastBlock.getLocations();
+      assertEquals(storageInfos.length, groupSize - toRecoverBlockNum);
+      
+      int[] targetDNs = new int[dnNum - groupSize];
+      n = 0;
+      for (int i = 0; i < dnNum; i++) {
+        if (!bitset.get(i)) { // not contain replica of the block.
+          targetDNs[n++] = i;
+        }
+      }
+      
+      waitForRecoveryFinished(file);
+      
+      targetDNs = sortTargetsByReplicas(blocks, targetDNs);
+      
+      // Check the replica on the new target node.
+      for (int i = 0; i < toRecoverBlockNum; i++) {
+        File replicaAfterRecovery = cluster.getBlockFile(targetDNs[i], blocks[i]);
+        File metadataAfterRecovery = 
+            cluster.getBlockMetadataFile(targetDNs[i], blocks[i]);
+        assertEquals(replicaAfterRecovery.length(), replicas[i].length());
+        assertTrue(metadataAfterRecovery.getName().
+            endsWith(blocks[i].getGenerationStamp() + ".meta"));
+        byte[] replicaContentAfterRecovery = readReplica(replicaAfterRecovery);
+        
+        Assert.assertArrayEquals(replicaContents[i], replicaContentAfterRecovery);
+      }
+    } finally {
+      for (int i = 0; i < toRecoverBlockNum; i++) {
+        restartDataNode(toDead[i]);
+      }
+      cluster.waitActive();
+    }
+    fs.delete(file, true);
+  }
+  
+  private void setDataNodesDead(DatanodeID[] dnIDs) throws IOException {
+    for (DatanodeID dn : dnIDs) {
+      DatanodeDescriptor dnd =
+          NameNodeAdapter.getDatanode(cluster.getNamesystem(), dn);
+      DFSTestUtil.setDatanodeDead(dnd);
+    }
+    
+    BlockManagerTestUtil.checkHeartbeat(cluster.getNamesystem().getBlockManager());
+  }
+  
+  private void restartDataNode(int dn) {
+    try {
+      cluster.restartDataNode(dn, true, true);
+    } catch (IOException e) {
+    }
+  }
+  
+  private int[] sortTargetsByReplicas(ExtendedBlock[] blocks, int[] targetDNs) {
+    int[] result = new int[blocks.length];
+    for (int i = 0; i < blocks.length; i++) {
+      result[i] = -1;
+      for (int j = 0; j < targetDNs.length; j++) {
+        if (targetDNs[j] != -1) {
+          File replica = cluster.getBlockFile(targetDNs[j], blocks[i]);
+          if (replica != null) {
+            result[i] = targetDNs[j];
+            targetDNs[j] = -1;
+            break;
+          }
+        }
+      }
+      if (result[i] == -1) {
+        Assert.fail("Failed to recover striped block: " + blocks[i].getBlockId());
+      }
+    }
+    return result;
+  }
+  
+  private byte[] readReplica(File replica) throws IOException {
+    int length = (int)replica.length();
+    ByteArrayOutputStream content = new ByteArrayOutputStream(length);
+    FileInputStream in = new FileInputStream(replica);
+    try {
+      byte[] buffer = new byte[1024];
+      int total = 0;
+      while (total < length) {
+        int n = in.read(buffer);
+        if (n <= 0) {
+          break;
+        }
+        content.write(buffer, 0, n);
+        total += n;
+      }
+      if (total < length) {
+        Assert.fail("Failed to read all content of replica");
+      }
+      return content.toByteArray();
+    } finally {
+      in.close();
+    }
+  }
+  
+  private LocatedBlocks waitForRecoveryFinished(Path file) throws Exception {
+    final int ATTEMPTS = 60;
+    for (int i = 0; i < ATTEMPTS; i++) {
+      LocatedBlocks locatedBlocks = getLocatedBlocks(file);
+      LocatedStripedBlock lastBlock = 
+          (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
+      DatanodeInfo[] storageInfos = lastBlock.getLocations();
+      if (storageInfos.length >= groupSize) {
+        return locatedBlocks;
+      }
+      Thread.sleep(1000);
+    }
+    throw new IOException ("Time out waiting for EC block recovery.");
+  }
+  
+  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];
+    DFSUtil.getRandom().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();
+    }
+  }
+}


[09/50] hadoop git commit: HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. Contributed by Hui Zheng.

Posted by zh...@apache.org.
HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. Contributed by Hui Zheng.


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

Branch: refs/heads/HDFS-7285
Commit: c1aa8f542b2e724aa954b6324320db040d3f742d
Parents: 0559d00
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Apr 17 12:05:31 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:15 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt          |  2 ++
 .../hdfs/server/blockmanagement/BlockManager.java | 18 ++++--------------
 2 files changed, 6 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1aa8f54/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 78ca6d3..0ed61cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -85,3 +85,5 @@
 
     HDFS-7994. Detect if resevered EC Block ID is already used during namenode
     startup. (Hui Zheng via szetszwo)
+
+    HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. (Hui Zheng via zhz).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1aa8f54/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 5b554b3..f7b1fda 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
@@ -2932,15 +2932,6 @@ public class BlockManager {
   }
 
   /**
-   * Set the value of whether there are any non-EC blocks using StripedID.
-   *
-   * @param has - the value of whether there are any non-EC blocks using StripedID.
-   */
-  public void hasNonEcBlockUsingStripedID(boolean has){
-    hasNonEcBlockUsingStripedID = has;
-  }
-
-  /**
    * Process a single possibly misreplicated block. This adds it to the
    * appropriate queues if necessary, and returns a result code indicating
    * what happened with it.
@@ -3547,7 +3538,7 @@ public class BlockManager {
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
       info = blocksMap.getStoredBlock(
           new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
-      if ((info == null) && hasNonEcBlockUsingStripedID()){
+      if ((info == null) && hasNonEcBlockUsingStripedID){
         info = blocksMap.getStoredBlock(block);
       }
     } else {
@@ -3731,10 +3722,9 @@ public class BlockManager {
    */
   public BlockInfo addBlockCollectionWithCheck(
       BlockInfo block, BlockCollection bc) {
-    if (!hasNonEcBlockUsingStripedID()){
-      if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
-        hasNonEcBlockUsingStripedID(true);
-      }
+    if (!hasNonEcBlockUsingStripedID && !block.isStriped() &&
+        BlockIdManager.isStripedBlockID(block.getBlockId())) {
+      hasNonEcBlockUsingStripedID = true;
     }
     return addBlockCollection(block, bc);
   }


[44/50] hadoop git commit: HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding. Contributed by Uma Maheswara Rao G

Posted by zh...@apache.org.
HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding. 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/94bee5de
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/94bee5de
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/94bee5de

Branch: refs/heads/HDFS-7285
Commit: 94bee5de42d4a78a9a27ebf2534d32992782d146
Parents: 9f929d3
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Thu May 7 16:26:01 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:40:58 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  10 +-
 .../hadoop/hdfs/DFSStripedInputStream.java      |   2 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |  10 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   4 +-
 .../org/apache/hadoop/hdfs/protocol/ECInfo.java |  41 ------
 .../apache/hadoop/hdfs/protocol/ECZoneInfo.java |  56 --------
 .../hadoop/hdfs/protocol/ErasureCodingInfo.java |  41 ++++++
 .../hdfs/protocol/ErasureCodingZoneInfo.java    |  56 ++++++++
 ...tNamenodeProtocolServerSideTranslatorPB.java |  18 +--
 .../ClientNamenodeProtocolTranslatorPB.java     |  16 +--
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  24 ++--
 .../hdfs/server/namenode/ECSchemaManager.java   | 127 -------------------
 .../namenode/ErasureCodingSchemaManager.java    | 127 +++++++++++++++++++
 .../namenode/ErasureCodingZoneManager.java      |  12 +-
 .../hdfs/server/namenode/FSDirectory.java       |   4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  24 ++--
 .../hdfs/server/namenode/NameNodeRpcServer.java |   8 +-
 .../hdfs/tools/erasurecode/ECCommand.java       |   4 +-
 .../src/main/proto/ClientNamenodeProtocol.proto |   4 +-
 .../src/main/proto/erasurecoding.proto          |  16 +--
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |   8 +-
 .../org/apache/hadoop/hdfs/TestECSchemas.java   |   2 +-
 .../hadoop/hdfs/TestErasureCodingZones.java     |  10 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |  10 +-
 .../server/namenode/TestStripedINodeFile.java   |  16 +--
 26 files changed, 328 insertions(+), 325 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 8729f8a..11e8376 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -183,3 +183,6 @@
 
     HDFS-8334. Erasure coding: rename DFSStripedInputStream related test 
     classes. (Zhe Zhang)
+
+    HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding
+    (umamahesh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 b149bcf..47706fc 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
@@ -118,8 +118,8 @@ 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.ECInfo;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -1188,7 +1188,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     //    Get block info from namenode
     TraceScope scope = getPathTraceScope("newDFSInputStream", src);
     try {
-      ECInfo info = getErasureCodingInfo(src);
+      ErasureCodingInfo info = getErasureCodingInfo(src);
       if (info != null) {
         return new DFSStripedInputStream(this, src, verifyChecksum, info);
       } else {
@@ -3121,7 +3121,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  public ECInfo getErasureCodingInfo(String src) throws IOException {
+  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
     checkOpen();
     TraceScope scope = getPathTraceScope("getErasureCodingInfo", src);
     try {
@@ -3338,7 +3338,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @return Returns the zone information if path is in EC Zone, null otherwise
    * @throws IOException
    */
-  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
     checkOpen();
     TraceScope scope = getPathTraceScope("getErasureCodingZoneInfo", src);
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 13c4743..7cb7b6d 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
@@ -127,7 +127,7 @@ public class DFSStripedInputStream extends DFSInputStream {
   private final CompletionService<Integer> readingService;
 
   DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
-      ECInfo ecInfo) throws IOException {
+      ErasureCodingInfo ecInfo) throws IOException {
     super(dfsClient, src, verifyChecksum);
     // ECInfo is restored from NN just before reading striped file.
     assert ecInfo != null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 ede4f48..723dbcd 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
@@ -72,7 +72,7 @@ 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.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -2306,18 +2306,18 @@ public class DistributedFileSystem extends FileSystem {
    * @return Returns the zone information if path is in EC zone, null otherwise
    * @throws IOException
    */
-  public ECZoneInfo getErasureCodingZoneInfo(final Path path)
+  public ErasureCodingZoneInfo getErasureCodingZoneInfo(final Path path)
       throws IOException {
     Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<ECZoneInfo>() {
+    return new FileSystemLinkResolver<ErasureCodingZoneInfo>() {
       @Override
-      public ECZoneInfo doCall(final Path p) throws IOException,
+      public ErasureCodingZoneInfo doCall(final Path p) throws IOException,
           UnresolvedLinkException {
         return dfs.getErasureCodingZoneInfo(getPathName(p));
       }
 
       @Override
-      public ECZoneInfo next(final FileSystem fs, final Path p)
+      public ErasureCodingZoneInfo next(final FileSystem fs, final Path p)
           throws IOException {
         if (fs instanceof DistributedFileSystem) {
           DistributedFileSystem myDfs = (DistributedFileSystem) fs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 76e2d12..b0b457c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1474,7 +1474,7 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  public ECInfo getErasureCodingInfo(String src) throws IOException;
+  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException;
 
   /**
    * Gets list of ECSchemas loaded in Namenode
@@ -1492,5 +1492,5 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException;
+  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java
deleted file mode 100644
index ca642c2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import org.apache.hadoop.io.erasurecode.ECSchema;
-
-/**
- * Class to provide information, such as ECSchema, for a file/block.
- */
-public class ECInfo {
-  private final String src;
-  private final ECSchema schema;
-
-  public ECInfo(String src, ECSchema schema) {
-    this.src = src;
-    this.schema = schema;
-  }
-
-  public String getSrc() {
-    return src;
-  }
-
-  public ECSchema getSchema() {
-    return schema;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java
deleted file mode 100644
index ecfb92e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java
+++ /dev/null
@@ -1,56 +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;
-
-import org.apache.hadoop.io.erasurecode.ECSchema;
-
-/**
- * Information about the EC Zone at the specified path.
- */
-public class ECZoneInfo {
-
-  private String dir;
-  private ECSchema schema;
-
-  public ECZoneInfo(String dir, ECSchema schema) {
-    this.dir = dir;
-    this.schema = schema;
-  }
-
-  /**
-   * Get directory of the EC zone.
-   * 
-   * @return
-   */
-  public String getDir() {
-    return dir;
-  }
-
-  /**
-   * Get the schema for the EC Zone
-   * 
-   * @return
-   */
-  public ECSchema getSchema() {
-    return schema;
-  }
-
-  @Override
-  public String toString() {
-    return "Dir: " + getDir() + ", Schema: " + schema;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java
new file mode 100644
index 0000000..bad09b3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+/**
+ * Class to provide information, such as ECSchema, for a file/block.
+ */
+public class ErasureCodingInfo {
+  private final String src;
+  private final ECSchema schema;
+
+  public ErasureCodingInfo(String src, ECSchema schema) {
+    this.src = src;
+    this.schema = schema;
+  }
+
+  public String getSrc() {
+    return src;
+  }
+
+  public ECSchema getSchema() {
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java
new file mode 100644
index 0000000..ec0efbd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java
@@ -0,0 +1,56 @@
+/**
+ * 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 org.apache.hadoop.io.erasurecode.ECSchema;
+
+/**
+ * Information about the EC Zone at the specified path.
+ */
+public class ErasureCodingZoneInfo {
+
+  private String dir;
+  private ECSchema schema;
+
+  public ErasureCodingZoneInfo(String dir, ECSchema schema) {
+    this.dir = dir;
+    this.schema = schema;
+  }
+
+  /**
+   * Get directory of the EC zone.
+   * 
+   * @return
+   */
+  public String getDir() {
+    return dir;
+  }
+
+  /**
+   * Get the schema for the EC Zone
+   * 
+   * @return
+   */
+  public ECSchema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public String toString() {
+    return "Dir: " + getDir() + ", Schema: " + schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 f9bdb71..bae753b 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,8 +35,8 @@ 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.ECInfo;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -202,8 +202,8 @@ 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.GetECSchemasRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
@@ -1527,7 +1527,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public GetErasureCodingInfoResponseProto getErasureCodingInfo(RpcController controller,
       GetErasureCodingInfoRequestProto request) throws ServiceException {
     try {
-      ECInfo ecInfo = server.getErasureCodingInfo(request.getSrc());
+      ErasureCodingInfo ecInfo = server.getErasureCodingInfo(request.getSrc());
       GetErasureCodingInfoResponseProto.Builder resBuilder = GetErasureCodingInfoResponseProto
           .newBuilder();
       if (ecInfo != null) {
@@ -1556,11 +1556,11 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public GetECZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller,
-      GetECZoneInfoRequestProto request) throws ServiceException {
+  public GetErasureCodingZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller,
+      GetErasureCodingZoneInfoRequestProto request) throws ServiceException {
     try {
-      ECZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc());
-      GetECZoneInfoResponseProto.Builder builder = GetECZoneInfoResponseProto.newBuilder();
+      ErasureCodingZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc());
+      GetErasureCodingZoneInfoResponseProto.Builder builder = GetErasureCodingZoneInfoResponseProto.newBuilder();
       if (ecZoneInfo != null) {
         builder.setECZoneInfo(PBHelper.convertECZoneInfo(ecZoneInfo));
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 6c1c971..014fcef 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,8 +58,8 @@ 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.ECInfo;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -168,8 +168,8 @@ 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.GetECSchemasRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
@@ -1550,7 +1550,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ECInfo getErasureCodingInfo(String src) throws IOException {
+  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
     GetErasureCodingInfoRequestProto req = GetErasureCodingInfoRequestProto.newBuilder()
         .setSrc(src).build();
     try {
@@ -1581,11 +1581,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
-    GetECZoneInfoRequestProto req = GetECZoneInfoRequestProto.newBuilder()
+  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+    GetErasureCodingZoneInfoRequestProto req = GetErasureCodingZoneInfoRequestProto.newBuilder()
         .setSrc(src).build();
     try {
-      GetECZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo(
+      GetErasureCodingZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo(
           null, req);
       if (response.hasECZoneInfo()) {
         return PBHelper.convertECZoneInfo(response.getECZoneInfo());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 e230232..26bdf34 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
@@ -77,13 +77,13 @@ 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.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 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.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -135,10 +135,10 @@ 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.ECInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECZoneInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneInfoProto;
 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.BlockStoragePolicyProto;
@@ -3117,13 +3117,13 @@ public class PBHelper {
         build();
   }
 
-  public static ECInfo convertECInfo(ECInfoProto ecInfoProto) {
-    return new ECInfo(ecInfoProto.getSrc(),
+  public static ErasureCodingInfo convertECInfo(ErasureCodingInfoProto ecInfoProto) {
+    return new ErasureCodingInfo(ecInfoProto.getSrc(),
         convertECSchema(ecInfoProto.getSchema()));
   }
 
-  public static ECInfoProto convertECInfo(ECInfo ecInfo) {
-    return ECInfoProto.newBuilder().setSrc(ecInfo.getSrc())
+  public static ErasureCodingInfoProto convertECInfo(ErasureCodingInfo ecInfo) {
+    return ErasureCodingInfoProto.newBuilder().setSrc(ecInfo.getSrc())
         .setSchema(convertECSchema(ecInfo.getSchema())).build();
   }
 
@@ -3151,13 +3151,13 @@ public class PBHelper {
     return builder.build();
   }
 
-  public static ECZoneInfoProto convertECZoneInfo(ECZoneInfo ecZoneInfo) {
-    return ECZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir())
+  public static ErasureCodingZoneInfoProto convertECZoneInfo(ErasureCodingZoneInfo ecZoneInfo) {
+    return ErasureCodingZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir())
         .setSchema(convertECSchema(ecZoneInfo.getSchema())).build();
   }
 
-  public static ECZoneInfo convertECZoneInfo(ECZoneInfoProto ecZoneInfoProto) {
-    return new ECZoneInfo(ecZoneInfoProto.getDir(),
+  public static ErasureCodingZoneInfo convertECZoneInfo(ErasureCodingZoneInfoProto ecZoneInfoProto) {
+    return new ErasureCodingZoneInfo(ecZoneInfoProto.getDir(),
         convertECSchema(ecZoneInfoProto.getSchema()));
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
deleted file mode 100644
index 2d63498..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
+++ /dev/null
@@ -1,127 +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 org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.erasurecode.ECSchema;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-/**
- * This manages EC schemas predefined and activated in the system.
- * It loads customized schemas and syncs with persisted ones in
- * NameNode image.
- *
- * This class is instantiated by the FSNamesystem.
- */
-@InterfaceAudience.LimitedPrivate({"HDFS"})
-public final class ECSchemaManager {
-
-  /**
-   * TODO: HDFS-8095
-   */
-  private static final int DEFAULT_DATA_BLOCKS = 6;
-  private static final int DEFAULT_PARITY_BLOCKS = 3;
-  private static final String DEFAULT_CODEC_NAME = "rs";
-  private static final String DEFAULT_SCHEMA_NAME = "RS-6-3";
-  private static final ECSchema SYS_DEFAULT_SCHEMA =
-      new ECSchema(DEFAULT_SCHEMA_NAME,
-               DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS);
-
-  //We may add more later.
-  private static ECSchema[] SYS_SCHEMAS = new ECSchema[] {
-      SYS_DEFAULT_SCHEMA
-  };
-
-  /**
-   * All active EC activeSchemas maintained in NN memory for fast querying,
-   * identified and sorted by its name.
-   */
-  private final Map<String, ECSchema> activeSchemas;
-
-  ECSchemaManager() {
-
-    this.activeSchemas = new TreeMap<String, ECSchema>();
-    for (ECSchema schema : SYS_SCHEMAS) {
-      activeSchemas.put(schema.getSchemaName(), schema);
-    }
-
-    /**
-     * TODO: HDFS-7859 persist into NameNode
-     * load persistent schemas from image and editlog, which is done only once
-     * during NameNode startup. This can be done here or in a separate method.
-     */
-  }
-
-  /**
-   * Get system defined schemas.
-   * @return system schemas
-   */
-  public static ECSchema[] getSystemSchemas() {
-    return SYS_SCHEMAS;
-  }
-
-  /**
-   * Get system-wide default EC schema, which can be used by default when no
-   * schema is specified for an EC zone.
-   * @return schema
-   */
-  public static ECSchema getSystemDefaultSchema() {
-    return SYS_DEFAULT_SCHEMA;
-  }
-
-  /**
-   * Tell the specified schema is the system default one or not.
-   * @param schema
-   * @return true if it's the default false otherwise
-   */
-  public static boolean isSystemDefault(ECSchema schema) {
-    if (schema == null) {
-      throw new IllegalArgumentException("Invalid schema parameter");
-    }
-
-    // schema name is the identifier.
-    return SYS_DEFAULT_SCHEMA.getSchemaName().equals(schema.getSchemaName());
-  }
-
-  /**
-   * Get all EC schemas that's available to use.
-   * @return all EC schemas
-   */
-  public ECSchema[] getSchemas() {
-    ECSchema[] results = new ECSchema[activeSchemas.size()];
-    return activeSchemas.values().toArray(results);
-  }
-
-  /**
-   * Get the EC schema specified by the schema name.
-   * @param schemaName
-   * @return EC schema specified by the schema name
-   */
-  public ECSchema getSchema(String schemaName) {
-    return activeSchemas.get(schemaName);
-  }
-
-  /**
-   * Clear and clean up
-   */
-  public void clear() {
-    activeSchemas.clear();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java
new file mode 100644
index 0000000..4c4aae9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingSchemaManager.java
@@ -0,0 +1,127 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * This manages EC schemas predefined and activated in the system.
+ * It loads customized schemas and syncs with persisted ones in
+ * NameNode image.
+ *
+ * This class is instantiated by the FSNamesystem.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+public final class ErasureCodingSchemaManager {
+
+  /**
+   * TODO: HDFS-8095
+   */
+  private static final int DEFAULT_DATA_BLOCKS = 6;
+  private static final int DEFAULT_PARITY_BLOCKS = 3;
+  private static final String DEFAULT_CODEC_NAME = "rs";
+  private static final String DEFAULT_SCHEMA_NAME = "RS-6-3";
+  private static final ECSchema SYS_DEFAULT_SCHEMA =
+      new ECSchema(DEFAULT_SCHEMA_NAME,
+               DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS);
+
+  //We may add more later.
+  private static ECSchema[] SYS_SCHEMAS = new ECSchema[] {
+      SYS_DEFAULT_SCHEMA
+  };
+
+  /**
+   * All active EC activeSchemas maintained in NN memory for fast querying,
+   * identified and sorted by its name.
+   */
+  private final Map<String, ECSchema> activeSchemas;
+
+  ErasureCodingSchemaManager() {
+
+    this.activeSchemas = new TreeMap<String, ECSchema>();
+    for (ECSchema schema : SYS_SCHEMAS) {
+      activeSchemas.put(schema.getSchemaName(), schema);
+    }
+
+    /**
+     * TODO: HDFS-7859 persist into NameNode
+     * load persistent schemas from image and editlog, which is done only once
+     * during NameNode startup. This can be done here or in a separate method.
+     */
+  }
+
+  /**
+   * Get system defined schemas.
+   * @return system schemas
+   */
+  public static ECSchema[] getSystemSchemas() {
+    return SYS_SCHEMAS;
+  }
+
+  /**
+   * Get system-wide default EC schema, which can be used by default when no
+   * schema is specified for an EC zone.
+   * @return schema
+   */
+  public static ECSchema getSystemDefaultSchema() {
+    return SYS_DEFAULT_SCHEMA;
+  }
+
+  /**
+   * Tell the specified schema is the system default one or not.
+   * @param schema
+   * @return true if it's the default false otherwise
+   */
+  public static boolean isSystemDefault(ECSchema schema) {
+    if (schema == null) {
+      throw new IllegalArgumentException("Invalid schema parameter");
+    }
+
+    // schema name is the identifier.
+    return SYS_DEFAULT_SCHEMA.getSchemaName().equals(schema.getSchemaName());
+  }
+
+  /**
+   * Get all EC schemas that's available to use.
+   * @return all EC schemas
+   */
+  public ECSchema[] getSchemas() {
+    ECSchema[] results = new ECSchema[activeSchemas.size()];
+    return activeSchemas.values().toArray(results);
+  }
+
+  /**
+   * Get the EC schema specified by the schema name.
+   * @param schemaName
+   * @return EC schema specified by the schema name
+   */
+  public ECSchema getSchema(String schemaName) {
+    return activeSchemas.get(schemaName);
+  }
+
+  /**
+   * Clear and clean up
+   */
+  public void clear() {
+    activeSchemas.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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
index 14d4e29..6b6add7 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
@@ -22,7 +22,7 @@ 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.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import java.io.IOException;
@@ -53,11 +53,11 @@ public class ErasureCodingZoneManager {
   }
 
   ECSchema getECSchema(INodesInPath iip) throws IOException {
-    ECZoneInfo ecZoneInfo = getECZoneInfo(iip);
+    ErasureCodingZoneInfo ecZoneInfo = getECZoneInfo(iip);
     return ecZoneInfo == null ? null : ecZoneInfo.getSchema();
   }
 
-  ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
+  ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     List<INode> inodes = iip.getReadOnlyINodes();
@@ -79,9 +79,9 @@ public class ErasureCodingZoneManager {
       for (XAttr xAttr : xAttrs) {
         if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
           String schemaName = new String(xAttr.getValue());
-          ECSchema schema = dir.getFSNamesystem().getSchemaManager()
+          ECSchema schema = dir.getFSNamesystem().getECSchemaManager()
               .getSchema(schemaName);
-          return new ECZoneInfo(inode.getFullPathName(), schema);
+          return new ErasureCodingZoneInfo(inode.getFullPathName(), schema);
         }
       }
     }
@@ -110,7 +110,7 @@ public class ErasureCodingZoneManager {
 
     // System default schema will be used since no specified.
     if (schema == null) {
-      schema = ECSchemaManager.getSystemDefaultSchema();
+      schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     }
 
     // Now persist the schema name in xattr

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 0d3ec51..3f1b182 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
@@ -45,7 +45,7 @@ import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
@@ -1416,7 +1416,7 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
+  ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
     readLock();
     try {
       return ecZoneManager.getECZoneInfo(iip);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 9e6f74a..8e8f576 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
@@ -181,8 +181,8 @@ 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.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ECInfo;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -431,7 +431,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private final BlockManager blockManager;
   private final SnapshotManager snapshotManager;
   private final CacheManager cacheManager;
-  private final ECSchemaManager schemaManager;
+  private final ErasureCodingSchemaManager ecSchemaManager;
   private final DatanodeStatistics datanodeStatistics;
 
   private String nameserviceId;
@@ -611,7 +611,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     leaseManager.removeAllLeases();
     snapshotManager.clearSnapshottableDirs();
     cacheManager.clear();
-    schemaManager.clear();
+    ecSchemaManager.clear();
     setImageLoaded(false);
     blockManager.clear();
   }
@@ -851,7 +851,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       this.dir = new FSDirectory(this, conf);
       this.snapshotManager = new SnapshotManager(dir);
       this.cacheManager = new CacheManager(this, conf, blockManager);
-      this.schemaManager = new ECSchemaManager();
+      this.ecSchemaManager = new ErasureCodingSchemaManager();
       this.safeMode = new SafeModeInfo(conf);
       this.topConf = new TopConf(conf);
       this.auditLoggers = initAuditLoggers(conf);
@@ -7223,8 +7223,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /** @return the schema manager. */
-  public ECSchemaManager getSchemaManager() {
-    return schemaManager;
+  public ErasureCodingSchemaManager getECSchemaManager() {
+    return ecSchemaManager;
   }
 
   @Override  // NameNodeMXBean
@@ -8170,11 +8170,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Get the erasure coding information for specified src
    */
-  ECInfo getErasureCodingInfo(String src) throws AccessControlException,
+  ErasureCodingInfo getErasureCodingInfo(String src) throws AccessControlException,
       UnresolvedLinkException, IOException {
     ECSchema schema = getECSchemaForPath(src);
     if (schema != null) {
-      return new ECInfo(src, schema);
+      return new ErasureCodingInfo(src, schema);
     }
     return null;
   }
@@ -8182,7 +8182,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Get the erasure coding zone information for specified path
    */
-  ECZoneInfo getErasureCodingZoneInfo(String src) throws AccessControlException,
+  ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws AccessControlException,
       UnresolvedLinkException, IOException {
     checkOperation(OperationCategory.READ);
     final byte[][] pathComponents = FSDirectory
@@ -8211,7 +8211,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return schemaManager.getSchemas();
+      return ecSchemaManager.getSchemas();
     } finally {
       readUnlock();
     }
@@ -8226,7 +8226,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return schemaManager.getSchema(schemaName);
+      return ecSchemaManager.getSchema(schemaName);
     } finally {
       readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 06701a9..ef8ee62 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
@@ -84,8 +84,8 @@ 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.ECInfo;
-import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
@@ -2057,7 +2057,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public ECInfo getErasureCodingInfo(String src) throws IOException {
+  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
     checkNNStartup();
     return namesystem.getErasureCodingInfo(src);
   }
@@ -2069,7 +2069,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
     checkNNStartup();
     return namesystem.getErasureCodingZoneInfo(src);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 802a46d..d53844d 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,7 @@ 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.ECZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.StringUtils;
@@ -164,7 +164,7 @@ public abstract class ECCommand extends Command {
       super.processPath(item);
       DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
       try {
-        ECZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path);
+        ErasureCodingZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path);
         out.println(ecZoneInfo.toString());
       } catch (IOException e) {
         throw new IOException("Unable to create EC zone for the path "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
index 046120d..0a0a4c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -867,6 +867,6 @@ service ClientNamenodeProtocol {
       returns(GetErasureCodingInfoResponseProto);
   rpc getECSchemas(GetECSchemasRequestProto)
       returns(GetECSchemasResponseProto);
-  rpc getErasureCodingZoneInfo(GetECZoneInfoRequestProto) 
-      returns(GetECZoneInfoResponseProto);
+  rpc getErasureCodingZoneInfo(GetErasureCodingZoneInfoRequestProto) 
+      returns(GetErasureCodingZoneInfoResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
index 702f6fd..7a19a80 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -43,17 +43,17 @@ message ECSchemaProto {
 }
 
 /**
- * ECInfo
+ * ErasureCodingInfo
  */
-message ECInfoProto {
+message ErasureCodingInfoProto {
  required string src = 1;
  required ECSchemaProto schema = 2;
 }
 
 /**
- * ECZoneInfo
+ * ErasureCodingZoneInfo
  */
-message ECZoneInfoProto {
+message ErasureCodingZoneInfoProto {
   required string dir = 1;
   required ECSchemaProto schema = 2;
 }
@@ -71,7 +71,7 @@ message GetErasureCodingInfoRequestProto {
 }
 
 message GetErasureCodingInfoResponseProto {
-  optional ECInfoProto ECInfo = 1;
+  optional ErasureCodingInfoProto ECInfo = 1;
 }
 
 message GetECSchemasRequestProto { // void request
@@ -81,12 +81,12 @@ message GetECSchemasResponseProto {
   repeated ECSchemaProto schemas = 1;
 }
 
-message GetECZoneInfoRequestProto {
+message GetErasureCodingZoneInfoRequestProto {
   required string src = 1; // path to get the zone info
 }
 
-message GetECZoneInfoResponseProto {
-  optional ECZoneInfoProto ECZoneInfo = 1;
+message GetErasureCodingZoneInfoResponseProto {
+  optional ErasureCodingZoneInfoProto ECZoneInfo = 1;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 a1f704d..4da9c26 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,7 @@ 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.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -34,7 +34,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
-import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -54,8 +54,8 @@ public class TestDFSStripedInputStream {
   private DistributedFileSystem fs;
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
-  private ECInfo info = new ECInfo(filePath.toString(),
-      ECSchemaManager.getSystemDefaultSchema());
+  private ErasureCodingInfo info = new ErasureCodingInfo(filePath.toString(),
+      ErasureCodingSchemaManager.getSystemDefaultSchema());
   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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
index 83d208a..88198c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
@@ -22,7 +22,7 @@ import static org.junit.Assert.*;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.After;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 f1aec82..59818c0 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
@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.ECInfo;
-import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.io.erasurecode.ECSchema;
@@ -158,7 +158,7 @@ public class TestErasureCodingZones {
     assertNull(fs.getClient().getErasureCodingInfo(src));
     // dir ECInfo after creating ec zone
     fs.getClient().createErasureCodingZone(src, null); //Default one will be used.
-    ECSchema sysDefaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema sysDefaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     verifyErasureCodingInfo(src, sysDefaultSchema);
     fs.create(new Path(ecDir, "/child1")).close();
     // verify for the files in ec zone
@@ -167,7 +167,7 @@ public class TestErasureCodingZones {
 
   @Test
   public void testGetErasureCodingInfo() throws Exception {
-    ECSchema[] sysSchemas = ECSchemaManager.getSystemSchemas();
+    ECSchema[] sysSchemas = ErasureCodingSchemaManager.getSystemSchemas();
     assertTrue("System schemas should be of only 1 for now",
         sysSchemas.length == 1);
 
@@ -187,7 +187,7 @@ public class TestErasureCodingZones {
 
   private void verifyErasureCodingInfo(
       String src, ECSchema usingSchema) throws IOException {
-    ECInfo ecInfo = fs.getClient().getErasureCodingInfo(src);
+    ErasureCodingInfo ecInfo = fs.getClient().getErasureCodingInfo(src);
     assertNotNull("ECInfo should have been non-null", ecInfo);
     assertEquals(src, ecInfo.getSrc());
     ECSchema schema = ecInfo.getSchema();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 f580cbb..1be00b6 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
@@ -71,7 +71,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
@@ -663,7 +663,7 @@ public class TestPBHelper {
     short[] liveBlkIndices0 = new short[2];
     BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo(
         new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0,
-        liveBlkIndices0, ECSchemaManager.getSystemDefaultSchema());
+        liveBlkIndices0, ErasureCodingSchemaManager.getSystemDefaultSchema());
     DatanodeInfo[] dnInfos1 = new DatanodeInfo[] {
         DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
     DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil
@@ -677,7 +677,7 @@ public class TestPBHelper {
     short[] liveBlkIndices1 = new short[2];
     BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo(
         new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1,
-        liveBlkIndices1, ECSchemaManager.getSystemDefaultSchema());
+        liveBlkIndices1, ErasureCodingSchemaManager.getSystemDefaultSchema());
     List<BlockECRecoveryInfo> blkRecoveryInfosList = new ArrayList<BlockECRecoveryInfo>();
     blkRecoveryInfosList.add(blkECRecoveryInfo0);
     blkRecoveryInfosList.add(blkECRecoveryInfo1);
@@ -723,8 +723,8 @@ public class TestPBHelper {
     ECSchema ecSchema2 = blkECRecoveryInfo2.getECSchema();
     // Compare ECSchemas same as default ECSchema as we used system default
     // ECSchema used in this test
-    compareECSchemas(ECSchemaManager.getSystemDefaultSchema(), ecSchema1);
-    compareECSchemas(ECSchemaManager.getSystemDefaultSchema(), ecSchema2);
+    compareECSchemas(ErasureCodingSchemaManager.getSystemDefaultSchema(), ecSchema1);
+    compareECSchemas(ErasureCodingSchemaManager.getSystemDefaultSchema(), ecSchema2);
   }
 
   private void compareECSchemas(ECSchema ecSchema1, ECSchema ecSchema2) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94bee5de/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 f97943b..7a330b9 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
@@ -60,7 +60,7 @@ public class TestStripedINodeFile {
 
   @Test
   public void testBlockStripedTotalBlockCount() {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
         = new BlockInfoStriped(blk,
@@ -72,7 +72,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedLength()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
@@ -87,7 +87,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedConsumedSpace()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
@@ -116,7 +116,7 @@ public class TestStripedINodeFile {
   @Test
   public void testMultipleBlockStripedConsumedSpace()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk1 = new Block(1);
@@ -141,7 +141,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedFileSize()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
@@ -160,7 +160,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedUCFileSize()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
@@ -177,7 +177,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedComputeQuotaUsage()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
@@ -204,7 +204,7 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedUCComputeQuotaUsage()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);


[05/50] hadoop git commit: HDFS-8120. Erasure coding: created util class to analyze striped block groups. Contributed by Zhe Zhang and Li Bo.

Posted by zh...@apache.org.
HDFS-8120. Erasure coding: created util class to analyze striped block groups. Contributed by Zhe Zhang and Li Bo.


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

Branch: refs/heads/HDFS-7285
Commit: ed68791cba9898300b16543ac13f3efcdc45d3eb
Parents: 43df7be
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Apr 15 12:59:27 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:14 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |   4 +-
 .../hadoop/hdfs/DFSStripedInputStream.java      |  77 +++--------
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  34 +++--
 .../apache/hadoop/hdfs/StripedDataStreamer.java |  58 ++------
 .../server/blockmanagement/BlockManager.java    |  26 +++-
 .../hadoop/hdfs/util/StripedBlockUtil.java      | 138 +++++++++++++++++++
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  91 +++++++-----
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |  83 +++++------
 .../apache/hadoop/hdfs/TestReadStripedFile.java |  92 +++----------
 .../server/namenode/TestAddStripedBlocks.java   | 107 ++++++++++++++
 .../namenode/TestRecoverStripedBlocks.java      |   3 +-
 .../hadoop/hdfs/util/TestStripedBlockUtil.java  | 125 +++++++++++++++++
 12 files changed, 562 insertions(+), 276 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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 9104f84..16250dd 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
@@ -1148,9 +1148,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         for (int i = 0; i < offsets.length; i++) {
           int nread = reader.readAll(buf, offsets[i], lengths[i]);
           updateReadStatistics(readStatistics, nread, reader);
-          if (nread != len) {
+          if (nread != lengths[i]) {
             throw new IOException("truncated return from reader.read(): " +
-                "excpected " + len + ", got " + nread);
+                "excpected " + lengths[i] + ", got " + nread);
           }
         }
         DFSClientFaultInjector.get().readFromDatanodeDelay();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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 8a431b1..d597407 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
@@ -25,6 +25,7 @@ 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.namenode.UnsupportedActionException;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
@@ -50,7 +51,7 @@ import java.util.concurrent.Future;
  *
  * | <- Striped Block Group -> |
  *  blk_0      blk_1       blk_2   <- A striped block group has
- *    |          |           |          {@link #groupSize} blocks
+ *    |          |           |          {@link #dataBlkNum} blocks
  *    v          v           v
  * +------+   +------+   +------+
  * |cell_0|   |cell_1|   |cell_2|  <- The logical read order should be
@@ -72,7 +73,7 @@ import java.util.concurrent.Future;
 public class DFSStripedInputStream extends DFSInputStream {
   /**
    * This method plans the read portion from each block in the stripe
-   * @param groupSize The size / width of the striping group
+   * @param dataBlkNum The number of data blocks in the striping group
    * @param cellSize The size of each striping cell
    * @param startInBlk Starting offset in the striped block
    * @param len Length of the read request
@@ -81,29 +82,29 @@ public class DFSStripedInputStream extends DFSInputStream {
    *         for an individual block in the group
    */
   @VisibleForTesting
-  static ReadPortion[] planReadPortions(final int groupSize,
+  static ReadPortion[] planReadPortions(final int dataBlkNum,
       final int cellSize, final long startInBlk, final int len, int bufOffset) {
-    ReadPortion[] results = new ReadPortion[groupSize];
-    for (int i = 0; i < groupSize; i++) {
+    ReadPortion[] results = new ReadPortion[dataBlkNum];
+    for (int i = 0; i < dataBlkNum; i++) {
       results[i] = new ReadPortion();
     }
 
     // cellIdxInBlk is the index of the cell in the block
     // E.g., cell_3 is the 2nd cell in blk_0
-    int cellIdxInBlk = (int) (startInBlk / (cellSize * groupSize));
+    int cellIdxInBlk = (int) (startInBlk / (cellSize * dataBlkNum));
 
     // blkIdxInGroup is the index of the block in the striped block group
     // E.g., blk_2 is the 3rd block in the group
-    final int blkIdxInGroup = (int) (startInBlk / cellSize % groupSize);
+    final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum);
     results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk +
         startInBlk % cellSize;
     boolean crossStripe = false;
-    for (int i = 1; i < groupSize; i++) {
-      if (blkIdxInGroup + i >= groupSize && !crossStripe) {
+    for (int i = 1; i < dataBlkNum; i++) {
+      if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) {
         cellIdxInBlk++;
         crossStripe = true;
       }
-      results[(blkIdxInGroup + i) % groupSize].startOffsetInBlock =
+      results[(blkIdxInGroup + i) % dataBlkNum].startOffsetInBlock =
           cellSize * cellIdxInBlk;
     }
 
@@ -112,57 +113,21 @@ public class DFSStripedInputStream extends DFSInputStream {
     results[blkIdxInGroup].lengths.add(firstCellLen);
     results[blkIdxInGroup].readLength += firstCellLen;
 
-    int i = (blkIdxInGroup + 1) % groupSize;
+    int i = (blkIdxInGroup + 1) % dataBlkNum;
     for (int done = firstCellLen; done < len; done += cellSize) {
       ReadPortion rp = results[i];
       rp.offsetsInBuf.add(done + bufOffset);
       final int readLen = Math.min(len - done, cellSize);
       rp.lengths.add(readLen);
       rp.readLength += readLen;
-      i = (i + 1) % groupSize;
+      i = (i + 1) % dataBlkNum;
     }
     return results;
   }
 
-  /**
-   * This method parses a striped block group into individual blocks.
-   *
-   * @param bg The striped block group
-   * @param dataBlkNum the number of data blocks
-   * @return An array containing the blocks in the group
-   */
-  @VisibleForTesting
-  static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg,
-      int dataBlkNum, int cellSize) {
-    int locatedBGSize = bg.getBlockIndices().length;
-    // TODO not considering missing blocks for now, only identify data blocks
-    LocatedBlock[] lbs = new LocatedBlock[dataBlkNum];
-    for (short i = 0; i < locatedBGSize; i++) {
-      final int idx = bg.getBlockIndices()[i];
-      if (idx < dataBlkNum && lbs[idx] == null) {
-        lbs[idx] = constructInternalBlock(bg, i, cellSize, idx);
-      }
-    }
-    return lbs;
-  }
-
-  private static LocatedBlock constructInternalBlock(LocatedStripedBlock bg,
-      int idxInReturnedLocs, int cellSize, int idxInBlockGroup) {
-    final ExtendedBlock blk = new ExtendedBlock(bg.getBlock());
-    blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup);
-    // TODO: fix the numBytes computation
-
-    return new LocatedBlock(blk,
-        new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
-        new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
-        new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
-        bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(),
-        null);
-  }
-
-
   private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-  private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS;
+  private final short dataBlkNum = HdfsConstants.NUM_DATA_BLOCKS;
+  private final short parityBlkNum = HdfsConstants.NUM_PARITY_BLOCKS;
 
   DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum)
       throws IOException {
@@ -199,7 +164,7 @@ public class DFSStripedInputStream extends DFSInputStream {
         "LocatedStripedBlock for a striped file";
 
     int idx = (int) (((blkStartOffset - lb.getStartOffset()) / cellSize)
-        % groupSize);
+        % dataBlkNum);
     // If indexing information is returned, iterate through the index array
     // to find the entry for position idx in the group
     LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
@@ -213,7 +178,8 @@ public class DFSStripedInputStream extends DFSInputStream {
       DFSClient.LOG.debug("getBlockAt for striped blocks, offset="
           + blkStartOffset + ". Obtained block " + lb + ", idx=" + idx);
     }
-    return constructInternalBlock(lsb, i, cellSize, idx);
+    return StripedBlockUtil.constructInternalBlock(lsb, i, cellSize,
+        dataBlkNum, idx);
   }
 
   private LocatedBlock getBlockGroupAt(long offset) throws IOException {
@@ -240,13 +206,14 @@ public class DFSStripedInputStream extends DFSInputStream {
     LocatedStripedBlock blockGroup = (LocatedStripedBlock) block;
 
     // Planning the portion of I/O for each shard
-    ReadPortion[] readPortions = planReadPortions(groupSize, cellSize, start,
+    ReadPortion[] readPortions = planReadPortions(dataBlkNum, cellSize, start,
         len, offset);
 
     // Parse group to get chosen DN location
-    LocatedBlock[] blks = parseStripedBlockGroup(blockGroup, groupSize, cellSize);
+    LocatedBlock[] blks = StripedBlockUtil.
+        parseStripedBlockGroup(blockGroup, cellSize, dataBlkNum, parityBlkNum);
 
-    for (short i = 0; i < groupSize; i++) {
+    for (short i = 0; i < dataBlkNum; i++) {
       ReadPortion rp = readPortions[i];
       if (rp.readLength <= 0) {
         continue;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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 1d0e1be..f11a657 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
@@ -32,6 +32,7 @@ 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.util.StripedBlockUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.util.DataChecksum;
@@ -309,10 +310,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
         streamer.closeSocket();
         if (streamer.isLeadingStreamer()) {
           leadingStreamer = streamer;
-        } else {
-          streamer.countTailingBlockGroupBytes();
         }
-
       } catch (InterruptedException e) {
         throw new IOException("Failed to shutdown streamer");
       } finally {
@@ -320,6 +318,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
         setClosed();
       }
     }
+    assert leadingStreamer != null : "One streamer should be leader";
     leadingStreamer.countTailingBlockGroupBytes();
   }
 
@@ -337,23 +336,28 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   }
 
   private void writeParityCellsForLastStripe() throws IOException{
-    if(currentBlockGroupBytes == 0 ||
-        currentBlockGroupBytes % stripeDataSize() == 0)
+    long parityBlkSize = StripedBlockUtil.getInternalBlockLength(
+        currentBlockGroupBytes, cellSize, blockGroupDataBlocks,
+        blockGroupDataBlocks + 1);
+    if (parityBlkSize == 0 || currentBlockGroupBytes % stripeDataSize() == 0) {
       return;
-    int lastStripeLen =(int)(currentBlockGroupBytes % stripeDataSize());
-    // Size of parity cells should equal the size of the first cell, if it
-    // is not full.
-    int parityCellSize = cellSize;
-    int index = lastStripeLen / cellSize;
-    if (lastStripeLen < cellSize) {
-      parityCellSize = lastStripeLen;
-      index++;
     }
+    int parityCellSize = parityBlkSize % cellSize == 0 ? cellSize :
+                        (int) (parityBlkSize % cellSize);
+
     for (int i = 0; i < blockGroupBlocks; i++) {
-      if (i >= index) {
+      long internalBlkLen = StripedBlockUtil.getInternalBlockLength(
+          currentBlockGroupBytes, cellSize, blockGroupDataBlocks, 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
+      if (internalBlkLen < parityBlkSize || i >= blockGroupDataBlocks) {
         int position = cellBuffers[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++) {
-          cellBuffers[i].put((byte)0);
+          cellBuffers[i].put((byte) 0);
         }
       }
       cellBuffers[i].flip();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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 710d92d..5614852 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,16 +19,16 @@
 package org.apache.hadoop.hdfs;
 
 import java.util.List;
-import org.apache.hadoop.fs.StorageType;
+
 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.security.token.block.BlockTokenIdentifier;
+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.security.token.Token;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 
@@ -134,19 +134,7 @@ public class StripedDataStreamer extends DataStreamer {
               "putting a block to stripeBlocks, ie = " + ie);
         }
       }
-    } else if (!isParityStreamer()) {
-      if (block == null || block.getNumBytes() == 0) {
-        LocatedBlock finishedBlock = new LocatedBlock(null, null);
-        try {
-          boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30,
-              TimeUnit.SECONDS);
-        } catch (InterruptedException ie) {
-          //TODO: Handle InterruptedException (HDFS-7786)
-          ie.printStackTrace();
-        }
-      }
     }
-
   }
 
   @Override
@@ -155,8 +143,10 @@ public class StripedDataStreamer extends DataStreamer {
     LocatedBlock lb = null;
     if (isLeadingStreamer()) {
       if(hasCommittedBlock) {
-        //when committing a block group, leading streamer has to adjust
-        // {@link block} including the size of block group
+        /**
+         * when committing a block group, leading streamer has to adjust
+         * {@link block} to include the size of block group
+         */
         for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) {
           try {
             LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30,
@@ -179,7 +169,13 @@ public class StripedDataStreamer extends DataStreamer {
 
       lb = super.locateFollowingBlock(excludedNodes);
       hasCommittedBlock = true;
-      LocatedBlock[] blocks = unwrapBlockGroup(lb);
+      assert lb instanceof LocatedStripedBlock;
+      DFSClient.LOG.debug("Leading streamer obtained bg " + lb);
+      LocatedBlock[] blocks = StripedBlockUtil.
+          parseStripedBlockGroup((LocatedStripedBlock) lb,
+              HdfsConstants.BLOCK_STRIPED_CELL_SIZE, HdfsConstants.NUM_DATA_BLOCKS,
+              HdfsConstants.NUM_PARITY_BLOCKS
+          );
       assert blocks.length == blockGroupSize :
           "Fail to get block group from namenode: blockGroupSize: " +
               blockGroupSize + ", blocks.length: " + blocks.length;
@@ -212,30 +208,4 @@ public class StripedDataStreamer extends DataStreamer {
     }
     return lb;
   }
-
-  /**
-   * Generate other blocks in a block group according to the first one.
-   *
-   * @param firstBlockInGroup the first block in a block group
-   * @return  other blocks in this group
-   */
-  public static LocatedBlock[] unwrapBlockGroup(
-      final LocatedBlock firstBlockInGroup) {
-    ExtendedBlock eb = firstBlockInGroup.getBlock();
-    DatanodeInfo[] locs = firstBlockInGroup.getLocations();
-    String[] storageIDs = firstBlockInGroup.getStorageIDs();
-    StorageType[] storageTypes = firstBlockInGroup.getStorageTypes();
-    Token<BlockTokenIdentifier> blockToken = firstBlockInGroup.getBlockToken();
-    LocatedBlock[] blocksInGroup = new LocatedBlock[locs.length];
-    for (int i = 0; i < blocksInGroup.length; i++) {
-      //each block in a group has the same number of bytes and timestamp
-      ExtendedBlock extendedBlock = new ExtendedBlock(eb.getBlockPoolId(),
-          eb.getBlockId() + i, eb.getNumBytes(), eb.getGenerationStamp());
-      blocksInGroup[i] = new LocatedBlock(extendedBlock,
-          new DatanodeInfo[] {locs[i]}, new String[]{storageIDs[i]},
-          new StorageType[] {storageTypes[i]});
-      blocksInGroup[i].setBlockToken(blockToken);
-    }
-    return blocksInGroup;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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 b6faacb..07b49c8 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
@@ -82,6 +82,7 @@ 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 static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Daemon;
@@ -1964,8 +1965,8 @@ public class BlockManager {
       metrics.addBlockReport((int) (endTime - startTime));
     }
     blockLog.info("BLOCK* processReport: from storage {} node {}, " +
-        "blocks: {}, hasStaleStorage: {}, processing time: {} msecs", storage
-        .getStorageID(), nodeID, newReport.getNumberOfBlocks(),
+            "blocks: {}, hasStaleStorage: {}, processing time: {} msecs", storage
+            .getStorageID(), nodeID, newReport.getNumberOfBlocks(),
         node.hasStaleStorages(), (endTime - startTime));
     return !node.hasStaleStorages();
   }
@@ -1992,8 +1993,8 @@ public class BlockManager {
     assert(zombie.numBlocks() == 0);
     LOG.warn("processReport 0x{}: removed {} replicas from storage {}, " +
             "which no longer exists on the DataNode.",
-            Long.toHexString(context.getReportId()), prevBlocks,
-            zombie.getStorageID());
+        Long.toHexString(context.getReportId()), prevBlocks,
+        zombie.getStorageID());
   }
 
   /**
@@ -2472,7 +2473,22 @@ public class BlockManager {
               "block is " + ucState + " and reported genstamp " + reportedGS
               + " does not match genstamp in block map "
               + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
-        } else if (storedBlock.getNumBytes() != reported.getNumBytes()) {
+        }
+        boolean wrongSize;
+        if (storedBlock.isStriped()) {
+          assert BlockIdManager.isStripedBlockID(reported.getBlockId());
+          assert storedBlock.getBlockId() ==
+              BlockIdManager.convertToStripedID(reported.getBlockId());
+          BlockInfoStriped stripedBlock = (BlockInfoStriped) storedBlock;
+          int reportedBlkIdx = BlockIdManager.getBlockIndex(reported);
+          wrongSize = reported.getNumBytes() !=
+              getInternalBlockLength(stripedBlock.getNumBytes(),
+                  HdfsConstants.BLOCK_STRIPED_CELL_SIZE,
+                  stripedBlock.getDataBlockNum(), reportedBlkIdx);
+        } else {
+          wrongSize = storedBlock.getNumBytes() != reported.getNumBytes();
+        }
+        if (wrongSize) {
           return new BlockToMarkCorrupt(new Block(reported), storedBlock,
               "block is " + ucState + " and reported length " +
               reported.getNumBytes() + " does not match " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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
new file mode 100644
index 0000000..2368021
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@ -0,0 +1,138 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
+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;
+
+/**
+ * Utility class for analyzing striped block groups
+ */
+@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;
+    // TODO not considering missing blocks for now, only identify data blocks
+    LocatedBlock[] lbs = new LocatedBlock[dataBlkNum + parityBlkNum];
+    for (short i = 0; i < locatedBGSize; i++) {
+      final int idx = bg.getBlockIndices()[i];
+      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 = new ExtendedBlock(bg.getBlock());
+    blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup);
+    blk.setNumBytes(getInternalBlockLength(bg.getBlockSize(),
+        cellSize, dataBlkNum, idxInBlockGroup));
+
+    return new LocatedBlock(blk,
+        new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
+        new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
+        new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
+        bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(),
+        null);
+  }
+
+  /**
+   * Get the size of an internal block at the given index of a block group
+   *
+   * @param numBytesInGroup Size of the block group only counting data blocks
+   * @param cellSize The size of a striping cell
+   * @param dataBlkNum The number of data blocks
+   * @param idxInGroup The logical index in the striped block group
+   * @return The size of the internal block at the specified index
+   */
+  public static long getInternalBlockLength(long numBytesInGroup,
+      int cellSize, int dataBlkNum, int idxInGroup) {
+    // Size of each stripe (only counting data blocks)
+    final long numBytesPerStripe = cellSize * dataBlkNum;
+    assert numBytesPerStripe  > 0:
+        "getInternalBlockLength should only be called on valid striped blocks";
+    // If block group ends at stripe boundary, each internal block has an equal
+    // share of the group
+    if (numBytesInGroup % numBytesPerStripe == 0) {
+      return numBytesInGroup / dataBlkNum;
+    }
+
+    int numStripes = (int) ((numBytesInGroup - 1) / numBytesPerStripe + 1);
+    assert numStripes >= 1 : "There should be at least 1 stripe";
+
+    // All stripes but the last one are full stripes. The block should at least
+    // contain (numStripes - 1) full cells.
+    long blkSize = (numStripes - 1) * cellSize;
+
+    long lastStripeLen = numBytesInGroup % numBytesPerStripe;
+    // Size of parity cells should equal the size of the first cell, if it
+    // is not full.
+    long lastParityCellLen = Math.min(cellSize, lastStripeLen);
+
+    if (idxInGroup >= dataBlkNum) {
+      // for parity blocks
+      blkSize += lastParityCellLen;
+    } else {
+      // for data blocks
+      blkSize +=  Math.min(cellSize,
+          Math.max(0, lastStripeLen - cellSize * idxInGroup));
+    }
+
+    return blkSize;
+  }
+
+  /**
+   * 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
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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 ed508fc..0c88842 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -108,7 +108,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-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.protocol.datatransfer.Sender;
@@ -1851,11 +1850,30 @@ public class DFSTestUtil {
     return reports;
   }
 
-  public static void createECFile(MiniDFSCluster cluster, Path file, Path dir,
-      int numBlocks, int numStripesPerBlk) throws Exception {
+  /**
+   * Creates the metadata of a file in striped layout. This method only
+   * manipulates the NameNode state without injecting data to DataNode.
+   *  @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();
-    dfs.mkdirs(dir);
-    dfs.getClient().createErasureCodingZone(dir.toString(), null);
+    // If outer test already created EC zone, dir should be left as null
+    if (toMkdir) {
+      assert dir != null;
+      dfs.mkdirs(dir);
+      try {
+        dfs.getClient().createErasureCodingZone(dir.toString(), null);
+      } catch (IOException e) {
+        if (!e.getMessage().contains("non-empty directory")) {
+          throw e;
+        }
+      }
+    }
 
     FSDataOutputStream out = null;
     try {
@@ -1867,7 +1885,7 @@ public class DFSTestUtil {
 
       ExtendedBlock previous = null;
       for (int i = 0; i < numBlocks; i++) {
-        Block newBlock = createBlock(cluster.getDataNodes(), dfs, ns,
+        Block newBlock = addStripedBlockToFile(cluster.getDataNodes(), dfs, ns,
             file.toString(), fileNode, dfs.getClient().getClientName(),
             previous, numStripesPerBlk);
         previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
@@ -1880,43 +1898,50 @@ public class DFSTestUtil {
     }
   }
 
-  static Block createBlock(List<DataNode> dataNodes, DistributedFileSystem fs,
-      FSNamesystem ns, String file, INodeFile fileNode, String clientName,
-      ExtendedBlock previous, int numStripes) throws Exception {
+  /**
+   * 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.
+   * @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.getBlockReplication();
+    assert dataNodes.size() >= groupSize;
     // 1. RECEIVING_BLOCK IBR
-    int i = 0;
-    for (DataNode dn : dataNodes) {
-      if (i < groupSize) {
-        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);
-        }
+    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
-    i = 0;
-    for (DataNode dn : dataNodes) {
-      if (i < groupSize) {
-        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);
-        }
+    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);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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 c78922e..4a09bda 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
@@ -2,7 +2,6 @@ package org.apache.hadoop.hdfs;
 
 import java.util.Arrays;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
@@ -14,10 +13,12 @@ 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;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
@@ -39,16 +40,16 @@ public class TestDFSStripedOutputStream {
   private MiniDFSCluster cluster;
   private Configuration conf = new Configuration();
   private DistributedFileSystem fs;
-  int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-  int blockSize = 8 * 1024 * 1024;
-  int cellsInBlock = blockSize / cellSize;
+  private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final int stripesPerBlock = 4;
+  int blockSize = cellSize * stripesPerBlock;
   private int mod = 29;
 
   @Before
   public void setup() throws IOException {
     int numDNs = dataBlocks + parityBlocks + 2;
     Configuration conf = new Configuration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, cellsInBlock * cellSize);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
     fs = cluster.getFileSystem();
@@ -103,8 +104,7 @@ public class TestDFSStripedOutputStream {
 
   @Test
   public void TestFileMoreThanOneStripe2() throws IOException {
-    testOneFile("/MoreThanOneStripe2",
-        cellSize * dataBlocks * (cellsInBlock >= 2 ? cellsInBlock / 2 : 1)
+    testOneFile("/MoreThanOneStripe2", cellSize * dataBlocks
             + cellSize * dataBlocks + 123);
   }
 
@@ -113,18 +113,22 @@ public class TestDFSStripedOutputStream {
     testOneFile("/FullBlockGroup", blockSize * dataBlocks);
   }
 
-  //TODO: The following tests will pass after HDFS-8121 fixed
-//  @Test
+  @Test
   public void TestFileMoreThanABlockGroup1() throws IOException {
     testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123);
   }
 
-  //  @Test
+  @Test
   public void TestFileMoreThanABlockGroup2() throws IOException {
-    testOneFile("/MoreThanABlockGroup2",
-        blockSize * dataBlocks * 3
-            + (cellsInBlock >= 2 ? cellsInBlock / 2 : 1) * cellSize * dataBlocks
-            + 123);
+    testOneFile("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize+ 123);
+  }
+
+
+  @Test
+  public void TestFileMoreThanABlockGroup3() throws IOException {
+    testOneFile("/MoreThanABlockGroup3",
+        blockSize * dataBlocks * 3 + cellSize * dataBlocks
+        + cellSize + 123);
   }
 
   private int stripeDataSize() {
@@ -193,7 +197,10 @@ public class TestDFSStripedOutputStream {
     LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L);
 
     for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
-      LocatedBlock[] blocks = StripedDataStreamer.unwrapBlockGroup(firstBlock);
+      assert firstBlock instanceof LocatedStripedBlock;
+      LocatedBlock[] blocks = StripedBlockUtil.
+          parseStripedBlockGroup((LocatedStripedBlock) firstBlock,
+              cellSize, dataBlocks, parityBlocks);
       List<LocatedBlock> oneGroup = Arrays.asList(blocks);
       blockGroupList.add(oneGroup);
     }
@@ -205,12 +212,6 @@ public class TestDFSStripedOutputStream {
       byte[][] dataBlockBytes = new byte[dataBlocks][];
       byte[][] parityBlockBytes = new byte[allBlocks - dataBlocks][];
 
-      //calculate the size of this block group
-      int lenOfBlockGroup = group < blockGroupList.size() - 1 ?
-          blockSize * dataBlocks :
-          writeBytes - blockSize * (blockGroupList.size() - 1) * dataBlocks;
-      int intactStripes = lenOfBlockGroup / stripeDataSize();
-      int lastStripeLen = lenOfBlockGroup % stripeDataSize();
 
       //for each block, use BlockReader to read data
       for (int i = 0; i < blockList.size(); i++) {
@@ -223,25 +224,17 @@ public class TestDFSStripedOutputStream {
         InetSocketAddress targetAddr = NetUtils.createSocketAddr(
             nodes[0].getXferAddr());
 
-        int lenOfCell = cellSize;
-        if (i == lastStripeLen / cellSize) {
-          lenOfCell = lastStripeLen % cellSize;
-        } else if (i > lastStripeLen / cellSize) {
-          lenOfCell = 0;
-        }
-        int lenOfBlock = cellSize * intactStripes + lenOfCell;
-        byte[] blockBytes = new byte[lenOfBlock];
+        byte[] blockBytes = new byte[(int)block.getNumBytes()];
         if (i < dataBlocks) {
           dataBlockBytes[i] = blockBytes;
         } else {
           parityBlockBytes[i - dataBlocks] = blockBytes;
         }
 
-        if (lenOfBlock == 0) {
+        if (block.getNumBytes() == 0) {
           continue;
         }
 
-        block.setNumBytes(lenOfBlock);
         BlockReader blockReader = new BlockReaderFactory(new DfsClientConf(conf)).
             setFileName(src).
             setBlock(block).
@@ -276,33 +269,33 @@ public class TestDFSStripedOutputStream {
               }
             }).build();
 
-        blockReader.readAll(blockBytes, 0, lenOfBlock);
+        blockReader.readAll(blockBytes, 0, (int)block.getNumBytes());
         blockReader.close();
       }
 
       //check if we write the data correctly
-      for (int i = 0; i < dataBlockBytes.length; i++) {
-        byte[] cells = dataBlockBytes[i];
-        if (cells == null) {
+      for (int blkIdxInGroup = 0; blkIdxInGroup < dataBlockBytes.length; blkIdxInGroup++) {
+        byte[] actualBlkBytes = dataBlockBytes[blkIdxInGroup];
+        if (actualBlkBytes == null) {
           continue;
         }
-        for (int j = 0; j < cells.length; j++) {
+        for (int posInBlk = 0; posInBlk < actualBlkBytes.length; posInBlk++) {
           byte expected;
           //calculate the postion of this byte in the file
-          long pos = group * dataBlocks * blockSize
-              + (i * cellSize + j / cellSize * cellSize * dataBlocks)
-              + j % cellSize;
-          if (pos >= writeBytes) {
+          long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(cellSize,
+              dataBlocks, posInBlk, blkIdxInGroup) +
+              group * blockSize * dataBlocks;
+          if (posInFile >= writeBytes) {
             expected = 0;
           } else {
-            expected = getByte(pos);
+            expected = getByte(posInFile);
           }
 
-          if (expected != cells[j]) {
-            Assert.fail("Unexpected byte " + cells[j] + ", expect " + expected
+          if (expected != actualBlkBytes[posInBlk]) {
+            Assert.fail("Unexpected byte " + actualBlkBytes[posInBlk] + ", expect " + expected
                 + ". Block group index is " + group +
-                ", stripe index is " + j / cellSize +
-                ", cell index is " + i + ", byte index is " + j % cellSize);
+                ", stripe index is " + posInBlk / cellSize +
+                ", cell index is " + blkIdxInGroup + ", byte index is " + posInBlk % cellSize);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
index 849e12e..90488c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
@@ -21,10 +21,7 @@ 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.StorageType;
 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.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -34,10 +31,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -54,17 +50,18 @@ public class TestReadStripedFile {
   private DistributedFileSystem fs;
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
-  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS;
-  private final short TOTAL_SIZE = HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS;
+  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_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM;
   private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
   private final int NUM_STRIPE_PER_BLOCK = 2;
-  private final int BLOCKSIZE = 2 * GROUP_SIZE * CELLSIZE;
+  private final int BLOCKSIZE = NUM_STRIPE_PER_BLOCK * DATA_BLK_NUM * CELLSIZE;
 
   @Before
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
     SimulatedFSDataset.setFactory(conf);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(TOTAL_SIZE)
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(BLK_GROUP_SIZE)
         .build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
@@ -77,72 +74,14 @@ public class TestReadStripedFile {
     }
   }
 
-  private LocatedStripedBlock createDummyLocatedBlock() {
-    final long blockGroupID = -1048576;
-    DatanodeInfo[] locs = new DatanodeInfo[TOTAL_SIZE];
-    String[] storageIDs = new String[TOTAL_SIZE];
-    StorageType[] storageTypes = new StorageType[TOTAL_SIZE];
-    int[] indices = new int[TOTAL_SIZE];
-    for (int i = 0; i < TOTAL_SIZE; i++) {
-      locs[i] = new DatanodeInfo(cluster.getDataNodes().get(i).getDatanodeId());
-      storageIDs[i] = cluster.getDataNodes().get(i).getDatanodeUuid();
-      storageTypes[i] = StorageType.DISK;
-      indices[i] = (i + 2) % GROUP_SIZE;
-    }
-    return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID),
-        locs, storageIDs, storageTypes, indices, 0, false, null);
-  }
-
-  @Test
-  public void testParseDummyStripedBlock() {
-    LocatedStripedBlock lsb = createDummyLocatedBlock();
-    LocatedBlock[] blocks = DFSStripedInputStream.parseStripedBlockGroup(
-        lsb, GROUP_SIZE, CELLSIZE);
-    assertEquals(GROUP_SIZE, blocks.length);
-    for (int j = 0; j < GROUP_SIZE; j++) {
-      assertFalse(blocks[j].isStriped());
-      assertEquals(j,
-          BlockIdManager.getBlockIndex(blocks[j].getBlock().getLocalBlock()));
-      assertEquals(j * CELLSIZE, blocks[j].getStartOffset());
-    }
-  }
-
-  @Test
-  public void testParseStripedBlock() throws Exception {
-    final int numBlocks = 4;
-    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
-        NUM_STRIPE_PER_BLOCK);
-    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
-        filePath.toString(), 0, BLOCKSIZE * numBlocks);
-
-    assertEquals(4, lbs.locatedBlockCount());
-    List<LocatedBlock> lbList = lbs.getLocatedBlocks();
-    for (LocatedBlock lb : lbList) {
-      assertTrue(lb.isStriped());
-    }
-
-    for (int i = 0; i < numBlocks; i++) {
-      LocatedStripedBlock lsb = (LocatedStripedBlock) (lbs.get(i));
-      LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb,
-          GROUP_SIZE, CELLSIZE);
-      assertEquals(GROUP_SIZE, blks.length);
-      for (int j = 0; j < GROUP_SIZE; j++) {
-        assertFalse(blks[j].isStriped());
-        assertEquals(j,
-            BlockIdManager.getBlockIndex(blks[j].getBlock().getLocalBlock()));
-        assertEquals(i * BLOCKSIZE + j * CELLSIZE, blks[j].getStartOffset());
-      }
-    }
-  }
-
   /**
    * Test {@link DFSStripedInputStream#getBlockAt(long)}
    */
   @Test
   public void testGetBlock() throws Exception {
     final int numBlocks = 4;
-    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
-        NUM_STRIPE_PER_BLOCK);
+    DFSTestUtil.createStripedFile(cluster, filePath, dirPath, numBlocks,
+        NUM_STRIPE_PER_BLOCK, true);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, BLOCKSIZE * numBlocks);
     final DFSStripedInputStream in =
@@ -151,9 +90,9 @@ public class TestReadStripedFile {
     List<LocatedBlock> lbList = lbs.getLocatedBlocks();
     for (LocatedBlock aLbList : lbList) {
       LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList;
-      LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb,
-          GROUP_SIZE, CELLSIZE);
-      for (int j = 0; j < GROUP_SIZE; j++) {
+      LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb,
+          CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM);
+      for (int j = 0; j < DATA_BLK_NUM; j++) {
         LocatedBlock refreshed = in.getBlockAt(blks[j].getStartOffset());
         assertEquals(blks[j].getBlock(), refreshed.getBlock());
         assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset());
@@ -165,15 +104,16 @@ public class TestReadStripedFile {
   @Test
   public void testPread() throws Exception {
     final int numBlocks = 4;
-    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
-        NUM_STRIPE_PER_BLOCK);
+    DFSTestUtil.createStripedFile(cluster, filePath, dirPath, numBlocks,
+        NUM_STRIPE_PER_BLOCK, true);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, BLOCKSIZE);
 
     assert lbs.get(0) instanceof LocatedStripedBlock;
     LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0));
-    for (int i = 0; i < GROUP_SIZE; i++) {
-      Block blk = new Block(bg.getBlock().getBlockId() + i, BLOCKSIZE,
+    for (int i = 0; i < DATA_BLK_NUM; i++) {
+      Block blk = new Block(bg.getBlock().getBlockId() + i,
+          NUM_STRIPE_PER_BLOCK * CELLSIZE,
           bg.getBlock().getGenerationStamp());
       blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
       cluster.injectBlocks(i, Arrays.asList(blk),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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 27df1cd..6bb1162 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,11 +34,13 @@ 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.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;
@@ -53,6 +55,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
 
+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 {
@@ -284,4 +288,107 @@ public class TestAddStripedBlocks {
       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();
+    DFSTestUtil.createStripedFile(cluster, filePath, null,
+        numBlocks, numStripes, false);
+
+    INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()).
+        asFile();
+    Assert.assertTrue(fileNode.isStriped());
+    BlockInfoStriped stored = fileNode.getStripedBlocksFeature().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());
+    Assert.assertEquals(2, ns.getCorruptReplicaBlocks());
+
+    // 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(3).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(3, ns.getCorruptReplicaBlocks());
+
+    // 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(3, ns.getCorruptReplicaBlocks());
+
+    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(1).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(3, ns.getCorruptReplicaBlocks());
+
+    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(3, ns.getCorruptReplicaBlocks());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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 4292f9a..ea18c3e 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
@@ -78,7 +78,8 @@ public class TestRecoverStripedBlocks {
   @Test
   public void testMissingStripedBlock() throws Exception {
     final int numBlocks = 4;
-    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, 1);
+    DFSTestUtil.createStripedFile(cluster, filePath,
+        dirPath, numBlocks, 1, true);
 
     // make sure the file is complete in NN
     final INodeFile fileNode = cluster.getNamesystem().getFSDirectory()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed68791c/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
new file mode 100644
index 0000000..ec0b1bb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
@@ -0,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.util;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+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;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.parseStripedBlockGroup;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+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_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM;
+  private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+
+  private LocatedStripedBlock createDummyLocatedBlock() {
+    final long blockGroupID = -1048576;
+    DatanodeInfo[] locs = new DatanodeInfo[BLK_GROUP_SIZE];
+    String[] storageIDs = new String[BLK_GROUP_SIZE];
+    StorageType[] storageTypes = new StorageType[BLK_GROUP_SIZE];
+    int[] indices = new int[BLK_GROUP_SIZE];
+    for (int i = 0; i < BLK_GROUP_SIZE; i++) {
+      indices[i] = (i + 2) % DATA_BLK_NUM;
+      // Location port always equal to logical index of a block,
+      // for easier verification
+      locs[i] = DFSTestUtil.getLocalDatanodeInfo(indices[i]);
+      storageIDs[i] = locs[i].getDatanodeUuid();
+      storageTypes[i] = StorageType.DISK;
+    }
+    return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID),
+        locs, storageIDs, storageTypes, indices, 0, false, null);
+  }
+
+  @Test
+  public void testParseDummyStripedBlock() {
+    LocatedStripedBlock lsb = createDummyLocatedBlock();
+    LocatedBlock[] blocks = parseStripedBlockGroup(
+        lsb, CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM);
+    assertEquals(DATA_BLK_NUM + PARITY_BLK_NUM, blocks.length);
+    for (int i = 0; i < DATA_BLK_NUM; i++) {
+      assertFalse(blocks[i].isStriped());
+      assertEquals(i,
+          BlockIdManager.getBlockIndex(blocks[i].getBlock().getLocalBlock()));
+      assertEquals(i * CELLSIZE, blocks[i].getStartOffset());
+      assertEquals(1, blocks[i].getLocations().length);
+      assertEquals(i, blocks[i].getLocations()[0].getIpcPort());
+      assertEquals(i, blocks[i].getLocations()[0].getXferPort());
+    }
+  }
+
+  private void verifyInternalBlocks (long numBytesInGroup, long[] expected) {
+    for (int i = 1; i < BLK_GROUP_SIZE; i++) {
+      assertEquals(expected[i],
+          getInternalBlockLength(numBytesInGroup, CELLSIZE, DATA_BLK_NUM, i));
+    }
+  }
+
+  @Test
+  public void testGetInternalBlockLength () {
+    // A small delta that is smaller than a cell
+    final int delta = 10;
+    assert delta < CELLSIZE;
+
+    // Block group is smaller than a cell
+    verifyInternalBlocks(CELLSIZE - delta,
+        new long[] {CELLSIZE - delta, 0, 0, 0, 0, 0,
+            CELLSIZE - delta, CELLSIZE - delta, CELLSIZE - delta});
+
+    // Block group is exactly as large as a cell
+    verifyInternalBlocks(CELLSIZE,
+        new long[] {CELLSIZE, 0, 0, 0, 0, 0,
+            CELLSIZE, CELLSIZE, CELLSIZE});
+
+    // Block group is a little larger than a cell
+    verifyInternalBlocks(CELLSIZE + delta,
+        new long[] {CELLSIZE, delta, 0, 0, 0, 0,
+            CELLSIZE, CELLSIZE, CELLSIZE});
+
+    // Block group contains multiple stripes and ends at stripe boundary
+    verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE,
+        new long[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE});
+
+    // Block group contains multiple stripes and ends at cell boundary
+    // (not ending at stripe boundary)
+    verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE + CELLSIZE,
+        new long[] {3 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            3 * CELLSIZE, 3 * CELLSIZE, 3 * CELLSIZE});
+
+    // Block group contains multiple stripes and doesn't end at cell boundary
+    verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE - delta,
+        new long[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE - delta,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE});
+  }
+
+}


[38/50] hadoop git commit: HDFS-8324. Add trace info to DFSClient#getErasureCodingZoneInfo(..). Contributed by Vinayakumar B

Posted by zh...@apache.org.
HDFS-8324. Add trace info to DFSClient#getErasureCodingZoneInfo(..). Contributed by Vinayakumar B


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

Branch: refs/heads/HDFS-7285
Commit: b7014e1e2a754aeb286d1352200a93cf6403a337
Parents: a7fac8b
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Tue May 5 19:25:21 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:24 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt          | 3 +++
 .../src/main/java/org/apache/hadoop/hdfs/DFSClient.java           | 3 +++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7014e1e/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 ef760fc..a8df3f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -169,3 +169,6 @@
 
     HDFS-8242. Erasure Coding: XML based end-to-end test for ECCli commands
     (Rakesh R via vinayakumarb)
+
+    HDFS-8324. Add trace info to DFSClient#getErasureCodingZoneInfo(..) (vinayakumarb via 
+    umamahesh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7014e1e/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 7682459..b149bcf 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
@@ -3340,11 +3340,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("getErasureCodingZoneInfo", src);
     try {
       return namenode.getErasureCodingZoneInfo(src);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(FileNotFoundException.class,
           AccessControlException.class, UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 }


[20/50] hadoop git commit: HDFS-8136. Client gets and uses EC schema when reads and writes a stripping file. Contributed by Kai Sasaki

Posted by zh...@apache.org.
HDFS-8136. Client gets and uses EC schema when reads and writes a stripping file. Contributed by Kai Sasaki


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

Branch: refs/heads/HDFS-7285
Commit: ef6c1b87c0b1c7241bd6203c6d53074d72d89c59
Parents: b8aff2d
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Apr 24 00:19:12 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:19 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../hadoop/hdfs/DFSStripedInputStream.java      |  17 +-
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  24 ++-
 .../hdfs/server/namenode/FSNamesystem.java      |   2 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  | 175 +++++++++++++++++++
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |   4 +-
 .../apache/hadoop/hdfs/TestReadStripedFile.java |   1 -
 7 files changed, 210 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef6c1b87/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 b2faac0..8977c46 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -119,3 +119,6 @@
 
     HDFS-8156. Add/implement necessary APIs even we just have the system default 
     schema. (Kai Zheng via Zhe Zhang)
+
+    HDFS-8136. Client gets and uses EC schema when reads and writes a stripping
+    file. (Kai Sasaki via Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef6c1b87/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 d597407..d0e2b68 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
@@ -21,9 +21,9 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.fs.StorageType;
 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.protocol.ECInfo;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.net.NetUtils;
@@ -125,13 +125,19 @@ public class DFSStripedInputStream extends DFSInputStream {
     return results;
   }
 
-  private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-  private final short dataBlkNum = HdfsConstants.NUM_DATA_BLOCKS;
-  private final short parityBlkNum = HdfsConstants.NUM_PARITY_BLOCKS;
+  private final int cellSize;
+  private final short dataBlkNum;
+  private final short parityBlkNum;
+  private final ECInfo ecInfo;
 
   DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum)
       throws IOException {
     super(dfsClient, src, verifyChecksum);
+    // ECInfo is restored from NN just before reading striped file.
+    ecInfo = dfsClient.getErasureCodingInfo(src);
+    cellSize = ecInfo.getSchema().getChunkSize();
+    dataBlkNum = (short)ecInfo.getSchema().getNumDataUnits();
+    parityBlkNum = (short)ecInfo.getSchema().getNumParityUnits();
     DFSClient.LOG.debug("Creating an striped input stream for file " + src);
   }
 
@@ -279,9 +285,6 @@ public class DFSStripedInputStream extends DFSInputStream {
     throw new InterruptedException("let's retry");
   }
 
-  public void setCellSize(int cellSize) {
-    this.cellSize = cellSize;
-  }
 
   /**
    * This class represents the portion of I/O associated with each block in the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef6c1b87/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 7dc0091..eeb9d7e 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
@@ -32,8 +32,8 @@ 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.protocol.ECInfo;
 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.util.StripedBlockUtil;
@@ -61,11 +61,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   /**
    * Size of each striping cell, must be a multiple of bytesPerChecksum
    */
-  private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final ECInfo ecInfo;
+  private final int cellSize;
   private ByteBuffer[] cellBuffers;
-  private final short numAllBlocks = HdfsConstants.NUM_DATA_BLOCKS
-      + HdfsConstants.NUM_PARITY_BLOCKS;
-  private final short numDataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
+
+  private final short numAllBlocks;
+  private final short numDataBlocks;
+
   private int curIdx = 0;
   /* bytes written in current block group */
   //private long currentBlockGroupBytes = 0;
@@ -77,6 +79,10 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     return streamers.get(0);
   }
 
+  private long getBlockGroupSize() {
+    return blockSize * numDataBlocks;
+  }
+
   /** Construct a new output stream for creating a file. */
   DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
                          EnumSet<CreateFlag> flag, Progressable progress,
@@ -84,6 +90,14 @@ public class DFSStripedOutputStream extends DFSOutputStream {
                          throws IOException {
     super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
     DFSClient.LOG.info("Creating striped output stream");
+
+    // ECInfo is restored from NN just before writing striped files.
+    ecInfo = dfsClient.getErasureCodingInfo(src);
+    cellSize = ecInfo.getSchema().getChunkSize();
+    numAllBlocks = (short)(ecInfo.getSchema().getNumDataUnits()
+        + ecInfo.getSchema().getNumParityUnits());
+    numDataBlocks = (short)ecInfo.getSchema().getNumDataUnits();
+
     checkConfiguration();
 
     cellBuffers = new ByteBuffer[numAllBlocks];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef6c1b87/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 73048bc..1290bb2 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
@@ -3356,7 +3356,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     return new FileState(pendingFile, src, iip);
   }
-  
+
   LocatedBlock makeLocatedBlock(BlockInfo blk, DatanodeStorageInfo[] locs,
       long offset) throws IOException {
     final LocatedBlock lBlk;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef6c1b87/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
new file mode 100644
index 0000000..6af4a7f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestDFSStripedInputStream {
+  private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
+  private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
+
+
+  private static DistributedFileSystem fs;
+  private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final static int stripesPerBlock = 4;
+  static int blockSize = cellSize * stripesPerBlock;
+  private int mod = 29;
+  static int numDNs = dataBlocks + parityBlocks + 2;
+
+  private static MiniDFSCluster cluster;
+  private static Configuration conf;
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    cluster
+        = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();;
+    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    fs = cluster.getFileSystem();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testFileEmpty() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/EmptyFile", 0);
+  }
+
+  @Test
+  public void testFileSmallerThanOneCell1() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1);
+  }
+
+  @Test
+  public void testFileSmallerThanOneCell2() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1);
+  }
+
+  @Test
+  public void testFileEqualsWithOneCell() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize);
+  }
+
+  @Test
+  public void testFileSmallerThanOneStripe1() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize * dataBlocks - 1);
+  }
+
+  @Test
+  public void testFileSmallerThanOneStripe2() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize + 123);
+  }
+
+  @Test
+  public void testFileEqualsWithOneStripe() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe", cellSize * dataBlocks);
+  }
+
+  @Test
+  public void testFileMoreThanOneStripe1() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1", cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void testFileMoreThanOneStripe2() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2", cellSize * dataBlocks
+        + cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void testFileFullBlockGroup() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/FullBlockGroup", blockSize * dataBlocks);
+  }
+
+  @Test
+  public void testFileMoreThanABlockGroup1() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1", blockSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void testFileMoreThanABlockGroup2() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize+ 123);
+  }
+
+
+  @Test
+  public void testFileMoreThanABlockGroup3() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3",
+        blockSize * dataBlocks * 3 + cellSize * dataBlocks
+            + 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) {
+    return (byte) (pos % mod + 1);
+  }
+
+  private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes)
+      throws IOException {
+    Path TestPath = new Path(src);
+    byte[] bytes = generateBytes(writeBytes);
+    DFSTestUtil.writeFile(fs, TestPath, new String(bytes));
+
+    //check file length
+    FileStatus status = fs.getFileStatus(TestPath);
+    long fileLength = status.getLen();
+    Assert.assertEquals("File length should be the same",
+        writeBytes, fileLength);
+
+    DFSStripedInputStream dis = new DFSStripedInputStream(
+        fs.getClient(), src, true);
+    try {
+      byte[] buf = new byte[writeBytes + 100];
+      int readLen = dis.read(0, buf, 0, buf.length);
+      readLen = readLen >= 0 ? readLen : 0;
+      Assert.assertEquals("The length of file should be the same to write size",
+          writeBytes, readLen);
+      for (int i = 0; i < writeBytes; i++) {
+        Assert.assertEquals("Byte at i should be the same",
+            getByte(i), buf[i]);
+      }
+    } finally {
+      dis.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef6c1b87/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 c213183..26f6d2c 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
@@ -159,7 +159,7 @@ public class TestDFSStripedOutputStream {
     // check file length
     FileStatus status = fs.getFileStatus(testPath);
     Assert.assertEquals(writeBytes, status.getLen());
-    
+
     checkData(src, writeBytes);
   }
 
@@ -236,7 +236,7 @@ public class TestDFSStripedOutputStream {
           cellSize, dataBlockBytes, parityBlockBytes);
     }
   }
-    
+
   static void verifyParity(final long size, final int cellSize,
       byte[][] dataBytes, byte[][] parityBytes) {
     // verify the parity blocks

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef6c1b87/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
index 90488c1..b0631ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
@@ -121,7 +121,6 @@ public class TestReadStripedFile {
     }
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(), false);
-    in.setCellSize(CELLSIZE);
     int readSize = BLOCKSIZE;
     byte[] readBuffer = new byte[readSize];
     int ret = in.read(0, readBuffer, 0, readSize);


[26/50] hadoop git commit: HDFS-8033. Erasure coding: stateful (non-positional) read from files in striped layout. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-8033. Erasure coding: stateful (non-positional) read from files in striped layout. Contributed by Zhe Zhang.


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

Branch: refs/heads/HDFS-7285
Commit: fc1609de4b239d664c99ea718aa48b0804266164
Parents: f2baca7
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Apr 24 22:36:15 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:20 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  55 ++--
 .../hadoop/hdfs/DFSStripedInputStream.java      | 311 ++++++++++++++++++-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |  43 +++
 .../apache/hadoop/hdfs/TestReadStripedFile.java | 110 ++++++-
 5 files changed, 465 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc1609de/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 cf41a9b..e8db485 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -131,3 +131,6 @@
 
     HDFS-8228. Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause 
     block id conflicts (Jing Zhao via Zhe Zhang)
+
+    HDFS-8033. Erasure coding: stateful (non-positional) read from files in 
+    striped layout (Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc1609de/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 16250dd..6eb25d0 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
@@ -95,34 +95,34 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   public static boolean tcpReadsDisabledForTesting = false;
   private long hedgedReadOpsLoopNumForTesting = 0;
   protected final DFSClient dfsClient;
-  private AtomicBoolean closed = new AtomicBoolean(false);
-  private final String src;
-  private final boolean verifyChecksum;
+  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;
-  private LocatedBlock currentLocatedBlock = null;
-  private long pos = 0;
-  private long blockEnd = -1;
+  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)
   ////
-  private LocatedBlocks locatedBlocks = null;
+  protected LocatedBlocks locatedBlocks = null;
   private long lastBlockBeingWrittenLength = 0;
   private FileEncryptionInfo fileEncryptionInfo = null;
-  private CachingStrategy cachingStrategy;
+  protected CachingStrategy cachingStrategy;
   ////
 
-  private final ReadStatistics readStatistics = new ReadStatistics();
+  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)
-  private final Object infoLock = new Object();
+  protected final Object infoLock = new Object();
 
   /**
    * Track the ByteBuffers that we have handed out to readers.
@@ -239,7 +239,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * back to the namenode to get a new list of block locations, and is
    * capped at maxBlockAcquireFailures
    */
-  private int failures = 0;
+  protected int failures = 0;
 
   /* XXX Use of CocurrentHashMap is temp fix. Need to fix 
    * parallel accesses to DFSInputStream (through ptreads) properly */
@@ -476,7 +476,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   }
 
   /** Fetch a block from namenode and cache it */
-  private void fetchBlockAt(long offset) throws IOException {
+  protected void fetchBlockAt(long offset) throws IOException {
     synchronized(infoLock) {
       int targetBlockIdx = locatedBlocks.findBlock(offset);
       if (targetBlockIdx < 0) { // block is not cached
@@ -579,7 +579,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
 
     // Will be getting a new BlockReader.
-    closeCurrentBlockReader();
+    closeCurrentBlockReaders();
 
     //
     // Connect to best DataNode for desired Block, with potential offset
@@ -620,7 +620,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         return chosenNode;
       } catch (IOException ex) {
         if (ex 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 " + targetAddr
               + " : " + ex);
           // The encryption key used is invalid.
@@ -696,7 +696,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           "unreleased ByteBuffers allocated by read().  " +
           "Please release " + builder.toString() + ".");
     }
-    closeCurrentBlockReader();
+    closeCurrentBlockReaders();
     super.close();
   }
 
@@ -718,7 +718,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         throws ChecksumException, IOException;
   }
 
-  private void updateReadStatistics(ReadStatistics readStatistics, 
+  protected void updateReadStatistics(ReadStatistics readStatistics,
         int nRead, BlockReader blockReader) {
     if (nRead <= 0) return;
     synchronized(infoLock) {
@@ -754,7 +754,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   /**
    * Used to read bytes into a user-supplied ByteBuffer
    */
-  private class ByteBufferStrategy implements ReaderStrategy {
+  protected class ByteBufferStrategy implements ReaderStrategy {
     final ByteBuffer buf;
     ByteBufferStrategy(ByteBuffer buf) {
       this.buf = buf;
@@ -770,6 +770,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         int ret = blockReader.read(buf);
         success = true;
         updateReadStatistics(readStatistics, ret, blockReader);
+        if (ret == 0) {
+          DFSClient.LOG.warn("zero");
+        }
         return ret;
       } finally {
         if (!success) {
@@ -837,7 +840,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
   }
 
-  private synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
+  protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
     dfsClient.checkOpen();
     if (closed.get()) {
       throw new IOException("Stream closed");
@@ -926,7 +929,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   /**
    * Add corrupted block replica into map.
    */
-  private void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node, 
+  protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
     Set<DatanodeInfo> dnSet = null;
     if((corruptedBlockMap.containsKey(blk))) {
@@ -996,7 +999,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * @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.
    */
-  private DNAddrPair getBestNodeDNAddrPair(LocatedBlock block,
+  protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block,
       Collection<DatanodeInfo> ignoredNodes) {
     DatanodeInfo[] nodes = block.getLocations();
     StorageType[] storageTypes = block.getStorageTypes();
@@ -1365,7 +1368,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * @return true if block access token has expired or invalid and it should be
    *         refetched
    */
-  private static boolean tokenRefetchNeeded(IOException ex,
+  protected static boolean tokenRefetchNeeded(IOException ex,
       InetSocketAddress targetAddr) {
     /*
      * Get a new access token and retry. Retry is needed in 2 cases. 1)
@@ -1472,7 +1475,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * @param corruptedBlockMap map of corrupted blocks
    * @param dataNodeCount number of data nodes who contains the block replicas
    */
-  private void reportCheckSumFailure(
+  protected void reportCheckSumFailure(
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap, 
       int dataNodeCount) {
     if (corruptedBlockMap.isEmpty()) {
@@ -1669,7 +1672,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
   }
 
-  private void closeCurrentBlockReader() {
+  protected void closeCurrentBlockReaders() {
     if (blockReader == null) return;
     // Close the current block reader so that the new caching settings can 
     // take effect immediately.
@@ -1689,7 +1692,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       this.cachingStrategy =
           new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
     }
-    closeCurrentBlockReader();
+    closeCurrentBlockReaders();
   }
 
   @Override
@@ -1699,7 +1702,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       this.cachingStrategy =
           new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
     }
-    closeCurrentBlockReader();
+    closeCurrentBlockReaders();
   }
 
   /**
@@ -1857,6 +1860,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
   @Override
   public synchronized void unbuffer() {
-    closeCurrentBlockReader();
+    closeCurrentBlockReaders();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc1609de/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 d0e2b68..fe9e101 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
@@ -18,20 +18,21 @@
 package org.apache.hadoop.hdfs;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.StorageType;
-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 org.apache.hadoop.hdfs.protocol.ECInfo;
-import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
+import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
@@ -125,6 +126,9 @@ public class DFSStripedInputStream extends DFSInputStream {
     return results;
   }
 
+  private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS;
+  private BlockReader[] blockReaders = null;
+  private DatanodeInfo[] currentNodes = null;
   private final int cellSize;
   private final short dataBlkNum;
   private final short parityBlkNum;
@@ -143,13 +147,285 @@ public class DFSStripedInputStream extends DFSInputStream {
 
   @Override
   public synchronized int read(final ByteBuffer buf) throws IOException {
-    throw new UnsupportedActionException("Stateful read is not supported");
+    ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
+    TraceScope scope =
+        dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src);
+    try {
+      return readWithStrategy(byteBufferReader, 0, buf.remaining());
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * When seeking into a new block group, create blockReader for each internal
+   * block in the group.
+   */
+  @VisibleForTesting
+  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[] chosenNodes = new DatanodeInfo[groupSize];
+    int refetchToken = 1; // only need to get a new access token once
+    int refetchEncryptionKey = 1; // only need to get a new encryption key once
+
+    // Compute desired striped block group
+    LocatedStripedBlock targetBlockGroup = getBlockGroupAt(target);
+
+    // Update current position
+    this.pos = target;
+    this.blockEnd = targetBlockGroup.getStartOffset() +
+        targetBlockGroup.getBlockSize() - 1;
+
+    long offsetIntoBlockGroup = target - targetBlockGroup.getStartOffset();
+    LocatedBlock[] targetBlocks = StripedBlockUtil.parseStripedBlockGroup(
+        targetBlockGroup, cellSize, dataBlkNum, parityBlkNum);
+    // The purpose is to get start offset into each block
+    ReadPortion[] readPortions = planReadPortions(groupSize, cellSize,
+        offsetIntoBlockGroup, 0, 0);
+    while (true) {
+      int i = 0;
+      InetSocketAddress targetAddr = null;
+      try {
+        blockReaders = new BlockReader[groupSize];
+        for (i = 0; i < groupSize; i++) {
+          LocatedBlock targetBlock = targetBlocks[i];
+          if (targetBlock == null) {
+            continue;
+          }
+          long offsetIntoBlock = readPortions[i].startOffsetInBlock;
+          DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null);
+          chosenNodes[i] = retval.info;
+          targetAddr = retval.addr;
+          StorageType storageType = retval.storageType;
+
+          ExtendedBlock blk = targetBlock.getBlock();
+          Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
+          CachingStrategy curCachingStrategy;
+          boolean shortCircuitForbidden;
+          synchronized(infoLock) {
+            curCachingStrategy = cachingStrategy;
+            shortCircuitForbidden = shortCircuitForbidden();
+          }
+          blockReaders[i] = new BlockReaderFactory(dfsClient.getConf()).
+              setInetSocketAddress(targetAddr).
+              setRemotePeerFactory(dfsClient).
+              setDatanodeInfo(chosenNodes[i]).
+              setStorageType(storageType).
+              setFileName(src).
+              setBlock(blk).
+              setBlockToken(accessToken).
+              setStartOffset(offsetIntoBlock).
+              setVerifyChecksum(verifyChecksum).
+              setClientName(dfsClient.clientName).
+              setLength(blk.getNumBytes() - offsetIntoBlock).
+              setCachingStrategy(curCachingStrategy).
+              setAllowShortCircuitLocalReads(!shortCircuitForbidden).
+              setClientCacheContext(dfsClient.getClientContext()).
+              setUserGroupInformation(dfsClient.ugi).
+              setConfiguration(dfsClient.getConfiguration()).
+              build();
+        }
+        currentLocatedBlock = targetBlockGroup;
+        return chosenNodes;
+      } catch (IOException ex) {
+        // Retry in case of encryption key or token exceptions. Otherwise throw
+        // IOException: since each internal block is singly replicated, it's
+        // not meaningful trying to locate another replica.
+        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 {
+          DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
+              + ", add to deadNodes and continue. " + ex, ex);
+          // Put chosen node into dead list and throw exception
+          addToDeadNodes(chosenNodes[i]);
+          throw ex;
+        }
+      }
+    }
+  }
+
+  /**
+   * 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() {
+    if (blockReaders ==  null || blockReaders.length == 0) {
+      return;
+    }
+    for (int i = 0; i < groupSize; i++) {
+      if (blockReaders[i] == null) {
+        continue;
+      }
+      try {
+        blockReaders[i].close();
+      } catch (IOException e) {
+        DFSClient.LOG.error("error closing blockReader", e);
+      }
+      blockReaders[i] = null;
+    }
+    blockEnd = -1;
   }
 
   @Override
-  public synchronized int read(final byte buf[], int off, int len)
+  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<>();
+    failures = 0;
+    if (pos < getFileLength()) {
+      int retries = 2;
+      /** Index of the target block in a stripe to read from */
+      int idxInGroup = (int) ((pos / cellSize) % dataBlkNum);
+      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 || currentNodes == null) {
+            currentNodes = 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) {
+            /**
+             * Temporary position into the file; {@link pos} might not proceed
+             * to this temporary position in case of exceptions.
+             */
+            long tmpPos = pos + result;
+            /** Start and end offsets of a cell in the file */
+            long cellStart = (tmpPos / cellSize) * cellSize;
+            long cellEnd = cellStart + cellSize - 1;
+
+            /** Number of bytes to read from the current cell */
+            int realLenInCell = (int) Math.min(realLen - result,
+                cellEnd - tmpPos + 1L);
+            assert realLenInCell > 0 : "Temporary position shouldn't be " +
+                "after cellEnd";
+            // Read from one blockReader up to cell boundary
+            int cellRet = readBuffer(blockReaders[idxInGroup],
+                currentNodes[idxInGroup], strategy, off + result,
+                realLenInCell);
+            if (cellRet >= 0) {
+              result += cellRet;
+              if (cellRet < realLenInCell) {
+                // A short read indicates the current blockReader buffer is
+                // already drained. Should return the read call. Otherwise
+                // should proceed to the next cell.
+                break;
+              }
+            } else {
+              // got a EOS from reader though we expect more data on it.
+              throw new IOException("Unexpected EOS from the reader");
+            }
+            idxInGroup = (idxInGroup + 1) % dataBlkNum;
+          }
+
+          pos += result;
+
+          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 (currentNodes[idxInGroup] != null) {
+            addToDeadNodes(currentNodes[idxInGroup]);
+          }
+          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;
+  }
+
+  private synchronized int readBuffer(BlockReader blockReader,
+      DatanodeInfo currentNode, ReaderStrategy readerStrategy, int off, int len)
+      throws IOException {
+    IOException ioe;
+    while (true) {
+      try {
+        return readerStrategy.doRead(blockReader, off, len);
+      } catch ( ChecksumException ce ) {
+        DFSClient.LOG.warn("Found Checksum error for "
+            + getCurrentBlock() + " from " + currentNode
+            + " at " + ce.getPos());
+        // If current block group is corrupt, it's meaningless to retry.
+        // TODO: this should trigger decoding logic (HDFS-7678)
+        throw ce;
+      } catch ( IOException e ) {
+        ioe = e;
+      }
+
+      boolean sourceFound = seekToBlockSource(pos);
+      if (!sourceFound) {
+        throw ioe;
+      }
+    }
+  }
+
+  private boolean seekToBlockSource(long targetPos)
       throws IOException {
-    throw new UnsupportedActionException("Stateful read is not supported");
+    currentNodes = blockSeekTo(targetPos);
+    return true;
+  }
+
+  protected class ByteBufferStrategy extends DFSInputStream.ByteBufferStrategy {
+    ByteBufferStrategy(ByteBuffer buf) {
+      super(buf);
+    }
+
+    @Override
+    public int doRead(BlockReader blockReader, int off, int len)
+        throws ChecksumException, IOException {
+      int oldlimit = buf.limit();
+      if (buf.remaining() > len) {
+        buf.limit(buf.position() + len);
+      }
+      int ret = super.doRead(blockReader, off, len);
+      buf.limit(oldlimit);
+      return ret;
+    }
   }
 
   /**
@@ -188,8 +464,11 @@ public class DFSStripedInputStream extends DFSInputStream {
         dataBlkNum, idx);
   }
 
-  private LocatedBlock getBlockGroupAt(long offset) throws IOException {
-    return super.getBlockAt(offset);
+  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;
   }
 
   /**
@@ -206,10 +485,8 @@ public class DFSStripedInputStream extends DFSInputStream {
     int len = (int) (end - start + 1);
 
     // Refresh the striped block group
-    LocatedBlock block = getBlockGroupAt(blockStartOffset);
-    assert block instanceof LocatedStripedBlock : "NameNode" +
-        " should return a LocatedStripedBlock for a striped file";
-    LocatedStripedBlock blockGroup = (LocatedStripedBlock) block;
+    LocatedStripedBlock blockGroup = getBlockGroupAt(blockStartOffset);
+
 
     // Planning the portion of I/O for each shard
     ReadPortion[] readPortions = planReadPortions(dataBlkNum, cellSize, start,
@@ -308,7 +585,7 @@ public class DFSStripedInputStream extends DFSInputStream {
      * +------------------------------------------------------+
      */
     private long startOffsetInBlock = 0;
-    private long readLength = 0;
+    private int readLength = 0;
     private final List<Integer> offsetsInBuf = new ArrayList<>();
     private final List<Integer> lengths = new ArrayList<>();
 
@@ -328,7 +605,7 @@ public class DFSStripedInputStream extends DFSInputStream {
       return lens;
     }
 
-    long getReadLength() {
+    int getReadLength() {
       return readLength;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc1609de/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 73c7350..cf10981 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
@@ -28,6 +28,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 public class TestDFSStripedInputStream {
   private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
@@ -165,6 +166,7 @@ public class TestDFSStripedInputStream {
     Assert.assertEquals("File length should be the same",
         writeBytes, fileLength);
 
+    // pread
     try (DFSStripedInputStream dis =
              new DFSStripedInputStream(fs.getClient(), src, true)) {
       byte[] buf = new byte[writeBytes + 100];
@@ -176,5 +178,46 @@ public class TestDFSStripedInputStream {
         Assert.assertEquals("Byte at i should be the same", getByte(i), buf[i]);
       }
     }
+
+    // stateful read with byte array
+    try (DFSStripedInputStream dis =
+             new DFSStripedInputStream(fs.getClient(), src, true)) {
+      byte[] buf = new byte[writeBytes + 100];
+      int readLen = 0;
+      int ret;
+      do {
+        ret = dis.read(buf, readLen, buf.length - readLen);
+        if (ret > 0) {
+          readLen += ret;
+        }
+      } while (ret >= 0);
+
+      readLen = readLen >= 0 ? readLen : 0;
+      Assert.assertEquals("The length of file should be the same to write size",
+          writeBytes, readLen);
+      for (int i = 0; i < writeBytes; i++) {
+        Assert.assertEquals("Byte at i should be the same", getByte(i), buf[i]);
+      }
+    }
+
+    // stateful read with ByteBuffer
+    try (DFSStripedInputStream dis =
+             new DFSStripedInputStream(fs.getClient(), src, true)) {
+      ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100);
+      int readLen = 0;
+      int ret;
+      do {
+        ret = dis.read(buf);
+        if (ret > 0) {
+          readLen += ret;
+        }
+      } while (ret >= 0);
+      readLen = readLen >= 0 ? readLen : 0;
+      Assert.assertEquals("The length of file should be the same to write size",
+          writeBytes, readLen);
+      for (int i = 0; i < writeBytes; i++) {
+        Assert.assertEquals("Byte at i should be the same", getByte(i), buf.array()[i]);
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc1609de/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
index b0631ce..d980bd6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+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;
@@ -28,7 +30,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
@@ -38,6 +39,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.List;
 
@@ -52,19 +54,21 @@ public class TestReadStripedFile {
   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 BLK_GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM;
   private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
   private final int NUM_STRIPE_PER_BLOCK = 2;
-  private final int BLOCKSIZE = NUM_STRIPE_PER_BLOCK * DATA_BLK_NUM * CELLSIZE;
+  private final int INTERNAL_BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE;
+  private final int BLOCK_GROUP_SIZE =  DATA_BLK_NUM * INTERNAL_BLOCK_SIZE;
 
   @Before
   public void setup() throws IOException {
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE);
     SimulatedFSDataset.setFactory(conf);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(BLK_GROUP_SIZE)
-        .build();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
+        DATA_BLK_NUM + PARITY_BLK_NUM).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
+    fs.mkdirs(dirPath);
+    fs.getClient().createErasureCodingZone(dirPath.toString(), null);
   }
 
   @After
@@ -80,10 +84,10 @@ public class TestReadStripedFile {
   @Test
   public void testGetBlock() throws Exception {
     final int numBlocks = 4;
-    DFSTestUtil.createStripedFile(cluster, filePath, dirPath, numBlocks,
-        NUM_STRIPE_PER_BLOCK, true);
+    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
+        NUM_STRIPE_PER_BLOCK, false);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
-        filePath.toString(), 0, BLOCKSIZE * numBlocks);
+        filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
     final DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(), false);
 
@@ -103,11 +107,11 @@ public class TestReadStripedFile {
 
   @Test
   public void testPread() throws Exception {
-    final int numBlocks = 4;
-    DFSTestUtil.createStripedFile(cluster, filePath, dirPath, numBlocks,
-        NUM_STRIPE_PER_BLOCK, true);
+    final int numBlocks = 2;
+    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
+        NUM_STRIPE_PER_BLOCK, false);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
-        filePath.toString(), 0, BLOCKSIZE);
+        filePath.toString(), 0, BLOCK_GROUP_SIZE);
 
     assert lbs.get(0) instanceof LocatedStripedBlock;
     LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0));
@@ -121,11 +125,89 @@ public class TestReadStripedFile {
     }
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(), false);
-    int readSize = BLOCKSIZE;
+    int readSize = BLOCK_GROUP_SIZE;
     byte[] readBuffer = new byte[readSize];
     int ret = in.read(0, readBuffer, 0, readSize);
 
     assertEquals(readSize, ret);
     // TODO: verify read results with patterned data from HDFS-8117
   }
+
+  @Test
+  public void testStatefulRead() throws Exception {
+    testStatefulRead(false, false);
+    testStatefulRead(true, false);
+    testStatefulRead(true, true);
+  }
+
+  private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket)
+      throws Exception {
+    final int numBlocks = 2;
+    final int fileSize = numBlocks * BLOCK_GROUP_SIZE;
+    if (cellMisalignPacket) {
+      conf.setInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT + 1);
+      tearDown();
+      setup();
+    }
+    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, fileSize);
+
+    assert lbs.getLocatedBlocks().size() == numBlocks;
+    for (LocatedBlock lb : lbs.getLocatedBlocks()) {
+      assert lb instanceof LocatedStripedBlock;
+      LocatedStripedBlock bg = (LocatedStripedBlock)(lb);
+      for (int i = 0; i < DATA_BLK_NUM; i++) {
+        Block blk = new Block(bg.getBlock().getBlockId() + i,
+            NUM_STRIPE_PER_BLOCK * CELLSIZE,
+            bg.getBlock().getGenerationStamp());
+        blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
+        cluster.injectBlocks(i, Arrays.asList(blk),
+            bg.getBlock().getBlockPoolId());
+      }
+    }
+
+    DFSStripedInputStream in =
+        new DFSStripedInputStream(fs.getClient(), filePath.toString(),
+            false);
+
+    byte[] expected = new byte[fileSize];
+
+    for (LocatedBlock bg : lbs.getLocatedBlocks()) {
+      /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
+      for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) {
+        for (int j = 0; j < DATA_BLK_NUM; j++) {
+          for (int k = 0; k < CELLSIZE; k++) {
+            int posInBlk = i * CELLSIZE + k;
+            int posInFile = (int) bg.getStartOffset() +
+                i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k;
+            expected[posInFile] = SimulatedFSDataset.simulatedByte(
+                new Block(bg.getBlock().getBlockId() + j), posInBlk);
+          }
+        }
+      }
+    }
+
+    if (useByteBuffer) {
+      ByteBuffer readBuffer = ByteBuffer.allocate(fileSize);
+      int done = 0;
+      while (done < fileSize) {
+        int ret = in.read(readBuffer);
+        assertTrue(ret > 0);
+        done += ret;
+      }
+      assertArrayEquals(expected, readBuffer.array());
+    } else {
+      byte[] readBuffer = new byte[fileSize];
+      int done = 0;
+      while (done < fileSize) {
+        int ret = in.read(readBuffer, done, fileSize - done);
+        assertTrue(ret > 0);
+        done += ret;
+      }
+      assertArrayEquals(expected, readBuffer);
+    }
+    fs.delete(filePath, true);
+  }
 }


[19/50] hadoop git commit: HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema in FileSystemLinkResolver. Contributed by Tsz Wo Nicholas Sze.

Posted by zh...@apache.org.
HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema in FileSystemLinkResolver. 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/fa8df98d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fa8df98d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fa8df98d

Branch: refs/heads/HDFS-7285
Commit: fa8df98d86f7702506b1a692b6266663f1f1d9f2
Parents: d387b82
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Apr 21 21:03:07 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:18 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt          | 3 +++
 .../main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java   | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa8df98d/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 d8f2e9d..3d86f05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -110,3 +110,6 @@
 
     HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to 
     create BlockReader. (szetszwo via Zhe Zhang)
+    
+    HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema
+    in FileSystemLinkResolver. (szetszwo via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa8df98d/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 4c8fff3..ede4f48 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
@@ -2281,7 +2281,7 @@ public class DistributedFileSystem extends FileSystem {
       @Override
       public Void doCall(final Path p) throws IOException,
           UnresolvedLinkException {
-        dfs.createErasureCodingZone(getPathName(p), null);
+        dfs.createErasureCodingZone(getPathName(p), schema);
         return null;
       }
 


[27/50] hadoop git commit: HDFS-8272. Erasure Coding: simplify the retry logic in DFSStripedInputStream (stateful read). Contributed by Jing Zhao

Posted by zh...@apache.org.
HDFS-8272. Erasure Coding: simplify the retry logic in DFSStripedInputStream (stateful read). 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/a71bfe0e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a71bfe0e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a71bfe0e

Branch: refs/heads/HDFS-7285
Commit: a71bfe0eb8014185f7b90618760952c24c96598a
Parents: 2e3fb43
Author: Zhe Zhang <zh...@apache.org>
Authored: Wed Apr 29 15:53:31 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:21 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../hadoop/hdfs/DFSStripedInputStream.java      | 336 ++++++++-----------
 2 files changed, 150 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a71bfe0e/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 9b4bf24..6a9bdee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -143,3 +143,6 @@
 
     HDFS-8235. Erasure Coding: Create DFSStripedInputStream in DFSClient#open.
     (Kai Sasaki via jing9)
+
+    HDFS-8272. Erasure Coding: simplify the retry logic in DFSStripedInputStream 
+    (stateful read). (Jing Zhao via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a71bfe0e/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 f6f7ed2..3da7306 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
@@ -22,11 +22,8 @@ import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.token.Token;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
@@ -126,23 +123,42 @@ public class DFSStripedInputStream extends DFSInputStream {
     return results;
   }
 
+  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;
+    }
+  }
+
   private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS;
-  private BlockReader[] blockReaders = null;
-  private DatanodeInfo[] currentNodes = null;
+  private final BlockReader[] blockReaders = new BlockReader[groupSize];
+  private final DatanodeInfo[] currentNodes = new DatanodeInfo[groupSize];
   private final int cellSize;
   private final short dataBlkNum;
   private final short parityBlkNum;
-  private final ECInfo ecInfo;
 
-  DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, ECInfo info)
-      throws IOException {
+  DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
+      ECInfo ecInfo) throws IOException {
     super(dfsClient, src, verifyChecksum);
     // ECInfo is restored from NN just before reading striped file.
-    assert info != null;
-    ecInfo = info;
+    assert ecInfo != null;
     cellSize = ecInfo.getSchema().getChunkSize();
-    dataBlkNum = (short)ecInfo.getSchema().getNumDataUnits();
-    parityBlkNum = (short)ecInfo.getSchema().getNumParityUnits();
+    dataBlkNum = (short) ecInfo.getSchema().getNumDataUnits();
+    parityBlkNum = (short) ecInfo.getSchema().getNumParityUnits();
     DFSClient.LOG.debug("Creating an striped input stream for file " + src);
   }
 
@@ -162,9 +178,7 @@ public class DFSStripedInputStream extends DFSInputStream {
    * When seeking into a new block group, create blockReader for each internal
    * block in the group.
    */
-  @VisibleForTesting
-  private synchronized DatanodeInfo[] blockSeekTo(long target)
-      throws IOException {
+  private synchronized void blockSeekTo(long target) throws IOException {
     if (target >= getFileLength()) {
       throw new IOException("Attempted to read past end of file");
     }
@@ -172,18 +186,13 @@ public class DFSStripedInputStream extends DFSInputStream {
     // Will be getting a new BlockReader.
     closeCurrentBlockReaders();
 
-    // Connect to best DataNode for desired Block, with potential offset
-    DatanodeInfo[] chosenNodes = new DatanodeInfo[groupSize];
-    int refetchToken = 1; // only need to get a new access token once
-    int refetchEncryptionKey = 1; // only need to get a new encryption key once
-
     // Compute desired striped block group
     LocatedStripedBlock targetBlockGroup = getBlockGroupAt(target);
-
     // Update current position
     this.pos = target;
     this.blockEnd = targetBlockGroup.getStartOffset() +
         targetBlockGroup.getBlockSize() - 1;
+    currentLocatedBlock = targetBlockGroup;
 
     long offsetIntoBlockGroup = target - targetBlockGroup.getStartOffset();
     LocatedBlock[] targetBlocks = StripedBlockUtil.parseStripedBlockGroup(
@@ -191,71 +200,50 @@ public class DFSStripedInputStream extends DFSInputStream {
     // The purpose is to get start offset into each block
     ReadPortion[] readPortions = planReadPortions(groupSize, cellSize,
         offsetIntoBlockGroup, 0, 0);
+
+    final ReaderRetryPolicy retry = new ReaderRetryPolicy();
+    for (int i = 0; i < groupSize; i++) {
+      LocatedBlock targetBlock = targetBlocks[i];
+      if (targetBlock != null) {
+        DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null);
+        if (retval != null) {
+          currentNodes[i] = retval.info;
+          blockReaders[i] = getBlockReaderWithRetry(targetBlock,
+              readPortions[i].startOffsetInBlock,
+              targetBlock.getBlockSize() - readPortions[i].startOffsetInBlock,
+              retval.addr, retval.storageType, retval.info, target, retry);
+        }
+      }
+    }
+  }
+
+  private BlockReader getBlockReaderWithRetry(LocatedBlock targetBlock,
+      long offsetInBlock, long length, InetSocketAddress targetAddr,
+      StorageType storageType, DatanodeInfo datanode, long offsetInFile,
+      ReaderRetryPolicy retry) throws IOException {
+    // only need to get a new access token or a new encryption key once
     while (true) {
-      int i = 0;
-      InetSocketAddress targetAddr = null;
       try {
-        blockReaders = new BlockReader[groupSize];
-        for (i = 0; i < groupSize; i++) {
-          LocatedBlock targetBlock = targetBlocks[i];
-          if (targetBlock == null) {
-            continue;
-          }
-          long offsetIntoBlock = readPortions[i].startOffsetInBlock;
-          DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null);
-          chosenNodes[i] = retval.info;
-          targetAddr = retval.addr;
-          StorageType storageType = retval.storageType;
-
-          ExtendedBlock blk = targetBlock.getBlock();
-          Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
-          CachingStrategy curCachingStrategy;
-          boolean shortCircuitForbidden;
-          synchronized(infoLock) {
-            curCachingStrategy = cachingStrategy;
-            shortCircuitForbidden = shortCircuitForbidden();
-          }
-          blockReaders[i] = new BlockReaderFactory(dfsClient.getConf()).
-              setInetSocketAddress(targetAddr).
-              setRemotePeerFactory(dfsClient).
-              setDatanodeInfo(chosenNodes[i]).
-              setStorageType(storageType).
-              setFileName(src).
-              setBlock(blk).
-              setBlockToken(accessToken).
-              setStartOffset(offsetIntoBlock).
-              setVerifyChecksum(verifyChecksum).
-              setClientName(dfsClient.clientName).
-              setLength(blk.getNumBytes() - offsetIntoBlock).
-              setCachingStrategy(curCachingStrategy).
-              setAllowShortCircuitLocalReads(!shortCircuitForbidden).
-              setClientCacheContext(dfsClient.getClientContext()).
-              setUserGroupInformation(dfsClient.ugi).
-              setConfiguration(dfsClient.getConfiguration()).
-              build();
-        }
-        currentLocatedBlock = targetBlockGroup;
-        return chosenNodes;
-      } catch (IOException ex) {
-        // Retry in case of encryption key or token exceptions. Otherwise throw
-        // IOException: since each internal block is singly replicated, it's
-        // not meaningful trying to locate another replica.
-        if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+        return getBlockReader(targetBlock, offsetInBlock, length, targetAddr,
+            storageType, datanode);
+      } 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 " + targetAddr
-              + " : " + ex);
-          // The encryption key used is invalid.
-          refetchEncryptionKey--;
+              + " : " + e);
           dfsClient.clearDataEncryptionKey();
-        } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
-          refetchToken--;
-          fetchBlockAt(target);
+          retry.refetchEncryptionKey();
+        } else if (retry.shouldRefetchToken() &&
+            tokenRefetchNeeded(e, targetAddr)) {
+          fetchBlockAt(offsetInFile);
+          retry.refetchToken();
         } else {
           DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
-              + ", add to deadNodes and continue. " + ex, ex);
-          // Put chosen node into dead list and throw exception
-          addToDeadNodes(chosenNodes[i]);
-          throw ex;
+              + ", add to deadNodes and continue.", e);
+          // Put chosen node into dead list, continue
+          addToDeadNodes(datanode);
+          return null;
         }
       }
     }
@@ -272,15 +260,15 @@ public class DFSStripedInputStream extends DFSInputStream {
       return;
     }
     for (int i = 0; i < groupSize; i++) {
-      if (blockReaders[i] == null) {
-        continue;
-      }
-      try {
-        blockReaders[i].close();
-      } catch (IOException e) {
-        DFSClient.LOG.error("error closing blockReader", e);
+      if (blockReaders[i] != null) {
+        try {
+          blockReaders[i].close();
+        } catch (IOException e) {
+          DFSClient.LOG.error("error closing blockReader", e);
+        }
+        blockReaders[i] = null;
       }
-      blockReaders[i] = null;
+      currentNodes[i] = null;
     }
     blockEnd = -1;
   }
@@ -292,123 +280,93 @@ public class DFSStripedInputStream extends DFSInputStream {
     if (closed.get()) {
       throw new IOException("Stream closed");
     }
-    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
-        = new HashMap<>();
+    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
     failures = 0;
     if (pos < getFileLength()) {
-      int retries = 2;
       /** Index of the target block in a stripe to read from */
       int idxInGroup = (int) ((pos / cellSize) % dataBlkNum);
-      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 || currentNodes == null) {
-            currentNodes = blockSeekTo(pos);
-          }
-          int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
-          synchronized(infoLock) {
-            if (locatedBlocks.isLastBlockComplete()) {
-              realLen = (int) Math.min(realLen,
-                  locatedBlocks.getFileLength() - pos);
-            }
+      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) {
-            /**
-             * Temporary position into the file; {@link pos} might not proceed
-             * to this temporary position in case of exceptions.
-             */
-            long tmpPos = pos + result;
-            /** Start and end offsets of a cell in the file */
-            long cellStart = (tmpPos / cellSize) * cellSize;
-            long cellEnd = cellStart + cellSize - 1;
-
-            /** Number of bytes to read from the current cell */
-            int realLenInCell = (int) Math.min(realLen - result,
-                cellEnd - tmpPos + 1L);
-            assert realLenInCell > 0 : "Temporary position shouldn't be " +
-                "after cellEnd";
-            // Read from one blockReader up to cell boundary
-            int cellRet = readBuffer(blockReaders[idxInGroup],
-                currentNodes[idxInGroup], strategy, off + result,
-                realLenInCell);
-            if (cellRet >= 0) {
-              result += cellRet;
-              if (cellRet < realLenInCell) {
-                // A short read indicates the current blockReader buffer is
-                // already drained. Should return the read call. Otherwise
-                // should proceed to the next cell.
-                break;
-              }
-            } else {
-              // got a EOS from reader though we expect more data on it.
-              throw new IOException("Unexpected EOS from the reader");
+        /** Number of bytes already read into buffer */
+        int result = 0;
+        while (result < realLen) {
+          /**
+           * Temporary position into the file; {@link pos} might not proceed
+           * to this temporary position in case of exceptions.
+           */
+          long tmpPos = pos + result;
+          /** Start and end offsets of a cell in the file */
+          long cellStart = (tmpPos / cellSize) * cellSize;
+          long cellEnd = cellStart + cellSize - 1;
+
+          /** Number of bytes to read from the current cell */
+          int realLenInCell = (int) Math.min(realLen - result,
+              cellEnd - tmpPos + 1L);
+          assert realLenInCell > 0 : "Temporary position shouldn't be "
+              + "after cellEnd";
+
+          // Read from one blockReader up to cell boundary
+          int cellRet = readBuffer(blockReaders[idxInGroup],
+              currentNodes[idxInGroup], strategy, off + result, realLenInCell,
+              corruptedBlockMap);
+          if (cellRet >= 0) {
+            result += cellRet;
+            if (cellRet < realLenInCell) {
+              // A short read indicates the current blockReader buffer is
+              // already drained. Should return the read call. Otherwise
+              // should proceed to the next cell.
+              break;
             }
-            idxInGroup = (idxInGroup + 1) % dataBlkNum;
-          }
-
-          pos += result;
-
-          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 (currentNodes[idxInGroup] != null) {
-            addToDeadNodes(currentNodes[idxInGroup]);
+          } else {
+            // got a EOS from reader though we expect more data on it.
+            throw new IOException("Unexpected EOS from the reader");
           }
-          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);
+          idxInGroup = (idxInGroup + 1) % dataBlkNum;
         }
+        pos += result;
+        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;
   }
 
   private synchronized int readBuffer(BlockReader blockReader,
-      DatanodeInfo currentNode, ReaderStrategy readerStrategy, int off, int len)
-      throws IOException {
-    IOException ioe;
-    while (true) {
-      try {
-        return readerStrategy.doRead(blockReader, off, len);
-      } catch ( ChecksumException ce ) {
-        DFSClient.LOG.warn("Found Checksum error for "
-            + getCurrentBlock() + " from " + currentNode
-            + " at " + ce.getPos());
-        // If current block group is corrupt, it's meaningless to retry.
-        // TODO: this should trigger decoding logic (HDFS-7678)
-        throw ce;
-      } catch ( IOException e ) {
-        ioe = e;
-      }
-
-      boolean sourceFound = seekToBlockSource(pos);
-      if (!sourceFound) {
-        throw ioe;
-      }
+      DatanodeInfo currentNode, ReaderStrategy readerStrategy, int off, int len,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
+    try {
+      return readerStrategy.doRead(blockReader, off, len);
+    } catch ( ChecksumException ce ) {
+      DFSClient.LOG.warn("Found Checksum error for "
+          + getCurrentBlock() + " from " + currentNode
+          + " at " + ce.getPos());
+      // we want to remember which block replicas we have tried
+      addIntoCorruptedBlockMap(getCurrentBlock(), currentNode,
+          corruptedBlockMap);
+    } catch (IOException e) {
+      DFSClient.LOG.warn("Exception while reading from "
+          + getCurrentBlock() + " of " + src + " from "
+          + currentNode, e);
     }
-  }
-
-  private boolean seekToBlockSource(long targetPos)
-      throws IOException {
-    currentNodes = blockSeekTo(targetPos);
-    return true;
+    // TODO: this should trigger decoding logic (HDFS-7678)
+    return -1;
   }
 
   protected class ByteBufferStrategy extends DFSInputStream.ByteBufferStrategy {
@@ -418,7 +376,7 @@ public class DFSStripedInputStream extends DFSInputStream {
 
     @Override
     public int doRead(BlockReader blockReader, int off, int len)
-        throws ChecksumException, IOException {
+        throws IOException {
       int oldlimit = buf.limit();
       if (buf.remaining() > len) {
         buf.limit(buf.position() + len);


[11/50] hadoop git commit: HDFS-8146. Protobuf changes for BlockECRecoveryCommand and its fields for making it ready for transfer to DN (Contributed by Uma Maheswara Rao G)

Posted by zh...@apache.org.
HDFS-8146. Protobuf changes for BlockECRecoveryCommand and its fields for making it ready for transfer to DN (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/0c37627b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0c37627b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0c37627b

Branch: refs/heads/HDFS-7285
Commit: 0c37627b0778651b5c2a9bfe77c44c28d123279d
Parents: 9219042
Author: Vinayakumar B <vi...@apache.org>
Authored: Sat Apr 18 23:20:45 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:16 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 137 ++++++++++++++++++-
 .../blockmanagement/DatanodeDescriptor.java     |  31 +----
 .../server/blockmanagement/DatanodeManager.java |   4 +-
 .../server/protocol/BlockECRecoveryCommand.java |  80 ++++++++++-
 .../hdfs/server/protocol/DatanodeProtocol.java  |   2 +-
 .../src/main/proto/DatanodeProtocol.proto       |   8 ++
 .../src/main/proto/erasurecoding.proto          |  13 ++
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |  88 ++++++++++++
 .../namenode/TestRecoverStripedBlocks.java      |  10 +-
 10 files changed, 335 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c37627b/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 0ed61cd..40517e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -87,3 +87,6 @@
     startup. (Hui Zheng via szetszwo)
 
     HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. (Hui Zheng via zhz).
+
+    HDFS-8146. Protobuf changes for BlockECRecoveryCommand and its fields for
+    making it ready for transfer to DN (Uma Maheswara Rao G via vinayakumarb)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c37627b/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 9ca73ae..c127b5f 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
@@ -28,6 +28,7 @@ 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;
@@ -100,7 +101,7 @@ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTyp
 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;
+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;
@@ -121,6 +122,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmI
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
 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;
@@ -132,11 +134,11 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDele
 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.ErasureCodingProtos.ECInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECZoneInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 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.BlockStoragePolicyProto;
@@ -184,7 +186,6 @@ 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.HdfsProtos.StripedBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
@@ -204,8 +205,10 @@ import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@@ -3150,4 +3153,132 @@ public class PBHelper {
     return new ECZoneInfo(ecZoneInfoProto.getDir(),
         convertECSchema(ecZoneInfoProto.getSchema()));
   }
+  
+  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();
+    }
+
+    return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos,
+        targetStorageUuids, convertStorageTypes, liveBlkIndices);
+  }
+
+  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));
+
+    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/0c37627b/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 7ec71a2..35cc31b 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
@@ -32,8 +32,8 @@ import java.util.Set;
 import java.util.Arrays;
 
 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;
@@ -44,6 +44,7 @@ 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.StorageReport;
@@ -99,34 +100,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  /** Block and targets pair */
-  @InterfaceAudience.Private
-  @InterfaceStability.Evolving
-  public static class BlockECRecoveryInfo {
-    public final ExtendedBlock block;
-    public final DatanodeDescriptor[] sources;
-    public final DatanodeStorageInfo[] targets;
-    public final short[] liveBlockIndices;
-
-    BlockECRecoveryInfo(ExtendedBlock block, DatanodeDescriptor[] sources,
-        DatanodeStorageInfo[] targets, short[] liveBlockIndices) {
-      this.block = block;
-      this.sources = sources;
-      this.targets = targets;
-      this.liveBlockIndices = liveBlockIndices;
-    }
-
-    @Override
-    public String toString() {
-      return new StringBuilder().append("BlockECRecoveryInfo(\n  ").
-          append("Recovering ").append(block).
-          append(" From: ").append(Arrays.asList(sources)).
-          append(" To: ").append(Arrays.asList(targets)).append(")\n").
-          append(" Block Indices: ").append(Arrays.asList(liveBlockIndices)).
-          toString();
-    }
-  }
-
   /** A BlockTargetPair queue. */
   private static class BlockQueue<E> {
     private final Queue<E> blockq = new LinkedList<E>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c37627b/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 fb5e904..e6ffd77 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
@@ -34,12 +34,12 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.protocol.*;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.*;
@@ -1441,7 +1441,7 @@ public class DatanodeManager {
         List<BlockECRecoveryInfo> pendingECList =
             nodeinfo.getErasureCodeCommand(maxTransfers);
         if (pendingECList != null) {
-          cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_CODEC,
+          cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY,
               pendingECList));
         }
         //check block invalidation

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c37627b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
index f7f02fd..9a387dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
@@ -18,10 +18,15 @@
 package org.apache.hadoop.hdfs.server.protocol;
 
 import com.google.common.base.Joiner;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 
+import java.util.Arrays;
 import java.util.Collection;
 
 /**
@@ -60,4 +65,77 @@ public class BlockECRecoveryCommand extends DatanodeCommand {
     sb.append("\n)");
     return sb.toString();
   }
+
+  /** Block and targets pair */
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  public static class BlockECRecoveryInfo {
+    private final ExtendedBlock block;
+    private final DatanodeInfo[] sources;
+    private DatanodeInfo[] targets;
+    private String[] targetStorageIDs;
+    private StorageType[] targetStorageTypes;
+    private final short[] liveBlockIndices;
+
+    public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources,
+        DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices) {
+      this.block = block;
+      this.sources = sources;
+      this.targets = DatanodeStorageInfo.toDatanodeInfos(targetDnStorageInfo);
+      this.targetStorageIDs = DatanodeStorageInfo
+          .toStorageIDs(targetDnStorageInfo);
+      this.targetStorageTypes = DatanodeStorageInfo
+          .toStorageTypes(targetDnStorageInfo);
+      this.liveBlockIndices = liveBlockIndices;
+    }
+    
+    public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources,
+        DatanodeInfo[] targets, String[] targetStorageIDs,
+        StorageType[] targetStorageTypes, short[] liveBlockIndices) {
+      this.block = block;
+      this.sources = sources;
+      this.targets = targets;
+      this.targetStorageIDs = targetStorageIDs;
+      this.targetStorageTypes = targetStorageTypes;
+      this.liveBlockIndices = liveBlockIndices;
+    }
+
+    public ExtendedBlock getExtendedBlock() {
+      return block;
+    }
+
+    public DatanodeInfo[] getSourceDnInfos() {
+      return sources;
+    }
+
+    public DatanodeInfo[] getTargetDnInfos() {
+      return targets;
+    }
+
+    public String[] getTargetStorageIDs() {
+      return targetStorageIDs;
+    }
+    
+    public StorageType[] getTargetStorageTypes() {
+      return targetStorageTypes;
+    }
+
+    public short[] getLiveBlockIndices() {
+      return liveBlockIndices;
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().append("BlockECRecoveryInfo(\n  ")
+          .append("Recovering ").append(block).append(" From: ")
+          .append(Arrays.asList(sources)).append(" To: [")
+          .append(Arrays.asList(targets)).append(")\n")
+          .append(" Block Indices: ").append(Arrays.asList(liveBlockIndices))
+          .toString();
+    }
+  }
+
+  public Collection<BlockECRecoveryInfo> getECTasks() {
+    return this.ecTasks;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c37627b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index b8ac165..1411fa9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -76,7 +76,7 @@ public interface DatanodeProtocol {
   final static int DNA_BALANCERBANDWIDTHUPDATE = 8; // update balancer bandwidth
   final static int DNA_CACHE = 9;      // cache blocks
   final static int DNA_UNCACHE = 10;   // uncache blocks
-  final static int DNA_CODEC = 11;   // uncache blocks
+  final static int DNA_ERASURE_CODING_RECOVERY = 11; // erasure coding recovery command
 
   /** 
    * Register Datanode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c37627b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 3083dc9..ac9ab46 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -33,6 +33,7 @@ package hadoop.hdfs.datanode;
 
 import "HAServiceProtocol.proto";
 import "hdfs.proto";
+import "erasurecoding.proto";
 
 /**
  * Information to identify a datanode to a namenode
@@ -145,6 +146,13 @@ message RegisterCommandProto {
 }
 
 /**
+ * Block Erasure coding recovery command
+ */
+message BlockECRecoveryCommandProto {
+  repeated BlockECRecoveryInfoProto blockECRecoveryinfo = 1;
+}
+
+/**
  * registration - Information of the datanode registering with the namenode
  */
 message RegisterDatanodeRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c37627b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
index d888f71..59bd949 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -21,6 +21,7 @@ option java_outer_classname = "ErasureCodingProtos";
 option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
+import "hdfs.proto";
 
 /**
  * ECSchema options entry
@@ -86,4 +87,16 @@ message GetECZoneInfoRequestProto {
 
 message GetECZoneInfoResponseProto {
   optional ECZoneInfoProto ECZoneInfo = 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;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c37627b/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 4b42f4c..4ec4ea5 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
@@ -24,6 +24,8 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -40,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 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.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
@@ -63,15 +66,20 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 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.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -639,4 +647,84 @@ public class TestPBHelper {
         .build();
     Assert.assertEquals(s, PBHelper.convert(PBHelper.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);
+    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);
+    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]);
+    }
+  }
+
+  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]);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c37627b/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 ea18c3e..ca4fbbc 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
@@ -29,9 +29,9 @@ 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.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -115,10 +115,10 @@ public class TestRecoverStripedBlocks {
         last.getNumberOfBlocksToBeErasureCoded());
     List<BlockECRecoveryInfo> recovery = last.getErasureCodeCommand(numBlocks);
     for (BlockECRecoveryInfo info : recovery) {
-      assertEquals(1, info.targets.length);
-      assertEquals(last, info.targets[0].getDatanodeDescriptor());
-      assertEquals(GROUP_SIZE - 1, info.sources.length);
-      assertEquals(GROUP_SIZE - 1, info.liveBlockIndices.length);
+      assertEquals(1, info.getTargetDnInfos().length);
+      assertEquals(last, info.getTargetDnInfos()[0]);
+      assertEquals(GROUP_SIZE - 1, info.getSourceDnInfos().length);
+      assertEquals(GROUP_SIZE - 1, info.getLiveBlockIndices().length);
     }
   }
 }


[50/50] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit is for HDFS-8327 and HDFS-8357). Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit is for HDFS-8327 and HDFS-8357). Contributed by Zhe Zhang.


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

Branch: refs/heads/HDFS-7285
Commit: 10e2d80c8bbcd9ec9380fc2f83a98debf37c1887
Parents: c31cb27
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon May 11 12:22:12 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 12:22:12 2015 -0700

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/BlockInfo.java  | 12 +++++--
 .../blockmanagement/BlockInfoContiguous.java    | 38 --------------------
 .../server/blockmanagement/BlockManager.java    |  4 +--
 .../erasurecode/ErasureCodingWorker.java        |  3 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 10 ++----
 .../server/namenode/TestStripedINodeFile.java   |  8 ++---
 .../namenode/TestTruncateQuotaUpdate.java       |  3 +-
 7 files changed, 23 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10e2d80c/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 aebfbb1..61068b9 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
@@ -88,13 +88,21 @@ public abstract class BlockInfo extends Block
   BlockInfo getPrevious(int index) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
-    return (BlockInfo) triplets[index*3+1];
+    BlockInfo info = (BlockInfo)triplets[index*3+1];
+    assert info == null ||
+        info.getClass().getName().startsWith(BlockInfo.class.getName()) :
+        "BlockInfo is expected at " + index*3;
+    return info;
   }
 
   BlockInfo getNext(int index) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
-    return (BlockInfo) triplets[index*3+2];
+    BlockInfo info = (BlockInfo)triplets[index*3+2];
+    assert info == null || info.getClass().getName().startsWith(
+        BlockInfo.class.getName()) :
+        "BlockInfo is expected at " + index*3;
+    return info;
   }
 
   void setStorageInfo(int index, DatanodeStorageInfo storage) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10e2d80c/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 a9ea625..be336b4 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
@@ -47,18 +47,6 @@ public class BlockInfoContiguous extends BlockInfo {
     this.setBlockCollection(from.getBlockCollection());
   }
 
-  public BlockCollection getBlockCollection() {
-    return bc;
-  }
-
-  public void setBlockCollection(BlockCollection bc) {
-    this.bc = bc;
-  }
-
-  public boolean isDeleted() {
-    return (bc == null);
-  }
-
   public DatanodeDescriptor getDatanode(int index) {
     DatanodeStorageInfo storage = getStorageInfo(index);
     return storage == null ? null : storage.getDatanodeDescriptor();
@@ -70,32 +58,6 @@ public class BlockInfoContiguous extends BlockInfo {
     return (DatanodeStorageInfo)triplets[index*3];
   }
 
-  private BlockInfoContiguous getPrevious(int index) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
-    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1];
-    assert info == null || 
-        info.getClass().getName().startsWith(BlockInfoContiguous.class.getName()) :
-              "BlockInfo is expected at " + index*3;
-    return info;
-  }
-
-  BlockInfoContiguous getNext(int index) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
-    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2];
-    assert info == null || info.getClass().getName().startsWith(
-        BlockInfoContiguous.class.getName()) :
-        "BlockInfo is expected at " + index*3;
-    return info;
-  }
-
-  private void setStorageInfo(int index, DatanodeStorageInfo storage) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-    triplets[index*3] = storage;
-  }
-
   /**
    * Return the previous block on the block list for the datanode at
    * position index. Set the previous block on the list to "to".

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10e2d80c/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 0ad391a..6b98dcf 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
@@ -2462,7 +2462,7 @@ public class BlockManager {
       if (rbi.getReportedState() == null) {
         // This is a DELETE_BLOCK request
         DatanodeStorageInfo storageInfo = rbi.getStorageInfo();
-        removeStoredBlock(rbi.getBlock(),
+        removeStoredBlock(getStoredBlock(rbi.getBlock()),
             storageInfo.getDatanodeDescriptor());
       } else {
         processAndHandleReportedBlock(rbi.getStorageInfo(),
@@ -3207,7 +3207,7 @@ public class BlockManager {
           QUEUE_REASON_FUTURE_GENSTAMP);
       return;
     }
-    removeStoredBlock(block, node);
+    removeStoredBlock(getStoredBlock(block), node);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10e2d80c/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 c4e568f..5ede508 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
@@ -49,6 +49,7 @@ 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.RemoteBlockReader2;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
@@ -872,7 +873,7 @@ public final class ErasureCodingWorker {
           unbufIn = saslStreams.in;
 
           out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-              HdfsServerConstants.SMALL_BUFFER_SIZE));
+              DFSUtil.getSmallBufferSize(conf)));
           in = new DataInputStream(unbufIn);
 
           DatanodeInfo source = new DatanodeInfo(datanode.getDatanodeId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10e2d80c/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 748c0be..79d4a29 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
@@ -708,11 +708,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   public final QuotaCounts computeQuotaUsageWithStriped(
       BlockStoragePolicySuite bsps, QuotaCounts counts) {
-    long nsDelta = 1;
-    final long ssDelta = storagespaceConsumed();
-    counts.addNameSpace(nsDelta);
-    counts.addStorageSpace(ssDelta);
-    return counts;
+    return null;
   }
 
   @Override
@@ -981,11 +977,11 @@ public class INodeFile extends INodeWithAdditionalFields
     }
 
     long size = 0;
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       size += b.getNumBytes();
     }
 
-    BlockInfoContiguous[] sblocks = null;
+    BlockInfo[] sblocks = null;
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
       FileDiff diff = sf.getDiffs().getLast();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10e2d80c/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 7a330b9..a725e6b 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
@@ -109,8 +109,8 @@ public class TestStripedINodeFile {
     //  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.storagespaceConsumedWithStriped());
-    assertEquals(4, inf.storagespaceConsumed());
+    assertEquals(4, inf.storagespaceConsumedWithStriped(null));
+    assertEquals(4, inf.storagespaceConsumed(null));
   }
 
   @Test
@@ -134,8 +134,8 @@ public class TestStripedINodeFile {
     inf.addBlock(blockInfoStriped1);
     inf.addBlock(blockInfoStriped2);
     // This is the double size of one block in above case.
-    assertEquals(4 * 2, inf.storagespaceConsumedWithStriped());
-    assertEquals(4 * 2, inf.storagespaceConsumed());
+    assertEquals(4 * 2, inf.storagespaceConsumedWithStriped(null));
+    assertEquals(4 * 2, inf.storagespaceConsumed(null));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10e2d80c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
index f6b18e6..57159db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 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.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
@@ -77,7 +78,7 @@ public class TestTruncateQuotaUpdate {
   @Test
   public void testTruncateWithSnapshotNoDivergence() {
     INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION);
-    addSnapshotFeature(file, file.getBlocks());
+    addSnapshotFeature(file, file.getContiguousBlocks());
 
     // case 4: truncate to 1.5 blocks
     // all the blocks are in snapshot. truncate need to allocate a new block


[13/50] hadoop git commit: HDFS-7937. Erasure Coding: INodeFile quota computation unit tests. Contributed by Kai Sasaki.

Posted by zh...@apache.org.
HDFS-7937. Erasure Coding: INodeFile quota computation unit tests. Contributed by Kai Sasaki.


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

Branch: refs/heads/HDFS-7285
Commit: ddf2a1ac72c587de6a3a254baa40fc5bdd4aa34b
Parents: 1d80df7
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Apr 17 18:07:07 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:16 2015 -0700

----------------------------------------------------------------------
 .../blockmanagement/BlockInfoStriped.java       |  23 +-
 .../server/namenode/TestStripedINodeFile.java   | 229 +++++++++++++++++++
 2 files changed, 250 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ddf2a1ac/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 20b0c5c..9f2f5ba 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
@@ -203,8 +203,27 @@ public class BlockInfoStriped extends BlockInfo {
     // 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 ((getNumBytes() - 1) / (dataBlockNum * BLOCK_STRIPED_CELL_SIZE) + 1)
-        * BLOCK_STRIPED_CELL_SIZE * parityBlockNum + getNumBytes();
+
+    // 0. Calculate the total bytes per stripes <Num Bytes per Stripes>
+    long numBytesPerStripe = dataBlockNum * BLOCK_STRIPED_CELL_SIZE;
+    if (getNumBytes() % numBytesPerStripe == 0) {
+      return getNumBytes() / dataBlockNum * getTotalBlockNum();
+    }
+    // 1. Calculate the number of stripes in this block group. <Num Stripes>
+    long numStripes = (getNumBytes() - 1) / numBytesPerStripe + 1;
+    // 2. Calculate the parity cell length in the last stripe. Note that the
+    //    size of parity cells should equal the size of the first cell, if it
+    //    is not full. <Last Stripe Parity Cell Length>
+    long lastStripeParityCellLen = Math.min(getNumBytes() % numBytesPerStripe,
+        BLOCK_STRIPED_CELL_SIZE);
+    // 3. Total consumed space is the total of
+    //     - The total of the full cells of data blocks and parity blocks.
+    //     - The remaining of data block which does not make a stripe.
+    //     - The last parity block cells. These size should be same
+    //       to the first cell in this stripe.
+    return getTotalBlockNum() * (BLOCK_STRIPED_CELL_SIZE * (numStripes - 1))
+        + getNumBytes() % numBytesPerStripe
+        + lastStripeParityCellLen * parityBlockNum;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ddf2a1ac/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
new file mode 100644
index 0000000..d251c30
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@ -0,0 +1,229 @@
+/**
+ * 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 java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+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.io.erasurecode.ECSchema;
+
+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 static INodeFile createStripedINodeFile() {
+    return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+        null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID);
+  }
+
+  @Test
+  public void testBlockStripedFeature()
+      throws IOException, InterruptedException{
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    assertTrue(inf.isStriped());
+  }
+
+  @Test
+  public void testBlockStripedTotalBlockCount() {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    assertEquals(9, blockInfoStriped.getTotalBlockNum());
+  }
+
+  @Test
+  public void testBlockStripedLength()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    inf.addBlock(blockInfoStriped);
+    assertEquals(1, inf.getBlocks().length);
+  }
+
+  @Test
+  public void testBlockStripedConsumedSpace()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    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.storagespaceConsumedWithStriped());
+    assertEquals(4, inf.storagespaceConsumed());
+  }
+
+  @Test
+  public void testMultipleBlockStripedConsumedSpace()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk1 = new Block(1);
+    BlockInfoStriped blockInfoStriped1
+        = new BlockInfoStriped(blk1,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    blockInfoStriped1.setNumBytes(1);
+    Block blk2 = new Block(2);
+    BlockInfoStriped blockInfoStriped2
+        = new BlockInfoStriped(blk2,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    blockInfoStriped2.setNumBytes(1);
+    inf.addBlock(blockInfoStriped1);
+    inf.addBlock(blockInfoStriped2);
+    // This is the double size of one block in above case.
+    assertEquals(4 * 2, inf.storagespaceConsumedWithStriped());
+    assertEquals(4 * 2, inf.storagespaceConsumed());
+  }
+
+  @Test
+  public void testBlockStripedFileSize()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    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 {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStripedUnderConstruction bInfoStripedUC
+        = new BlockInfoStripedUnderConstruction(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    bInfoStripedUC.setNumBytes(100);
+    inf.addBlock(bInfoStripedUC);
+    assertEquals(100, inf.computeFileSize());
+    assertEquals(0, inf.computeFileSize(false, false));
+  }
+
+  @Test
+  public void testBlockStripedComputeQuotaUsage()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    blockInfoStriped.setNumBytes(100);
+    inf.addBlock(blockInfoStriped);
+
+    BlockStoragePolicySuite suite =
+        BlockStoragePolicySuite.createDefaultSuite();
+    QuotaCounts counts =
+        inf.computeQuotaUsageWithStriped(suite,
+            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 {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStripedUnderConstruction bInfoStripedUC
+        = new BlockInfoStripedUnderConstruction(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    bInfoStripedUC.setNumBytes(100);
+    inf.addBlock(bInfoStripedUC);
+
+    BlockStoragePolicySuite suite
+        = BlockStoragePolicySuite.createDefaultSuite();
+    QuotaCounts counts
+        = inf.computeQuotaUsageWithStriped(suite,
+              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());
+  }
+}


[06/50] hadoop git commit: HDFS-7349. Support DFS command for the EC encoding (Contributed by Vinayakumar B)

Posted by zh...@apache.org.
HDFS-7349. Support DFS command for the EC encoding (Contributed by Vinayakumar B)


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

Branch: refs/heads/HDFS-7285
Commit: 43df7bef600a9668eca44cf6a705d5da8e073e81
Parents: e594f84
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Apr 15 16:38:22 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:14 2015 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/fs/FsShell.java |   8 +-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   4 +-
 .../hadoop-hdfs/src/main/bin/hdfs               |   5 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  18 ++
 .../hadoop/hdfs/DistributedFileSystem.java      |  32 +++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   9 +
 .../apache/hadoop/hdfs/protocol/ECZoneInfo.java |  56 +++++
 ...tNamenodeProtocolServerSideTranslatorPB.java |  18 ++
 .../ClientNamenodeProtocolTranslatorPB.java     |  19 ++
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  12 ++
 .../namenode/ErasureCodingZoneManager.java      |  11 +-
 .../hdfs/server/namenode/FSDirectory.java       |  10 +
 .../hdfs/server/namenode/FSNamesystem.java      |  24 +++
 .../hdfs/server/namenode/NameNodeRpcServer.java |   7 +
 .../hadoop/hdfs/tools/erasurecode/ECCli.java    |  48 +++++
 .../hdfs/tools/erasurecode/ECCommand.java       | 209 +++++++++++++++++++
 .../src/main/proto/ClientNamenodeProtocol.proto |   2 +
 .../src/main/proto/erasurecoding.proto          |  15 ++
 18 files changed, 502 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
index db73f6d..f873a01 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
@@ -111,6 +111,10 @@ public class FsShell extends Configured implements Tool {
     return getTrash().getCurrentTrashDir();
   }
 
+  protected String getUsagePrefix() {
+    return usagePrefix;
+  }
+
   // NOTE: Usage/Help are inner classes to allow access to outer methods
   // that access commandFactory
   
@@ -194,7 +198,7 @@ public class FsShell extends Configured implements Tool {
       }
     } else {
       // display help or usage for all commands 
-      out.println(usagePrefix);
+      out.println(getUsagePrefix());
       
       // display list of short usages
       ArrayList<Command> instances = new ArrayList<Command>();
@@ -218,7 +222,7 @@ public class FsShell extends Configured implements Tool {
   }
 
   private void printInstanceUsage(PrintStream out, Command instance) {
-    out.println(usagePrefix + " " + instance.getUsage());
+    out.println(getUsagePrefix() + " " + instance.getUsage());
   }
 
   private void printInstanceHelp(PrintStream out, Command instance) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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 9fdac98..b9fc6fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -79,4 +79,6 @@
     operation fails. (Rakesh R via Zhe Zhang)
 
     HDFS-8123. Erasure Coding: Better to move EC related proto messages to a
-    separate erasurecoding proto file (Rakesh R via vinayakumarb)
\ No newline at end of file
+    separate erasurecoding proto file (Rakesh R via vinayakumarb)
+
+    HDFS-7349. Support DFS command for the EC encoding (vinayakumarb)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index f464261..84c79b8 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -134,6 +134,11 @@ case ${COMMAND} in
     hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS"
     HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
   ;;
+  erasurecode)
+    CLASS=org.apache.hadoop.hdfs.tools.erasurecode.ECCli
+    hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS"
+    HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
+  ;;
   fetchdt)
     CLASS=org.apache.hadoop.hdfs.tools.DelegationTokenFetcher
   ;;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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 aadb4f6..b1dab46 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
@@ -119,6 +119,7 @@ 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.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -3333,4 +3334,21 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     return scope;
   }
+
+  /**
+   * Get the erasure coding zone 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
+   * @throws IOException
+   */
+  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+    checkOpen();
+    try {
+      return namenode.getErasureCodingZoneInfo(src);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(FileNotFoundException.class,
+          AccessControlException.class, UnresolvedPathException.class);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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 4c3e0a5..4c8fff3 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
@@ -72,6 +72,7 @@ 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.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -2297,4 +2298,35 @@ public class DistributedFileSystem extends FileSystem {
       }
     }.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
+   * @throws IOException
+   */
+  public ECZoneInfo getErasureCodingZoneInfo(final Path path)
+      throws IOException {
+    Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<ECZoneInfo>() {
+      @Override
+      public ECZoneInfo doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        return dfs.getErasureCodingZoneInfo(getPathName(p));
+      }
+
+      @Override
+      public ECZoneInfo next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          return myDfs.getErasureCodingZoneInfo(p);
+        }
+        throw new UnsupportedOperationException(
+            "Cannot getErasureCodingZoneInfo through a symlink to a "
+                + "non-DistributedFileSystem: " + path + " -> " + p);
+      }
+    }.resolve(this, absF);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index d485311..bba7697 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1484,4 +1484,13 @@ public interface ClientProtocol {
    */
   @Idempotent
   public ECSchema[] getECSchemas() throws IOException;
+
+  /**
+   * Get the information about the EC zone for the path
+   * 
+   * @param src path to get the info for
+   * @throws IOException
+   */
+  @Idempotent
+  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java
new file mode 100644
index 0000000..ecfb92e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECZoneInfo.java
@@ -0,0 +1,56 @@
+/**
+ * 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 org.apache.hadoop.io.erasurecode.ECSchema;
+
+/**
+ * Information about the EC Zone at the specified path.
+ */
+public class ECZoneInfo {
+
+  private String dir;
+  private ECSchema schema;
+
+  public ECZoneInfo(String dir, ECSchema schema) {
+    this.dir = dir;
+    this.schema = schema;
+  }
+
+  /**
+   * Get directory of the EC zone.
+   * 
+   * @return
+   */
+  public String getDir() {
+    return dir;
+  }
+
+  /**
+   * Get the schema for the EC Zone
+   * 
+   * @return
+   */
+  public ECSchema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public String toString() {
+    return "Dir: " + getDir() + ", Schema: " + schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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 17141c3..f9bdb71 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
@@ -36,6 +36,7 @@ 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.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -201,6 +202,8 @@ 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.GetECSchemasRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
@@ -1551,4 +1554,19 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetECZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller,
+      GetECZoneInfoRequestProto request) throws ServiceException {
+    try {
+      ECZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc());
+      GetECZoneInfoResponseProto.Builder builder = GetECZoneInfoResponseProto.newBuilder();
+      if (ecZoneInfo != null) {
+        builder.setECZoneInfo(PBHelper.convertECZoneInfo(ecZoneInfo));
+      }
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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 44d1258..6c1c971 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
@@ -59,6 +59,7 @@ 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.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -167,6 +168,8 @@ 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.GetECSchemasRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
@@ -1576,4 +1579,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+    GetECZoneInfoRequestProto req = GetECZoneInfoRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      GetECZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo(
+          null, req);
+      if (response.hasECZoneInfo()) {
+        return PBHelper.convertECZoneInfo(response.getECZoneInfo());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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 10afc73..9ca73ae 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
@@ -76,6 +76,7 @@ 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.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -134,6 +135,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportC
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECZoneInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -3138,4 +3140,14 @@ public class PBHelper {
     }
     return builder.build();
   }
+
+  public static ECZoneInfoProto convertECZoneInfo(ECZoneInfo ecZoneInfo) {
+    return ECZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir())
+        .setSchema(convertECSchema(ecZoneInfo.getSchema())).build();
+  }
+
+  public static ECZoneInfo convertECZoneInfo(ECZoneInfoProto ecZoneInfoProto) {
+    return new ECZoneInfo(ecZoneInfoProto.getDir(),
+        convertECSchema(ecZoneInfoProto.getSchema()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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
index 5320c1c..0a84083 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
@@ -22,6 +22,7 @@ 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.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.io.erasurecode.ECSchema;
@@ -57,7 +58,12 @@ public class ErasureCodingZoneManager {
     return getECSchema(iip) != null;
   }
 
-  ECSchema getECSchema(INodesInPath iip) throws IOException{
+  ECSchema getECSchema(INodesInPath iip) throws IOException {
+    ECZoneInfo ecZoneInfo = getECZoneInfo(iip);
+    return ecZoneInfo == null ? null : ecZoneInfo.getSchema();
+  }
+
+  ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     List<INode> inodes = iip.getReadOnlyINodes();
@@ -80,7 +86,8 @@ public class ErasureCodingZoneManager {
         if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
           ECSchemaProto ecSchemaProto;
           ecSchemaProto = ECSchemaProto.parseFrom(xAttr.getValue());
-          return PBHelper.convertECSchema(ecSchemaProto);
+          ECSchema schema = PBHelper.convertECSchema(ecSchemaProto);
+          return new ECZoneInfo(inode.getFullPathName(), schema);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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 6349dc4..46852d6 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
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
@@ -1415,6 +1416,15 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
+    readLock();
+    try {
+      return ecZoneManager.getECZoneInfo(iip);
+    } finally {
+      readUnlock();
+    }
+  }
+
   static INode resolveLastINode(INodesInPath iip) throws FileNotFoundException {
     INode inode = iip.getLastINode();
     if (inode == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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 9dee676..f176128 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
@@ -182,6 +182,7 @@ 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.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -8185,6 +8186,29 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
+   * Get the erasure coding zone information for specified path
+   */
+  ECZoneInfo getErasureCodingZoneInfo(String src) throws AccessControlException,
+      UnresolvedLinkException, IOException {
+    checkOperation(OperationCategory.READ);
+    final byte[][] pathComponents = FSDirectory
+        .getPathComponentsForReservedPath(src);
+    final FSPermissionChecker pc = getPermissionChecker();
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      src = dir.resolvePath(pc, src, pathComponents);
+      final INodesInPath iip = dir.getINodesInPath(src, true);
+      if (isPermissionEnabled) {
+        dir.checkPathAccess(pc, iip, FsAction.READ);
+      }
+      return dir.getECZoneInfo(iip);
+    } finally {
+      readUnlock();
+    }
+  }
+
+  /**
    * Get available ECSchemas
    */
   ECSchema[] getECSchemas() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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 0eb8982..023f863 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,6 +85,7 @@ 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.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
@@ -2065,4 +2066,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
     checkNNStartup();
     return namesystem.getECSchemas();
   }
+
+  @Override // ClientProtocol
+  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+    checkNNStartup();
+    return namesystem.getErasureCodingZoneInfo(src);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
new file mode 100644
index 0000000..4ed9d0a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.tools.erasurecode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.shell.CommandFactory;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * CLI for the erasure code encoding operations.
+ */
+public class ECCli extends FsShell {
+
+  private final static String usagePrefix =
+      "Usage: hdfs erasurecode [generic options]";
+
+  @Override
+  protected String getUsagePrefix() {
+    return usagePrefix;
+  }
+
+  @Override
+  protected void registerCommands(CommandFactory factory) {
+    factory.registerCommands(ECCommand.class);
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    int res = ToolRunner.run(conf, new ECCli(), args);
+    System.exit(res);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/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
new file mode 100644
index 0000000..84c2275
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
@@ -0,0 +1,209 @@
+/**
+ * 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.tools.erasurecode;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+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.ECZoneInfo;
+import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Erasure Coding CLI commands
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+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(GetECZoneInfoCommand.class, "-"
+        + GetECZoneInfoCommand.NAME);
+    factory.addClass(ListECSchemas.class, "-" + ListECSchemas.NAME);
+  }
+
+  @Override
+  public String getCommandName() {
+    return getName();
+  }
+
+  @Override
+  protected void run(Path path) throws IOException {
+    throw new RuntimeException("Not suppose to get here");
+  }
+
+  @Deprecated
+  @Override
+  public int runAll() {
+    return run(args);
+  }
+
+  @Override
+  protected void processPath(PathData item) throws IOException {
+    if (!(item.fs instanceof DistributedFileSystem)) {
+      throw new UnsupportedActionException(
+          "Erasure commands are only supported for the HDFS paths");
+    }
+  }
+
+  /**
+   * Create EC encoding zone command. Zones are created to use specific EC
+   * encoding schema, other than default while encoding the files under some
+   * specific directory.
+   */
+  static class CreateECZoneCommand extends ECCommand {
+    public static final String NAME = "createZone";
+    public static final String USAGE = "[-s <schemaName>] <path>";
+    public static final String DESCRIPTION = 
+        "Create a zone to encode files using a specified schema\n"
+        + "Options :\n"
+        + "  -s <schemaName> : EC schema name to encode files. "
+        + "If not passed default schema will be used\n"
+        + "  <path>  : Path to an empty directory. Under this directory "
+        + "files will be encoded using specified schema";
+    private String schemaName;
+    private ECSchema schema = null;
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      schemaName = StringUtils.popOptionWithArgument("-s", args);
+      if (args.isEmpty()) {
+        throw new HadoopIllegalArgumentException("<path> is missing");
+      }
+      if (args.size() > 1) {
+        throw new HadoopIllegalArgumentException("Too many arguments");
+      }
+    }
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      super.processPath(item);
+      DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
+      try {
+        if (schemaName != null) {
+          ECSchema[] ecSchemas = dfs.getClient().getECSchemas();
+          for (ECSchema ecSchema : ecSchemas) {
+            if (schemaName.equals(ecSchema.getSchemaName())) {
+              schema = ecSchema;
+              break;
+            }
+          }
+          if (schema == null) {
+            StringBuilder sb = new StringBuilder();
+            sb.append("Schema '");
+            sb.append(schemaName);
+            sb.append("' does not match any of the supported schemas.");
+            sb.append("Please select any one of [");
+            for (ECSchema ecSchema : ecSchemas) {
+              sb.append(ecSchema.getSchemaName());
+              sb.append(", ");
+            }
+            throw new HadoopIllegalArgumentException(sb.toString());
+          }
+        }
+        dfs.createErasureCodingZone(item.path, schema);
+        out.println("EC Zone created successfully at " + item.path);
+      } catch (IOException e) {
+        throw new IOException("Unable to create EC zone for the path "
+            + item.path, e);
+      }
+    }
+  }
+
+  /**
+   * Get the information about the zone
+   */
+  static class GetECZoneInfoCommand extends ECCommand {
+    public static final String NAME = "getZoneInfo";
+    public static final String USAGE = "<path>";
+    public static final String DESCRIPTION =
+        "Get information about the EC zone at specified path\n";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      if (args.isEmpty()) {
+        throw new HadoopIllegalArgumentException("<path> is missing");
+      }
+      if (args.size() > 1) {
+        throw new HadoopIllegalArgumentException("Too many arguments");
+      }
+    }
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      super.processPath(item);
+      DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
+      try {
+        ECZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path);
+        out.println(ecZoneInfo.toString());
+      } catch (IOException e) {
+        throw new IOException("Unable to create EC zone for the path "
+            + item.path, e);
+      }
+    }
+  }
+
+  /**
+   * List all supported EC Schemas
+   */
+  static class ListECSchemas extends ECCommand {
+    public static final String NAME = "listSchemas";
+    public static final String USAGE = "";
+    public static final String DESCRIPTION = 
+        "Get the list of ECSchemas supported\n";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      if (!args.isEmpty()) {
+        throw new HadoopIllegalArgumentException("Too many parameters");
+      }
+
+      FileSystem fs = FileSystem.get(getConf());
+      if (fs instanceof DistributedFileSystem == false) {
+        throw new UnsupportedActionException(
+            "Erasure commands are only supported for the HDFS");
+      }
+      DistributedFileSystem dfs = (DistributedFileSystem) fs;
+
+      ECSchema[] ecSchemas = dfs.getClient().getECSchemas();
+      StringBuilder sb = new StringBuilder();
+      int i = 0;
+      while (i < ecSchemas.length) {
+        ECSchema ecSchema = ecSchemas[i];
+        sb.append(ecSchema.getSchemaName());
+        i++;
+        if (i < ecSchemas.length) {
+          sb.append(", ");
+        }
+      }
+      out.println(sb.toString());
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
index 89c38e9..046120d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -867,4 +867,6 @@ service ClientNamenodeProtocol {
       returns(GetErasureCodingInfoResponseProto);
   rpc getECSchemas(GetECSchemasRequestProto)
       returns(GetECSchemasResponseProto);
+  rpc getErasureCodingZoneInfo(GetECZoneInfoRequestProto) 
+      returns(GetECZoneInfoResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df7bef/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
index 4d5731b..d888f71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -49,6 +49,13 @@ message ECInfoProto {
  required ECSchemaProto schema = 2;
 }
 
+/**
+ * ECZoneInfo
+ */
+message ECZoneInfoProto {
+  required string dir = 1;
+  required ECSchemaProto schema = 2;
+}
 
 message CreateErasureCodingZoneRequestProto {
   required string src = 1;
@@ -72,3 +79,11 @@ message GetECSchemasRequestProto { // void request
 message GetECSchemasResponseProto {
   repeated ECSchemaProto schemas = 1;
 }
+
+message GetECZoneInfoRequestProto {
+  required string src = 1; // path to get the zone info
+}
+
+message GetECZoneInfoResponseProto {
+  optional ECZoneInfoProto ECZoneInfo = 1;
+}
\ No newline at end of file


[48/50] hadoop git commit: HDFS-8203. Erasure Coding: Seek and other Ops in DFSStripedInputStream. Contributed by Yi Liu.

Posted by zh...@apache.org.
HDFS-8203. Erasure Coding: Seek and other Ops in DFSStripedInputStream. Contributed by Yi Liu.


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

Branch: refs/heads/HDFS-7285
Commit: de30e663231de504303e4b2ba4aef32ee06b753b
Parents: 49d0ac8
Author: Jing Zhao <ji...@apache.org>
Authored: Thu May 7 11:06:40 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:40:59 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +
 .../hadoop/hdfs/DFSStripedInputStream.java      | 88 +++++++++++++++++---
 .../hadoop/hdfs/TestWriteReadStripedFile.java   | 83 +++++++++++++++---
 3 files changed, 151 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/de30e663/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 11e8376..fed08e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -186,3 +186,6 @@
 
     HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding
     (umamahesh)
+
+    HDFS-8203. Erasure Coding: Seek and other Ops in DFSStripedInputStream.
+    (Yi Liu via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de30e663/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 7cb7b6d..9011192 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
@@ -19,10 +19,13 @@ 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.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.*;
 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.ReadPortion;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.planReadPortions;
 
@@ -31,9 +34,11 @@ import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
+import java.io.EOFException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
 import java.util.Set;
 import java.util.Map;
 import java.util.HashMap;
@@ -263,6 +268,10 @@ public class DFSStripedInputStream extends DFSInputStream {
   }
 
   private long getOffsetInBlockGroup() {
+    return getOffsetInBlockGroup(pos);
+  }
+
+  private long getOffsetInBlockGroup(long pos) {
     return pos - currentLocatedBlock.getStartOffset();
   }
 
@@ -278,18 +287,22 @@ public class DFSStripedInputStream extends DFSInputStream {
     // compute stripe range based on pos
     final long offsetInBlockGroup = getOffsetInBlockGroup();
     final long stripeLen = cellSize * dataBlkNum;
-    int stripeIndex = (int) (offsetInBlockGroup / stripeLen);
-    curStripeRange = new StripeRange(stripeIndex * stripeLen,
-        Math.min(currentLocatedBlock.getBlockSize() - (stripeIndex * stripeLen),
-            stripeLen));
-    final int numCell = (int) ((curStripeRange.length - 1) / cellSize + 1);
+    final int stripeIndex = (int) (offsetInBlockGroup / stripeLen);
+    final int stripeBufOffset = (int) (offsetInBlockGroup % stripeLen);
+    final int stripeLimit = (int) Math.min(currentLocatedBlock.getBlockSize()
+        - (stripeIndex * stripeLen), stripeLen);
+    curStripeRange = new StripeRange(offsetInBlockGroup,
+        stripeLimit - stripeBufOffset);
+
+    final int startCell = stripeBufOffset / cellSize;
+    final int numCell = (stripeLimit - 1) / cellSize + 1;
 
     // read the whole stripe in parallel
     Map<Future<Integer>, Integer> futures = new HashMap<>();
-    for (int i = 0; i < numCell; i++) {
-      curStripeBuf.position(cellSize * i);
-      curStripeBuf.limit((int) Math.min(cellSize * (i + 1),
-          curStripeRange.length));
+    for (int i = startCell; i < numCell; i++) {
+      int bufPos = i == startCell ? stripeBufOffset : cellSize * i;
+      curStripeBuf.position(bufPos);
+      curStripeBuf.limit(Math.min(cellSize * (i + 1), stripeLimit));
       ByteBuffer buf = curStripeBuf.slice();
       ByteBufferStrategy strategy = new ByteBufferStrategy(buf);
       final int targetLength = buf.remaining();
@@ -329,6 +342,39 @@ public class DFSStripedInputStream extends DFSInputStream {
     };
   }
 
+  /**
+   * 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
   protected synchronized int readWithStrategy(ReaderStrategy strategy,
       int off, int len) throws IOException {
@@ -405,10 +451,8 @@ public class DFSStripedInputStream extends DFSInputStream {
    * @return number of bytes copied
    */
   private int copy(ReaderStrategy strategy, int offset, int length) {
-    final long stripeLen = cellSize * dataBlkNum;
-    final long offsetInBlk = pos - currentLocatedBlock.getStartOffset();
-    // compute the position in the curStripeBuf based on "pos"
-    int bufOffset = (int) (offsetInBlk % stripeLen);
+    final long offsetInBlk = getOffsetInBlockGroup();
+    int bufOffset = getStripedBufOffset(offsetInBlk);
     curStripeBuf.position(bufOffset);
     return strategy.copyFrom(curStripeBuf, offset,
         Math.min(length, curStripeBuf.remaining()));
@@ -546,4 +590,22 @@ public class DFSStripedInputStream extends DFSInputStream {
     }
     throw new InterruptedException("let's retry");
   }
+
+  /**
+   * 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.");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de30e663/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 eacc6ed..5c6f449 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
@@ -22,12 +22,12 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
@@ -150,11 +150,35 @@ public class TestWriteReadStripedFile {
     return bytes;
   }
 
+  private int readAll(FSDataInputStream in, byte[] buf) throws IOException {
+    int readLen = 0;
+    int ret;
+    do {
+      ret = in.read(buf, readLen, buf.length - readLen);
+      if (ret > 0) {
+        readLen += ret;
+      }
+    } while (ret >= 0 && readLen < buf.length);
+    return readLen;
+  }
+
   private byte getByte(long pos) {
     final int mod = 29;
     return (byte) (pos % mod + 1);
   }
 
+  private void assertSeekAndRead(FSDataInputStream fsdis, int pos,
+      int writeBytes) throws IOException {
+    fsdis.seek(pos);
+    byte[] buf = new byte[writeBytes];
+    int readLen = readAll(fsdis, buf);
+    Assert.assertEquals(readLen, writeBytes - pos);
+    for (int i = 0; i < readLen; i++) {
+      Assert.assertEquals("Byte at " + i + " should be the same",
+          getByte(pos + i), buf[i]);
+    }
+  }
+
   private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes)
       throws IOException {
     Path testPath = new Path(src);
@@ -183,15 +207,7 @@ public class TestWriteReadStripedFile {
     // stateful read with byte array
     try (FSDataInputStream fsdis = fs.open(new Path(src))) {
       byte[] buf = new byte[writeBytes + 100];
-      int readLen = 0;
-      int ret;
-      do {
-        ret = fsdis.read(buf, readLen, buf.length - readLen);
-        if (ret > 0) {
-          readLen += ret;
-        }
-      } while (ret >= 0);
-      readLen = readLen >= 0 ? readLen : 0;
+      int readLen = readAll(fsdis, buf);
       Assert.assertEquals("The length of file should be the same to write size",
           writeBytes, readLen);
       for (int i = 0; i < writeBytes; i++) {
@@ -200,6 +216,53 @@ public class TestWriteReadStripedFile {
       }
     }
 
+    // seek and stateful read
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
+      // seek to 1/2 of content
+      int pos = writeBytes/2;
+      assertSeekAndRead(fsdis, pos, writeBytes);
+
+      // seek to 1/3 of content
+      pos = writeBytes/3;
+      assertSeekAndRead(fsdis, pos, writeBytes);
+
+      // seek to 0 pos
+      pos = 0;
+      assertSeekAndRead(fsdis, pos, writeBytes);
+
+      if (writeBytes > cellSize) {
+        // seek to cellSize boundary
+        pos = cellSize -1;
+        assertSeekAndRead(fsdis, pos, writeBytes);
+      }
+
+      if (writeBytes > cellSize * dataBlocks) {
+        // seek to striped cell group boundary
+        pos = cellSize * dataBlocks - 1;
+        assertSeekAndRead(fsdis, pos, writeBytes);
+      }
+
+      if (writeBytes > blockSize * dataBlocks) {
+        // seek to striped block group boundary
+        pos = blockSize * dataBlocks - 1;
+        assertSeekAndRead(fsdis, pos, writeBytes);
+      }
+
+      try {
+        fsdis.seek(-1);
+        Assert.fail("Should be failed if seek to negative offset");
+      } catch (EOFException e) {
+        // expected
+      }
+
+      try {
+        fsdis.seek(writeBytes + 1);
+        Assert.fail("Should be failed if seek after EOF");
+      } catch (EOFException e) {
+        // expected
+      }
+    }
+
     // stateful read with ByteBuffer
     try (FSDataInputStream fsdis = fs.open(new Path(src))) {
       ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100);


[37/50] hadoop git commit: HDFS-8242. Erasure Coding: XML based end-to-end test for ECCli commands (Contributed by Rakesh R)

Posted by zh...@apache.org.
HDFS-8242. Erasure Coding: XML based end-to-end test for ECCli commands (Contributed by Rakesh R)


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

Branch: refs/heads/HDFS-7285
Commit: a7fac8b6447b9f256206dc9bbc39b3a7e0c93025
Parents: 3809261
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue May 5 11:54:30 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:24 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../hdfs/tools/erasurecode/ECCommand.java       |   9 +-
 .../hadoop/cli/CLITestCmdErasureCoding.java     |  38 +++
 .../apache/hadoop/cli/TestErasureCodingCLI.java | 114 +++++++
 .../cli/util/CLICommandErasureCodingCli.java    |  21 ++
 .../cli/util/ErasureCodingCliCmdExecutor.java   |  37 ++
 .../test/resources/testErasureCodingConf.xml    | 342 +++++++++++++++++++
 7 files changed, 561 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7fac8b6/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 faec023..ef760fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -166,3 +166,6 @@
     (jing9)
 
     HDFS-8137. Send the EC schema to DataNode via EC encoding/recovering command(umamahesh)
+
+    HDFS-8242. Erasure Coding: XML based end-to-end test for ECCli commands
+    (Rakesh R via vinayakumarb)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7fac8b6/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 84c2275..802a46d 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
@@ -17,7 +17,9 @@
 package org.apache.hadoop.hdfs.tools.erasurecode;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.LinkedList;
+import java.util.List;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -120,11 +122,12 @@ public abstract class ECCommand extends Command {
             sb.append("Schema '");
             sb.append(schemaName);
             sb.append("' does not match any of the supported schemas.");
-            sb.append("Please select any one of [");
+            sb.append(" Please select any one of ");
+            List<String> schemaNames = new ArrayList<String>();
             for (ECSchema ecSchema : ecSchemas) {
-              sb.append(ecSchema.getSchemaName());
-              sb.append(", ");
+              schemaNames.add(ecSchema.getSchemaName());
             }
+            sb.append(schemaNames);
             throw new HadoopIllegalArgumentException(sb.toString());
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7fac8b6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
----------------------------------------------------------------------
diff --git 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
new file mode 100644
index 0000000..6c06a8d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.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 {
+    if (getType() instanceof CLICommandErasureCodingCli)
+      return new ErasureCodingCliCmdExecutor(tag, new ECCli());
+    return super.getExecutor(tag);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7fac8b6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
----------------------------------------------------------------------
diff --git 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
new file mode 100644
index 0000000..5f01ea2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.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());
+  }
+
+  @Test
+  @Override
+  public void testAll() {
+    super.testAll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7fac8b6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java
new file mode 100644
index 0000000..aafcd9f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java
@@ -0,0 +1,21 @@
+/**
+ * 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.util;
+
+public class CLICommandErasureCodingCli implements CLICommandTypes {
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7fac8b6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java
new file mode 100644
index 0000000..e993313
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.util;
+
+import org.apache.hadoop.hdfs.tools.erasurecode.ECCli;
+import org.apache.hadoop.util.ToolRunner;
+
+public class ErasureCodingCliCmdExecutor extends CommandExecutor {
+  protected String namenode = null;
+  protected ECCli admin = null;
+
+  public ErasureCodingCliCmdExecutor(String namenode, ECCli admin) {
+    this.namenode = namenode;
+    this.admin = admin;
+  }
+
+  @Override
+  protected void execute(final String cmd) throws Exception {
+    String[] args = getCommandAsArgs(cmd, "NAMENODE", this.namenode);
+    ToolRunner.run(admin, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7fac8b6/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
new file mode 100644
index 0000000..b7b29d3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -0,0 +1,342 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="testConf.xsl"?>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<configuration>
+  <!-- Normal mode is test. To run just the commands and dump the output
+       to the log, set it to nocompare -->
+  <mode>test</mode>
+
+  <!--  Comparator types:
+           ExactComparator
+           SubstringComparator
+           RegexpComparator
+           TokenComparator
+           -->
+  <tests>
+
+  <!-- Test help options -->
+    <test>
+      <description>help: help for erasure coding command</description>
+      <test-commands>
+        <ec-admin-command>-help</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Usage: hdfs erasurecode [generic options]</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>help: createZone command</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -help createZone</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 schema( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-createZone \[-s &lt;schemaName&gt;\] &lt;path&gt;(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>help: getZoneInfo command</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -help getZoneInfo</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>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-getZoneInfo &lt;path&gt;(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>help: listSchemas command</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -help listSchemas</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Get the list of ECSchemas supported</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-listSchemas (.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+  <!-- Test erasure code commands -->
+    <test>
+      <description>createZone : create a zone to encode files</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /eczone</command>
+        <ec-admin-command>-fs NAMENODE -createZone -s RS-6-3 /eczone</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /eczone</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>EC Zone created successfully at NAMENODE/eczone</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>createZone : default schema</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /eczone</command>
+        <ec-admin-command>-fs NAMENODE -createZone /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /eczone</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Dir: /eczone, Schema: ECSchema=[Name=RS-6-3</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getZoneInfo : get information about the EC zone at specified path</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /eczone</command>
+        <ec-admin-command>-fs NAMENODE -createZone -s RS-6-3 /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /eczone</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Dir: /eczone, Schema: ECSchema=[Name=RS-6-3</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getZoneInfo : get EC zone at specified file path</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /eczone</command>
+        <ec-admin-command>-fs NAMENODE -createZone -s RS-6-3 /eczone</ec-admin-command>
+        <command>-fs NAMENODE -touchz /eczone/ecfile</command>
+        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone/ecfile</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /eczone/ecfile</command>
+        <command>-fs NAMENODE -rmdir /eczone</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Dir: /eczone, Schema: ECSchema=[Name=RS-6-3</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>listSchemas : get the list of ECSchemas supported</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -listSchemas</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>RS-6-3</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+<!-- Test illegal parameters -->
+    <test>
+      <description>createZone : illegal parameters - path is missing</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /eczone</command>
+        <ec-admin-command>-fs NAMENODE -createZone</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /eczone</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-createZone: &lt;path&gt; is missing(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>createZone : illegal parameters - schema name is missing</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /eczone</command>
+        <ec-admin-command>-fs NAMENODE -createZone -s</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /eczone</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-createZone: option -s requires 1 argument(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>createZone : 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>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /eczone</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-createZone: Too many arguments</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>createZone : illegal parameters - invalidschema</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /eczone</command>
+        <ec-admin-command>-fs NAMENODE -createZone -s invalidschema /eczone</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /eczone</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Schema 'invalidschema' does not match any of the supported schemas. Please select any one of [RS-6-3]</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>createZone : illegal parameters - no such file</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -createZone /eczone</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>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getZoneInfo : illegal parameters - path is missing</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -getZoneInfo </ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-getZoneInfo: &lt;path&gt; is missing(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getZoneInfo : illegal parameters - too many arguments</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone /eczone</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /eczone</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-getZoneInfo: Too many arguments</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getZoneInfo : illegal parameters - no such file</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^getZoneInfo: `/eczone': No such file or directory(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>listSchemas : illegal parameters - too many parameters</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -listSchemas /eczone</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-listSchemas: Too many parameters</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+  </tests>
+</configuration>


[18/50] hadoop git commit: HDFS-8024. Erasure Coding: ECworker frame, basics, bootstraping and configuration. (Contributed by Uma Maheswara Rao G)

Posted by zh...@apache.org.
HDFS-8024. Erasure Coding: ECworker frame, basics, bootstraping and configuration. (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/eba862a6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/eba862a6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/eba862a6

Branch: refs/heads/HDFS-7285
Commit: eba862a6d5f44075f3e4f98364a4b83b8eac0b89
Parents: fa8df98
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Wed Apr 22 19:30:14 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:18 2015 -0700

----------------------------------------------------------------------
 .../erasurecode/coder/AbstractErasureCoder.java |  2 +-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  7 ++
 .../hdfs/server/datanode/BPOfferService.java    |  6 ++
 .../hadoop/hdfs/server/datanode/DataNode.java   | 10 +++
 .../erasurecode/ErasureCodingWorker.java        | 83 ++++++++++++++++++++
 .../src/main/proto/DatanodeProtocol.proto       |  2 +
 7 files changed, 112 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eba862a6/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
index e5bf11a..7403e35 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -66,7 +66,7 @@ public abstract class AbstractErasureCoder
    * @param isEncoder
    * @return raw coder
    */
-  protected static RawErasureCoder createRawCoder(Configuration conf,
+  public static RawErasureCoder createRawCoder(Configuration conf,
       String rawCoderFactoryKey, boolean isEncoder) {
 
     if (conf == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eba862a6/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 3d86f05..1acde41 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -113,3 +113,6 @@
     
     HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema
     in FileSystemLinkResolver. (szetszwo via Zhe Zhang)
+
+    HDFS-8024. Erasure Coding: ECworker frame, basics, bootstraping and configuration.
+    (umamahesh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eba862a6/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 c127b5f..68cfe7f 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
@@ -973,6 +973,8 @@ public class PBHelper {
       return REG_CMD;
     case BlockIdCommand:
       return PBHelper.convert(proto.getBlkIdCmd());
+    case BlockECRecoveryCommand:
+      return PBHelper.convert(proto.getBlkECRecoveryCmd());
     default:
       return null;
     }
@@ -1123,6 +1125,11 @@ public class PBHelper {
       builder.setCmdType(DatanodeCommandProto.Type.BlockIdCommand).
         setBlkIdCmd(PBHelper.convert((BlockIdCommand) datanodeCommand));
       break;
+    case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY:
+      builder.setCmdType(DatanodeCommandProto.Type.BlockECRecoveryCommand)
+          .setBlkECRecoveryCmd(
+              convert((BlockECRecoveryCommand) datanodeCommand));
+      break;
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     default:
       builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eba862a6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index 92323f1..69baac7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -32,11 +32,13 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.protocol.*;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
@@ -722,6 +724,10 @@ class BPOfferService {
         dxcs.balanceThrottler.setBandwidth(bandwidth);
       }
       break;
+    case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY:
+      LOG.info("DatanodeCommand action: DNA_ERASURE_CODING_RECOVERY");
+      Collection<BlockECRecoveryInfo> ecTasks = ((BlockECRecoveryCommand) cmd).getECTasks();
+      dn.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
     default:
       LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eba862a6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index d2b2939..221ba38 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -87,6 +87,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.management.ObjectName;
 
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -152,6 +153,7 @@ import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
+import org.apache.hadoop.hdfs.server.datanode.erasurecode.ErasureCodingWorker;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
@@ -354,6 +356,8 @@ public class DataNode extends ReconfigurableBase
   private String dnUserName = null;
 
   private SpanReceiverHost spanReceiverHost;
+
+  private ErasureCodingWorker ecWorker;
   private static final int NUM_CORES = Runtime.getRuntime()
       .availableProcessors();
   private static final double CONGESTION_RATIO = 1.5;
@@ -1157,6 +1161,7 @@ public class DataNode extends ReconfigurableBase
     saslClient = new SaslDataTransferClient(dnConf.conf, 
         dnConf.saslPropsResolver, dnConf.trustedChannelResolver);
     saslServer = new SaslDataTransferServer(dnConf, blockPoolTokenSecretManager);
+    ecWorker = new ErasureCodingWorker(conf); // Initialize ErasureCoding worker
   }
 
   /**
@@ -3258,4 +3263,9 @@ public class DataNode extends ReconfigurableBase
     checkSuperuserPrivilege();
     spanReceiverHost.removeSpanReceiver(id);
   }
+  
+  public ErasureCodingWorker getErasureCodingWorker(){
+    return ecWorker;
+    
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eba862a6/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
new file mode 100644
index 0000000..6430308
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@ -0,0 +1,83 @@
+/**
+ * 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.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
+import org.apache.hadoop.io.erasurecode.coder.AbstractErasureCoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
+
+/**
+ * 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 Configuration conf;
+  RawErasureCoder rawEncoder = null;
+  RawErasureCoder rawDecoder = null;
+
+  public ErasureCodingWorker(Configuration conf) {
+    this.conf = conf;
+    initialize();
+  }
+
+  /**
+   * Initializes the required resources for handling the erasure coding recovery
+   * work.
+   */
+  public void initialize() {
+    // Right now directly used RS coder. Once other coders integration ready, we
+    // can load preferred codec here.
+    initializeErasureEncoder();
+    initializeErasureDecoder();
+  }
+
+  private void initializeErasureDecoder() {
+    rawDecoder = AbstractErasureCoder.createRawCoder(conf,
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, false);
+    if (rawDecoder == null) {
+      rawDecoder = new RSRawDecoder();
+    }
+  }
+
+  private void initializeErasureEncoder() {
+    rawEncoder = AbstractErasureCoder.createRawCoder(conf,
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, true);
+    if (rawEncoder == null) {
+      rawEncoder = new RSRawEncoder();
+    }
+  }
+
+  /**
+   * Handles the Erasure Coding recovery work commands.
+   * 
+   * @param ecTasks
+   *          BlockECRecoveryInfo
+   */
+  public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
+    // HDFS-7348 : Implement the actual recovery process
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eba862a6/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index ac9ab46..482e2e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -59,6 +59,7 @@ message DatanodeCommandProto {
     UnusedUpgradeCommand = 6;
     NullDatanodeCommand = 7;
     BlockIdCommand = 8;
+    BlockECRecoveryCommand = 9;
   }
 
   required Type cmdType = 1;    // Type of the command
@@ -72,6 +73,7 @@ message DatanodeCommandProto {
   optional KeyUpdateCommandProto keyUpdateCmd = 6;
   optional RegisterCommandProto registerCmd = 7;
   optional BlockIdCommandProto blkIdCmd = 8;
+  optional BlockECRecoveryCommandProto blkECRecoveryCmd = 9;
 }
 
 /**


[15/50] hadoop git commit: HDFS-8190. StripedBlockUtil.getInternalBlockLength may have overflow error.

Posted by zh...@apache.org.
HDFS-8190. StripedBlockUtil.getInternalBlockLength may have overflow error.


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

Branch: refs/heads/HDFS-7285
Commit: b59a0ba7090acb11c4448ddd2bce43f3c1a0a2a4
Parents: 166e565
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Apr 20 17:42:02 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:17 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  61 ++++---
 .../hadoop/hdfs/TestDFSStripedOutputStream.java | 178 +++++++------------
 3 files changed, 100 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b59a0ba7/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 c8dbf08..8f28285 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -104,3 +104,6 @@
 
     HDFS-8181. createErasureCodingZone sets retryCache state as false always
     (Uma Maheswara Rao G via vinayakumarb)
+
+    HDFS-8190. StripedBlockUtil.getInternalBlockLength may have overflow error.
+    (szetszwo)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b59a0ba7/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 2368021..d622d4d 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
@@ -25,6 +25,8 @@ 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;
+
 /**
  * Utility class for analyzing striped block groups
  */
@@ -81,46 +83,43 @@ public class StripedBlockUtil {
   /**
    * Get the size of an internal block at the given index of a block group
    *
-   * @param numBytesInGroup Size of the block group only counting data blocks
+   * @param dataSize Size of the block group only counting data blocks
    * @param cellSize The size of a striping cell
-   * @param dataBlkNum The number of data blocks
-   * @param idxInGroup The logical index in the striped block group
+   * @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 numBytesInGroup,
-      int cellSize, int dataBlkNum, int idxInGroup) {
+  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 long numBytesPerStripe = cellSize * dataBlkNum;
-    assert numBytesPerStripe  > 0:
-        "getInternalBlockLength should only be called on valid striped blocks";
+    final int stripeSize = cellSize * numDataBlocks;
     // If block group ends at stripe boundary, each internal block has an equal
     // share of the group
-    if (numBytesInGroup % numBytesPerStripe == 0) {
-      return numBytesInGroup / dataBlkNum;
+    final int lastStripeDataLen = (int)(dataSize % stripeSize);
+    if (lastStripeDataLen == 0) {
+      return dataSize / numDataBlocks;
     }
 
-    int numStripes = (int) ((numBytesInGroup - 1) / numBytesPerStripe + 1);
-    assert numStripes >= 1 : "There should be at least 1 stripe";
-
-    // All stripes but the last one are full stripes. The block should at least
-    // contain (numStripes - 1) full cells.
-    long blkSize = (numStripes - 1) * cellSize;
-
-    long lastStripeLen = numBytesInGroup % numBytesPerStripe;
-    // Size of parity cells should equal the size of the first cell, if it
-    // is not full.
-    long lastParityCellLen = Math.min(cellSize, lastStripeLen);
-
-    if (idxInGroup >= dataBlkNum) {
-      // for parity blocks
-      blkSize += lastParityCellLen;
-    } else {
-      // for data blocks
-      blkSize +=  Math.min(cellSize,
-          Math.max(0, lastStripeLen - cellSize * idxInGroup));
+    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 blkSize;
+    return size > cellSize? cellSize: size;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b59a0ba7/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 cc20f40..160b190 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
@@ -1,7 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hdfs;
 
-import java.nio.ByteBuffer;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+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;
@@ -17,7 +41,6 @@ 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.server.common.HdfsServerConstants;
-
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.IOUtils;
@@ -30,13 +53,8 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.ArrayList;
-import java.util.List;
-
 public class TestDFSStripedOutputStream {
+  public static final Log LOG = LogFactory.getLog(TestDFSStripedOutputStream.class);
   private int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
   private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
 
@@ -46,7 +64,6 @@ public class TestDFSStripedOutputStream {
   private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
   private final int stripesPerBlock = 4;
   private final int blockSize = cellSize * stripesPerBlock;
-  private final RawErasureEncoder encoder = new RSRawEncoder();
 
   @Before
   public void setup() throws IOException {
@@ -56,7 +73,6 @@ public class TestDFSStripedOutputStream {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
     fs = cluster.getFileSystem();
-    encoder.initialize(dataBlocks, parityBlocks, cellSize);
   }
 
   @After
@@ -67,78 +83,74 @@ public class TestDFSStripedOutputStream {
   }
 
   @Test
-  public void TestFileEmpty() throws IOException {
+  public void testFileEmpty() throws IOException {
     testOneFile("/EmptyFile", 0);
   }
 
   @Test
-  public void TestFileSmallerThanOneCell1() throws IOException {
+  public void testFileSmallerThanOneCell1() throws IOException {
     testOneFile("/SmallerThanOneCell", 1);
   }
 
   @Test
-  public void TestFileSmallerThanOneCell2() throws IOException {
+  public void testFileSmallerThanOneCell2() throws IOException {
     testOneFile("/SmallerThanOneCell", cellSize - 1);
   }
 
   @Test
-  public void TestFileEqualsWithOneCell() throws IOException {
+  public void testFileEqualsWithOneCell() throws IOException {
     testOneFile("/EqualsWithOneCell", cellSize);
   }
 
   @Test
-  public void TestFileSmallerThanOneStripe1() throws IOException {
+  public void testFileSmallerThanOneStripe1() throws IOException {
     testOneFile("/SmallerThanOneStripe", cellSize * dataBlocks - 1);
   }
 
   @Test
-  public void TestFileSmallerThanOneStripe2() throws IOException {
+  public void testFileSmallerThanOneStripe2() throws IOException {
     testOneFile("/SmallerThanOneStripe", cellSize + 123);
   }
 
   @Test
-  public void TestFileEqualsWithOneStripe() throws IOException {
+  public void testFileEqualsWithOneStripe() throws IOException {
     testOneFile("/EqualsWithOneStripe", cellSize * dataBlocks);
   }
 
   @Test
-  public void TestFileMoreThanOneStripe1() throws IOException {
+  public void testFileMoreThanOneStripe1() throws IOException {
     testOneFile("/MoreThanOneStripe1", cellSize * dataBlocks + 123);
   }
 
   @Test
-  public void TestFileMoreThanOneStripe2() throws IOException {
+  public void testFileMoreThanOneStripe2() throws IOException {
     testOneFile("/MoreThanOneStripe2", cellSize * dataBlocks
             + cellSize * dataBlocks + 123);
   }
 
   @Test
-  public void TestFileFullBlockGroup() throws IOException {
+  public void testFileFullBlockGroup() throws IOException {
     testOneFile("/FullBlockGroup", blockSize * dataBlocks);
   }
 
   @Test
-  public void TestFileMoreThanABlockGroup1() throws IOException {
+  public void testFileMoreThanABlockGroup1() throws IOException {
     testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123);
   }
 
   @Test
-  public void TestFileMoreThanABlockGroup2() throws IOException {
+  public void testFileMoreThanABlockGroup2() throws IOException {
     testOneFile("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize+ 123);
   }
 
 
   @Test
-  public void TestFileMoreThanABlockGroup3() throws IOException {
+  public void testFileMoreThanABlockGroup3() throws IOException {
     testOneFile("/MoreThanABlockGroup3",
         blockSize * dataBlocks * 3 + cellSize * dataBlocks
         + cellSize + 123);
   }
 
-  private int stripeDataSize() {
-    return cellSize * dataBlocks;
-  }
-
   private byte[] generateBytes(int cnt) {
     byte[] bytes = new byte[cnt];
     for (int i = 0; i < cnt; i++) {
@@ -152,8 +164,7 @@ 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 IOException {
     Path testPath = new Path(src);
 
     byte[] bytes = generateBytes(writeBytes);
@@ -161,8 +172,7 @@ public class TestDFSStripedOutputStream {
 
     // check file length
     FileStatus status = fs.getFileStatus(testPath);
-    long fileLength = status.getLen();
-    Assert.assertEquals(writeBytes, fileLength);
+    Assert.assertEquals(writeBytes, status.getLen());
 
     List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
     LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L);
@@ -251,16 +261,12 @@ public class TestDFSStripedOutputStream {
           continue;
         }
         for (int posInBlk = 0; posInBlk < actualBlkBytes.length; posInBlk++) {
-          byte expected;
           // calculate the position of this byte in the file
           long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(cellSize,
               dataBlocks, posInBlk, blkIdxInGroup) +
               group * blockSize * dataBlocks;
-          if (posInFile >= writeBytes) {
-            expected = 0;
-          } else {
-            expected = getByte(posInFile);
-          }
+          Assert.assertTrue(posInFile < writeBytes);
+          final byte expected = getByte(posInFile);
 
           String s = "Unexpected byte " + actualBlkBytes[posInBlk]
               + ", expect " + expected
@@ -272,84 +278,34 @@ public class TestDFSStripedOutputStream {
         }
       }
 
-      // verify the parity blocks
-      final ByteBuffer[] parityBuffers = new ByteBuffer[parityBlocks];
-      final long groupSize = lbs.getLocatedBlocks().get(group).getBlockSize();
-      int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength(groupSize,
-          cellSize, dataBlocks, dataBlocks);
-      for (int i = 0; i < parityBlocks; i++) {
-        parityBuffers[i] = ByteBuffer.allocate(parityBlkSize);
-      }
-      final int numStripes = (int) (groupSize - 1) / stripeDataSize() + 1;
-      for (int i = 0; i < numStripes; i++) {
-        final int parityCellSize = i < numStripes - 1 || parityBlkSize % cellSize == 0
-            ? cellSize : parityBlkSize % cellSize;
-        ByteBuffer[] stripeBuf = new ByteBuffer[dataBlocks];
-        for (int k = 0; k < stripeBuf.length; k++) {
-          stripeBuf[k] = ByteBuffer.allocate(cellSize);
-        }
-        for (int j = 0; j < dataBlocks; j++) {
-          if (dataBlockBytes[j] != null) {
-            int length = Math.min(cellSize,
-                dataBlockBytes[j].length - cellSize * i);
-            if (length > 0) {
-              stripeBuf[j].put(dataBlockBytes[j], cellSize * i, length);
-            }
-          }
-          final long pos = stripeBuf[j].position();
-          for (int k = 0; k < parityCellSize - pos; k++) {
-            stripeBuf[j].put((byte) 0);
-          }
-          stripeBuf[j].flip();
-        }
-        ByteBuffer[] parityBuf = new ByteBuffer[parityBlocks];
-        for (int j = 0; j < parityBlocks; j++) {
-          parityBuf[j] = ByteBuffer.allocate(cellSize);
-          for (int k = 0; k < parityCellSize; k++) {
-            parityBuf[j].put((byte) 0);
-          }
-          parityBuf[j].flip();
-        }
-
-        encoder.encode(stripeBuf, parityBuf);
-        for (int j = 0; j < parityBlocks; j++) {
-          parityBuffers[j].put(parityBuf[j]);
-        }
-      }
-
-      for (int i = 0; i < parityBlocks; i++) {
-        Assert.assertArrayEquals(parityBuffers[i].array(), parityBlockBytes[i]);
-      }
+      verifyParity(lbs.getLocatedBlocks().get(group).getBlockSize(),
+          cellSize, dataBlockBytes, parityBlockBytes);
     }
   }
-
-  private void testReadWriteOneFile(String src, int writeBytes)
-      throws IOException {
-    Path TestPath = new Path(src);
-    byte[] bytes = generateBytes(writeBytes);
-    DFSTestUtil.writeFile(fs, TestPath, new String(bytes));
-
-    //check file length
-    FileStatus status = fs.getFileStatus(TestPath);
-    long fileLength = status.getLen();
-    if (fileLength != writeBytes) {
-      Assert.fail("File Length error: expect=" + writeBytes
-          + ", actual=" + fileLength);
+    
+  static void verifyParity(final long size, final int cellSize,
+      byte[][] dataBytes, byte[][] parityBytes) {
+    // 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];
     }
-
-    DFSStripedInputStream dis = new DFSStripedInputStream(
-        fs.getClient(), src, true);
-    byte[] buf = new byte[writeBytes + 100];
-    int readLen = dis.read(0, buf, 0, buf.length);
-    readLen = readLen >= 0 ? readLen : 0;
-    if (readLen != writeBytes) {
-      Assert.fail("The length of file is not correct.");
-    }
-
-    for (int i = 0; i < writeBytes; i++) {
-      if (getByte(i) != buf[i]) {
-        Assert.fail("Byte at i = " + i + " is wrongly written.");
+    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 = new RSRawEncoder();
+    encoder.initialize(dataBytes.length, parityBytes.length, cellSize);
+    encoder.encode(dataBytes, expectedParityBytes);
+    for (int i = 0; i < parityBytes.length; i++) {
+      Assert.assertArrayEquals(expectedParityBytes[i], parityBytes[i]);
+    }
   }
 }


[35/50] hadoop git commit: HDFS-8316. Erasure coding: refactor EC constants to be consistent with HDFS-8249. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-8316. Erasure coding: refactor EC constants to be consistent with HDFS-8249. Contributed by Zhe Zhang.


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

Branch: refs/heads/HDFS-7285
Commit: 1e348817ec040a5847d1adfd8629cd8b65ce969e
Parents: 74dfef3
Author: Jing Zhao <ji...@apache.org>
Authored: Mon May 4 11:24:35 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:23 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/protocol/HdfsConstants.java   | 11 +++++++++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt |  3 +++
 .../org/apache/hadoop/hdfs/DFSStripedOutputStream.java   |  2 +-
 .../hdfs/server/blockmanagement/BlockIdManager.java      |  4 ++--
 .../blockmanagement/SequentialBlockGroupIdGenerator.java |  4 ++--
 .../hadoop/hdfs/server/common/HdfsServerConstants.java   |  5 ++++-
 .../hdfs/server/namenode/TestAddStripedBlocks.java       |  4 ++--
 .../hdfs/server/namenode/TestStripedINodeFile.java       |  6 +++---
 8 files changed, 28 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e348817/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 58c7ea1..32ca81c 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
@@ -75,6 +75,17 @@ 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 schema.
+   * TODO: to be removed once all places use schema.
+   */
+
+  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 = 256 * 1024;
+
   // SafeMode actions
   public enum SafeModeAction {
     SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e348817/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 145494f..e30b2ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -158,3 +158,6 @@
 
     HDFS-7949. WebImageViewer need support file size calculation with striped 
     blocks. (Rakesh R via Zhe Zhang)
+
+    HDFS-8316. Erasure coding: refactor EC constants to be consistent with HDFS-8249.
+    (Zhe Zhang via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e348817/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 5e2a534..71cdbb9 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
@@ -419,7 +419,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   @Override
   protected synchronized void closeImpl() throws IOException {
     if (isClosed()) {
-      getLeadingStreamer().getLastException().check();
+      getLeadingStreamer().getLastException().check(true);
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e348817/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index fc82d5d..746802c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -241,11 +241,11 @@ public class BlockIdManager {
    * data/parity block id in the same striped block group.
    */
   public static long convertToStripedID(long id) {
-    return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK);
+    return id & (~HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
   }
 
   public static int getBlockIndex(Block reportedBlock) {
     return (int) (reportedBlock.getBlockId() &
-        HdfsConstants.BLOCK_GROUP_INDEX_MASK);
+        HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e348817/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
----------------------------------------------------------------------
diff --git 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
index de8e379..479ee4c 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
@@ -21,8 +21,8 @@ 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.protocol.HdfsConstants.BLOCK_GROUP_INDEX_MASK;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_BLOCKS_IN_GROUP;
+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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e348817/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 e7807c5..39ccebe 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
@@ -379,6 +379,9 @@ public interface HdfsServerConstants {
       "raw.hdfs.crypto.file.encryption.info";
   String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER =
       "security.hdfs.unreadable.by.superuser";
-  public static final String XATTR_ERASURECODING_ZONE =
+  String XATTR_ERASURECODING_ZONE =
       "raw.hdfs.erasurecoding.zone";
+
+  long BLOCK_GROUP_INDEX_MASK = 15;
+  byte MAX_BLOCKS_IN_GROUP = 16;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e348817/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 297db14..ba76360 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
@@ -102,7 +102,7 @@ public class TestAddStripedBlocks {
     DFSTestUtil.writeFile(dfs, testPath, "hello again");
     lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0);
     final long secondId = lb.get(0).getBlock().getBlockId();
-    Assert.assertEquals(firstId + HdfsConstants.MAX_BLOCKS_IN_GROUP, secondId);
+    Assert.assertEquals(firstId + HdfsServerConstants.MAX_BLOCKS_IN_GROUP, secondId);
   }
 
   @Test (timeout=60000)
@@ -159,7 +159,7 @@ public class TestAddStripedBlocks {
     Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS,
         block.getParityBlockNum());
     Assert.assertEquals(0,
-        block.getBlockId() & HdfsConstants.BLOCK_GROUP_INDEX_MASK);
+        block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
 
     final BlockInfoStripedUnderConstruction blockUC =
         (BlockInfoStripedUnderConstruction) block;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e348817/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 4a6d6cc..f97943b 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
@@ -28,10 +28,10 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 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;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import org.junit.Test;
@@ -46,8 +46,8 @@ public class TestStripedINodeFile {
       "userName", null, FsPermission.getDefault());
 
   private static INodeFile createStripedINodeFile() {
-    return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-        null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID);
+    return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+        null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID);
   }
 
   @Test


[46/50] hadoop git commit: HDFS-8289. Erasure Coding: add ECSchema to HdfsFileStatus. Contributed by Yong Zhang.

Posted by zh...@apache.org.
HDFS-8289. Erasure Coding: add ECSchema to HdfsFileStatus. Contributed by Yong Zhang.


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

Branch: refs/heads/HDFS-7285
Commit: 95287672de6414ad2dc31d03d90d5c1146581c6c
Parents: de30e66
Author: Jing Zhao <ji...@apache.org>
Authored: Thu May 7 11:52:49 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:40:59 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsFileStatus.java    | 10 ++-
 .../protocol/SnapshottableDirectoryStatus.java  |  2 +-
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |  2 +-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  6 +-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  2 +-
 .../hadoop/hdfs/DFSStripedInputStream.java      | 13 ++--
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  4 +-
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |  5 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  2 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 10 ++-
 .../server/namenode/FSDirStatAndListingOp.java  | 16 +++--
 .../src/main/proto/erasurecoding.proto          | 19 ------
 .../hadoop-hdfs/src/main/proto/hdfs.proto       | 22 +++++++
 .../hadoop/hdfs/TestDFSClientRetries.java       |  4 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  | 16 +++--
 .../apache/hadoop/hdfs/TestEncryptionZones.java |  2 +-
 .../hadoop/hdfs/TestFileStatusWithECschema.java | 65 ++++++++++++++++++++
 .../java/org/apache/hadoop/hdfs/TestLease.java  |  4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  2 +-
 .../apache/hadoop/hdfs/web/TestJsonUtil.java    |  2 +-
 21 files changed, 149 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
index 34f429a..f07973a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 /** Interface that represents the over the wire information for a file.
  */
@@ -48,6 +49,8 @@ public class HdfsFileStatus {
 
   private final FileEncryptionInfo feInfo;
 
+  private final ECSchema schema;
+  
   // Used by dir, not including dot and dotdot. Always zero for a regular file.
   private final int childrenNum;
   private final byte storagePolicy;
@@ -73,7 +76,7 @@ public class HdfsFileStatus {
       long blocksize, long modification_time, long access_time,
       FsPermission permission, String owner, String group, byte[] symlink,
       byte[] path, long fileId, int childrenNum, FileEncryptionInfo feInfo,
-      byte storagePolicy) {
+      byte storagePolicy, ECSchema schema) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
@@ -93,6 +96,7 @@ public class HdfsFileStatus {
     this.childrenNum = childrenNum;
     this.feInfo = feInfo;
     this.storagePolicy = storagePolicy;
+    this.schema = schema;
   }
 
   /**
@@ -250,6 +254,10 @@ public class HdfsFileStatus {
     return feInfo;
   }
 
+  public ECSchema getECSchema() {
+    return schema;
+  }
+
   public final int getChildrenNum() {
     return childrenNum;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
index ac19d44..813ea26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
@@ -61,7 +61,7 @@ public class SnapshottableDirectoryStatus {
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
         access_time, permission, owner, group, null, localName, inodeId,
-        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
     this.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index ca94840..62f679b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -132,7 +132,7 @@ class JsonUtilClient {
         blockSize, mTime, aTime, permission, owner, group,
         symlink, DFSUtilClient.string2Bytes(localName),
         fileId, childrenNum, null,
-        storagePolicy);
+        storagePolicy, null);
   }
 
   /** Convert a Json map to an ExtendedBlock object. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/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 fed08e1..ab8a748 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -189,3 +189,6 @@
 
     HDFS-8203. Erasure Coding: Seek and other Ops in DFSStripedInputStream.
     (Yi Liu via jing9)
+
+    HDFS-8289. Erasure Coding: add ECSchema to HdfsFileStatus. (Yong Zhang via
+    jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/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 47706fc..1942bc9 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
@@ -1188,9 +1188,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     //    Get block info from namenode
     TraceScope scope = getPathTraceScope("newDFSInputStream", src);
     try {
-      ErasureCodingInfo info = getErasureCodingInfo(src);
-      if (info != null) {
-        return new DFSStripedInputStream(this, src, verifyChecksum, info);
+      ECSchema schema = getFileInfo(src).getECSchema();
+      if (schema != null) {
+        return new DFSStripedInputStream(this, src, verifyChecksum, schema);
       } else {
         return new DFSInputStream(this, src, verifyChecksum);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/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 8580357..ea1ea26 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
@@ -271,7 +271,7 @@ public class DFSOutputStream extends FSOutputSummer
       }
       Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!");
       final DFSOutputStream out;
-      if(stat.getReplication() == 0) {
+      if(stat.getECSchema() != null) {
         out = new DFSStripedOutputStream(dfsClient, src, stat,
             flag, progress, checksum, favoredNodes);
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/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 9011192..7425e75 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.io.ByteBufferPool;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.planReadPortions;
 
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
@@ -132,13 +133,13 @@ public class DFSStripedInputStream extends DFSInputStream {
   private final CompletionService<Integer> readingService;
 
   DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
-      ErasureCodingInfo ecInfo) throws IOException {
+      ECSchema schema) throws IOException {
     super(dfsClient, src, verifyChecksum);
-    // ECInfo is restored from NN just before reading striped file.
-    assert ecInfo != null;
-    cellSize = ecInfo.getSchema().getChunkSize();
-    dataBlkNum = (short) ecInfo.getSchema().getNumDataUnits();
-    parityBlkNum = (short) ecInfo.getSchema().getNumParityUnits();
+
+    assert schema != null;
+    cellSize = schema.getChunkSize();
+    dataBlkNum = (short) schema.getNumDataUnits();
+    parityBlkNum = (short) schema.getNumParityUnits();
     curStripeRange = new StripeRange(0, 0);
     readingService =
         new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/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 bbc8ba0..b99afab 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
@@ -219,9 +219,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       LOG.debug("Creating DFSStripedOutputStream for " + src);
     }
 
-    // ECInfo is restored from NN just before writing striped files.
-    //TODO reduce an rpc call HDFS-8289
-    final ECSchema schema = dfsClient.getErasureCodingInfo(src).getSchema();
+    final ECSchema schema = stat.getECSchema();
     final int numParityBlocks = schema.getNumParityUnits();
     cellSize = schema.getChunkSize();
     numDataBlocks = schema.getNumDataUnits();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
index 23e8f57..9194d26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
@@ -26,6 +26,7 @@ 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;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 /** 
  * Interface that represents the over the wire information
@@ -58,10 +59,10 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
       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, ECSchema schema) {
     super(length, isdir, block_replication, blocksize, modification_time,
         access_time, permission, owner, group, symlink, path, fileId,
-        childrenNum, feInfo, storagePolicy);
+        childrenNum, feInfo, storagePolicy, schema);
     this.locations = locations;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/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 014fcef..67b1457 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
@@ -173,7 +173,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodin
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/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 26bdf34..94b2ff9 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
@@ -136,9 +136,9 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailur
 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.ErasureCodingInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneInfoProto;
+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.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
@@ -1505,7 +1505,8 @@ public class PBHelper {
         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.hasEcSchema() ? PBHelper.convertECSchema(fs.getEcSchema()) : null);
   }
 
   public static SnapshottableDirectoryStatus convert(
@@ -1566,6 +1567,9 @@ public class PBHelper {
         builder.setLocations(PBHelper.convert(locations));
       }
     }
+    if(fs.getECSchema() != null) {
+      builder.setEcSchema(PBHelper.convertECSchema(fs.getECSchema()));
+    }
     return builder.build();
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/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 c636d93..7133cf1 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.commons.io.Charsets;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
@@ -38,6 +39,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -315,7 +317,7 @@ class FSDirStatAndListingOp {
       if (fsd.getINode4DotSnapshot(srcs) != null) {
         return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
             HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
-            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
       }
       return null;
     }
@@ -382,7 +384,9 @@ class FSDirStatAndListingOp {
 
     final FileEncryptionInfo feInfo = isRawPath ? null :
         fsd.getFileEncryptionInfo(node, snapshot, iip);
-
+    
+    final ECSchema schema = fsd.getECSchema(iip);
+    
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
       size = fileNode.computeFileSize(snapshot);
@@ -412,7 +416,8 @@ class FSDirStatAndListingOp {
         node.getId(),
         childrenNum,
         feInfo,
-        storagePolicy);
+        storagePolicy,
+        schema);
   }
 
   private static INodeAttributes getINodeAttributes(
@@ -459,7 +464,8 @@ class FSDirStatAndListingOp {
     }
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
-
+    final ECSchema schema = fsd.getECSchema(iip);
+        
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
           blocksize, node.getModificationTime(snapshot),
@@ -467,7 +473,7 @@ class FSDirStatAndListingOp {
           getPermissionForFileStatus(nodeAttrs, isEncrypted),
           nodeAttrs.getUserName(), nodeAttrs.getGroupName(),
           node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
-          node.getId(), loc, childrenNum, feInfo, storagePolicy);
+          node.getId(), loc, childrenNum, feInfo, storagePolicy, schema);
     // Set caching information for the located blocks.
     if (loc != null) {
       CacheManager cacheManager = fsd.getFSNamesystem().getCacheManager();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
index 7a19a80..2302d1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -24,25 +24,6 @@ package hadoop.hdfs;
 import "hdfs.proto";
 
 /**
- * ECSchema options entry
- */
-message ECSchemaOptionEntryProto {
-  required string key = 1;
-  required string value = 2;
-}
-
-/**
- * ECSchema for erasurecoding
- */
-message ECSchemaProto {
-  required string schemaName = 1;
-  required string codecName = 2;
-  required uint32 dataUnits = 3;
-  required uint32 parityUnits = 4;
-  repeated ECSchemaOptionEntryProto options = 5;
-}
-
-/**
  * ErasureCodingInfo
  */
 message ErasureCodingInfoProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index 67e2058..64030be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -304,6 +304,25 @@ message LocatedBlocksProto {
 }
 
 /**
+ * ECSchema options entry
+ */
+message ECSchemaOptionEntryProto {
+  required string key = 1;
+  required string value = 2;
+}
+
+/**
+ * ECSchema for erasurecoding
+ */
+message ECSchemaProto {
+  required string schemaName = 1;
+  required string codecName = 2;
+  required uint32 dataUnits = 3;
+  required uint32 parityUnits = 4;
+  repeated ECSchemaOptionEntryProto options = 5;
+}
+
+/**
  * Status of a file, directory or symlink
  * Optionally includes a file's block locations if requested by client on the rpc call.
  */
@@ -337,6 +356,9 @@ message HdfsFileStatusProto {
   optional FileEncryptionInfoProto fileEncryptionInfo = 15;
 
   optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
+
+  // Optional field for erasure coding
+  optional ECSchemaProto ecSchema = 17;
 } 
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index 68cc155..ec88a54 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -255,12 +255,12 @@ public class TestDFSClientRetries {
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null, (byte) 0)).when(mockNN).getFileInfo(anyString());
+                1010, 0, null, (byte) 0, null)).when(mockNN).getFileInfo(anyString());
     
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null, (byte) 0))
+                1010, 0, null, (byte) 0, null))
         .when(mockNN)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/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 4da9c26..3f79933 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.ErasureCodingInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -36,6 +35,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -54,8 +54,7 @@ public class TestDFSStripedInputStream {
   private DistributedFileSystem fs;
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
-  private ErasureCodingInfo info = new ErasureCodingInfo(filePath.toString(),
-      ErasureCodingSchemaManager.getSystemDefaultSchema());
+  private final ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
   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;
@@ -92,8 +91,8 @@ public class TestDFSStripedInputStream {
         NUM_STRIPE_PER_BLOCK, false);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
-    final DFSStripedInputStream in =
-        new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, info);
+    final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
+        filePath.toString(), false, schema);
 
     List<LocatedBlock> lbList = lbs.getLocatedBlocks();
     for (LocatedBlock aLbList : lbList) {
@@ -129,7 +128,7 @@ public class TestDFSStripedInputStream {
     }
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(),
-            filePath.toString(), false, info);
+            filePath.toString(), false, schema);
     int readSize = BLOCK_GROUP_SIZE;
     byte[] readBuffer = new byte[readSize];
     int ret = in.read(0, readBuffer, 0, readSize);
@@ -156,8 +155,7 @@ public class TestDFSStripedInputStream {
     }
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
         NUM_STRIPE_PER_BLOCK, false);
-    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
-        filePath.toString(), 0, fileSize);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(filePath.toString(), 0, fileSize);
 
     assert lbs.getLocatedBlocks().size() == numBlocks;
     for (LocatedBlock lb : lbs.getLocatedBlocks()) {
@@ -175,7 +173,7 @@ public class TestDFSStripedInputStream {
 
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(),
-            false, info);
+            false, schema);
 
     byte[] expected = new byte[fileSize];
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index b211ffb..8542131 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -737,7 +737,7 @@ public class TestEncryptionZones {
             version, new byte[suite.getAlgorithmBlockSize()],
             new byte[suite.getAlgorithmBlockSize()],
             "fakeKey", "fakeVersion"),
-            (byte) 0))
+            (byte) 0, null))
         .when(mcp)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java
new file mode 100644
index 0000000..f8c0667
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECschema.java
@@ -0,0 +1,65 @@
+package org.apache.hadoop.hdfs;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFileStatusWithECschema {
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private DFSClient client;
+
+  @Before
+  public void before() throws IOException {
+    cluster =
+        new MiniDFSCluster.Builder(new Configuration()).numDataNodes(1).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    client = fs.getClient();
+  }
+
+  @After
+  public void after() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testFileStatusWithECschema() throws Exception {
+    // test directory not in EC zone
+    final Path dir = new Path("/foo");
+    assertTrue(fs.mkdir(dir, FsPermission.getDirDefault()));
+    assertNull(client.getFileInfo(dir.toString()).getECSchema());
+    // test file not in EC zone
+    final Path file = new Path(dir, "foo");
+    fs.create(file).close();
+    assertNull(client.getFileInfo(file.toString()).getECSchema());
+    fs.delete(file, true);
+
+    final ECSchema schema1 = ErasureCodingSchemaManager.getSystemDefaultSchema();
+    // create EC zone on dir
+    fs.createErasureCodingZone(dir, schema1);
+    final ECSchema schame2 = client.getFileInfo(dir.toUri().getPath()).getECSchema();
+    assertNotNull(schame2);
+    assertTrue(schema1.equals(schame2));
+
+    // test file in EC zone
+    fs.create(file).close();
+    final ECSchema schame3 =
+        fs.getClient().getFileInfo(file.toUri().getPath()).getECSchema();
+    assertNotNull(schame3);
+    assertTrue(schema1.equals(schame3));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
index 985f43e..b77ff3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
@@ -354,12 +354,12 @@ public class TestLease {
     Mockito.doReturn(
         new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
             (short) 777), "owner", "group", new byte[0], new byte[0],
-            1010, 0, null, (byte) 0)).when(mcp).getFileInfo(anyString());
+            1010, 0, null, (byte) 0, null)).when(mcp).getFileInfo(anyString());
     Mockito
         .doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null, (byte) 0))
+                1010, 0, null, (byte) 0, null))
         .when(mcp)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git 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
index cc7e799..6edc49b 100644
--- 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
@@ -1198,7 +1198,7 @@ public class TestFsck {
 
     HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication,
         blockSize, modTime, accessTime, perms, owner, group, symlink,
-        path, fileId, numChildren, null, storagePolicy);
+        path, fileId, numChildren, null, storagePolicy, null);
     Result res = new Result(conf);
 
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95287672/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
index 391f190..8947c5b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
@@ -65,7 +65,7 @@ public class TestJsonUtil {
     final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
         now, now + 10, new FsPermission((short) 0644), "user", "group",
         DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
-        HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0);
+        HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null);
     final FileStatus fstatus = toFileStatus(status, parent);
     System.out.println("status  = " + status);
     System.out.println("fstatus = " + fstatus);


[31/50] hadoop git commit: HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of datastreamer threads. Contributed by Rakesh R.

Posted by zh...@apache.org.
HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of datastreamer threads. Contributed by Rakesh R.


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

Branch: refs/heads/HDFS-7285
Commit: b962860fed3e53a9049ac8a83ff116418032def0
Parents: d2e70cb
Author: Zhe Zhang <zh...@apache.org>
Authored: Thu Apr 30 00:13:32 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:22 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt                |  3 +++
 .../org/apache/hadoop/hdfs/DFSStripedOutputStream.java  | 12 ++++++++++--
 2 files changed, 13 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b962860f/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 ca60487..3c75152 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -149,3 +149,6 @@
 
     HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil.
     (Zhe Zhang)
+
+    HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of 
+    datastreamer threads. (Rakesh R via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b962860f/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 c930187..5e2a534 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
@@ -331,18 +331,26 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   // interrupt datastreamer if force is true
   @Override
   protected void closeThreads(boolean force) throws IOException {
+    int index = 0;
+    boolean exceptionOccurred = false;
     for (StripedDataStreamer streamer : streamers) {
       try {
         streamer.close(force);
         streamer.join();
         streamer.closeSocket();
-      } catch (InterruptedException e) {
-        throw new IOException("Failed to shutdown streamer");
+      } catch (InterruptedException | IOException e) {
+        DFSClient.LOG.error("Failed to shutdown streamer: name="
+            + streamer.getName() + ", index=" + index + ", file=" + src, e);
+        exceptionOccurred = true;
       } finally {
         streamer.setSocketToNull();
         setClosed();
+        index++;
       }
     }
+    if (exceptionOccurred) {
+      throw new IOException("Failed to shutdown streamer");
+    }
   }
 
   /**


[23/50] hadoop git commit: HDFS-8189. ClientProtocol#createErasureCodingZone API was wrongly annotated as Idempotent (Contributed by Vinayakumar B)

Posted by zh...@apache.org.
HDFS-8189. ClientProtocol#createErasureCodingZone API was wrongly annotated as Idempotent (Contributed by Vinayakumar B)


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

Branch: refs/heads/HDFS-7285
Commit: 61923aafafca424ed2ff39e9633797b928cb2a64
Parents: e4a4fd0
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 28 14:24:17 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:20 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt            |  5 ++++-
 .../apache/hadoop/hdfs/protocol/ClientProtocol.java | 16 ++++++++--------
 2 files changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/61923aaf/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 c28473b..6c5d7ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -136,4 +136,7 @@
     striped layout (Zhe Zhang)
 
     HDFS-8230. Erasure Coding: Ignore DatanodeProtocol#DNA_ERASURE_CODING_RECOVERY 
-    commands from standbynode if any (vinayakumarb)
\ No newline at end of file
+    commands from standbynode if any (vinayakumarb)
+
+    HDFS-8189. ClientProtocol#createErasureCodingZone API was wrongly annotated
+    as Idempotent (vinayakumarb)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/61923aaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index bba7697..76e2d12 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1364,14 +1364,6 @@ public interface ClientProtocol {
       long prevId) throws IOException;
 
   /**
-   * Create an erasure coding zone with specified schema, if any, otherwise
-   * default
-   */
-  @Idempotent
-  public void createErasureCodingZone(String src, ECSchema schema)
-      throws IOException;
-
-  /**
    * Set xattr of a file or directory.
    * The name must be prefixed with the namespace followed by ".". For example,
    * "user.attr".
@@ -1467,6 +1459,14 @@ public interface ClientProtocol {
   public EventBatchList getEditsFromTxid(long txid) throws IOException;
 
   /**
+   * Create an erasure coding zone with specified schema, if any, otherwise
+   * default
+   */
+  @AtMostOnce
+  public void createErasureCodingZone(String src, ECSchema schema)
+      throws IOException;
+
+  /**
    * Gets the ECInfo for the specified file/directory
    * 
    * @param src


[21/50] hadoop git commit: HDFS-8233. Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last stripe is at the block group boundary. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-8233. Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last stripe is at the block group boundary. 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/613e0a8d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/613e0a8d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/613e0a8d

Branch: refs/heads/HDFS-7285
Commit: 613e0a8d46ea7908edb7831e1cfcf60baa44a4d5
Parents: ef6c1b8
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Apr 23 15:43:04 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:19 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  5 +-
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 51 +++++++++-----------
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |  6 +++
 3 files changed, 34 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/613e0a8d/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 8977c46..48791b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -121,4 +121,7 @@
     schema. (Kai Zheng via Zhe Zhang)
 
     HDFS-8136. Client gets and uses EC schema when reads and writes a stripping
-    file. (Kai Sasaki via Kai Zheng)
\ No newline at end of file
+    file. (Kai Sasaki via Kai Zheng)
+
+    HDFS-8233. Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last
+    stripe is at the block group boundary. (jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/613e0a8d/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 eeb9d7e..245dfc1 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
@@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.protocol.ECInfo;
 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.util.StripedBlockUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.util.DataChecksum;
@@ -278,14 +277,6 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     return numDataBlocks * cellSize;
   }
 
-  private long getCurrentBlockGroupBytes() {
-    long sum = 0;
-    for (int i = 0; i < numDataBlocks; i++) {
-      sum += streamers.get(i).getBytesCurBlock();
-    }
-    return sum;
-  }
-
   private void notSupported(String headMsg)
       throws IOException{
       throw new IOException(
@@ -347,37 +338,43 @@ 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 void writeParityCellsForLastStripe() throws IOException {
-    final long currentBlockGroupBytes = getCurrentBlockGroupBytes();
-    long parityBlkSize = StripedBlockUtil.getInternalBlockLength(
-        currentBlockGroupBytes, cellSize, numDataBlocks,
-        numDataBlocks + 1);
-    if (parityBlkSize == 0 || currentBlockGroupBytes % stripeDataSize() == 0) {
+    final long currentBlockGroupBytes = getCurrentSumBytes();
+    if (currentBlockGroupBytes % stripeDataSize() == 0) {
       return;
     }
-    int parityCellSize = parityBlkSize % cellSize == 0 ? cellSize :
-                        (int) (parityBlkSize % cellSize);
+    long firstCellSize = getLeadingStreamer().getBytesCurBlock() % cellSize;
+    long parityCellSize = firstCellSize > 0 && firstCellSize < cellSize ?
+        firstCellSize : cellSize;
 
     for (int i = 0; i < numAllBlocks; i++) {
-      long internalBlkLen = StripedBlockUtil.getInternalBlockLength(
-          currentBlockGroupBytes, cellSize, numDataBlocks, 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
-      if (internalBlkLen < parityBlkSize || i >= numDataBlocks) {
-        int position = cellBuffers[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++) {
-          cellBuffers[i].put((byte) 0);
-        }
+      int position = cellBuffers[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++) {
+        cellBuffers[i].put((byte) 0);
       }
       cellBuffers[i].flip();
     }
     encode(cellBuffers);
 
-    //write parity cells
+    // write parity cells
     curIdx = numDataBlocks;
     refreshStreamer();
     for (int i = numDataBlocks; i < numAllBlocks; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/613e0a8d/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 26f6d2c..5ce94ee 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
@@ -115,6 +115,12 @@ public class TestDFSStripedOutputStream {
   }
 
   @Test
+  public void testFileLessThanFullBlockGroup() throws IOException {
+    testOneFile("/LessThanFullBlockGroup",
+        cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize);
+  }
+
+  @Test
   public void testFileFullBlockGroup() throws IOException {
     testOneFile("/FullBlockGroup", blockSize * dataBlocks);
   }


[04/50] hadoop git commit: HDFS-7994. Detect if resevered EC Block ID is already used during namenode startup. Contributed by Hui Zheng

Posted by zh...@apache.org.
HDFS-7994. Detect if resevered EC Block ID is already used during namenode startup. Contributed by Hui Zheng


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

Branch: refs/heads/HDFS-7285
Commit: 0559d00437130c09dcbac8225af95ede4bca6ac5
Parents: ed68791
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Apr 16 13:16:37 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:14 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   5 +-
 .../server/blockmanagement/BlockManager.java    |  42 ++++-
 .../hdfs/server/namenode/FSEditLogLoader.java   |   4 +-
 .../hdfs/server/namenode/FSImageFormat.java     |   6 +-
 .../server/namenode/FSImageFormatPBINode.java   |   2 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |   2 +-
 .../server/namenode/TestFSEditLogLoader.java    | 106 ++++++++++++
 .../hdfs/server/namenode/TestFSImage.java       | 169 ++++++++++++++++++-
 8 files changed, 321 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0559d004/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 b9fc6fa..78ca6d3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -81,4 +81,7 @@
     HDFS-8123. Erasure Coding: Better to move EC related proto messages to a
     separate erasurecoding proto file (Rakesh R via vinayakumarb)
 
-    HDFS-7349. Support DFS command for the EC encoding (vinayakumarb)
\ No newline at end of file
+    HDFS-7349. Support DFS command for the EC encoding (vinayakumarb)
+
+    HDFS-7994. Detect if resevered EC Block ID is already used during namenode
+    startup. (Hui Zheng via szetszwo)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0559d004/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 07b49c8..5b554b3 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
@@ -273,6 +273,9 @@ public class BlockManager {
   /** Check whether name system is running before terminating */
   private boolean checkNSRunning = true;
 
+  /** Check whether there are any non-EC blocks using StripedID */
+  private boolean hasNonEcBlockUsingStripedID = false;
+
   public BlockManager(final Namesystem namesystem, final Configuration conf)
     throws IOException {
     this.namesystem = namesystem;
@@ -2920,6 +2923,24 @@ public class BlockManager {
   }
 
   /**
+   * Get the value of whether there are any non-EC blocks using StripedID.
+   *
+   * @return Returns the value of whether there are any non-EC blocks using StripedID.
+   */
+  public boolean hasNonEcBlockUsingStripedID(){
+    return hasNonEcBlockUsingStripedID;
+  }
+
+  /**
+   * Set the value of whether there are any non-EC blocks using StripedID.
+   *
+   * @param has - the value of whether there are any non-EC blocks using StripedID.
+   */
+  public void hasNonEcBlockUsingStripedID(boolean has){
+    hasNonEcBlockUsingStripedID = has;
+  }
+
+  /**
    * Process a single possibly misreplicated block. This adds it to the
    * appropriate queues if necessary, and returns a result code indicating
    * what happened with it.
@@ -3526,8 +3547,10 @@ public class BlockManager {
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
       info = blocksMap.getStoredBlock(
           new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
-    }
-    if (info == null) {
+      if ((info == null) && hasNonEcBlockUsingStripedID()){
+        info = blocksMap.getStoredBlock(block);
+      }
+    } else {
       info = blocksMap.getStoredBlock(block);
     }
     return info;
@@ -3701,6 +3724,21 @@ public class BlockManager {
     return blocksMap.addBlockCollection(block, bc);
   }
 
+  /**
+   * 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()){
+      if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
+        hasNonEcBlockUsingStripedID(true);
+      }
+    }
+    return addBlockCollection(block, bc);
+  }
+
   public BlockCollection getBlockCollection(Block b) {
     return blocksMap.getBlockCollection(b);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0559d004/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 d6698ff..d3b920e 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
@@ -992,7 +992,7 @@ public class FSEditLogLoader {
       newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
           file.getBlockReplication());
     }
-    fsNamesys.getBlockManager().addBlockCollection(newBlockInfo, file);
+    fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file);
     file.addBlock(newBlockInfo);
     fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
   }
@@ -1082,7 +1082,7 @@ public class FSEditLogLoader {
           // OP_ADD operations as each block is allocated.
           newBI = new BlockInfoContiguous(newBlock, file.getBlockReplication());
         }
-        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/0559d004/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 0582a0a..54d0d30 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
@@ -701,7 +701,7 @@ public class FSImageFormat {
       if (blocks != null) {
         final BlockManager bm = namesystem.getBlockManager();
         for (int i = 0; i < blocks.length; i++) {
-          file.setBlock(i, bm.addBlockCollection(blocks[i], file));
+          file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file));
         } 
       }
     }
@@ -1006,8 +1006,8 @@ public class FSImageFormat {
         if (oldnode.numBlocks() > 0) {
           BlockInfo ucBlock = cons.getLastBlock();
           // we do not replace the inode, just replace the last block of oldnode
-          BlockInfo info = namesystem.getBlockManager().addBlockCollection(
-              ucBlock, oldnode);
+          BlockInfo info = namesystem.getBlockManager()
+              .addBlockCollectionWithCheck(ucBlock, oldnode);
           oldnode.setBlock(oldnode.numBlocks() - 1, info);
         }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0559d004/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 82398ee..583f4b0 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
@@ -219,7 +219,7 @@ public final class FSImageFormatPBINode {
       final BlockInfo[] blocks = file.getBlocks();
       if (blocks != null) {
         for (int i = 0; i < blocks.length; i++) {
-          file.setBlock(i, bm.addBlockCollection(blocks[i], file));
+          file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0559d004/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --git 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
index 74baec5..717c8fb 100644
--- 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
@@ -247,7 +247,7 @@ public class FSImageFormatPBSnapshot {
               (BlockInfoContiguous) fsn.getBlockManager().getStoredBlock(blk);
           if(storedBlock == null) {
             storedBlock = (BlockInfoContiguous) fsn.getBlockManager()
-                .addBlockCollection(new BlockInfoContiguous(blk,
+                .addBlockCollectionWithCheck(new BlockInfoContiguous(blk,
                     copy.getFileReplication()), file);
           }
           blocks[j] = storedBlock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0559d004/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 a5c26e9..57f62c9 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
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.junit.Assert.assertFalse;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.spy;
 
@@ -47,6 +48,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 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;
@@ -570,4 +572,108 @@ public class TestFSEditLogLoader {
       }
     }
   }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForAddBlock() throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_addblock";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_addblock";
+      String clientMachine = "testMachine_addblock";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      //check whether the hasNonEcBlockUsingStripedID is set
+      //after loading a addblock-editlog
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      fns.getEditLog().logAddBlock(testFilePath, file);
+      file.toCompleteFile(System.currentTimeMillis());
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForUpdateBlocks()
+      throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_002";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser2";
+      String clientMachine = "testMachine1";
+      long blkId = 100;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      file.toCompleteFile(System.currentTimeMillis());
+
+      long newBlkNumBytes = 1024*8;
+      long newTimestamp = 1426222918+3600;
+      file.toUnderConstruction(clientName, clientMachine);
+      file.getLastBlock().setBlockId(-100);
+      file.getLastBlock().setNumBytes(newBlkNumBytes);
+      file.getLastBlock().setGenerationStamp(newTimestamp);
+      fns.getEditLog().logUpdateBlocks(testFilePath, file, true);
+      file.toCompleteFile(System.currentTimeMillis());
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0559d004/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 d36d78e..c482f1f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-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 java.io.File;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
@@ -62,6 +57,12 @@ 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 =
@@ -426,4 +427,162 @@ public class TestFSImage {
       cluster.shutdown();
     }
   }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForLoadFile() throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_loadfile";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_loadfile";
+      String clientMachine = "testMachine_loadfile";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      file.toCompleteFile(System.currentTimeMillis());
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      //after nonEcBlockUsingStripedID is deleted
+      //the hasNonEcBlockUsingStripedID is set to false
+      fs = cluster.getFileSystem();
+      fs.delete(p,false);
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertFalse(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForLoadUCFile()
+      throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_loaducfile";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_loaducfile";
+      String clientMachine = "testMachine_loaducfile";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForLoadSnapshot()
+      throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_loadSnapshot";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_loadSnapshot";
+      String clientMachine = "testMachine_loadSnapshot";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      Path d = new Path(testDir);
+      fs.mkdir(d, new FsPermission("755"));
+      fs.allowSnapshot(d);
+
+      Path p = new Path(testFilePath);
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      file.toCompleteFile(System.currentTimeMillis());
+
+      fs.createSnapshot(d,"testHasNonEcBlockUsingStripeID");
+      fs.truncate(p,0);
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }


[29/50] hadoop git commit: Fix merge conflicts.

Posted by zh...@apache.org.
Fix merge conflicts.


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

Branch: refs/heads/HDFS-7285
Commit: 2e3fb43e954c143f5740242fa2f69830c6dda431
Parents: 453e390
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Apr 29 11:35:58 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:21 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSInputStream.java  |  7 +++----
 .../apache/hadoop/hdfs/DFSStripedOutputStream.java   | 15 ++++-----------
 .../java/org/apache/hadoop/hdfs/DataStreamer.java    |  1 -
 .../org/apache/hadoop/hdfs/StripedDataStreamer.java  |  7 ++++---
 4 files changed, 11 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e3fb43e/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 6eb25d0..bef4da0 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
@@ -1116,7 +1116,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   /**
    * Read data from one DataNode.
    * @param datanode the datanode from which to read data
-   * @param block the block to read
+   * @param blockStartOffset starting offset in the file
    * @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
@@ -1146,7 +1146,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       BlockReader reader = null;
       try {
         DFSClientFaultInjector.get().fetchFromDatanodeException();
-        reader = getBlockReader(block, start, len, datanode.addr,
+        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]);
@@ -1203,8 +1203,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * with each other.
    */
   private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) {
-    Preconditions.checkArgument(offsets.length == lengths.length &&
-        offsets.length > 0);
+    Preconditions.checkArgument(offsets.length == lengths.length && offsets.length > 0);
     int sum = 0;
     for (int i = 0; i < lengths.length; i++) {
       if (i > 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e3fb43e/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 6842267..c930187 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
@@ -124,10 +124,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     for (short i = 0; i < numAllBlocks; i++) {
       StripedDataStreamer streamer = new StripedDataStreamer(stat, null,
           dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
-          i, stripeBlocks);
-      if (favoredNodes != null && favoredNodes.length != 0) {
-        streamer.setFavoredNodes(favoredNodes);
-      }
+          i, stripeBlocks, favoredNodes);
       s.add(streamer);
     }
     streamers = Collections.unmodifiableList(s);
@@ -316,7 +313,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       return;
     }
     for (StripedDataStreamer streamer : streamers) {
-      streamer.setLastException(new IOException("Lease timeout of "
+      streamer.getLastException().set(new IOException("Lease timeout of "
           + (dfsClient.getConf().getHdfsTimeout()/1000) +
           " seconds expired."));
     }
@@ -414,12 +411,8 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   @Override
   protected synchronized void closeImpl() throws IOException {
     if (isClosed()) {
-      IOException e = getLeadingStreamer().getLastException().getAndSet(null);
-      if (e != null) {
-        throw e;
-      } else {
-        return;
-      }
+      getLeadingStreamer().getLastException().check();
+      return;
     }
 
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e3fb43e/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 98eaa27..11d50e0 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
@@ -217,7 +217,6 @@ class DataStreamer extends Daemon {
   private volatile DatanodeInfo[] nodes = null; // list of targets for current block
   private volatile StorageType[] storageTypes = null;
   private volatile String[] storageIDs = null;
-  protected String[] favoredNodes;
   volatile boolean hasError = false;
   volatile int errorIndex = -1;
   // Restarting node index

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e3fb43e/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 19c205e..ef7e2a6 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
@@ -58,9 +58,10 @@ public class StripedDataStreamer extends DataStreamer {
                       Progressable progress, DataChecksum checksum,
                       AtomicReference<CachingStrategy> cachingStrategy,
                       ByteArrayManager byteArrayManage, short index,
-                      List<BlockingQueue<LocatedBlock>> stripedBlocks) {
-    super(stat,block, dfsClient, src, progress, checksum, cachingStrategy,
-        byteArrayManage);
+                      List<BlockingQueue<LocatedBlock>> stripedBlocks,
+                      String[] favoredNodes) {
+    super(stat, block, dfsClient, src, progress, checksum, cachingStrategy,
+        byteArrayManage, favoredNodes);
     this.index = index;
     this.stripedBlocks = stripedBlocks;
   }


[47/50] hadoop git commit: HADOOP-11920. Refactor some codes for erasure coders. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11920. Refactor some codes for erasure coders. Contributed by Kai Zheng


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

Branch: refs/heads/HDFS-7285
Commit: 49d0ac8d18344437c8c2824c64cf33582ef896bf
Parents: 94bee5d
Author: Kai Zheng <ka...@intel.com>
Authored: Thu May 7 21:02:50 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:40:59 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  2 +
 .../hadoop/fs/CommonConfigurationKeys.java      |  4 --
 .../apache/hadoop/io/erasurecode/ECChunk.java   |  2 +-
 .../erasurecode/coder/AbstractErasureCoder.java |  6 +-
 .../io/erasurecode/coder/RSErasureDecoder.java  | 40 +------------
 .../rawcoder/AbstractRawErasureCoder.java       | 62 +++++++++++++++++++-
 .../rawcoder/AbstractRawErasureDecoder.java     | 54 ++++++++++-------
 .../rawcoder/AbstractRawErasureEncoder.java     | 52 ++++++++--------
 .../erasurecode/rawcoder/RawErasureCoder.java   |  8 +--
 .../erasurecode/rawcoder/RawErasureDecoder.java | 24 +++++---
 .../io/erasurecode/rawcoder/XORRawDecoder.java  | 24 ++------
 .../io/erasurecode/rawcoder/XORRawEncoder.java  |  6 +-
 .../hadoop/io/erasurecode/TestCoderBase.java    |  4 +-
 .../erasurecode/coder/TestRSErasureCoder.java   |  6 +-
 14 files changed, 155 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index 7a344a8..06e7acf 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -46,3 +46,5 @@
     HADOOP-11841. Remove unused ecschema-def.xml files.  (szetszwo)
 
     HADOOP-11921 Enhance tests for erasure coders. (Kai Zheng)
+
+    HADOOP-11920 Refactor some codes for erasure coders. (Kai Zheng)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index bd2a24b..3f2871b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -143,10 +143,6 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   /** Supported erasure codec classes */
   public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs";
 
-  /** Use XOR raw coder when possible for the RS codec */
-  public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY =
-      "io.erasurecode.codec.rs.usexor";
-
   /** Raw coder factory for the RS codec */
   public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY =
       "io.erasurecode.codec.rs.rawcoder";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
index 01e8f35..436e13e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
@@ -71,7 +71,7 @@ public class ECChunk {
    * @param chunks
    * @return an array of byte array
    */
-  public static byte[][] toArray(ECChunk[] chunks) {
+  public static byte[][] toArrays(ECChunk[] chunks) {
     byte[][] bytesArr = new byte[chunks.length][];
 
     ByteBuffer buffer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
index 7403e35..d491570 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -90,11 +90,7 @@ public abstract class AbstractErasureCoder
       throw new RuntimeException("Failed to create raw coder", e);
     }
 
-    if (fact != null) {
-      return isEncoder ? fact.createEncoder() : fact.createDecoder();
-    }
-
-    return null;
+    return isEncoder ? fact.createEncoder() : fact.createDecoder();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
index fc664a5..ec7cbb5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -17,13 +17,11 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
-import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 
 /**
  * Reed-Solomon erasure decoder that decodes a block group.
@@ -32,38 +30,14 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
  */
 public class RSErasureDecoder extends AbstractErasureDecoder {
   private RawErasureDecoder rsRawDecoder;
-  private RawErasureDecoder xorRawDecoder;
-  private boolean useXorWhenPossible = true;
 
   @Override
-  public void setConf(Configuration conf) {
-    super.setConf(conf);
-
-    if (conf != null) {
-      this.useXorWhenPossible = conf.getBoolean(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, true);
-    }
-  }
-
-    @Override
   protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
 
-    RawErasureDecoder rawDecoder;
-
     ECBlock[] inputBlocks = getInputBlocks(blockGroup);
     ECBlock[] outputBlocks = getOutputBlocks(blockGroup);
 
-    /**
-     * Optimization: according to some benchmark, when only one block is erased
-     * and to be recovering, the most simple XOR scheme can be much efficient.
-     * We will have benchmark tests to verify this opt is effect or not.
-     */
-    if (outputBlocks.length == 1 && useXorWhenPossible) {
-      rawDecoder = checkCreateXorRawDecoder();
-    } else {
-      rawDecoder = checkCreateRSRawDecoder();
-    }
-
+    RawErasureDecoder rawDecoder = checkCreateRSRawDecoder();
     return new ErasureDecodingStep(inputBlocks,
         getErasedIndexes(inputBlocks), outputBlocks, rawDecoder);
   }
@@ -81,19 +55,9 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
     return rsRawDecoder;
   }
 
-  private RawErasureDecoder checkCreateXorRawDecoder() {
-    if (xorRawDecoder == null) {
-      xorRawDecoder = new XORRawDecoder();
-      xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize());
-    }
-    return xorRawDecoder;
-  }
-
   @Override
   public void release() {
-    if (xorRawDecoder != null) {
-      xorRawDecoder.release();
-    } else if (rsRawDecoder != null) {
+    if (rsRawDecoder != null) {
       rsRawDecoder.release();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
index e6f3d92..2378bb5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 
 import org.apache.hadoop.conf.Configured;
 
+import java.nio.ByteBuffer;
+
 /**
  * A common class of basic facilities to be shared by encoder and decoder
  *
@@ -27,6 +29,9 @@ import org.apache.hadoop.conf.Configured;
 public abstract class AbstractRawErasureCoder
     extends Configured implements RawErasureCoder {
 
+  // Hope to reset coding buffers a little faster using it
+  private byte[] zeroChunkBytes;
+
   private int numDataUnits;
   private int numParityUnits;
   private int chunkSize;
@@ -37,6 +42,8 @@ public abstract class AbstractRawErasureCoder
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
     this.chunkSize = chunkSize;
+
+    zeroChunkBytes = new byte[chunkSize]; // With ZERO by default
   }
 
   @Override
@@ -55,7 +62,7 @@ public abstract class AbstractRawErasureCoder
   }
 
   @Override
-  public boolean preferNativeBuffer() {
+  public boolean preferDirectBuffer() {
     return false;
   }
 
@@ -63,4 +70,57 @@ public abstract class AbstractRawErasureCoder
   public void release() {
     // Nothing to do by default
   }
+
+  /**
+   * Convert an array of heap ByteBuffers to an array of byte array.
+   * @param buffers
+   * @return an array of byte array
+   */
+  protected static byte[][] toArrays(ByteBuffer[] buffers) {
+    byte[][] bytesArr = new byte[buffers.length][];
+
+    ByteBuffer buffer;
+    for (int i = 0; i < buffers.length; i++) {
+      buffer = buffers[i];
+      if (buffer == null) {
+        bytesArr[i] = null;
+        continue;
+      }
+
+      if (buffer.hasArray()) {
+        bytesArr[i] = buffer.array();
+      } else {
+        throw new IllegalArgumentException("Invalid ByteBuffer passed, " +
+            "expecting heap buffer");
+      }
+    }
+
+    return bytesArr;
+  }
+
+  /**
+   * Ensure the buffer (either input or output) ready to read or write with ZERO
+   * bytes fully in chunkSize.
+   * @param buffer
+   * @return the buffer itself
+   */
+  protected ByteBuffer resetBuffer(ByteBuffer buffer) {
+    buffer.clear();
+    buffer.put(zeroChunkBytes);
+    buffer.position(0);
+
+    return buffer;
+  }
+
+  /**
+   * Ensure the buffer (either input or output) ready to read or write with ZERO
+   * bytes fully in chunkSize.
+   * @param buffer bytes array buffer
+   * @return the buffer itself
+   */
+  protected byte[] resetBuffer(byte[] buffer) {
+    System.arraycopy(zeroChunkBytes, 0, buffer, 0, buffer.length);
+
+    return buffer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
index 4613b25..b247543 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
@@ -32,27 +32,30 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   @Override
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs) {
-    if (erasedIndexes.length == 0) {
-      return;
-    }
+    checkParameters(inputs, erasedIndexes, outputs);
 
-    doDecode(inputs, erasedIndexes, outputs);
+    boolean hasArray = inputs[0].hasArray();
+    if (hasArray) {
+      byte[][] newInputs = toArrays(inputs);
+      byte[][] newOutputs = toArrays(outputs);
+      doDecode(newInputs, erasedIndexes, newOutputs);
+    } else {
+      doDecode(inputs, erasedIndexes, outputs);
+    }
   }
 
   /**
-   * Perform the real decoding using ByteBuffer
-   * @param inputs
+   * Perform the real decoding using Direct ByteBuffer.
+   * @param inputs Direct ByteBuffers expected
    * @param erasedIndexes
-   * @param outputs
+   * @param outputs Direct ByteBuffers expected
    */
   protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
                                    ByteBuffer[] outputs);
 
   @Override
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) {
-    if (erasedIndexes.length == 0) {
-      return;
-    }
+    checkParameters(inputs, erasedIndexes, outputs);
 
     doDecode(inputs, erasedIndexes, outputs);
   }
@@ -69,25 +72,32 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   @Override
   public void decode(ECChunk[] inputs, int[] erasedIndexes,
                      ECChunk[] outputs) {
-    doDecode(inputs, erasedIndexes, outputs);
+    ByteBuffer[] newInputs = ECChunk.toBuffers(inputs);
+    ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs);
+    decode(newInputs, erasedIndexes, newOutputs);
   }
 
   /**
-   * Perform the real decoding using chunks
+   * Check and validate decoding parameters, throw exception accordingly. The
+   * checking assumes it's a MDS code. Other code  can override this.
    * @param inputs
    * @param erasedIndexes
    * @param outputs
    */
-  protected void doDecode(ECChunk[] inputs, int[] erasedIndexes,
-                          ECChunk[] outputs) {
-    if (inputs[0].getBuffer().hasArray()) {
-      byte[][] inputBytesArr = ECChunk.toArray(inputs);
-      byte[][] outputBytesArr = ECChunk.toArray(outputs);
-      doDecode(inputBytesArr, erasedIndexes, outputBytesArr);
-    } else {
-      ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
-      ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
-      doDecode(inputBuffers, erasedIndexes, outputBuffers);
+  protected void checkParameters(Object[] inputs, int[] erasedIndexes,
+                                 Object[] outputs) {
+    if (inputs.length != getNumParityUnits() + getNumDataUnits()) {
+      throw new IllegalArgumentException("Invalid inputs length");
+    }
+
+    if (erasedIndexes.length != outputs.length) {
+      throw new IllegalArgumentException(
+          "erasedIndexes and outputs mismatch in length");
+    }
+
+    if (erasedIndexes.length > getNumParityUnits()) {
+      throw new IllegalArgumentException(
+          "Too many erased, not recoverable");
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
index 4feaf39..06e88bf 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
@@ -31,23 +31,28 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
 
   @Override
   public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    assert (inputs.length == getNumDataUnits());
-    assert (outputs.length == getNumParityUnits());
+    checkParameters(inputs, outputs);
 
-    doEncode(inputs, outputs);
+    boolean hasArray = inputs[0].hasArray();
+    if (hasArray) {
+      byte[][] newInputs = toArrays(inputs);
+      byte[][] newOutputs = toArrays(outputs);
+      doEncode(newInputs, newOutputs);
+    } else {
+      doEncode(inputs, outputs);
+    }
   }
 
   /**
-   * Perform the real encoding work using ByteBuffer
-   * @param inputs
-   * @param outputs
+   * Perform the real encoding work using direct ByteBuffer
+   * @param inputs Direct ByteBuffers expected
+   * @param outputs Direct ByteBuffers expected
    */
   protected abstract void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs);
 
   @Override
   public void encode(byte[][] inputs, byte[][] outputs) {
-    assert (inputs.length == getNumDataUnits());
-    assert (outputs.length == getNumParityUnits());
+    checkParameters(inputs, outputs);
 
     doEncode(inputs, outputs);
   }
@@ -61,33 +66,22 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
 
   @Override
   public void encode(ECChunk[] inputs, ECChunk[] outputs) {
-    assert (inputs.length == getNumDataUnits());
-    assert (outputs.length == getNumParityUnits());
-
-    doEncode(inputs, outputs);
+    ByteBuffer[] newInputs = ECChunk.toBuffers(inputs);
+    ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs);
+    encode(newInputs, newOutputs);
   }
 
   /**
-   * Perform the real encoding work using chunks.
+   * Check and validate decoding parameters, throw exception accordingly.
    * @param inputs
    * @param outputs
    */
-  protected void doEncode(ECChunk[] inputs, ECChunk[] outputs) {
-    /**
-     * Note callers may pass byte array, or ByteBuffer via ECChunk according
-     * to how ECChunk is created. Some implementations of coder use byte array
-     * (ex: pure Java), some use native ByteBuffer (ex: ISA-L), all for the
-     * better performance.
-     */
-    if (inputs[0].getBuffer().hasArray()) {
-      byte[][] inputBytesArr = ECChunk.toArray(inputs);
-      byte[][] outputBytesArr = ECChunk.toArray(outputs);
-      doEncode(inputBytesArr, outputBytesArr);
-    } else {
-      ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
-      ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
-      doEncode(inputBuffers, outputBuffers);
+  protected void checkParameters(Object[] inputs, Object[] outputs) {
+    if (inputs.length != getNumDataUnits()) {
+      throw new IllegalArgumentException("Invalid inputs length");
+    }
+    if (outputs.length != getNumParityUnits()) {
+      throw new IllegalArgumentException("Invalid outputs length");
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
index 9af5b6c..3fb211f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -64,13 +64,13 @@ public interface RawErasureCoder extends Configurable {
   public int getChunkSize();
 
   /**
-   * Tell if native or off-heap buffer is preferred or not. It's for callers to
-   * decide how to allocate coding chunk buffers, either on heap or off heap.
-   * It will return false by default.
+   * Tell if direct buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, using DirectByteBuffer or
+   * bytes array. It will return false by default.
    * @return true if native buffer is preferred for performance consideration,
    * otherwise false.
    */
-  public boolean preferNativeBuffer();
+  public boolean preferDirectBuffer();
 
   /**
    * Should be called when release this coder. Good chance to release encoding

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
index 1358b7d..1807da7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
@@ -31,24 +31,30 @@ import java.nio.ByteBuffer;
 public interface RawErasureDecoder extends RawErasureCoder {
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs
-   * @param inputs
-   * @param outputs
+   * Decode with inputs and erasedIndexes, generates outputs.
+   * @param inputs inputs to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes
    */
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs);
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs
-   * @param inputs
-   * @param outputs
+   * Decode with inputs and erasedIndexes, generates outputs.
+   * @param inputs inputs to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes
    */
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs);
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs
-   * @param inputs
-   * @param outputs
+   * Decode with inputs and erasedIndexes, generates outputs.
+   * @param inputs inputs to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes
    */
   public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
index b6b1633..2ea1b3d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
@@ -27,17 +27,11 @@ public class XORRawDecoder extends AbstractRawErasureDecoder {
   @Override
   protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
                           ByteBuffer[] outputs) {
-    assert(erasedIndexes.length == outputs.length);
-    assert(erasedIndexes.length <= 1);
+    resetBuffer(outputs[0]);
 
-    int bufSize = inputs[0].remaining();
+    int bufSize = getChunkSize();
     int erasedIdx = erasedIndexes[0];
 
-    // Set the output to zeros.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0].put(j, (byte) 0);
-    }
-
     // Process the inputs.
     for (int i = 0; i < inputs.length; i++) {
       // Skip the erased location.
@@ -52,19 +46,13 @@ public class XORRawDecoder extends AbstractRawErasureDecoder {
   }
 
   @Override
-  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
-                          byte[][] outputs) {
-    assert(erasedIndexes.length == outputs.length);
-    assert(erasedIndexes.length <= 1);
+  protected void doDecode(byte[][] inputs,
+                          int[] erasedIndexes, byte[][] outputs) {
+    resetBuffer(outputs[0]);
 
-    int bufSize = inputs[0].length;
+    int bufSize = getChunkSize();
     int erasedIdx = erasedIndexes[0];
 
-    // Set the output to zeros.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0][j] = 0;
-    }
-
     // Process the inputs.
     for (int i = 0; i < inputs.length; i++) {
       // Skip the erased location.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
index dbfab5d..116cb91 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
@@ -26,8 +26,9 @@ public class XORRawEncoder extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    int bufSize = inputs[0].remaining();
+    resetBuffer(outputs[0]);
 
+    int bufSize = getChunkSize();
     // Get the first buffer's data.
     for (int j = 0; j < bufSize; j++) {
       outputs[0].put(j, inputs[0].get(j));
@@ -43,8 +44,9 @@ public class XORRawEncoder extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(byte[][] inputs, byte[][] outputs) {
-    int bufSize = inputs[0].length;
+    resetBuffer(outputs[0]);
 
+    int bufSize = getChunkSize();
     // Get the first buffer's data.
     for (int j = 0; j < bufSize; j++) {
       outputs[0][j] = inputs[0][j];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
index be1924c..704b3f0 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -75,8 +75,8 @@ public abstract class TestCoderBase {
    */
   protected void compareAndVerify(ECChunk[] erasedChunks,
                                   ECChunk[] recoveredChunks) {
-    byte[][] erased = ECChunk.toArray(erasedChunks);
-    byte[][] recovered = ECChunk.toArray(recoveredChunks);
+    byte[][] erased = ECChunk.toArrays(erasedChunks);
+    byte[][] recovered = ECChunk.toArrays(recoveredChunks);
     boolean result = Arrays.deepEquals(erased, recovered);
     assertTrue("Decoding and comparing failed.", result);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49d0ac8d/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
index 7d9d37a..3ae6a93 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
@@ -56,14 +56,12 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
      * This tests if the two configuration items work or not.
      */
     Configuration conf = new Configuration();
-    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        RSRawErasureCoderFactory.class.getCanonicalName());
-    conf.setBoolean(
-        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false);
+    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, RSRawErasureCoderFactory.class.getCanonicalName());
 
     prepare(conf, 10, 4, new int[]{0});
 
     testCoding(true);
+    testCoding(true);
   }
 
   @Test


[39/50] hadoop git commit: HDFS-7672. Handle write failure for stripping blocks and refactor the existing code in DFSStripedOutputStream and StripedDataStreamer.

Posted by zh...@apache.org.
HDFS-7672. Handle write failure for stripping blocks and refactor the existing code in DFSStripedOutputStream and StripedDataStreamer.


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

Branch: refs/heads/HDFS-7285
Commit: 8eeea750288cbc7ffc17ab43b1b98802e174dea6
Parents: b7014e1
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue May 5 16:26:49 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:37:51 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  69 +--
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 501 ++++++++++++-------
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  10 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  15 +-
 .../apache/hadoop/hdfs/StripedDataStreamer.java | 156 ++----
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |   2 -
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |  18 +-
 .../TestDFSStripedOutputStreamWithFailure.java  | 323 ++++++++++++
 9 files changed, 764 insertions(+), 333 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8eeea750/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 a8df3f2..7efaa5a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -172,3 +172,6 @@
 
     HDFS-8324. Add trace info to DFSClient#getErasureCodingZoneInfo(..) (vinayakumarb via 
     umamahesh)
+    
+    HDFS-7672. Handle write failure for stripping blocks and refactor the
+    existing code in DFSStripedOutputStream and StripedDataStreamer.  (szetszwo)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8eeea750/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 0280d71..8580357 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
@@ -24,6 +24,8 @@ import java.nio.channels.ClosedChannelException;
 import java.util.EnumSet;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
@@ -86,6 +88,8 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 public class DFSOutputStream extends FSOutputSummer
     implements Syncable, CanSetDropBehind {
+  static final Log LOG = LogFactory.getLog(DFSOutputStream.class);
+
   /**
    * Number of times to retry creating a file when there are transient 
    * errors (typically related to encryption zones and KeyProvider operations).
@@ -419,24 +423,35 @@ public class DFSOutputStream extends FSOutputSummer
     streamer.incBytesCurBlock(len);
 
     // If packet is full, enqueue it for transmission
-    //
     if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
         streamer.getBytesCurBlock() == blockSize) {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("DFSClient writeChunk packet full seqno=" +
-            currentPacket.getSeqno() +
-            ", src=" + src +
-            ", bytesCurBlock=" + streamer.getBytesCurBlock() +
-            ", blockSize=" + blockSize +
-            ", appendChunk=" + streamer.getAppendChunk());
-      }
-      streamer.waitAndQueuePacket(currentPacket);
-      currentPacket = null;
+      enqueueCurrentPacketFull();
+    }
+  }
 
-      adjustChunkBoundary();
+  void enqueueCurrentPacket() throws IOException {
+    streamer.waitAndQueuePacket(currentPacket);
+    currentPacket = null;
+  }
 
-      endBlock();
+  void enqueueCurrentPacketFull() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("enqueue full " + currentPacket + ", src=" + src
+          + ", bytesCurBlock=" + streamer.getBytesCurBlock()
+          + ", blockSize=" + blockSize
+          + ", appendChunk=" + streamer.getAppendChunk()
+          + ", " + streamer);
     }
+    enqueueCurrentPacket();
+    adjustChunkBoundary();
+    endBlock();
+  }
+
+  /** create an empty packet to mark the end of the block */
+  void setCurrentPacket2Empty() throws InterruptedIOException {
+    currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
+        streamer.getAndIncCurrentSeqno(), true);
+    currentPacket.setSyncBlock(shouldSyncBlock);
   }
 
   /**
@@ -444,7 +459,7 @@ public class DFSOutputStream extends FSOutputSummer
    * write filled up its partial chunk. Tell the summer to generate full
    * crc chunks from now on.
    */
-  protected void adjustChunkBoundary() {
+  private void adjustChunkBoundary() {
     if (streamer.getAppendChunk() &&
         streamer.getBytesCurBlock() % bytesPerChecksum == 0) {
       streamer.setAppendChunk(false);
@@ -466,11 +481,8 @@ public class DFSOutputStream extends FSOutputSummer
    */
   protected void endBlock() throws IOException {
     if (streamer.getBytesCurBlock() == blockSize) {
-      currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
-          streamer.getAndIncCurrentSeqno(), true);
-      currentPacket.setSyncBlock(shouldSyncBlock);
-      streamer.waitAndQueuePacket(currentPacket);
-      currentPacket = null;
+      setCurrentPacket2Empty();
+      enqueueCurrentPacket();
       streamer.setBytesCurBlock(0);
       lastFlushOffset = 0;
     }
@@ -592,8 +604,7 @@ public class DFSOutputStream extends FSOutputSummer
         }
         if (currentPacket != null) {
           currentPacket.setSyncBlock(isSync);
-          streamer.waitAndQueuePacket(currentPacket);
-          currentPacket = null;
+          enqueueCurrentPacket();
         }
         if (endBlock && streamer.getBytesCurBlock() > 0) {
           // Need to end the current block, thus send an empty packet to
@@ -601,8 +612,7 @@ public class DFSOutputStream extends FSOutputSummer
           currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
               streamer.getAndIncCurrentSeqno(), true);
           currentPacket.setSyncBlock(shouldSyncBlock || isSync);
-          streamer.waitAndQueuePacket(currentPacket);
-          currentPacket = null;
+          enqueueCurrentPacket();
           streamer.setBytesCurBlock(0);
           lastFlushOffset = 0;
         } else {
@@ -779,15 +789,11 @@ public class DFSOutputStream extends FSOutputSummer
       flushBuffer();       // flush from all upper layers
 
       if (currentPacket != null) {
-        streamer.waitAndQueuePacket(currentPacket);
-        currentPacket = null;
+        enqueueCurrentPacket();
       }
 
       if (streamer.getBytesCurBlock() != 0) {
-        // send an empty packet to mark the end of the block
-        currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
-            streamer.getAndIncCurrentSeqno(), true);
-        currentPacket.setSyncBlock(shouldSyncBlock);
+        setCurrentPacket2Empty();
       }
 
       flushInternal();             // flush all data to Datanodes
@@ -901,4 +907,9 @@ public class DFSOutputStream extends FSOutputSummer
   public long getFileId() {
     return fileId;
   }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + ":" + streamer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8eeea750/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 71cdbb9..bbc8ba0 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
@@ -28,14 +28,16 @@ import java.util.EnumSet;
 import java.util.List;
 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.protocol.ECInfo;
 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.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.util.DataChecksum;
@@ -44,6 +46,8 @@ import org.apache.htrace.Sampler;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 
+import com.google.common.base.Preconditions;
+
 
 /****************************************************************
  * The DFSStripedOutputStream class supports writing files in striped
@@ -55,33 +59,154 @@ import org.apache.htrace.TraceScope;
 
 @InterfaceAudience.Private
 public class DFSStripedOutputStream extends DFSOutputStream {
+  /** Coordinate the communication between the streamers. */
+  static class Coordinator {
+    private final List<BlockingQueue<ExtendedBlock>> endBlocks;
+    private final List<BlockingQueue<LocatedBlock>> stripedBlocks;
+    private volatile boolean shouldLocateFollowingBlock = false;
+
+    Coordinator(final int numDataBlocks, final int numAllBlocks) {
+      endBlocks = new ArrayList<>(numDataBlocks);
+      for (int i = 0; i < numDataBlocks; i++) {
+        endBlocks.add(new LinkedBlockingQueue<ExtendedBlock>(1));
+      }
 
-  private final List<StripedDataStreamer> streamers;
-  /**
-   * Size of each striping cell, must be a multiple of bytesPerChecksum
-   */
-  private final ECInfo ecInfo;
-  private final int cellSize;
-  // checksum buffer, we only need to calculate checksum for parity blocks
-  private byte[] checksumBuf;
-  private ByteBuffer[] cellBuffers;
+      stripedBlocks = new ArrayList<>(numAllBlocks);
+      for (int i = 0; i < numAllBlocks; i++) {
+        stripedBlocks.add(new LinkedBlockingQueue<LocatedBlock>(1));
+      }
+    }
 
-  private final short numAllBlocks;
-  private final short numDataBlocks;
+    boolean shouldLocateFollowingBlock() {
+      return shouldLocateFollowingBlock;
+    }
 
-  private int curIdx = 0;
-  /* bytes written in current block group */
-  //private long currentBlockGroupBytes = 0;
+    void putEndBlock(int i, ExtendedBlock block) {
+      shouldLocateFollowingBlock = true;
 
-  //TODO: Use ErasureCoder interface (HDFS-7781)
-  private RawErasureEncoder encoder;
+      final boolean b = endBlocks.get(i).offer(block);
+      Preconditions.checkState(b, "Failed to add " + block
+          + " to endBlocks queue, i=" + i);
+    }
 
-  private StripedDataStreamer getLeadingStreamer() {
-    return streamers.get(0);
+    ExtendedBlock getEndBlock(int i) throws InterruptedIOException {
+      try {
+        return endBlocks.get(i).poll(30, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        throw DFSUtil.toInterruptedIOException(
+            "getEndBlock interrupted, i=" + i, e);
+      }
+    }
+
+    void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) {
+      ExtendedBlock b = endBlocks.get(i).peek();
+      if (b == null) {
+        // streamer just has failed, put end block and continue
+        b = block;
+        putEndBlock(i, b);
+      }
+      b.setNumBytes(newBytes);
+    }
+
+    void putStripedBlock(int i, LocatedBlock block) throws IOException {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("putStripedBlock " + block + ", i=" + i);
+      }
+      final boolean b = stripedBlocks.get(i).offer(block);
+      if (!b) {
+        throw new IOException("Failed: " + block + ", i=" + i);
+      }
+    }
+
+    LocatedBlock getStripedBlock(int i) throws IOException {
+      final LocatedBlock lb;
+      try {
+        lb = stripedBlocks.get(i).poll(90, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        throw DFSUtil.toInterruptedIOException("getStripedBlock interrupted", e);
+      }
+
+      if (lb == null) {
+        throw new IOException("Failed: i=" + i);
+      }
+      return lb;
+    }
   }
 
-  private long getBlockGroupSize() {
-    return blockSize * numDataBlocks;
+  /** Buffers for writing the data and parity cells of a strip. */
+  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;
+
+  /** Size of each striping cell, must be a multiple of bytesPerChecksum */
+  private final int cellSize;
+  private final int numAllBlocks;
+  private final int numDataBlocks;
+
+  private StripedDataStreamer getLeadingStreamer() {
+    return streamers.get(0);
   }
 
   /** Construct a new output stream for creating a file. */
@@ -90,82 +215,94 @@ public class DFSStripedOutputStream extends DFSOutputStream {
                          DataChecksum checksum, String[] favoredNodes)
                          throws IOException {
     super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
-    DFSClient.LOG.info("Creating striped output stream");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating DFSStripedOutputStream for " + src);
+    }
 
     // ECInfo is restored from NN just before writing striped files.
-    ecInfo = dfsClient.getErasureCodingInfo(src);
-    cellSize = ecInfo.getSchema().getChunkSize();
-    numAllBlocks = (short)(ecInfo.getSchema().getNumDataUnits()
-        + ecInfo.getSchema().getNumParityUnits());
-    numDataBlocks = (short)ecInfo.getSchema().getNumDataUnits();
+    //TODO reduce an rpc call HDFS-8289
+    final ECSchema schema = dfsClient.getErasureCodingInfo(src).getSchema();
+    final int numParityBlocks = schema.getNumParityUnits();
+    cellSize = schema.getChunkSize();
+    numDataBlocks = schema.getNumDataUnits();
+    numAllBlocks = numDataBlocks + numParityBlocks;
 
-    checkConfiguration();
-
-    checksumBuf = new byte[getChecksumSize() * (cellSize / bytesPerChecksum)];
-    cellBuffers = new ByteBuffer[numAllBlocks];
-    List<BlockingQueue<LocatedBlock>> stripeBlocks = new ArrayList<>();
+    encoder = new RSRawEncoder();
+    encoder.initialize(numDataBlocks, numParityBlocks, cellSize);
 
-    for (int i = 0; i < numAllBlocks; i++) {
-      stripeBlocks.add(new LinkedBlockingQueue<LocatedBlock>(numAllBlocks));
-      try {
-        cellBuffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
-      } catch (InterruptedException ie) {
-        final InterruptedIOException iioe = new InterruptedIOException(
-            "create cell buffers");
-        iioe.initCause(ie);
-        throw iioe;
-      }
+    coordinator = new Coordinator(numDataBlocks, numAllBlocks);
+    try {
+      cellBuffers = new CellBuffers(numParityBlocks);
+    } catch (InterruptedException ie) {
+      throw DFSUtil.toInterruptedIOException("Failed to create cell buffers", ie);
     }
-    encoder = new RSRawEncoder();
-    encoder.initialize(numDataBlocks,
-        numAllBlocks - numDataBlocks, cellSize);
 
     List<StripedDataStreamer> s = new ArrayList<>(numAllBlocks);
     for (short i = 0; i < numAllBlocks; i++) {
-      StripedDataStreamer streamer = new StripedDataStreamer(stat, null,
+      StripedDataStreamer streamer = new StripedDataStreamer(stat,
           dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
-          i, stripeBlocks, favoredNodes);
+          favoredNodes, i, coordinator);
       s.add(streamer);
     }
     streamers = Collections.unmodifiableList(s);
+    setCurrentStreamer(0);
+  }
 
-    refreshStreamer();
+  StripedDataStreamer getStripedDataStreamer(int i) {
+    return streamers.get(i);
   }
 
-  private void checkConfiguration() {
-    if (cellSize % bytesPerChecksum != 0) {
-      throw new HadoopIllegalArgumentException("Invalid values: "
-          + DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
-          + ") must divide cell size (=" + cellSize + ").");
-    }
+  int getCurrentIndex() {
+    return getCurrentStreamer().getIndex();
   }
 
-  private void refreshStreamer() {
-    streamer = streamers.get(curIdx);
+  StripedDataStreamer getCurrentStreamer() {
+    return (StripedDataStreamer)streamer;
   }
 
-  private void moveToNextStreamer() {
-    curIdx = (curIdx + 1) % numAllBlocks;
-    refreshStreamer();
+  private StripedDataStreamer setCurrentStreamer(int i) {
+    streamer = streamers.get(i);
+    return getCurrentStreamer();
   }
 
   /**
-   * encode the buffers.
-   * After encoding, flip each buffer.
+   * Encode the buffers, i.e. compute parities.
    *
    * @param buffers data buffers + parity buffers
    */
-  private void encode(ByteBuffer[] buffers) {
-    ByteBuffer[] dataBuffers = new ByteBuffer[numDataBlocks];
-    ByteBuffer[] parityBuffers = new ByteBuffer[numAllBlocks - numDataBlocks];
-    for (int i = 0; i < numAllBlocks; i++) {
-      if (i < numDataBlocks) {
-        dataBuffers[i] = buffers[i];
-      } else {
-        parityBuffers[i - numDataBlocks] = buffers[i];
+  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()) {
+        count++;
       }
     }
-    encoder.encode(dataBuffers, parityBuffers);
+    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().setIsFailed(true);
+    checkStreamers();
+    currentPacket = null;
   }
 
   /**
@@ -173,11 +310,12 @@ public class DFSStripedOutputStream extends DFSOutputStream {
    * writing parity blocks.
    *
    * @param byteBuffer the given buffer to generate packets
+   * @param checksumBuf the checksum buffer
    * @return packets generated
    * @throws IOException
    */
-  private List<DFSPacket> generatePackets(ByteBuffer byteBuffer)
-      throws IOException{
+  private List<DFSPacket> generatePackets(
+      ByteBuffer byteBuffer, byte[] checksumBuf) throws IOException{
     List<DFSPacket> packets = new ArrayList<>();
     assert byteBuffer.hasArray();
     getDataChecksum().calculateChunkedSums(byteBuffer.array(), 0,
@@ -201,82 +339,47 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   }
 
   @Override
-  protected synchronized void writeChunk(byte[] b, int offset, int len,
+  protected synchronized void writeChunk(byte[] bytes, int offset, int len,
       byte[] checksum, int ckoff, int cklen) throws IOException {
-    super.writeChunk(b, offset, len, checksum, ckoff, cklen);
-
-    if (getSizeOfCellnBuffer(curIdx) <= cellSize) {
-      addToCellBuffer(b, offset, len);
-    } else {
-      String msg = "Writing a chunk should not overflow the cell buffer.";
-      DFSClient.LOG.info(msg);
-      throw new IOException(msg);
-    }
-
-    // If current packet has not been enqueued for transmission,
-    // but the cell buffer is full, we need to enqueue the packet
-    if (currentPacket != null && getSizeOfCellnBuffer(curIdx) == cellSize) {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("DFSClient writeChunk cell buffer full seqno=" +
-            currentPacket.getSeqno() +
-            ", curIdx=" + curIdx +
-            ", src=" + src +
-            ", bytesCurBlock=" + streamer.getBytesCurBlock() +
-            ", blockSize=" + blockSize +
-            ", appendChunk=" + streamer.getAppendChunk());
+    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);
+
+        // cell is full and current packet has not been enqueued,
+        if (cellFull && currentPacket != null) {
+          enqueueCurrentPacketFull();
+        }
+      } catch(Exception e) {
+        handleStreamerFailure("offset=" + offset + ", length=" + len, e);
       }
-      streamer.waitAndQueuePacket(currentPacket);
-      currentPacket = null;
-      adjustChunkBoundary();
-      endBlock();
+    }
+
+    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 (getSizeOfCellnBuffer(curIdx) == cellSize) {
-      //move curIdx to next cell
-      moveToNextStreamer();
+    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 (curIdx == numDataBlocks) {
-        //encode the data cells
-        for (int k = 0; k < numDataBlocks; k++) {
-          cellBuffers[k].flip();
-        }
-        encode(cellBuffers);
-        for (int i = numDataBlocks; i < numAllBlocks; i++) {
-          ByteBuffer parityBuffer = cellBuffers[i];
-          List<DFSPacket> packets = generatePackets(parityBuffer);
-          for (DFSPacket p : packets) {
-            currentPacket = p;
-            streamer.waitAndQueuePacket(currentPacket);
-            currentPacket = null;
-          }
-          endBlock();
-          moveToNextStreamer();
-        }
-        //read next stripe to cellBuffers
-        clearCellBuffers();
-      }
-    }
-  }
-
-  private void addToCellBuffer(byte[] b, int off, int len) {
-    cellBuffers[curIdx].put(b, off, len);
-  }
-
-  private int getSizeOfCellnBuffer(int cellIndex) {
-    return cellBuffers[cellIndex].position();
-  }
-
-  private void clearCellBuffers() {
-    for (int i = 0; i< numAllBlocks; i++) {
-      cellBuffers[i].clear();
-      if (i >= numDataBlocks) {
-        Arrays.fill(cellBuffers[i].array(), (byte) 0);
+      if (next == numDataBlocks) {
+        cellBuffers.flipDataBuffers();
+        writeParityCells();
+        next = 0;
       }
+      setCurrentStreamer(next);
     }
   }
 
@@ -284,20 +387,14 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     return numDataBlocks * cellSize;
   }
 
-  private void notSupported(String headMsg)
-      throws IOException{
-      throw new IOException(
-          headMsg + " is now not supported for striping layout.");
-  }
-
   @Override
-  public void hflush() throws IOException {
-    notSupported("hflush");
+  public void hflush() {
+    throw new UnsupportedOperationException();
   }
 
   @Override
-  public void hsync() throws IOException {
-    notSupported("hsync");
+  public void hsync() {
+    throw new UnsupportedOperationException();
   }
 
   @Override
@@ -327,29 +424,28 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     return closed || getLeadingStreamer().streamerClosed();
   }
 
-  // shutdown datastreamer and responseprocessor threads.
-  // interrupt datastreamer if force is true
   @Override
   protected void closeThreads(boolean force) throws IOException {
-    int index = 0;
-    boolean exceptionOccurred = false;
+    final MultipleIOException.Builder b = new MultipleIOException.Builder();
     for (StripedDataStreamer streamer : streamers) {
       try {
         streamer.close(force);
         streamer.join();
         streamer.closeSocket();
-      } catch (InterruptedException | IOException e) {
-        DFSClient.LOG.error("Failed to shutdown streamer: name="
-            + streamer.getName() + ", index=" + index + ", file=" + src, e);
-        exceptionOccurred = true;
+      } catch(Exception e) {
+        try {
+          handleStreamerFailure("force=" + force, e);
+        } catch(IOException ioe) {
+          b.add(ioe);
+        }
       } finally {
         streamer.setSocketToNull();
         setClosed();
-        index++;
       }
     }
-    if (exceptionOccurred) {
-      throw new IOException("Failed to shutdown streamer");
+    final IOException ioe = b.build();
+    if (ioe != null) {
+      throw ioe;
     }
   }
 
@@ -370,50 +466,69 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     if (currentBlockGroupBytes % stripeDataSize() == 0) {
       return;
     }
-    long firstCellSize = getLeadingStreamer().getBytesCurBlock() % cellSize;
-    long parityCellSize = firstCellSize > 0 && firstCellSize < cellSize ?
+
+    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
-      int position = cellBuffers[i].position();
+      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++) {
-        cellBuffers[i].put((byte) 0);
+        buffers[i].put((byte) 0);
       }
-      cellBuffers[i].flip();
+      buffers[i].flip();
     }
-    encode(cellBuffers);
 
-    // write parity cells
-    curIdx = numDataBlocks;
-    refreshStreamer();
+    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++) {
-      ByteBuffer parityBuffer = cellBuffers[i];
-      List<DFSPacket> packets = generatePackets(parityBuffer);
-      for (DFSPacket p : packets) {
-        currentPacket = p;
-        streamer.waitAndQueuePacket(currentPacket);
-        currentPacket = null;
+      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 {
+        for (DFSPacket p : generatePackets(buffer, checksumBuf)) {
+          streamer.waitAndQueuePacket(p);
+        }
+        endBlock();
+      } catch(Exception e) {
+        handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e);
       }
-      endBlock();
-      moveToNextStreamer();
     }
 
-    clearCellBuffers();
+    if (current.isFailed()) {
+      final long newBytes = oldBytes + len;
+      current.setBytesCurBlock(newBytes);
+    }
   }
 
   @Override
   void setClosed() {
     super.setClosed();
     for (int i = 0; i < numAllBlocks; i++) {
-      byteArrayManager.release(cellBuffers[i].array());
       streamers.get(i).release();
     }
+    cellBuffers.release();
   }
 
   @Override
@@ -425,25 +540,31 @@ public class DFSStripedOutputStream extends DFSOutputStream {
 
     try {
       // flush from all upper layers
-      flushBuffer();
-      if (currentPacket != null) {
-        streamer.waitAndQueuePacket(currentPacket);
-        currentPacket = null;
+      try {
+        flushBuffer();
+        if (currentPacket != null) {
+          enqueueCurrentPacket();
+        }
+      } catch(Exception e) {
+        handleStreamerFailure("closeImpl", e);
       }
+
       // if the last stripe is incomplete, generate and write parity cells
       writeParityCellsForLastStripe();
 
       for (int i = 0; i < numAllBlocks; i++) {
-        curIdx = i;
-        refreshStreamer();
-        if (streamer.getBytesCurBlock() > 0) {
-          // send an empty packet to mark the end of the block
-          currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
-              streamer.getAndIncCurrentSeqno(), true);
-          currentPacket.setSyncBlock(shouldSyncBlock);
+        final StripedDataStreamer s = setCurrentStreamer(i);
+        if (!s.isFailed()) {
+          try {
+            if (s.getBytesCurBlock() > 0) {
+              setCurrentPacket2Empty();
+            }
+            // flush all data to Datanode
+            flushInternal();
+          } catch(Exception e) {
+            handleStreamerFailure("closeImpl", e);
+          }
         }
-        // flush all data to Datanode
-        flushInternal();
       }
 
       closeThreads(false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8eeea750/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index a925a60..d2ce9fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -36,6 +36,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_PAS
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.io.PrintStream;
 import java.io.UnsupportedEncodingException;
 import java.net.InetAddress;
@@ -55,7 +56,6 @@ import java.util.Set;
 
 import javax.net.SocketFactory;
 
-import com.google.common.collect.Sets;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.Option;
@@ -96,6 +96,7 @@ import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 import com.google.protobuf.BlockingService;
 
 @InterfaceAudience.Private
@@ -1525,4 +1526,11 @@ public class DFSUtil {
   public static int getSmallBufferSize(Configuration conf) {
     return Math.min(getIoFileBufferSize(conf) / 2, 512);
   }
+
+  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/8eeea750/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 11d50e0..ddab845 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
@@ -451,7 +451,7 @@ class DataStreamer extends Daemon {
         // get new block from namenode.
         if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
           if(LOG.isDebugEnabled()) {
-            LOG.debug("Allocating new block");
+            LOG.debug("Allocating new block " + this);
           }
           setPipeline(nextBlockOutputStream());
           initDataStreaming();
@@ -469,10 +469,7 @@ class DataStreamer extends Daemon {
         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()) {
@@ -1684,7 +1681,7 @@ class DataStreamer extends Daemon {
       dataQueue.addLast(packet);
       lastQueuedSeqno = packet.getSeqno();
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Queued packet " + packet.getSeqno());
+        LOG.debug("Queued " + packet + ", " + this);
       }
       dataQueue.notifyAll();
     }
@@ -1834,4 +1831,10 @@ class DataStreamer extends Daemon {
       s.close();
     }
   }
+
+  @Override
+  public String toString() {
+    return  (block == null? null: block.getLocalBlock())
+        + "@" + Arrays.toString(getNodes());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8eeea750/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 ef7e2a6..258fc65 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,8 +18,14 @@
 
 package org.apache.hadoop.hdfs;
 
-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 java.io.IOException;
+import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -31,15 +37,6 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 
-import java.io.IOException;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-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;
-
 /****************************************************************************
  * The StripedDataStreamer class is used by {@link DFSStripedOutputStream}.
  * There are two kinds of StripedDataStreamer, leading streamer and ordinary
@@ -49,40 +46,32 @@ import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
  *
  ****************************************************************************/
 public class StripedDataStreamer extends DataStreamer {
-  private final short index;
-  private final List<BlockingQueue<LocatedBlock>> stripedBlocks;
-  private boolean hasCommittedBlock = false;
+  private final Coordinator coordinator;
+  private final int index;
+  private volatile boolean isFailed;
 
-  StripedDataStreamer(HdfsFileStatus stat, ExtendedBlock block,
+  StripedDataStreamer(HdfsFileStatus stat,
                       DFSClient dfsClient, String src,
                       Progressable progress, DataChecksum checksum,
                       AtomicReference<CachingStrategy> cachingStrategy,
-                      ByteArrayManager byteArrayManage, short index,
-                      List<BlockingQueue<LocatedBlock>> stripedBlocks,
-                      String[] favoredNodes) {
-    super(stat, block, dfsClient, src, progress, checksum, cachingStrategy,
+                      ByteArrayManager byteArrayManage, String[] favoredNodes,
+                      short index, Coordinator coordinator) {
+    super(stat, null, dfsClient, src, progress, checksum, cachingStrategy,
         byteArrayManage, favoredNodes);
     this.index = index;
-    this.stripedBlocks = stripedBlocks;
+    this.coordinator = coordinator;
   }
 
-  /**
-   * 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
-   */
-  StripedDataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat,
-                      DFSClient dfsClient, String src,
-                      Progressable progress, DataChecksum checksum,
-                      AtomicReference<CachingStrategy> cachingStrategy,
-                      ByteArrayManager byteArrayManage, short index,
-                      List<BlockingQueue<LocatedBlock>> stripedBlocks)
-      throws IOException {
-    super(lastBlock, stat, dfsClient, src, progress, checksum, cachingStrategy,
-        byteArrayManage);
-    this.index = index;
-    this.stripedBlocks = stripedBlocks;
+  int getIndex() {
+    return index;
+  }
+
+  void setIsFailed(boolean isFailed) {
+    this.isFailed = isFailed;
+  }
+
+  boolean isFailed() {
+    return isFailed;
   }
 
   public boolean isLeadingStreamer () {
@@ -95,18 +84,8 @@ public class StripedDataStreamer extends DataStreamer {
 
   @Override
   protected void endBlock() {
-    if (!isLeadingStreamer() && !isParityStreamer()) {
-      // before retrieving a new block, transfer the finished block to
-      // leading streamer
-      LocatedBlock finishedBlock = new LocatedBlock(
-          new ExtendedBlock(block.getBlockPoolId(), block.getBlockId(),
-              block.getNumBytes(), block.getGenerationStamp()), null);
-      try {
-        boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30,
-            TimeUnit.SECONDS);
-      } catch (InterruptedException ie) {
-        // TODO: Handle InterruptedException (HDFS-7786)
-      }
+    if (!isParityStreamer()) {
+      coordinator.putEndBlock(index, block);
     }
     super.endBlock();
   }
@@ -114,71 +93,40 @@ public class StripedDataStreamer extends DataStreamer {
   @Override
   protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
       throws IOException {
-    LocatedBlock lb = null;
     if (isLeadingStreamer()) {
-      if (hasCommittedBlock) {
-        /**
-         * when committing a block group, leading streamer has to adjust
-         * {@link block} to include the size of block group
-         */
-        for (int i = 1; i < NUM_DATA_BLOCKS; i++) {
-          try {
-            LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30,
-                TimeUnit.SECONDS);
-            if (finishedLocatedBlock == null) {
-              throw new IOException("Fail to get finished LocatedBlock " +
-                  "from streamer, i=" + i);
-            }
-            ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock();
-            long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes();
-            if (block != null) {
-              block.setNumBytes(block.getNumBytes() + bytes);
-            }
-          } catch (InterruptedException ie) {
-            DFSClient.LOG.info("InterruptedException received when putting" +
-                " a block to stripeBlocks, ie = " + ie);
-          }
+      if (coordinator.shouldLocateFollowingBlock()) {
+        // set numByte for the previous block group
+        long bytes = 0;
+        for (int i = 0; i < NUM_DATA_BLOCKS; i++) {
+          final ExtendedBlock b = coordinator.getEndBlock(i);
+          bytes += b == null ? 0 : b.getNumBytes();
         }
+        block.setNumBytes(bytes);
       }
 
-      lb = super.locateFollowingBlock(excludedNodes);
-      hasCommittedBlock = true;
-      assert lb instanceof LocatedStripedBlock;
-      DFSClient.LOG.debug("Leading streamer obtained bg " + lb);
-      LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
-          (LocatedStripedBlock) lb, BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS,
-          NUM_PARITY_BLOCKS);
+      final LocatedStripedBlock lsb
+          = (LocatedStripedBlock)super.locateFollowingBlock(excludedNodes);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Obtained block group " + lsb);
+      }
+      LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(lsb,
+          BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+
       assert blocks.length == (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) :
           "Fail to get block group from namenode: blockGroupSize: " +
               (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) + ", blocks.length: " +
               blocks.length;
-      lb = blocks[0];
-      for (int i = 1; i < blocks.length; i++) {
-        try {
-          boolean offSuccess = stripedBlocks.get(i).offer(blocks[i],
-              90, TimeUnit.SECONDS);
-          if(!offSuccess){
-            String msg = "Fail to put block to stripeBlocks. i = " + i;
-            DFSClient.LOG.info(msg);
-            throw new IOException(msg);
-          } else {
-            DFSClient.LOG.info("Allocate a new block to a streamer. i = " + i
-                + ", block: " + blocks[i]);
-          }
-        } catch (InterruptedException ie) {
-          DFSClient.LOG.info("InterruptedException received when putting" +
-              " a block to stripeBlocks, ie = " + ie);
-        }
-      }
-    } else {
-      try {
-        // wait 90 seconds to get a block from the queue
-        lb = stripedBlocks.get(index).poll(90, TimeUnit.SECONDS);
-      } catch (InterruptedException ie) {
-        DFSClient.LOG.info("InterruptedException received when retrieving " +
-            "a block from stripeBlocks, ie = " + ie);
+      for (int i = 0; i < blocks.length; i++) {
+        coordinator.putStripedBlock(i, blocks[i]);
       }
     }
-    return lb;
+
+    return coordinator.getStripedBlock(index);
+  }
+
+  @Override
+  public String toString() {
+    return "#" + index + ": isFailed? " + Boolean.toString(isFailed).charAt(0)
+        + ", " + super.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8eeea750/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 12ad23e..0edc542 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
@@ -1998,8 +1998,6 @@ public class MiniDFSCluster {
     int node = -1;
     for (int i = 0; i < dataNodes.size(); i++) {
       DataNode dn = dataNodes.get(i).datanode;
-      LOG.info("DN name=" + dnName + " found DN=" + dn +
-          " with name=" + dn.getDisplayName());
       if (dnName.equals(dn.getDatanodeId().getXferAddr())) {
         node = i;
         break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8eeea750/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 5ce94ee..ec98e68 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
@@ -35,6 +35,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 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;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -42,6 +44,12 @@ import org.junit.Test;
 
 public class TestDFSStripedOutputStream {
   public static final Log LOG = LogFactory.getLog(TestDFSStripedOutputStream.class);
+
+  static {
+    GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
+  }
+
   private int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
   private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
 
@@ -245,6 +253,11 @@ public class TestDFSStripedOutputStream {
 
   static void verifyParity(final long size, final int cellSize,
       byte[][] dataBytes, byte[][] parityBytes) {
+    verifyParity(size, cellSize, dataBytes, parityBytes, -1);
+  }
+
+  static void verifyParity(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);
@@ -265,7 +278,10 @@ public class TestDFSStripedOutputStream {
     encoder.initialize(dataBytes.length, parityBytes.length, cellSize);
     encoder.encode(dataBytes, expectedParityBytes);
     for (int i = 0; i < parityBytes.length; i++) {
-      Assert.assertArrayEquals(expectedParityBytes[i], parityBytes[i]);
+      if (i != killedDnIndex) {
+        Assert.assertArrayEquals("i=" + i + ", killedDnIndex=" + killedDnIndex,
+            expectedParityBytes[i], parityBytes[i]);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8eeea750/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
new file mode 100644
index 0000000..c2e588a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -0,0 +1,323 @@
+/**
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+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.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.util.StripedBlockUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestDFSStripedOutputStreamWithFailure {
+  public static final Log LOG = LogFactory.getLog(
+      TestDFSStripedOutputStreamWithFailure.class);
+  static {
+    GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL);
+    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 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;
+
+  private final HdfsConfiguration conf = new HdfsConfiguration();
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem dfs;
+  private final Path dir = new Path("/"
+      + TestDFSStripedOutputStreamWithFailure.class.getSimpleName());
+
+
+  @Before
+  public void setup() throws IOException {
+    final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.waitActive();
+    dfs = cluster.getFileSystem();
+    dfs.mkdirs(dir);
+    dfs.createErasureCodingZone(dir, null);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private static byte getByte(long pos) {
+    return (byte)pos;
+  }
+
+  @Test(timeout=120000)
+  public void testDatanodeFailure1() {
+    final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
+    final int dn = 1;
+    runTest("file" + dn, length, dn);
+  }
+
+  @Test(timeout=120000)
+  public void testDatanodeFailure2() {
+    final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
+    final int dn = 2;
+    runTest("file" + dn, length, dn);
+  }
+
+  @Test(timeout=120000)
+  public void testDatanodeFailure3() {
+    final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
+    final int dn = 3;
+    runTest("file" + dn, length, dn);
+  }
+
+  @Test(timeout=120000)
+  public void testDatanodeFailure4() {
+    final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
+    final int dn = 4;
+    runTest("file" + dn, length, dn);
+  }
+
+  @Test(timeout=120000)
+  public void testDatanodeFailure5() {
+    final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
+    final int dn = 5;
+    runTest("file" + dn, length, dn);
+  }
+
+  @Test(timeout=120000)
+  public void testDatanodeFailure6() {
+    final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
+    final int dn = 6;
+    runTest("file" + dn, length, dn);
+  }
+
+  @Test(timeout=120000)
+  public void testDatanodeFailure7() {
+    final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
+    final int dn = 7;
+    runTest("file" + dn, length, dn);
+  }
+
+  @Test(timeout=120000)
+  public void testDatanodeFailure8() {
+    final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
+    final int dn = 8;
+    runTest("file" + dn, length, dn);
+  }
+
+  private void runTest(final String src, final int length, final int dnIndex) {
+    try {
+      cluster.startDataNodes(conf, 1, true, null, null);
+      cluster.waitActive();
+
+      runTest(new Path(dir, src), length, dnIndex);
+    } catch(Exception e) {
+      LOG.info("FAILED", e);
+      Assert.fail(StringUtils.stringifyException(e));
+    }
+  }
+
+  private void runTest(final Path p, final int length,
+      final int dnIndex) throws Exception {
+    LOG.info("p=" + p + ", length=" + length + ", dnIndex=" + dnIndex);
+    final String fullPath = p.toString();
+
+    final AtomicInteger pos = new AtomicInteger();
+    final FSDataOutputStream out = dfs.create(p);
+    final AtomicBoolean killed = new AtomicBoolean();
+    final Thread killer = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        killDatanode(cluster, (DFSStripedOutputStream)out.getWrappedStream(),
+            dnIndex, pos);
+        killed.set(true);
+      }
+    });
+    killer.start();
+
+    final int mask = (1 << 16) - 1;
+    for(; pos.get() < length; ) {
+      final int i = pos.getAndIncrement();
+      write(out, i);
+      if ((i & mask) == 0) {
+        final long ms = 100;
+        LOG.info("i=" + i + " sleep " + ms);
+        Thread.sleep(ms);
+      }
+    }
+    killer.join(10000);
+    Assert.assertTrue(killed.get());
+    out.close();
+
+    // check file length
+    final FileStatus status = dfs.getFileStatus(p);
+    Assert.assertEquals(length, status.getLen());
+
+    checkData(dfs, fullPath, length, dnIndex);
+  }
+
+  static void write(FSDataOutputStream out, int i) throws IOException {
+    try {
+      out.write(getByte(i));
+    } catch(IOException ioe) {
+      throw new IOException("Failed at i=" + i, ioe);
+    }
+  }
+
+  static DatanodeInfo getDatanodes(StripedDataStreamer streamer) {
+    for(;;) {
+      final DatanodeInfo[] datanodes = streamer.getNodes();
+      if (datanodes != null) {
+        Assert.assertEquals(1, datanodes.length);
+        Assert.assertNotNull(datanodes[0]);
+        return datanodes[0];
+      }
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException ignored) {
+        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) 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);
+      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 = (int)((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 && i == 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("XXX i=" + i + ", 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(
+          lbs.getLocatedBlocks().get(group).getBlockSize(),
+          CELL_SIZE, dataBlockBytes, parityBlockBytes,
+          killedDnIndex - dataBlockBytes.length);
+    }
+  }
+}


[49/50] hadoop git commit: HDFS-8355. Erasure Coding: Refactor BlockInfo and BlockInfoUnderConstruction. Contributed by Tsz Wo Nicholas Sze.

Posted by zh...@apache.org.
HDFS-8355. Erasure Coding: Refactor BlockInfo and BlockInfoUnderConstruction. 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/c31cb274
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c31cb274
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c31cb274

Branch: refs/heads/HDFS-7285
Commit: c31cb27427e5f48c05b71b26966bc630f14c58ee
Parents: 9528767
Author: Jing Zhao <ji...@apache.org>
Authored: Fri May 8 13:56:56 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:41:00 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +
 .../hdfs/server/blockmanagement/BlockInfo.java  | 95 +-------------------
 .../BlockInfoContiguousUnderConstruction.java   | 27 ++----
 .../BlockInfoStripedUnderConstruction.java      | 25 ++----
 .../BlockInfoUnderConstruction.java             | 27 ++++++
 .../server/blockmanagement/BlockManager.java    | 51 ++++++++---
 .../hdfs/server/namenode/FSNamesystem.java      | 52 +++++------
 7 files changed, 108 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c31cb274/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 ab8a748..c7d01c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -192,3 +192,6 @@
 
     HDFS-8289. Erasure Coding: add ECSchema to HdfsFileStatus. (Yong Zhang via
     jing9)
+
+    HDFS-8355. Erasure Coding: Refactor BlockInfo and BlockInfoUnderConstruction.
+    (Tsz Wo Nicholas Sze via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c31cb274/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 8b71925..aebfbb1 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,13 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import java.util.LinkedList;
+
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.LightWeightGSet;
 
-import java.io.IOException;
-import java.util.LinkedList;
-
 /**
  * For a given block (or an erasure coding block group), BlockInfo class
  * maintains 1) the {@link BlockCollection} it is part of, and 2) datanodes
@@ -336,94 +335,4 @@ public abstract class BlockInfo extends Block
   public void setNext(LightWeightGSet.LinkedElement next) {
     this.nextLinkedElement = next;
   }
-
-  static BlockInfo copyOf(BlockInfo b) {
-    if (!b.isStriped()) {
-      return new BlockInfoContiguous((BlockInfoContiguous) b);
-    } else {
-      return new BlockInfoStriped((BlockInfoStriped) b);
-    }
-  }
-
-  static BlockInfo convertToCompleteBlock(BlockInfo blk) throws IOException {
-    if (blk instanceof BlockInfoContiguousUnderConstruction) {
-      return ((BlockInfoContiguousUnderConstruction) blk)
-          .convertToCompleteBlock();
-    } else if (blk instanceof BlockInfoStripedUnderConstruction) {
-      return ((BlockInfoStripedUnderConstruction) blk).convertToCompleteBlock();
-    } else {
-      return blk;
-    }
-  }
-
-  static void commitBlock(BlockInfo blockInfo, Block reported)
-      throws IOException {
-    if (blockInfo instanceof BlockInfoContiguousUnderConstruction) {
-      ((BlockInfoContiguousUnderConstruction) blockInfo).commitBlock(reported);
-    } else if (blockInfo instanceof BlockInfoStripedUnderConstruction) {
-      ((BlockInfoStripedUnderConstruction) blockInfo).commitBlock(reported);
-    }
-  }
-
-  static void addReplica(BlockInfo ucBlock, DatanodeStorageInfo storageInfo,
-      Block reportedBlock, HdfsServerConstants.ReplicaState reportedState) {
-    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
-        ucBlock instanceof BlockInfoStripedUnderConstruction;
-    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
-      ((BlockInfoContiguousUnderConstruction) ucBlock).addReplicaIfNotPresent(
-          storageInfo, reportedBlock, reportedState);
-    } else { // StripedUC
-      ((BlockInfoStripedUnderConstruction) ucBlock).addReplicaIfNotPresent(
-          storageInfo, reportedBlock, reportedState);
-    }
-  }
-
-  static int getNumExpectedLocations(BlockInfo ucBlock) {
-    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
-        ucBlock instanceof BlockInfoStripedUnderConstruction;
-    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
-      return ((BlockInfoContiguousUnderConstruction) ucBlock)
-          .getNumExpectedLocations();
-    } else { // StripedUC
-      return ((BlockInfoStripedUnderConstruction) ucBlock)
-          .getNumExpectedLocations();
-    }
-  }
-
-  public static DatanodeStorageInfo[] getExpectedStorageLocations(
-      BlockInfo ucBlock) {
-    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
-        ucBlock instanceof BlockInfoStripedUnderConstruction;
-    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
-      return ((BlockInfoContiguousUnderConstruction) ucBlock)
-          .getExpectedStorageLocations();
-    } else { // StripedUC
-      return ((BlockInfoStripedUnderConstruction) ucBlock)
-          .getExpectedStorageLocations();
-    }
-  }
-
-  public static void setExpectedLocations(BlockInfo ucBlock,
-      DatanodeStorageInfo[] targets) {
-    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
-        ucBlock instanceof BlockInfoStripedUnderConstruction;
-    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
-      ((BlockInfoContiguousUnderConstruction) ucBlock)
-          .setExpectedLocations(targets);
-    } else { // StripedUC
-      ((BlockInfoStripedUnderConstruction) ucBlock)
-          .setExpectedLocations(targets);
-    }
-  }
-
-  public static long getBlockRecoveryId(BlockInfo ucBlock) {
-    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
-        ucBlock instanceof BlockInfoStripedUnderConstruction;
-    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
-      return ((BlockInfoContiguousUnderConstruction) ucBlock)
-          .getBlockRecoveryId();
-    } else { // StripedUC
-      return ((BlockInfoStripedUnderConstruction) ucBlock).getBlockRecoveryId();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c31cb274/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
index 9ba2978..ce2219a 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
@@ -80,16 +80,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous
     setExpectedLocations(targets);
   }
 
-  /**
-   * Convert an under construction block to a complete block.
-   * 
-   * @return BlockInfoContiguous - 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. 
-   */
-  BlockInfoContiguous convertToCompleteBlock() throws IOException {
+  @Override
+  public BlockInfoContiguous convertToCompleteBlock() throws IOException {
     assert getBlockUCState() != BlockUCState.COMPLETE :
       "Trying to convert a COMPLETE block";
     return new BlockInfoContiguous(this);
@@ -170,13 +162,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous
     }
   }
 
-  /**
-   * 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());
@@ -235,9 +222,9 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous
     }
   }
 
-  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();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c31cb274/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
index 0373314..b991615 100644
--- 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
@@ -73,16 +73,8 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
     setExpectedLocations(targets);
   }
 
-  /**
-   * Convert an under construction striped block to a complete striped block.
-   *
-   * @return BlockInfoStriped - 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.
-   */
-  BlockInfoStriped convertToCompleteBlock() throws IOException {
+  @Override
+  public BlockInfoStriped convertToCompleteBlock() throws IOException {
     assert getBlockUCState() != COMPLETE :
       "Trying to convert a COMPLETE block";
     return new BlockInfoStriped(this);
@@ -177,12 +169,8 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
     }
   }
 
-  /**
-   * 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
-   */
-  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());
@@ -242,8 +230,9 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
     }
   }
 
-  void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block reportedBlock,
-      ReplicaState rState) {
+  @Override
+  public void addReplicaIfNotPresent(DatanodeStorageInfo storage,
+      Block reportedBlock, ReplicaState rState) {
     if (replicas == null) {
       replicas = new ReplicaUnderConstruction[1];
       replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage, rState);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c31cb274/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
index bfdd386..10a8cae 100644
--- 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
@@ -17,7 +17,11 @@
  */
 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 {
   /**
@@ -54,4 +58,27 @@ public interface BlockInfoUnderConstruction {
    * 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/c31cb274/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 0307792..0ad391a 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
@@ -636,13 +636,19 @@ public class BlockManager {
    */
   private static boolean commitBlock(final BlockInfo block,
       final Block commitBlock) throws IOException {
-    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();
-    BlockInfo.commitBlock(block, commitBlock);
-    return true;
+    if (block instanceof BlockInfoUnderConstruction
+        && block.getBlockUCState() != BlockUCState.COMMITTED) {
+      final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block;
+
+      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;
   }
   
   /**
@@ -699,7 +705,10 @@ public class BlockManager {
           "Cannot complete block: block has not been COMMITTED by the client");
     }
 
-    final BlockInfo completeBlock = BlockInfo.convertToCompleteBlock(curBlock);
+    final BlockInfo completeBlock
+        = !(curBlock instanceof BlockInfoUnderConstruction)? curBlock
+            : ((BlockInfoUnderConstruction)curBlock).convertToCompleteBlock();
+
     // replace penultimate block in file
     bc.setBlock(blkIndex, completeBlock);
     
@@ -737,7 +746,9 @@ public class BlockManager {
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
       final BlockInfo block) throws IOException {
-    BlockInfo.commitBlock(block, block);
+    if (block instanceof BlockInfoUnderConstruction) {
+      ((BlockInfoUnderConstruction)block).commitBlock(block);
+    }
     return completeBlock(bc, block, true);
   }
 
@@ -2235,12 +2246,13 @@ public class BlockManager {
       
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-        BlockInfo.addReplica(storedBlock, storageInfo, iblk, reportedState);
+        final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)storedBlock;
+        uc.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 = BlockInfo.getNumExpectedLocations(storedBlock);
+          int numOfReplicas = uc.getNumExpectedLocations();
           namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock);
         }
         //and fall through to next clause
@@ -2602,7 +2614,8 @@ public class BlockManager {
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
       DatanodeStorageInfo storageInfo) throws IOException {
     BlockInfo block = ucBlock.storedBlock;
-    BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock,
+    final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block;
+    uc.addReplicaIfNotPresent(storageInfo, ucBlock.reportedBlock,
         ucBlock.reportedState);
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
@@ -3930,6 +3943,20 @@ public class BlockManager {
         null);
   }
 
+  public LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info,
+      DatanodeStorageInfo[] locs, long offset) throws IOException {
+    final LocatedBlock lb;
+    if (info.isStriped()) {
+      lb = newLocatedStripedBlock(eb, locs,
+          ((BlockInfoStripedUnderConstruction)info).getBlockIndices(),
+          offset, false);
+    } else {
+      lb = newLocatedBlock(eb, locs, offset, false);
+    }
+    setBlockToken(lb, BlockTokenIdentifier.AccessMode.WRITE);
+    return lb;
+  }
+
   /**
    * This class is used internally by {@link this#computeRecoveryWorkForBlocks}
    * to represent a task to recover a block through replication or erasure

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c31cb274/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 8e8f576..fb67263 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
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
@@ -60,9 +59,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROL
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
@@ -88,8 +87,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROU
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER;
-import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.monotonicNow;
+import static org.apache.hadoop.util.Time.now;
 
 import java.io.BufferedWriter;
 import java.io.ByteArrayInputStream;
@@ -140,6 +139,7 @@ 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.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;
@@ -153,6 +153,7 @@ import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
@@ -161,10 +162,8 @@ 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.fs.permission.PermissionStatus;
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.ServiceFailedException;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -173,6 +172,7 @@ 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;
@@ -181,15 +181,15 @@ 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.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
-import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 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.LocatedStripedBlock;
@@ -210,7 +210,7 @@ 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.BlockInfoStripedUnderConstruction;
+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;
@@ -218,7 +218,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -3199,9 +3198,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         } else {
           // add new chosen targets to already allocated block and return
           BlockInfo lastBlockInFile = pendingFile.getLastBlock();
-          BlockInfo.setExpectedLocations(lastBlockInFile, targets);
+          final BlockInfoUnderConstruction uc
+              = (BlockInfoUnderConstruction)lastBlockInFile;
+          uc.setExpectedLocations(targets);
           offset = pendingFile.computeFileSize();
-          return makeLocatedBlock(lastBlockInFile, targets, offset);
+          return newLocatedBlock(lastBlockInFile, targets, offset);
         }
       }
 
@@ -3223,7 +3224,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     getEditLog().logSync();
 
     // Return located block
-    return makeLocatedBlock(getStoredBlock(newBlock), targets, offset);
+    return newLocatedBlock(getStoredBlock(newBlock), targets, offset);
   }
 
   /*
@@ -3344,8 +3345,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             "caught retry for allocation of a new block in " +
             src + ". Returning previously allocated block " + lastBlockInFile);
         long offset = pendingFile.computeFileSize();
-        onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
-            BlockInfo.getExpectedStorageLocations(lastBlockInFile), offset);
+        final BlockInfoUnderConstruction uc
+            = (BlockInfoUnderConstruction)lastBlockInFile;
+        onRetryBlock[0] = newLocatedBlock(lastBlockInFile,
+            uc.getExpectedStorageLocations(), offset);
         return new FileState(pendingFile, src, iip);
       } else {
         // Case 3
@@ -3357,20 +3360,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new FileState(pendingFile, src, iip);
   }
 
-  LocatedBlock makeLocatedBlock(BlockInfo blk, DatanodeStorageInfo[] locs,
-      long offset) throws IOException {
-    final LocatedBlock lBlk;
-    if (blk.isStriped()) {
-      assert blk instanceof BlockInfoStripedUnderConstruction;
-      lBlk = BlockManager.newLocatedStripedBlock(getExtendedBlock(blk), locs,
-          ((BlockInfoStripedUnderConstruction) blk).getBlockIndices(),
-          offset, false);
-    } else {
-      lBlk = BlockManager.newLocatedBlock(getExtendedBlock(blk), locs, offset, false);
-    }
-    getBlockManager().setBlockToken(
-        lBlk, BlockTokenIdentifier.AccessMode.WRITE);
-    return lBlk;
+  LocatedBlock newLocatedBlock(BlockInfo info,
+      DatanodeStorageInfo[] locs, long offset) throws IOException {
+    final ExtendedBlock b = getExtendedBlock(info);
+    return getBlockManager().newLocatedBlock(b, info, locs, offset);
   }
 
   /** @see ClientProtocol#getAdditionalDatanode */
@@ -4369,7 +4362,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
 
       truncatedBlock = iFile.getLastBlock();
-      long recoveryId = BlockInfo.getBlockRecoveryId(truncatedBlock);
+      final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)truncatedBlock;
+      final long recoveryId = uc.getBlockRecoveryId();
       copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId();
       if(recoveryId != newgenerationstamp) {
         throw new IOException("The recovery id " + newgenerationstamp


[02/50] hadoop git commit: HDFS-8123. Erasure Coding: Better to move EC related proto messages to a separate erasurecoding proto file (Contrubuted by Rakesh R)

Posted by zh...@apache.org.
HDFS-8123. Erasure Coding: Better to move EC related proto messages to a separate erasurecoding proto file (Contrubuted by Rakesh R)


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

Branch: refs/heads/HDFS-7285
Commit: 11fef75ffe317909cf52e66b341bf4cb546f2baa
Parents: d7eacf5
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Apr 15 12:09:16 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:13 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  5 +-
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |  1 +
 ...tNamenodeProtocolServerSideTranslatorPB.java | 12 ++--
 .../ClientNamenodeProtocolTranslatorPB.java     | 13 ++--
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  6 +-
 .../namenode/ErasureCodingZoneManager.java      |  2 +-
 .../src/main/proto/ClientNamenodeProtocol.proto | 24 +------
 .../src/main/proto/erasurecoding.proto          | 74 ++++++++++++++++++++
 .../hadoop-hdfs/src/main/proto/hdfs.proto       | 27 -------
 9 files changed, 96 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/11fef75f/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 5250dfa..07bbd4a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -61,4 +61,7 @@
     HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng)
 
     HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all
-    ECSchemas loaded in Namenode. (vinayakumarb)
\ No newline at end of file
+    ECSchemas loaded in Namenode. (vinayakumarb)
+
+    HDFS-8123. Erasure Coding: Better to move EC related proto messages to a
+    separate erasurecoding proto file (Rakesh R via vinayakumarb)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11fef75f/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index c11b963..a13a2bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -343,6 +343,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <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>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11fef75f/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 d103cf0..17141c3 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
@@ -107,12 +107,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto;
 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;
@@ -197,14 +193,18 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto;
 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.GetECSchemasRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
+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.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/11fef75f/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 2e17823..44d1258 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
@@ -107,11 +107,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 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.GetECSchemasRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto;
 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;
@@ -165,13 +161,16 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Trunca
 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.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
 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.HdfsProtos.ECSchemaProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11fef75f/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 c169515..10afc73 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
@@ -131,6 +131,9 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDele
 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.ECInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -151,9 +154,6 @@ 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.ECInfoProto;
-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.ExportedBlockKeysProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11fef75f/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
index c7daa2b..5320c1c 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
@@ -22,7 +22,7 @@ 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.proto.HdfsProtos.ECSchemaProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11fef75f/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
index c9059bb..89c38e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -34,6 +34,7 @@ import "acl.proto";
 import "xattr.proto";
 import "encryption.proto";
 import "inotify.proto";
+import "erasurecoding.proto";
 
 /**
  * The ClientNamenodeProtocol Service defines the interface between a client 
@@ -714,29 +715,6 @@ message GetEditsFromTxidResponseProto {
   required EventsListProto eventsList = 1;
 }
 
-message CreateErasureCodingZoneRequestProto {
-  required string src = 1;
-  optional ECSchemaProto schema = 2;
-}
-
-message CreateErasureCodingZoneResponseProto {
-}
-
-message GetErasureCodingInfoRequestProto {
-  required string src = 1;
-}
-
-message GetErasureCodingInfoResponseProto {
-  optional ECInfoProto ECInfo = 1;
-}
-
-message GetECSchemasRequestProto { // void request
-}
-
-message GetECSchemasResponseProto {
-  repeated ECSchemaProto schemas = 1;
-}
-
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11fef75f/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
new file mode 100644
index 0000000..4d5731b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -0,0 +1,74 @@
+/**
+ * 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;
+
+
+/**
+ * ECSchema options entry
+ */
+message ECSchemaOptionEntryProto {
+  required string key = 1;
+  required string value = 2;
+}
+
+/**
+ * ECSchema for erasurecoding
+ */
+message ECSchemaProto {
+  required string schemaName = 1;
+  required string codecName = 2;
+  required uint32 dataUnits = 3;
+  required uint32 parityUnits = 4;
+  repeated ECSchemaOptionEntryProto options = 5;
+}
+
+/**
+ * ECInfo
+ */
+message ECInfoProto {
+ required string src = 1;
+ required ECSchemaProto schema = 2;
+}
+
+
+message CreateErasureCodingZoneRequestProto {
+  required string src = 1;
+  optional ECSchemaProto schema = 2;
+}
+
+message CreateErasureCodingZoneResponseProto {
+}
+
+message GetErasureCodingInfoRequestProto {
+  required string src = 1;
+}
+
+message GetErasureCodingInfoResponseProto {
+  optional ECInfoProto ECInfo = 1;
+}
+
+message GetECSchemasRequestProto { // void request
+}
+
+message GetECSchemasResponseProto {
+  repeated ECSchemaProto schemas = 1;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11fef75f/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index 0507538..67e2058 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -620,30 +620,3 @@ message RollingUpgradeStatusProto {
   required string blockPoolId = 1;
   optional bool finalized = 2 [default = false];
 }
-
-/**
- * ECSchema options entry
- */
-message ECSchemaOptionEntryProto {
-  required string key = 1;
-  required string value = 2;
-}
-
-/**
- * ECShema for erasurecoding
- */
-message ECSchemaProto {
-  required string schemaName = 1;
-  required string codecName = 2;
-  required uint32 dataUnits = 3;
-  required uint32 parityUnits = 4;
-  repeated ECSchemaOptionEntryProto options = 5;
-}
-
-/**
- * ECInfo
- */
-message ECInfoProto {
- required string src = 1;
- required ECSchemaProto schema = 2;
-}
\ No newline at end of file


[25/50] hadoop git commit: HDFS-8228. Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause block id conflicts. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-8228. Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause block id conflicts. 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/f2baca72
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f2baca72
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f2baca72

Branch: refs/heads/HDFS-7285
Commit: f2baca72c5c274f97451aec20ded5d4260c4e4c7
Parents: 3627397
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Apr 24 09:30:38 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:20 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 ++
 .../SequentialBlockGroupIdGenerator.java        | 39 +++++++-------
 .../SequentialBlockIdGenerator.java             |  2 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  | 57 +++++++++++---------
 .../server/namenode/TestAddStripedBlocks.java   | 21 ++++++++
 5 files changed, 77 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2baca72/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 9357e23..cf41a9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -128,3 +128,6 @@
 
     HDFS-8223. Should calculate checksum for parity blocks in DFSStripedOutputStream.
     (Yi Liu via jing9)
+
+    HDFS-8228. Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause 
+    block id conflicts (Jing Zhao via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2baca72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
----------------------------------------------------------------------
diff --git 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
index e9e22ee..de8e379 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
@@ -19,9 +19,11 @@ 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.protocol.HdfsConstants;
 import org.apache.hadoop.util.SequentialNumber;
 
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_GROUP_INDEX_MASK;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.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.
@@ -34,6 +36,9 @@ import org.apache.hadoop.util.SequentialNumber;
  * 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 {
@@ -47,32 +52,30 @@ public class SequentialBlockGroupIdGenerator extends SequentialNumber {
 
   @Override // NumberGenerator
   public long nextValue() {
-    // Skip to next legitimate block group ID based on the naming protocol
-    while (super.getCurrentValue() % HdfsConstants.MAX_BLOCKS_IN_GROUP > 0) {
-      super.nextValue();
-    }
+    skipTo((getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK) + MAX_BLOCKS_IN_GROUP);
     // Make sure there's no conflict with existing random block IDs
-    while (hasValidBlockInRange(super.getCurrentValue())) {
-      super.skipTo(super.getCurrentValue() +
-          HdfsConstants.MAX_BLOCKS_IN_GROUP);
+    final Block b = new Block(getCurrentValue());
+    while (hasValidBlockInRange(b)) {
+      skipTo(getCurrentValue() + MAX_BLOCKS_IN_GROUP);
+      b.setBlockId(getCurrentValue());
     }
-    if (super.getCurrentValue() >= 0) {
-      BlockManager.LOG.warn("All negative block group IDs are used, " +
-          "growing into positive IDs, " +
-          "which might conflict with non-erasure coded blocks.");
+    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 super.getCurrentValue();
+    return getCurrentValue();
   }
 
   /**
-   *
-   * @param id The starting ID of the range
+   * @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
    */
-  private boolean hasValidBlockInRange(long id) {
-    for (int i = 0; i < HdfsConstants.MAX_BLOCKS_IN_GROUP; i++) {
-      Block b = new Block(id + i);
+  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) {
         return true;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2baca72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java
----------------------------------------------------------------------
diff --git 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
index c97de4b..6074784 100644
--- 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
@@ -54,7 +54,7 @@ public class SequentialBlockIdGenerator extends SequentialNumber {
       b.setBlockId(super.nextValue());
     }
     if (b.getBlockId() < 0) {
-      BlockManager.LOG.warn("All positive block IDs are used, " +
+      throw new IllegalStateException("All positive block IDs are used, " +
           "wrapping to negative IDs, " +
           "which might conflict with erasure coded block groups.");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2baca72/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 6af4a7f..73c7350 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
@@ -22,10 +22,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -40,18 +38,15 @@ public class TestDFSStripedInputStream {
   private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
   private final static int stripesPerBlock = 4;
   static int blockSize = cellSize * stripesPerBlock;
-  private int mod = 29;
   static int numDNs = dataBlocks + parityBlocks + 2;
 
   private static MiniDFSCluster cluster;
-  private static Configuration conf;
 
   @BeforeClass
   public static void setup() throws IOException {
-    conf = new Configuration();
+    Configuration conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
-    cluster
-        = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();;
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
     fs = cluster.getFileSystem();
   }
@@ -85,43 +80,56 @@ public class TestDFSStripedInputStream {
 
   @Test
   public void testFileSmallerThanOneStripe1() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize * dataBlocks - 1);
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe",
+        cellSize * dataBlocks - 1);
   }
 
   @Test
   public void testFileSmallerThanOneStripe2() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", cellSize + 123);
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe",
+        cellSize + 123);
   }
 
   @Test
   public void testFileEqualsWithOneStripe() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe", cellSize * dataBlocks);
+    testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe",
+        cellSize * dataBlocks);
   }
 
   @Test
   public void testFileMoreThanOneStripe1() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1", cellSize * dataBlocks + 123);
+    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1",
+        cellSize * dataBlocks + 123);
   }
 
   @Test
   public void testFileMoreThanOneStripe2() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2", cellSize * dataBlocks
-        + cellSize * dataBlocks + 123);
+    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2",
+        cellSize * dataBlocks + cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void testLessThanFullBlockGroup() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup",
+        cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize);
   }
 
   @Test
   public void testFileFullBlockGroup() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/FullBlockGroup", blockSize * dataBlocks);
+    testOneFileUsingDFSStripedInputStream("/FullBlockGroup",
+        blockSize * dataBlocks);
   }
 
   @Test
   public void testFileMoreThanABlockGroup1() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1", blockSize * dataBlocks + 123);
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1",
+        blockSize * dataBlocks + 123);
   }
 
   @Test
   public void testFileMoreThanABlockGroup2() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", blockSize * dataBlocks + cellSize+ 123);
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2",
+        blockSize * dataBlocks + cellSize+ 123);
   }
 
 
@@ -141,35 +149,32 @@ public class TestDFSStripedInputStream {
   }
 
   private byte getByte(long pos) {
+    final int mod = 29;
     return (byte) (pos % mod + 1);
   }
 
   private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes)
       throws IOException {
-    Path TestPath = new Path(src);
+    Path testPath = new Path(src);
     byte[] bytes = generateBytes(writeBytes);
-    DFSTestUtil.writeFile(fs, TestPath, new String(bytes));
+    DFSTestUtil.writeFile(fs, testPath, new String(bytes));
 
     //check file length
-    FileStatus status = fs.getFileStatus(TestPath);
+    FileStatus status = fs.getFileStatus(testPath);
     long fileLength = status.getLen();
     Assert.assertEquals("File length should be the same",
         writeBytes, fileLength);
 
-    DFSStripedInputStream dis = new DFSStripedInputStream(
-        fs.getClient(), src, true);
-    try {
+    try (DFSStripedInputStream dis =
+             new DFSStripedInputStream(fs.getClient(), src, true)) {
       byte[] buf = new byte[writeBytes + 100];
       int readLen = dis.read(0, buf, 0, buf.length);
       readLen = readLen >= 0 ? readLen : 0;
       Assert.assertEquals("The length of file should be the same to write size",
           writeBytes, readLen);
       for (int i = 0; i < writeBytes; i++) {
-        Assert.assertEquals("Byte at i should be the same",
-            getByte(i), buf[i]);
+        Assert.assertEquals("Byte at i should be the same", getByte(i), buf[i]);
       }
-    } finally {
-      dis.close();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2baca72/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 6bb1162..d03e938 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
@@ -82,6 +82,27 @@ public class TestAddStripedBlocks {
     }
   }
 
+  /**
+   * 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 + HdfsConstants.MAX_BLOCKS_IN_GROUP, secondId);
+  }
+
   @Test
   public void testAddStripedBlock() throws Exception {
     final Path file = new Path("/file1");


[22/50] hadoop git commit: HDFS-8223. Should calculate checksum for parity blocks in DFSStripedOutputStream. Contributed by Yi Liu.

Posted by zh...@apache.org.
HDFS-8223. Should calculate checksum for parity blocks in DFSStripedOutputStream. Contributed by Yi Liu.


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

Branch: refs/heads/HDFS-7285
Commit: 3627397fb73ee5b8292e61a35290a54f48f13cf1
Parents: 613e0a8
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Apr 23 15:48:21 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:19 2015 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/fs/FSOutputSummer.java    |  4 ++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt  |  3 +++
 .../org/apache/hadoop/hdfs/DFSStripedOutputStream.java    | 10 ++++++++++
 3 files changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3627397f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
----------------------------------------------------------------------
diff --git 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
index bdc5585..a8a7494 100644
--- 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,6 +196,10 @@ abstract public class FSOutputSummer extends OutputStream {
     return sum.getChecksumSize();
   }
 
+  protected DataChecksum getDataChecksum() {
+    return sum;
+  }
+
   protected TraceScope createWriteTraceScope() {
     return NullScope.INSTANCE;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3627397f/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 48791b1..9357e23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -125,3 +125,6 @@
 
     HDFS-8233. Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last
     stripe is at the block group boundary. (jing9)
+
+    HDFS-8223. Should calculate checksum for parity blocks in DFSStripedOutputStream.
+    (Yi Liu via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3627397f/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 245dfc1..6842267 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
@@ -62,6 +62,8 @@ public class DFSStripedOutputStream extends DFSOutputStream {
    */
   private final ECInfo ecInfo;
   private final int cellSize;
+  // checksum buffer, we only need to calculate checksum for parity blocks
+  private byte[] checksumBuf;
   private ByteBuffer[] cellBuffers;
 
   private final short numAllBlocks;
@@ -99,6 +101,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
 
     checkConfiguration();
 
+    checksumBuf = new byte[getChecksumSize() * (cellSize / bytesPerChecksum)];
     cellBuffers = new ByteBuffer[numAllBlocks];
     List<BlockingQueue<LocatedBlock>> stripeBlocks = new ArrayList<>();
 
@@ -179,6 +182,10 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   private List<DFSPacket> generatePackets(ByteBuffer byteBuffer)
       throws IOException{
     List<DFSPacket> packets = new ArrayList<>();
+    assert byteBuffer.hasArray();
+    getDataChecksum().calculateChunkedSums(byteBuffer.array(), 0,
+        byteBuffer.remaining(), checksumBuf, 0);
+    int ckOff = 0;
     while (byteBuffer.remaining() > 0) {
       DFSPacket p = createPacket(packetSize, chunksPerPacket,
           streamer.getBytesCurBlock(),
@@ -186,6 +193,9 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       int maxBytesToPacket = p.getMaxChunks() * bytesPerChecksum;
       int toWrite = byteBuffer.remaining() > maxBytesToPacket ?
           maxBytesToPacket: byteBuffer.remaining();
+      int ckLen = ((toWrite - 1) / bytesPerChecksum + 1) * getChecksumSize();
+      p.writeChecksum(checksumBuf, ckOff, ckLen);
+      ckOff += ckLen;
       p.writeData(byteBuffer, toWrite);
       streamer.incBytesCurBlock(toWrite);
       packets.add(p);


[10/50] hadoop git commit: HDFS-8181. createErasureCodingZone sets retryCache state as false always (Contributed by Uma Maheswara Rao G)

Posted by zh...@apache.org.
HDFS-8181. createErasureCodingZone sets retryCache state as false always (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/c3a866ee
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c3a866ee
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c3a866ee

Branch: refs/heads/HDFS-7285
Commit: c3a866ee37d41ceb13944910f686969a2bc5f2ae
Parents: 0c37627
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Apr 20 15:04:49 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:16 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt              | 14 ++++++++++++++
 .../hdfs/server/namenode/NameNodeRpcServer.java       |  1 +
 2 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3a866ee/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 40517e7..c8dbf08 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -83,10 +83,24 @@
 
     HDFS-7349. Support DFS command for the EC encoding (vinayakumarb)
 
+    HDFS-8120. Erasure coding: created util class to analyze striped block groups.
+    (Contributed by Zhe Zhang and Li Bo via Jing Zhao)
+
     HDFS-7994. Detect if resevered EC Block ID is already used during namenode
     startup. (Hui Zheng via szetszwo)
 
     HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. (Hui Zheng via zhz).
 
+    HDFS-8166. DFSStripedOutputStream should not create empty blocks. (Jing Zhao)
+
+    HDFS-7937. Erasure Coding: INodeFile quota computation unit tests.
+    (Kai Sasaki via Jing Zhao)
+
+    HDFS-8145. Fix the editlog corruption exposed by failed TestAddStripedBlocks.
+    (Jing Zhao)
+
     HDFS-8146. Protobuf changes for BlockECRecoveryCommand and its fields for
     making it ready for transfer to DN (Uma Maheswara Rao G via vinayakumarb)
+
+    HDFS-8181. createErasureCodingZone sets retryCache state as false always
+    (Uma Maheswara Rao G via vinayakumarb)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3a866ee/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 023f863..06701a9 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
@@ -1856,6 +1856,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     boolean success = false;
     try {
       namesystem.createErasureCodingZone(src, schema, cacheEntry != null);
+      success = true;
     } finally {
       RetryCache.setState(cacheEntry, success);
     }


[32/50] hadoop git commit: HDFS-8308. Erasure Coding: NameNode may get blocked in waitForLoadingFSImage() when loading editlog. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-8308. Erasure Coding: NameNode may get blocked in waitForLoadingFSImage() when loading editlog. 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/2bd2dd7e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2bd2dd7e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2bd2dd7e

Branch: refs/heads/HDFS-7285
Commit: 2bd2dd7e0e346c5e63c22f20692b7ee3318fd159
Parents: b962860
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Apr 30 19:42:29 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:22 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +
 .../namenode/ErasureCodingZoneManager.java      |  3 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  4 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     | 12 ++++
 .../hadoop/hdfs/TestErasureCodingZones.java     |  6 +-
 .../server/namenode/TestAddStripedBlocks.java   | 61 ++++++++++----------
 6 files changed, 52 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bd2dd7e/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 3c75152..596bbcf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -152,3 +152,6 @@
 
     HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of 
     datastreamer threads. (Rakesh R via Zhe Zhang)
+
+    HDFS-8308. Erasure Coding: NameNode may get blocked in waitForLoadingFSImage()
+    when loading editlog. (jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bd2dd7e/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
index 8cda289..14d4e29 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
@@ -79,7 +79,8 @@ public class ErasureCodingZoneManager {
       for (XAttr xAttr : xAttrs) {
         if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
           String schemaName = new String(xAttr.getValue());
-          ECSchema schema = dir.getFSNamesystem().getECSchema(schemaName);
+          ECSchema schema = dir.getFSNamesystem().getSchemaManager()
+              .getSchema(schemaName);
           return new ECZoneInfo(inode.getFullPathName(), schema);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bd2dd7e/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 1290bb2..86745ec 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
@@ -8120,9 +8120,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Create an erasure coding zone on directory src.
-   * @param schema  ECSchema for the erasure coding zone
-   * @param src     the path of a directory which will be the root of the
+   * @param srcArg  the path of a directory which will be the root of the
    *                erasure coding zone. The directory must be empty.
+   * @param schema  ECSchema for the erasure coding zone
    *
    * @throws AccessControlException  if the caller is not the superuser.
    * @throws UnresolvedLinkException if the path can't be resolved.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bd2dd7e/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 0c88842..0165189 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
@@ -1948,4 +1948,16 @@ public class DFSTestUtil {
     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 void writeAndFlushStripedOutputStream(
+      DFSStripedOutputStream out, int chunkSize) throws IOException {
+    // FSOutputSummer.BUFFER_NUM_CHUNKS == 9
+    byte[] toWrite = new byte[chunkSize * 9 + 1];
+    out.write(toWrite);
+    out.flushInternal();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bd2dd7e/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 ae2bdd8..f1aec82 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
@@ -35,8 +35,6 @@ import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.*;
 
 public class TestErasureCodingZones {
-  private final int NUM_OF_DATANODES = 3;
-  private Configuration conf;
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
   private static final int BLOCK_SIZE = 1024;
@@ -44,10 +42,10 @@ public class TestErasureCodingZones {
 
   @Before
   public void setupCluster() throws IOException {
-    conf = new HdfsConfiguration();
+    Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     cluster = new MiniDFSCluster.Builder(conf).
-        numDataNodes(NUM_OF_DATANODES).build();
+        numDataNodes(1).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
     namesystem = cluster.getNamesystem();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bd2dd7e/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 d03e938..297db14 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
@@ -19,6 +19,7 @@ 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;
@@ -55,6 +56,7 @@ 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;
@@ -103,52 +105,50 @@ public class TestAddStripedBlocks {
     Assert.assertEquals(firstId + HdfsConstants.MAX_BLOCKS_IN_GROUP, secondId);
   }
 
-  @Test
+  @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);
+      DFSTestUtil.writeAndFlushStripedOutputStream(
+          (DFSStripedOutputStream) out.getWrappedStream(),
+          DFS_BYTES_PER_CHECKSUM_DEFAULT);
 
       FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
       INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
-      LocatedBlock newBlock = cluster.getNamesystem().getAdditionalBlock(
-          file.toString(), fileNode.getId(), dfs.getClient().getClientName(),
-          null, null, null);
-      assertEquals(GROUP_SIZE, newBlock.getLocations().length);
-      assertEquals(GROUP_SIZE, newBlock.getStorageIDs().length);
 
       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);
     }
-
-    // restart NameNode to check editlog
-    cluster.restartNameNode(true);
-    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(), 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);
   }
 
   private void checkStripedBlockUC(BlockInfoStriped block,
@@ -190,11 +190,12 @@ public class TestAddStripedBlocks {
     FSDataOutputStream out = null;
     try {
       out = dfs.create(file, (short) 1);
+      DFSTestUtil.writeAndFlushStripedOutputStream(
+          (DFSStripedOutputStream) out.getWrappedStream(),
+          DFS_BYTES_PER_CHECKSUM_DEFAULT);
 
       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);
       BlockInfoStripedUnderConstruction lastBlk =
           (BlockInfoStripedUnderConstruction) fileNode.getLastBlock();
       DatanodeInfo[] expectedDNs = DatanodeStorageInfo


[07/50] hadoop git commit: HDFS-8166. DFSStripedOutputStream should not create empty blocks. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-8166. DFSStripedOutputStream should not create empty blocks. 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/1d80df7a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1d80df7a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1d80df7a

Branch: refs/heads/HDFS-7285
Commit: 1d80df7a6ff8fbc11842faac249a370ab7720852
Parents: 44a30f9
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Apr 17 17:55:19 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:15 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 163 +++++++++++--------
 .../apache/hadoop/hdfs/StripedDataStreamer.java |  72 +++-----
 .../server/blockmanagement/BlockManager.java    |  17 +-
 .../hadoop/hdfs/TestDFSStripedOutputStream.java | 162 +++++++++++-------
 4 files changed, 236 insertions(+), 178 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d80df7a/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 f11a657..7dc0091 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
@@ -22,10 +22,14 @@ 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.protocol.ExtendedBlock;
@@ -59,12 +63,12 @@ public class DFSStripedOutputStream extends DFSOutputStream {
    */
   private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
   private ByteBuffer[] cellBuffers;
-  private final short blockGroupBlocks = HdfsConstants.NUM_DATA_BLOCKS
+  private final short numAllBlocks = HdfsConstants.NUM_DATA_BLOCKS
       + HdfsConstants.NUM_PARITY_BLOCKS;
-  private final short blockGroupDataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
+  private final short numDataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
   private int curIdx = 0;
   /* bytes written in current block group */
-  private long currentBlockGroupBytes = 0;
+  //private long currentBlockGroupBytes = 0;
 
   //TODO: Use ErasureCoder interface (HDFS-7781)
   private RawErasureEncoder encoder;
@@ -73,10 +77,6 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     return streamers.get(0);
   }
 
-  private long getBlockGroupSize() {
-    return blockSize * HdfsConstants.NUM_DATA_BLOCKS;
-  }
-
   /** Construct a new output stream for creating a file. */
   DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
                          EnumSet<CreateFlag> flag, Progressable progress,
@@ -84,15 +84,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
                          throws IOException {
     super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
     DFSClient.LOG.info("Creating striped output stream");
-    if (blockGroupBlocks <= 1) {
-      throw new IOException("The block group must contain more than one block.");
-    }
+    checkConfiguration();
 
-    cellBuffers = new ByteBuffer[blockGroupBlocks];
+    cellBuffers = new ByteBuffer[numAllBlocks];
     List<BlockingQueue<LocatedBlock>> stripeBlocks = new ArrayList<>();
 
-    for (int i = 0; i < blockGroupBlocks; i++) {
-      stripeBlocks.add(new LinkedBlockingQueue<LocatedBlock>(blockGroupBlocks));
+    for (int i = 0; i < numAllBlocks; i++) {
+      stripeBlocks.add(new LinkedBlockingQueue<LocatedBlock>(numAllBlocks));
       try {
         cellBuffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
       } catch (InterruptedException ie) {
@@ -103,29 +101,38 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       }
     }
     encoder = new RSRawEncoder();
-    encoder.initialize(blockGroupDataBlocks,
-        blockGroupBlocks - blockGroupDataBlocks, cellSize);
+    encoder.initialize(numDataBlocks,
+        numAllBlocks - numDataBlocks, cellSize);
 
-    streamers = new ArrayList<>(blockGroupBlocks);
-    for (short i = 0; i < blockGroupBlocks; i++) {
+    List<StripedDataStreamer> s = new ArrayList<>(numAllBlocks);
+    for (short i = 0; i < numAllBlocks; i++) {
       StripedDataStreamer streamer = new StripedDataStreamer(stat, null,
           dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
           i, stripeBlocks);
       if (favoredNodes != null && favoredNodes.length != 0) {
         streamer.setFavoredNodes(favoredNodes);
       }
-      streamers.add(streamer);
+      s.add(streamer);
     }
+    streamers = Collections.unmodifiableList(s);
 
     refreshStreamer();
   }
 
+  private void checkConfiguration() {
+    if (cellSize % bytesPerChecksum != 0) {
+      throw new HadoopIllegalArgumentException("Invalid values: "
+          + DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
+          + ") must divide cell size (=" + cellSize + ").");
+    }
+  }
+
   private void refreshStreamer() {
     streamer = streamers.get(curIdx);
   }
 
   private void moveToNextStreamer() {
-    curIdx = (curIdx + 1) % blockGroupBlocks;
+    curIdx = (curIdx + 1) % numAllBlocks;
     refreshStreamer();
   }
 
@@ -136,20 +143,21 @@ public class DFSStripedOutputStream extends DFSOutputStream {
    * @param buffers data buffers + parity buffers
    */
   private void encode(ByteBuffer[] buffers) {
-    ByteBuffer[] dataBuffers = new ByteBuffer[blockGroupDataBlocks];
-    ByteBuffer[] parityBuffers = new ByteBuffer[blockGroupBlocks - blockGroupDataBlocks];
-    for (int i = 0; i < blockGroupBlocks; i++) {
-      if (i < blockGroupDataBlocks) {
+    ByteBuffer[] dataBuffers = new ByteBuffer[numDataBlocks];
+    ByteBuffer[] parityBuffers = new ByteBuffer[numAllBlocks - numDataBlocks];
+    for (int i = 0; i < numAllBlocks; i++) {
+      if (i < numDataBlocks) {
         dataBuffers[i] = buffers[i];
       } else {
-        parityBuffers[i - blockGroupDataBlocks] = buffers[i];
+        parityBuffers[i - numDataBlocks] = buffers[i];
       }
     }
     encoder.encode(dataBuffers, parityBuffers);
   }
 
   /**
-   * Generate packets from a given buffer
+   * Generate packets from a given buffer. This is only used for streamers
+   * writing parity blocks.
    *
    * @param byteBuffer the given buffer to generate packets
    * @return packets generated
@@ -185,7 +193,6 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       throw new IOException(msg);
     }
 
-
     // If current packet has not been enqueued for transmission,
     // but the cell buffer is full, we need to enqueue the packet
     if (currentPacket != null && getSizeOfCellnBuffer(curIdx) == cellSize) {
@@ -213,13 +220,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       //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 (curIdx == blockGroupDataBlocks) {
+      if (curIdx == numDataBlocks) {
         //encode the data cells
-        for (int k = 0; k < blockGroupDataBlocks; k++) {
+        for (int k = 0; k < numDataBlocks; k++) {
           cellBuffers[k].flip();
         }
         encode(cellBuffers);
-        for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) {
+        for (int i = numDataBlocks; i < numAllBlocks; i++) {
           ByteBuffer parityBuffer = cellBuffers[i];
           List<DFSPacket> packets = generatePackets(parityBuffer);
           for (DFSPacket p : packets) {
@@ -245,13 +252,24 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   }
 
   private void clearCellBuffers() {
-    for (int i = 0; i< blockGroupBlocks; i++) {
+    for (int i = 0; i< numAllBlocks; i++) {
       cellBuffers[i].clear();
+      if (i >= numDataBlocks) {
+        Arrays.fill(cellBuffers[i].array(), (byte) 0);
+      }
     }
   }
 
   private int stripeDataSize() {
-    return blockGroupDataBlocks * cellSize;
+    return numDataBlocks * cellSize;
+  }
+
+  private long getCurrentBlockGroupBytes() {
+    long sum = 0;
+    for (int i = 0; i < numDataBlocks; i++) {
+      sum += streamers.get(i).getBytesCurBlock();
+    }
+    return sum;
   }
 
   private void notSupported(String headMsg)
@@ -270,7 +288,6 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     notSupported("hsync");
   }
 
-
   @Override
   protected synchronized void start() {
     for (StripedDataStreamer streamer : streamers) {
@@ -302,15 +319,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   // interrupt datastreamer if force is true
   @Override
   protected void closeThreads(boolean force) throws IOException {
-    StripedDataStreamer leadingStreamer = null;
     for (StripedDataStreamer streamer : streamers) {
       try {
         streamer.close(force);
         streamer.join();
         streamer.closeSocket();
-        if (streamer.isLeadingStreamer()) {
-          leadingStreamer = streamer;
-        }
       } catch (InterruptedException e) {
         throw new IOException("Failed to shutdown streamer");
       } finally {
@@ -318,40 +331,26 @@ public class DFSStripedOutputStream extends DFSOutputStream {
         setClosed();
       }
     }
-    assert leadingStreamer != null : "One streamer should be leader";
-    leadingStreamer.countTailingBlockGroupBytes();
-  }
-
-  @Override
-  public synchronized void write(int b) throws IOException {
-    super.write(b);
-    currentBlockGroupBytes = (currentBlockGroupBytes + 1) % getBlockGroupSize();
-  }
-
-  @Override
-  public synchronized void write(byte b[], int off, int len)
-      throws IOException {
-    super.write(b, off, len);
-    currentBlockGroupBytes = (currentBlockGroupBytes + len) % getBlockGroupSize();
   }
 
-  private void writeParityCellsForLastStripe() throws IOException{
+  private void writeParityCellsForLastStripe() throws IOException {
+    final long currentBlockGroupBytes = getCurrentBlockGroupBytes();
     long parityBlkSize = StripedBlockUtil.getInternalBlockLength(
-        currentBlockGroupBytes, cellSize, blockGroupDataBlocks,
-        blockGroupDataBlocks + 1);
+        currentBlockGroupBytes, cellSize, numDataBlocks,
+        numDataBlocks + 1);
     if (parityBlkSize == 0 || currentBlockGroupBytes % stripeDataSize() == 0) {
       return;
     }
     int parityCellSize = parityBlkSize % cellSize == 0 ? cellSize :
                         (int) (parityBlkSize % cellSize);
 
-    for (int i = 0; i < blockGroupBlocks; i++) {
+    for (int i = 0; i < numAllBlocks; i++) {
       long internalBlkLen = StripedBlockUtil.getInternalBlockLength(
-          currentBlockGroupBytes, cellSize, blockGroupDataBlocks, i);
+          currentBlockGroupBytes, cellSize, numDataBlocks, 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
-      if (internalBlkLen < parityBlkSize || i >= blockGroupDataBlocks) {
+      if (internalBlkLen < parityBlkSize || i >= numDataBlocks) {
         int position = cellBuffers[i].position();
         assert position <= parityCellSize : "If an internal block is smaller" +
             " than parity block, then its last cell should be small than last" +
@@ -365,9 +364,9 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     encode(cellBuffers);
 
     //write parity cells
-    curIdx = blockGroupDataBlocks;
+    curIdx = numDataBlocks;
     refreshStreamer();
-    for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) {
+    for (int i = numDataBlocks; i < numAllBlocks; i++) {
       ByteBuffer parityBuffer = cellBuffers[i];
       List<DFSPacket> packets = generatePackets(parityBuffer);
       for (DFSPacket p : packets) {
@@ -385,7 +384,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   @Override
   void setClosed() {
     super.setClosed();
-    for (int i = 0; i < blockGroupBlocks; i++) {
+    for (int i = 0; i < numAllBlocks; i++) {
       byteArrayManager.release(cellBuffers[i].array());
       streamers.get(i).release();
     }
@@ -395,10 +394,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   protected synchronized void closeImpl() throws IOException {
     if (isClosed()) {
       IOException e = getLeadingStreamer().getLastException().getAndSet(null);
-      if (e == null)
-        return;
-      else
+      if (e != null) {
         throw e;
+      } else {
+        return;
+      }
     }
 
     try {
@@ -408,14 +408,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
         streamer.waitAndQueuePacket(currentPacket);
         currentPacket = null;
       }
-      //if the last stripe is incomplete, generate and write parity cells
+      // if the last stripe is incomplete, generate and write parity cells
       writeParityCellsForLastStripe();
 
-      for (int i = 0; i < blockGroupBlocks; i++) {
+      for (int i = 0; i < numAllBlocks; i++) {
         curIdx = i;
         refreshStreamer();
-        if (streamer.getBytesCurBlock()!= 0 ||
-            currentBlockGroupBytes < getBlockGroupSize()) {
+        if (streamer.getBytesCurBlock() > 0) {
           // send an empty packet to mark the end of the block
           currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
               streamer.getAndIncCurrentSeqno(), true);
@@ -425,9 +424,8 @@ public class DFSStripedOutputStream extends DFSOutputStream {
         flushInternal();
       }
 
-      // get last block before destroying the streamer
-      ExtendedBlock lastBlock = streamers.get(0).getBlock();
       closeThreads(false);
+      final ExtendedBlock lastBlock = getCommittedBlock();
       TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
       try {
         completeFile(lastBlock);
@@ -435,10 +433,35 @@ public class DFSStripedOutputStream extends DFSOutputStream {
         scope.close();
       }
       dfsClient.endFileLease(fileId);
-    } catch (ClosedChannelException e) {
+    } catch (ClosedChannelException ignored) {
     } finally {
       setClosed();
     }
   }
 
+  /**
+   * Generate the block which is reported and will be committed in NameNode.
+   * Need to go through all the streamers writing data blocks and add their
+   * bytesCurBlock together. Note that at this time all streamers have been
+   * closed. Also this calculation can cover streamers with writing failures.
+   *
+   * @return An ExtendedBlock with size of the whole block group.
+   */
+  ExtendedBlock getCommittedBlock() throws IOException {
+    ExtendedBlock b = getLeadingStreamer().getBlock();
+    if (b == null) {
+      return null;
+    }
+    final ExtendedBlock block = new ExtendedBlock(b);
+    final boolean atBlockGroupBoundary =
+        getLeadingStreamer().getBytesCurBlock() == 0 &&
+            getLeadingStreamer().getBlock() != null &&
+            getLeadingStreamer().getBlock().getNumBytes() > 0;
+    for (int i = 1; i < numDataBlocks; i++) {
+      block.setNumBytes(block.getNumBytes() +
+          (atBlockGroupBoundary ? streamers.get(i).getBlock().getNumBytes() :
+              streamers.get(i).getBytesCurBlock()));
+    }
+    return block;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d80df7a/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 5614852..19c205e 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
@@ -22,7 +22,6 @@ import java.util.List;
 
 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;
@@ -37,6 +36,10 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
+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;
+
 /****************************************************************************
  * The StripedDataStreamer class is used by {@link DFSStripedOutputStream}.
  * There are two kinds of StripedDataStreamer, leading streamer and ordinary
@@ -47,9 +50,7 @@ import java.util.concurrent.atomic.AtomicReference;
  ****************************************************************************/
 public class StripedDataStreamer extends DataStreamer {
   private final short index;
-  private final  List<BlockingQueue<LocatedBlock>> stripedBlocks;
-  private static short blockGroupSize = HdfsConstants.NUM_DATA_BLOCKS
-      + HdfsConstants.NUM_PARITY_BLOCKS;
+  private final List<BlockingQueue<LocatedBlock>> stripedBlocks;
   private boolean hasCommittedBlock = false;
 
   StripedDataStreamer(HdfsFileStatus stat, ExtendedBlock block,
@@ -88,66 +89,38 @@ public class StripedDataStreamer extends DataStreamer {
   }
 
   private boolean isParityStreamer() {
-    return index >= HdfsConstants.NUM_DATA_BLOCKS;
+    return index >= NUM_DATA_BLOCKS;
   }
 
   @Override
   protected void endBlock() {
     if (!isLeadingStreamer() && !isParityStreamer()) {
-      //before retrieving a new block, transfer the finished block to
-      //leading streamer
+      // before retrieving a new block, transfer the finished block to
+      // leading streamer
       LocatedBlock finishedBlock = new LocatedBlock(
           new ExtendedBlock(block.getBlockPoolId(), block.getBlockId(),
-                       block.getNumBytes(),block.getGenerationStamp()), null);
-      try{
+              block.getNumBytes(), block.getGenerationStamp()), null);
+      try {
         boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30,
             TimeUnit.SECONDS);
-      }catch (InterruptedException ie) {
-      //TODO: Handle InterruptedException (HDFS-7786)
+      } catch (InterruptedException ie) {
+        // TODO: Handle InterruptedException (HDFS-7786)
       }
     }
     super.endBlock();
   }
 
-  /**
-   * This function is called after the streamer is closed.
-   */
-  void countTailingBlockGroupBytes () throws IOException {
-    if (isLeadingStreamer()) {
-      //when committing a block group, leading streamer has to adjust
-      // {@link block} including the size of block group
-      for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) {
-        try {
-          LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30,
-              TimeUnit.SECONDS);
-          if (finishedLocatedBlock == null) {
-            throw new IOException("Fail to get finished LocatedBlock " +
-                "from streamer, i=" + i);
-          }
-          ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock();
-          long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes();
-          if (block != null) {
-            block.setNumBytes(block.getNumBytes() + bytes);
-          }
-        } catch (InterruptedException ie) {
-          DFSClient.LOG.info("InterruptedException received when " +
-              "putting a block to stripeBlocks, ie = " + ie);
-        }
-      }
-    }
-  }
-
   @Override
   protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
       throws IOException {
     LocatedBlock lb = null;
     if (isLeadingStreamer()) {
-      if(hasCommittedBlock) {
+      if (hasCommittedBlock) {
         /**
          * when committing a block group, leading streamer has to adjust
          * {@link block} to include the size of block group
          */
-        for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) {
+        for (int i = 1; i < NUM_DATA_BLOCKS; i++) {
           try {
             LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30,
                 TimeUnit.SECONDS);
@@ -157,7 +130,7 @@ public class StripedDataStreamer extends DataStreamer {
             }
             ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock();
             long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes();
-            if(block != null) {
+            if (block != null) {
               block.setNumBytes(block.getNumBytes() + bytes);
             }
           } catch (InterruptedException ie) {
@@ -171,14 +144,13 @@ public class StripedDataStreamer extends DataStreamer {
       hasCommittedBlock = true;
       assert lb instanceof LocatedStripedBlock;
       DFSClient.LOG.debug("Leading streamer obtained bg " + lb);
-      LocatedBlock[] blocks = StripedBlockUtil.
-          parseStripedBlockGroup((LocatedStripedBlock) lb,
-              HdfsConstants.BLOCK_STRIPED_CELL_SIZE, HdfsConstants.NUM_DATA_BLOCKS,
-              HdfsConstants.NUM_PARITY_BLOCKS
-          );
-      assert blocks.length == blockGroupSize :
+      LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
+          (LocatedStripedBlock) lb, BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS,
+          NUM_PARITY_BLOCKS);
+      assert blocks.length == (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) :
           "Fail to get block group from namenode: blockGroupSize: " +
-              blockGroupSize + ", blocks.length: " + blocks.length;
+              (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) + ", blocks.length: " +
+              blocks.length;
       lb = blocks[0];
       for (int i = 1; i < blocks.length; i++) {
         try {
@@ -199,7 +171,7 @@ public class StripedDataStreamer extends DataStreamer {
       }
     } else {
       try {
-        //wait 90 seconds to get a block from the queue
+        // wait 90 seconds to get a block from the queue
         lb = stripedBlocks.get(index).poll(90, TimeUnit.SECONDS);
       } catch (InterruptedException ie) {
         DFSClient.LOG.info("InterruptedException received when retrieving " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d80df7a/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 f7b1fda..712c665 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
@@ -65,6 +65,7 @@ import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
 import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
+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.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -597,8 +598,20 @@ public class BlockManager {
   }
 
   public short getMinStorageNum(BlockInfo block) {
-    return block.isStriped() ?
-        ((BlockInfoStriped) block).getDataBlockNum() : minReplication;
+    if (block.isStriped()) {
+      final BlockInfoStriped sblock = (BlockInfoStriped) block;
+      short dataBlockNum = sblock.getDataBlockNum();
+      if (sblock.isComplete() ||
+          sblock.getBlockUCState() == BlockUCState.COMMITTED) {
+        // if the sblock is committed/completed and its length is less than a
+        // full stripe, the minimum storage number needs to be adjusted
+        dataBlockNum = (short) Math.min(dataBlockNum,
+            (sblock.getNumBytes() - 1) / HdfsConstants.BLOCK_STRIPED_CELL_SIZE + 1);
+      }
+      return dataBlockNum;
+    } else {
+      return minReplication;
+    }
   }
 
   public boolean hasMinStorage(BlockInfo block) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d80df7a/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 4a09bda..cc20f40 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
@@ -1,5 +1,6 @@
 package org.apache.hadoop.hdfs;
 
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -20,6 +21,8 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
 import org.junit.After;
@@ -42,8 +45,8 @@ public class TestDFSStripedOutputStream {
   private DistributedFileSystem fs;
   private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
   private final int stripesPerBlock = 4;
-  int blockSize = cellSize * stripesPerBlock;
-  private int mod = 29;
+  private final int blockSize = cellSize * stripesPerBlock;
+  private final RawErasureEncoder encoder = new RSRawEncoder();
 
   @Before
   public void setup() throws IOException {
@@ -53,6 +56,7 @@ public class TestDFSStripedOutputStream {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
     fs = cluster.getFileSystem();
+    encoder.initialize(dataBlocks, parityBlocks, cellSize);
   }
 
   @After
@@ -144,60 +148,27 @@ public class TestDFSStripedOutputStream {
   }
 
   private byte getByte(long pos) {
+    int mod = 29;
     return (byte) (pos % mod + 1);
   }
 
-  private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes)
-      throws IOException {
-    Path TestPath = new Path(src);
-    byte[] bytes = generateBytes(writeBytes);
-    DFSTestUtil.writeFile(fs, TestPath, new String(bytes));
-
-    //check file length
-    FileStatus status = fs.getFileStatus(TestPath);
-    long fileLength = status.getLen();
-    if (fileLength != writeBytes) {
-      Assert.fail("File Length error: expect=" + writeBytes
-          + ", actual=" + fileLength);
-    }
-
-    DFSStripedInputStream dis = new DFSStripedInputStream(
-        fs.getClient(), src, true);
-    byte[] buf = new byte[writeBytes + 100];
-    int readLen = dis.read(0, buf, 0, buf.length);
-    readLen = readLen >= 0 ? readLen : 0;
-    if (readLen != writeBytes) {
-      Assert.fail("The length of file is not correct.");
-    }
-
-    for (int i = 0; i < writeBytes; i++) {
-      if (getByte(i) != buf[i]) {
-        Assert.fail("Byte at i = " + i + " is wrongly written.");
-      }
-    }
-  }
-
   private void testOneFile(String src, int writeBytes)
       throws IOException {
-    Path TestPath = new Path(src);
+    Path testPath = new Path(src);
 
-    int allBlocks = dataBlocks + parityBlocks;
     byte[] bytes = generateBytes(writeBytes);
-    DFSTestUtil.writeFile(fs, TestPath, new String(bytes));
+    DFSTestUtil.writeFile(fs, testPath, new String(bytes));
 
-    //check file length
-    FileStatus status = fs.getFileStatus(TestPath);
+    // check file length
+    FileStatus status = fs.getFileStatus(testPath);
     long fileLength = status.getLen();
-    if (fileLength != writeBytes) {
-      Assert.fail("File Length error: expect=" + writeBytes
-          + ", actual=" + fileLength);
-    }
+    Assert.assertEquals(writeBytes, fileLength);
 
     List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
     LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L);
 
     for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
-      assert firstBlock instanceof LocatedStripedBlock;
+      Assert.assertTrue(firstBlock instanceof LocatedStripedBlock);
       LocatedBlock[] blocks = StripedBlockUtil.
           parseStripedBlockGroup((LocatedStripedBlock) firstBlock,
               cellSize, dataBlocks, parityBlocks);
@@ -205,15 +176,14 @@ public class TestDFSStripedOutputStream {
       blockGroupList.add(oneGroup);
     }
 
-    //test each block group
+    // 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[allBlocks - dataBlocks][];
+      byte[][] parityBlockBytes = new byte[parityBlocks][];
 
-
-      //for each block, use BlockReader to read data
+      // for each block, use BlockReader to read data
       for (int i = 0; i < blockList.size(); i++) {
         LocatedBlock lblock = blockList.get(i);
         if (lblock == null) {
@@ -269,19 +239,20 @@ public class TestDFSStripedOutputStream {
               }
             }).build();
 
-        blockReader.readAll(blockBytes, 0, (int)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++) {
-        byte[] actualBlkBytes = dataBlockBytes[blkIdxInGroup];
+      // 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++) {
           byte expected;
-          //calculate the postion of this byte in the file
+          // calculate the position of this byte in the file
           long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(cellSize,
               dataBlocks, posInBlk, blkIdxInGroup) +
               group * blockSize * dataBlocks;
@@ -291,15 +262,94 @@ public class TestDFSStripedOutputStream {
             expected = getByte(posInFile);
           }
 
-          if (expected != actualBlkBytes[posInBlk]) {
-            Assert.fail("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);
+          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]);
+        }
+      }
+
+      // verify the parity blocks
+      final ByteBuffer[] parityBuffers = new ByteBuffer[parityBlocks];
+      final long groupSize = lbs.getLocatedBlocks().get(group).getBlockSize();
+      int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength(groupSize,
+          cellSize, dataBlocks, dataBlocks);
+      for (int i = 0; i < parityBlocks; i++) {
+        parityBuffers[i] = ByteBuffer.allocate(parityBlkSize);
+      }
+      final int numStripes = (int) (groupSize - 1) / stripeDataSize() + 1;
+      for (int i = 0; i < numStripes; i++) {
+        final int parityCellSize = i < numStripes - 1 || parityBlkSize % cellSize == 0
+            ? cellSize : parityBlkSize % cellSize;
+        ByteBuffer[] stripeBuf = new ByteBuffer[dataBlocks];
+        for (int k = 0; k < stripeBuf.length; k++) {
+          stripeBuf[k] = ByteBuffer.allocate(cellSize);
+        }
+        for (int j = 0; j < dataBlocks; j++) {
+          if (dataBlockBytes[j] != null) {
+            int length = Math.min(cellSize,
+                dataBlockBytes[j].length - cellSize * i);
+            if (length > 0) {
+              stripeBuf[j].put(dataBlockBytes[j], cellSize * i, length);
+            }
+          }
+          final long pos = stripeBuf[j].position();
+          for (int k = 0; k < parityCellSize - pos; k++) {
+            stripeBuf[j].put((byte) 0);
           }
+          stripeBuf[j].flip();
         }
+        ByteBuffer[] parityBuf = new ByteBuffer[parityBlocks];
+        for (int j = 0; j < parityBlocks; j++) {
+          parityBuf[j] = ByteBuffer.allocate(cellSize);
+          for (int k = 0; k < parityCellSize; k++) {
+            parityBuf[j].put((byte) 0);
+          }
+          parityBuf[j].flip();
+        }
+
+        encoder.encode(stripeBuf, parityBuf);
+        for (int j = 0; j < parityBlocks; j++) {
+          parityBuffers[j].put(parityBuf[j]);
+        }
+      }
+
+      for (int i = 0; i < parityBlocks; i++) {
+        Assert.assertArrayEquals(parityBuffers[i].array(), parityBlockBytes[i]);
       }
     }
   }
 
+  private void testReadWriteOneFile(String src, int writeBytes)
+      throws IOException {
+    Path TestPath = new Path(src);
+    byte[] bytes = generateBytes(writeBytes);
+    DFSTestUtil.writeFile(fs, TestPath, new String(bytes));
+
+    //check file length
+    FileStatus status = fs.getFileStatus(TestPath);
+    long fileLength = status.getLen();
+    if (fileLength != writeBytes) {
+      Assert.fail("File Length error: expect=" + writeBytes
+          + ", actual=" + fileLength);
+    }
+
+    DFSStripedInputStream dis = new DFSStripedInputStream(
+        fs.getClient(), src, true);
+    byte[] buf = new byte[writeBytes + 100];
+    int readLen = dis.read(0, buf, 0, buf.length);
+    readLen = readLen >= 0 ? readLen : 0;
+    if (readLen != writeBytes) {
+      Assert.fail("The length of file is not correct.");
+    }
+
+    for (int i = 0; i < writeBytes; i++) {
+      if (getByte(i) != buf[i]) {
+        Assert.fail("Byte at i = " + i + " is wrongly written.");
+      }
+    }
+  }
 }


[12/50] hadoop git commit: HDFS-8145. Fix the editlog corruption exposed by failed TestAddStripedBlocks. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-8145. Fix the editlog corruption exposed by failed TestAddStripedBlocks. 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/92190424
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/92190424
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/92190424

Branch: refs/heads/HDFS-7285
Commit: 92190424e6eef2253c523ab08dca22b65740436e
Parents: ddf2a1a
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Apr 17 18:13:47 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:16 2015 -0700

----------------------------------------------------------------------
 .../blockmanagement/BlockInfoStriped.java       |  7 ------
 .../namenode/ErasureCodingZoneManager.java      | 12 +++++-----
 .../hdfs/server/namenode/FSDirectory.java       |  6 ++---
 .../hdfs/server/namenode/FSEditLogLoader.java   | 13 ++++++-----
 .../hdfs/server/namenode/FSImageFormat.java     |  4 +---
 .../server/namenode/FSImageSerialization.java   | 13 +++++------
 .../blockmanagement/TestBlockInfoStriped.java   | 23 ++++++--------------
 .../hdfs/server/namenode/TestFSImage.java       |  2 +-
 8 files changed, 31 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/92190424/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 9f2f5ba..23e3153 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
@@ -244,13 +244,6 @@ public class BlockInfoStriped extends BlockInfo {
     return num;
   }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeShort(dataBlockNum);
-    out.writeShort(parityBlockNum);
-    super.write(out);
-  }
-
   /**
    * Convert a complete block to an under construction block.
    * @return BlockInfoUnderConstruction -  an under construction block.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92190424/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
index 0a84083..3f94227 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
@@ -54,10 +54,6 @@ public class ErasureCodingZoneManager {
     this.dir = dir;
   }
 
-  boolean getECPolicy(INodesInPath iip) throws IOException {
-    return getECSchema(iip) != null;
-  }
-
   ECSchema getECSchema(INodesInPath iip) throws IOException {
     ECZoneInfo ecZoneInfo = getECZoneInfo(iip);
     return ecZoneInfo == null ? null : ecZoneInfo.getSchema();
@@ -109,7 +105,7 @@ public class ErasureCodingZoneManager {
       throw new IOException("Attempt to create an erasure coding zone " +
           "for a file.");
     }
-    if (getECPolicy(srcIIP)) {
+    if (getECSchema(srcIIP) != null) {
       throw new IOException("Directory " + src + " is already in an " +
           "erasure coding zone.");
     }
@@ -132,8 +128,10 @@ public class ErasureCodingZoneManager {
   void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src)
       throws IOException {
     assert dir.hasReadLock();
-    if (getECPolicy(srcIIP)
-        != getECPolicy(dstIIP)) {
+    final ECSchema srcSchema = getECSchema(srcIIP);
+    final ECSchema dstSchema = getECSchema(dstIIP);
+    if ((srcSchema != null && !srcSchema.equals(dstSchema)) ||
+        (dstSchema != null && !dstSchema.equals(srcSchema))) {
       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/92190424/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 46852d6..0d3ec51 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
@@ -433,7 +433,7 @@ public class FSDirectory implements Closeable {
     writeLock();
     try {
       newiip = addINode(existing, newNode);
-      if (newiip != null && getECPolicy(newiip)) {
+      if (newiip != null && isInECZone(newiip)) {
         newNode.addStripedBlocksFeature();
       }
     } finally {
@@ -472,7 +472,7 @@ public class FSDirectory implements Closeable {
       INodesInPath iip = addINode(existing, newNode);
       if (iip != null) {
         // check if the file is in an EC zone
-        if (getECPolicy(iip)) {
+        if (isInECZone(iip)) {
           newNode.addStripedBlocksFeature();
         }
         if (aclEntries != null) {
@@ -1403,7 +1403,7 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  public boolean getECPolicy(INodesInPath iip) throws IOException {
+  public boolean isInECZone(INodesInPath iip) throws IOException {
     return getECSchema(iip) != null;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92190424/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 d3b920e..5906ecf 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
@@ -40,6 +40,7 @@ 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.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -419,7 +420,7 @@ public class FSEditLogLoader {
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.getECPolicy(iip));
+      updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.isInECZone(iip));
       break;
     }
     case OP_CLOSE: {
@@ -440,7 +441,7 @@ public class FSEditLogLoader {
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, addCloseOp, iip, file, fsDir.getECPolicy(iip));
+      updateBlocks(fsDir, addCloseOp, iip, file, fsDir.isInECZone(iip));
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -499,7 +500,7 @@ public class FSEditLogLoader {
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.getECPolicy(iip));
+      updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.isInECZone(iip));
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
@@ -517,7 +518,7 @@ public class FSEditLogLoader {
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // add the new block to the INodeFile
       // TODO whether the file is striped should later be retrieved from iip
-      addNewBlock(addBlockOp, oldFile, fsDir.getECPolicy(iip));
+      addNewBlock(addBlockOp, oldFile, fsDir.isInECZone(iip));
       break;
     }
     case OP_SET_REPLICATION: {
@@ -1080,7 +1081,9 @@ public class FSEditLogLoader {
           // 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.getBlockReplication());
+          newBI = isStriped ? new BlockInfoStriped(newBlock,
+              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS) :
+              new BlockInfoContiguous(newBlock, file.getBlockReplication());
         }
         fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);
         file.addBlock(newBI);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92190424/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 54d0d30..6f485f5 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
@@ -771,10 +771,8 @@ public class FSImageFormat {
       if (isStriped) {
         blocks = new Block[numBlocks];
         for (int j = 0; j < numBlocks; j++) {
-          short dataBlockNum = in.readShort();
-          short parityBlockNum = in.readShort();
           blocks[j] = new BlockInfoStriped(new Block(),
-                  dataBlockNum, parityBlockNum);
+              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
           blocks[j].readFields(in);
         }
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92190424/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 58244e5..25febd4 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
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
@@ -139,17 +140,15 @@ public class FSImageSerialization {
       blocksStriped = new BlockInfoStriped[numBlocks];
       int i = 0;
       for (; i < numBlocks - 1; i++) {
-        short dataBlockNum = in.readShort();
-        short parityBlockNum = in.readShort();
-        blocksStriped[i] = new BlockInfoStriped(new Block(), dataBlockNum,
-            parityBlockNum);
+        blocksStriped[i] = new BlockInfoStriped(new Block(),
+            HdfsConstants.NUM_DATA_BLOCKS,
+            HdfsConstants.NUM_PARITY_BLOCKS);
         blocksStriped[i].readFields(in);
       }
       if (numBlocks > 0) {
-        short dataBlockNum = in.readShort();
-        short parityBlockNum = in.readShort();
         blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(),
-            dataBlockNum, parityBlockNum, BlockUCState.UNDER_CONSTRUCTION, null);
+            HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS,
+            BlockUCState.UNDER_CONSTRUCTION, null);
         blocksStriped[i].readFields(in);
       }
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92190424/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 c4db5d4..3b689eb 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
@@ -21,7 +21,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
@@ -43,12 +42,8 @@ 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 BlockInfoStriped info;
-
-  @Before
-  public void setup() {
-    info = new BlockInfoStriped(baseBlock, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
-  }
+  private final BlockInfoStriped info = new BlockInfoStriped(baseBlock,
+      NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
 
   private Block[] createReportedBlocks(int num) {
     Block[] blocks = new Block[num];
@@ -230,17 +225,14 @@ public class TestBlockInfoStriped {
     long blkID = 1;
     long numBytes = 1;
     long generationStamp = 1;
-    short dataBlockNum = 6;
-    short parityBlockNum = 3;
-    ByteBuffer byteBuffer = ByteBuffer.allocate(Long.SIZE/Byte.SIZE*3
-            + Short.SIZE/Byte.SIZE*2);
-    byteBuffer.putShort(dataBlockNum).putShort(parityBlockNum)
-            .putLong(blkID).putLong(numBytes).putLong(generationStamp);
+    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(1,1,1),
-            (short)6,(short)3);
+    BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes,
+        generationStamp), NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+
     try {
       blk.write(out);
     } catch(Exception ex) {
@@ -249,5 +241,4 @@ public class TestBlockInfoStriped {
     assertEquals(byteBuffer.array().length, byteStream.toByteArray().length);
     assertArrayEquals(byteBuffer.array(), byteStream.toByteArray());
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92190424/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 c482f1f..bb37534 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
@@ -158,7 +158,7 @@ public class TestFSImage {
     for (int i = 0; i < stripedBlks.length; i++) {
       stripedBlks[i] = new BlockInfoStriped(
               new Block(stripedBlkId + i, preferredBlockSize, timestamp),
-              (short) 6, (short) 3);
+              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
       file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
     }
 


[17/50] hadoop git commit: HDFS-8156. Add/implement necessary APIs even we just have the system default schema. Contributed by Kai Zheng.

Posted by zh...@apache.org.
HDFS-8156. Add/implement necessary APIs even we just have the system default schema. Contributed by Kai Zheng.


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

Branch: refs/heads/HDFS-7285
Commit: b8aff2dc5da24c486b39e666805f8c60402ff147
Parents: eba862a
Author: Zhe Zhang <zh...@apache.org>
Authored: Wed Apr 22 14:48:54 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:18 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/io/erasurecode/ECSchema.java  | 173 +++++++++++--------
 .../hadoop/io/erasurecode/TestECSchema.java     |   2 +-
 .../hadoop/io/erasurecode/TestSchemaLoader.java |   6 +-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |   2 +-
 .../hdfs/server/namenode/ECSchemaManager.java   |  79 ++++++++-
 .../namenode/ErasureCodingZoneManager.java      |  16 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  29 +++-
 .../org/apache/hadoop/hdfs/TestECSchemas.java   |   5 +-
 .../hadoop/hdfs/TestErasureCodingZones.java     |  45 +++--
 10 files changed, 249 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8aff2dc/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
index 32077f6..f058ea7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.io.erasurecode;
 
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 
 /**
@@ -30,55 +31,80 @@ public final class ECSchema {
   public static final String CHUNK_SIZE_KEY = "chunkSize";
   public static final int DEFAULT_CHUNK_SIZE = 256 * 1024; // 256K
 
-  private String schemaName;
-  private String codecName;
-  private Map<String, String> options;
-  private int numDataUnits;
-  private int numParityUnits;
-  private int chunkSize;
+  /**
+   * A friendly and understandable name that can mean what's it, also serves as
+   * the identifier that distinguish it from other schemas.
+   */
+  private final String schemaName;
+
+  /**
+   * The erasure codec name associated.
+   */
+  private final String codecName;
+
+  /**
+   * Number of source data units coded
+   */
+  private final int numDataUnits;
+
+  /**
+   * Number of parity units generated in a coding
+   */
+  private final int numParityUnits;
+
+  /**
+   * Unit data size for each chunk in a coding
+   */
+  private final int chunkSize;
+
+  /*
+   * An erasure code can have its own specific advanced parameters, subject to
+   * itself to interpret these key-value settings.
+   */
+  private final Map<String, String> extraOptions;
 
   /**
-   * Constructor with schema name and provided options. Note the options may
+   * Constructor with schema name and provided all options. Note the options may
    * contain additional information for the erasure codec to interpret further.
    * @param schemaName schema name
-   * @param options schema options
+   * @param allOptions all schema options
    */
-  public ECSchema(String schemaName, Map<String, String> options) {
+  public ECSchema(String schemaName, Map<String, String> allOptions) {
     assert (schemaName != null && ! schemaName.isEmpty());
 
     this.schemaName = schemaName;
 
-    if (options == null || options.isEmpty()) {
+    if (allOptions == null || allOptions.isEmpty()) {
       throw new IllegalArgumentException("No schema options are provided");
     }
 
-    String codecName = options.get(CODEC_NAME_KEY);
+    this.codecName = allOptions.get(CODEC_NAME_KEY);
     if (codecName == null || codecName.isEmpty()) {
       throw new IllegalArgumentException("No codec option is provided");
     }
 
-    int dataUnits = 0, parityUnits = 0;
-    try {
-      if (options.containsKey(NUM_DATA_UNITS_KEY)) {
-        dataUnits = Integer.parseInt(options.get(NUM_DATA_UNITS_KEY));
-      }
-    } catch (NumberFormatException e) {
-      throw new IllegalArgumentException("Option value " +
-          options.get(NUM_DATA_UNITS_KEY) + " for " + NUM_DATA_UNITS_KEY +
-          " is found. It should be an integer");
+    int tmpNumDataUnits = extractIntOption(NUM_DATA_UNITS_KEY, allOptions);
+    int tmpNumParityUnits = extractIntOption(NUM_PARITY_UNITS_KEY, allOptions);
+    if (tmpNumDataUnits < 0 || tmpNumParityUnits < 0) {
+      throw new IllegalArgumentException(
+          "No good option for numDataUnits or numParityUnits found ");
     }
-
-    try {
-      if (options.containsKey(NUM_PARITY_UNITS_KEY)) {
-        parityUnits = Integer.parseInt(options.get(NUM_PARITY_UNITS_KEY));
-      }
-    } catch (NumberFormatException e) {
-      throw new IllegalArgumentException("Option value " +
-          options.get(NUM_PARITY_UNITS_KEY) + " for " + NUM_PARITY_UNITS_KEY +
-          " is found. It should be an integer");
+    this.numDataUnits = tmpNumDataUnits;
+    this.numParityUnits = tmpNumParityUnits;
+
+    int tmpChunkSize = extractIntOption(CHUNK_SIZE_KEY, allOptions);
+    if (tmpChunkSize > 0) {
+      this.chunkSize = tmpChunkSize;
+    } else {
+      this.chunkSize = DEFAULT_CHUNK_SIZE;
     }
 
-    initWith(codecName, dataUnits, parityUnits, options);
+    allOptions.remove(CODEC_NAME_KEY);
+    allOptions.remove(NUM_DATA_UNITS_KEY);
+    allOptions.remove(NUM_PARITY_UNITS_KEY);
+    allOptions.remove(CHUNK_SIZE_KEY);
+    // After some cleanup
+    this.extraOptions = Collections.unmodifiableMap(allOptions);
   }
 
   /**
@@ -94,48 +120,60 @@ public final class ECSchema {
   }
 
   /**
-   * Constructor with key parameters provided. Note the options may contain
+   * Constructor with key parameters provided. Note the extraOptions may contain
    * additional information for the erasure codec to interpret further.
    * @param schemaName
    * @param codecName
    * @param numDataUnits
    * @param numParityUnits
-   * @param options
+   * @param extraOptions
    */
-  public ECSchema(String schemaName, String codecName,
-                  int numDataUnits, int numParityUnits,
-                  Map<String, String> options) {
+  public ECSchema(String schemaName, String codecName, int numDataUnits,
+                  int numParityUnits, Map<String, String> extraOptions) {
+
     assert (schemaName != null && ! schemaName.isEmpty());
     assert (codecName != null && ! codecName.isEmpty());
+    assert (numDataUnits > 0 && numParityUnits > 0);
 
     this.schemaName = schemaName;
-    initWith(codecName, numDataUnits, numParityUnits, options);
-  }
-
-  private void initWith(String codecName, int numDataUnits, int numParityUnits,
-                        Map<String, String> options) {
     this.codecName = codecName;
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
 
-    this.options = options != null ? Collections.unmodifiableMap(options) :
-        Collections.EMPTY_MAP;
+    if (extraOptions == null) {
+      extraOptions = new HashMap<>();
+    }
+
+    int tmpChunkSize = extractIntOption(CHUNK_SIZE_KEY, extraOptions);
+    if (tmpChunkSize > 0) {
+      this.chunkSize = tmpChunkSize;
+    } else {
+      this.chunkSize = DEFAULT_CHUNK_SIZE;
+    }
+
+    extraOptions.remove(CHUNK_SIZE_KEY);
+    // After some cleanup
+    this.extraOptions = Collections.unmodifiableMap(extraOptions);
+  }
+
+  private int extractIntOption(String optionKey, Map<String, String> options) {
+    int result = -1;
 
-    this.chunkSize = DEFAULT_CHUNK_SIZE;
     try {
-      if (this.options.containsKey(CHUNK_SIZE_KEY)) {
-        this.chunkSize = Integer.parseInt(options.get(CHUNK_SIZE_KEY));
+      if (options.containsKey(optionKey)) {
+        result = Integer.parseInt(options.get(optionKey));
+        if (result <= 0) {
+          throw new IllegalArgumentException("Bad option value " + result +
+              " found for " + optionKey);
+        }
       }
     } catch (NumberFormatException e) {
       throw new IllegalArgumentException("Option value " +
-          this.options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
+          options.get(optionKey) + " for " + optionKey +
           " is found. It should be an integer");
     }
 
-    boolean isFine = numDataUnits > 0 && numParityUnits > 0 && chunkSize > 0;
-    if (! isFine) {
-      throw new IllegalArgumentException("Bad codec options are found");
-    }
+    return result;
   }
 
   /**
@@ -155,11 +193,11 @@ public final class ECSchema {
   }
 
   /**
-   * Get erasure coding options
-   * @return encoding options
+   * Get extra options specific to a erasure code.
+   * @return extra options
    */
-  public Map<String, String> getOptions() {
-    return options;
+  public Map<String, String> getExtraOptions() {
+    return extraOptions;
   }
 
   /**
@@ -194,18 +232,17 @@ public final class ECSchema {
   public String toString() {
     StringBuilder sb = new StringBuilder("ECSchema=[");
 
-    sb.append("Name=" + schemaName + ",");
-    sb.append(NUM_DATA_UNITS_KEY + "=" + numDataUnits + ",");
-    sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits + ",");
-    sb.append(CHUNK_SIZE_KEY + "=" + chunkSize + ",");
-
-    for (String opt : options.keySet()) {
-      boolean skip = (opt.equals(NUM_DATA_UNITS_KEY) ||
-          opt.equals(NUM_PARITY_UNITS_KEY) ||
-          opt.equals(CHUNK_SIZE_KEY));
-      if (! skip) {
-        sb.append(opt + "=" + options.get(opt) + ",");
-      }
+    sb.append("Name=" + schemaName + ", ");
+    sb.append("Codec=" + codecName + ", ");
+    sb.append(NUM_DATA_UNITS_KEY + "=" + numDataUnits + ", ");
+    sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits + ", ");
+    sb.append(CHUNK_SIZE_KEY + "=" + chunkSize +
+        (extraOptions.isEmpty() ? "" : ", "));
+
+    int i = 0;
+    for (String opt : extraOptions.keySet()) {
+      sb.append(opt + "=" + extraOptions.get(opt) +
+          (++i < extraOptions.size() ? ", " : ""));
     }
 
     sb.append("]");
@@ -239,14 +276,14 @@ public final class ECSchema {
     if (!codecName.equals(ecSchema.codecName)) {
       return false;
     }
-    return options.equals(ecSchema.options);
+    return extraOptions.equals(ecSchema.extraOptions);
   }
 
   @Override
   public int hashCode() {
     int result = schemaName.hashCode();
     result = 31 * result + codecName.hashCode();
-    result = 31 * result + options.hashCode();
+    result = 31 * result + extraOptions.hashCode();
     result = 31 * result + numDataUnits;
     result = 31 * result + numParityUnits;
     result = 31 * result + chunkSize;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8aff2dc/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java
index 4285ef0..15e672f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java
@@ -49,6 +49,6 @@ public class TestECSchema {
     assertEquals(numParityUnits, schema.getNumParityUnits());
     assertEquals(chunkSize, schema.getChunkSize());
     assertEquals(codec, schema.getCodecName());
-    assertEquals(extraOptionValue, schema.getOptions().get(extraOption));
+    assertEquals(extraOptionValue, schema.getExtraOptions().get(extraOption));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8aff2dc/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
index 6caeedb..939fa9b 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
@@ -59,16 +59,16 @@ public class TestSchemaLoader {
 
     ECSchema schema1 = schemas.get(0);
     assertEquals("RSk6m3", schema1.getSchemaName());
-    assertEquals(3, schema1.getOptions().size());
+    assertEquals(0, schema1.getExtraOptions().size());
     assertEquals(6, schema1.getNumDataUnits());
     assertEquals(3, schema1.getNumParityUnits());
     assertEquals("RS", schema1.getCodecName());
 
     ECSchema schema2 = schemas.get(1);
     assertEquals("RSk10m4", schema2.getSchemaName());
-    assertEquals(3, schema2.getOptions().size());
+    assertEquals(0, schema2.getExtraOptions().size());
     assertEquals(10, schema2.getNumDataUnits());
     assertEquals(4, schema2.getNumParityUnits());
     assertEquals("RS", schema2.getCodecName());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8aff2dc/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 1acde41..b2faac0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -116,3 +116,6 @@
 
     HDFS-8024. Erasure Coding: ECworker frame, basics, bootstraping and configuration.
     (umamahesh)
+
+    HDFS-8156. Add/implement necessary APIs even we just have the system default 
+    schema. (Kai Zheng via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8aff2dc/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 68cfe7f..3cd3e03 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
@@ -3143,7 +3143,7 @@ public class PBHelper {
         .setCodecName(schema.getCodecName())
         .setDataUnits(schema.getNumDataUnits())
         .setParityUnits(schema.getNumParityUnits());
-    Set<Entry<String, String>> entrySet = schema.getOptions().entrySet();
+    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());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8aff2dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
index b001c57..2d63498 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
@@ -20,22 +20,62 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 
+import java.util.Map;
+import java.util.TreeMap;
+
 /**
- * This manages EC schemas predefined and activated in the system. It loads from
- * predefined ones in XML and syncs with persisted ones in NameNode image.
+ * This manages EC schemas predefined and activated in the system.
+ * It loads customized schemas and syncs with persisted ones in
+ * NameNode image.
  *
  * This class is instantiated by the FSNamesystem.
  */
 @InterfaceAudience.LimitedPrivate({"HDFS"})
 public final class ECSchemaManager {
 
+  /**
+   * TODO: HDFS-8095
+   */
   private static final int DEFAULT_DATA_BLOCKS = 6;
   private static final int DEFAULT_PARITY_BLOCKS = 3;
   private static final String DEFAULT_CODEC_NAME = "rs";
-  private static final String DEFAULT_SCHEMA_NAME = "SYS-DEFAULT-RS-6-3";
+  private static final String DEFAULT_SCHEMA_NAME = "RS-6-3";
+  private static final ECSchema SYS_DEFAULT_SCHEMA =
+      new ECSchema(DEFAULT_SCHEMA_NAME,
+               DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS);
+
+  //We may add more later.
+  private static ECSchema[] SYS_SCHEMAS = new ECSchema[] {
+      SYS_DEFAULT_SCHEMA
+  };
+
+  /**
+   * All active EC activeSchemas maintained in NN memory for fast querying,
+   * identified and sorted by its name.
+   */
+  private final Map<String, ECSchema> activeSchemas;
 
-  private static ECSchema SYS_DEFAULT_SCHEMA = new ECSchema(DEFAULT_SCHEMA_NAME,
-      DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS);
+  ECSchemaManager() {
+
+    this.activeSchemas = new TreeMap<String, ECSchema>();
+    for (ECSchema schema : SYS_SCHEMAS) {
+      activeSchemas.put(schema.getSchemaName(), schema);
+    }
+
+    /**
+     * TODO: HDFS-7859 persist into NameNode
+     * load persistent schemas from image and editlog, which is done only once
+     * during NameNode startup. This can be done here or in a separate method.
+     */
+  }
+
+  /**
+   * Get system defined schemas.
+   * @return system schemas
+   */
+  public static ECSchema[] getSystemSchemas() {
+    return SYS_SCHEMAS;
+  }
 
   /**
    * Get system-wide default EC schema, which can be used by default when no
@@ -56,7 +96,32 @@ public final class ECSchemaManager {
       throw new IllegalArgumentException("Invalid schema parameter");
     }
 
-    // schema name is the identifier, but for safety we check all properties.
-    return SYS_DEFAULT_SCHEMA.equals(schema);
+    // schema name is the identifier.
+    return SYS_DEFAULT_SCHEMA.getSchemaName().equals(schema.getSchemaName());
+  }
+
+  /**
+   * Get all EC schemas that's available to use.
+   * @return all EC schemas
+   */
+  public ECSchema[] getSchemas() {
+    ECSchema[] results = new ECSchema[activeSchemas.size()];
+    return activeSchemas.values().toArray(results);
+  }
+
+  /**
+   * Get the EC schema specified by the schema name.
+   * @param schemaName
+   * @return EC schema specified by the schema name
+   */
+  public ECSchema getSchema(String schemaName) {
+    return activeSchemas.get(schemaName);
+  }
+
+  /**
+   * Clear and clean up
+   */
+  public void clear() {
+    activeSchemas.clear();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8aff2dc/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
index 3f94227..8cda289 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
@@ -23,8 +23,6 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import java.io.IOException;
@@ -80,9 +78,8 @@ public class ErasureCodingZoneManager {
           : inode.getXAttrFeature().getXAttrs();
       for (XAttr xAttr : xAttrs) {
         if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
-          ECSchemaProto ecSchemaProto;
-          ecSchemaProto = ECSchemaProto.parseFrom(xAttr.getValue());
-          ECSchema schema = PBHelper.convertECSchema(ecSchemaProto);
+          String schemaName = new String(xAttr.getValue());
+          ECSchema schema = dir.getFSNamesystem().getECSchema(schemaName);
           return new ECZoneInfo(inode.getFullPathName(), schema);
         }
       }
@@ -109,13 +106,14 @@ public class ErasureCodingZoneManager {
       throw new IOException("Directory " + src + " is already in an " +
           "erasure coding zone.");
     }
-    // TODO HDFS-7859 Need to persist the schema in xattr in efficient way
-    // As of now storing the protobuf format
+
+    // System default schema will be used since no specified.
     if (schema == null) {
       schema = ECSchemaManager.getSystemDefaultSchema();
     }
-    ECSchemaProto schemaProto = PBHelper.convertECSchema(schema);
-    byte[] schemaBytes = schemaProto.toByteArray();
+
+    // Now persist the schema name in xattr
+    byte[] schemaBytes = schema.getSchemaName().getBytes();
     final XAttr ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE,
         schemaBytes);
     final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8aff2dc/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 c77b193..73048bc 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
@@ -431,6 +431,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private final BlockManager blockManager;
   private final SnapshotManager snapshotManager;
   private final CacheManager cacheManager;
+  private final ECSchemaManager schemaManager;
   private final DatanodeStatistics datanodeStatistics;
 
   private String nameserviceId;
@@ -610,6 +611,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     leaseManager.removeAllLeases();
     snapshotManager.clearSnapshottableDirs();
     cacheManager.clear();
+    schemaManager.clear();
     setImageLoaded(false);
     blockManager.clear();
   }
@@ -849,6 +851,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       this.dir = new FSDirectory(this, conf);
       this.snapshotManager = new SnapshotManager(dir);
       this.cacheManager = new CacheManager(this, conf, blockManager);
+      this.schemaManager = new ECSchemaManager();
       this.safeMode = new SafeModeInfo(conf);
       this.topConf = new TopConf(conf);
       this.auditLoggers = initAuditLoggers(conf);
@@ -7207,16 +7210,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   public FSDirectory getFSDirectory() {
     return dir;
   }
+
   /** Set the FSDirectory. */
   @VisibleForTesting
   public void setFSDirectory(FSDirectory dir) {
     this.dir = dir;
   }
+
   /** @return the cache manager. */
   public CacheManager getCacheManager() {
     return cacheManager;
   }
 
+  /** @return the schema manager. */
+  public ECSchemaManager getSchemaManager() {
+    return schemaManager;
+  }
+
   @Override  // NameNodeMXBean
   public String getCorruptFiles() {
     List<String> list = new ArrayList<String>();
@@ -8217,9 +8227,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      // TODO HDFS-7866 Need to return all schemas maintained by Namenode
-      ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
-      return new ECSchema[] { defaultSchema };
+      return schemaManager.getSchemas();
+    } finally {
+      readUnlock();
+    }
+  }
+
+  /**
+   * Get the ECSchema specified by the name
+   */
+  ECSchema getECSchema(String schemaName) throws IOException {
+    checkOperation(OperationCategory.READ);
+    waitForLoadingFSImage();
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      return schemaManager.getSchema(schemaName);
     } finally {
       readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8aff2dc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
index 07e1359..83d208a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
@@ -48,10 +48,7 @@ public class TestECSchemas {
   @Test
   public void testGetECSchemas() throws Exception {
     ECSchema[] ecSchemas = cluster.getFileSystem().getClient().getECSchemas();
-    // TODO update assertion after HDFS-7866
     assertNotNull(ecSchemas);
-    assertEquals("Should have only one ecSchema", 1, ecSchemas.length);
-    assertEquals("Returned schemas should have only default schema",
-        ECSchemaManager.getSystemDefaultSchema(), ecSchemas[0]);
+    assertTrue("Should have at least one schema", ecSchemas.length > 0);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8aff2dc/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 699df4e..ae2bdd8 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
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.ECInfo;
+import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.io.erasurecode.ECSchema;
@@ -151,35 +152,49 @@ public class TestErasureCodingZones {
   }
 
   @Test
-  public void testGetErasureCodingInfo() throws Exception {
+  public void testGetErasureCodingInfoWithSystemDefaultSchema() 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().getErasureCodingInfo(src));
     // dir ECInfo after creating ec zone
-    fs.getClient().createErasureCodingZone(src, null);
-    verifyErasureCodingInfo(src);
+    fs.getClient().createErasureCodingZone(src, null); //Default one will be used.
+    ECSchema sysDefaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    verifyErasureCodingInfo(src, sysDefaultSchema);
+    fs.create(new Path(ecDir, "/child1")).close();
+    // verify for the files in ec zone
+    verifyErasureCodingInfo(src + "/child1", sysDefaultSchema);
+  }
+
+  @Test
+  public void testGetErasureCodingInfo() throws Exception {
+    ECSchema[] sysSchemas = ECSchemaManager.getSystemSchemas();
+    assertTrue("System schemas should be of only 1 for now",
+        sysSchemas.length == 1);
+
+    ECSchema usingSchema = sysSchemas[0];
+    String src = "/ec2";
+    final Path ecDir = new Path(src);
+    fs.mkdir(ecDir, FsPermission.getDirDefault());
+    // dir ECInfo before creating ec zone
+    assertNull(fs.getClient().getErasureCodingInfo(src));
+    // dir ECInfo after creating ec zone
+    fs.getClient().createErasureCodingZone(src, usingSchema);
+    verifyErasureCodingInfo(src, usingSchema);
     fs.create(new Path(ecDir, "/child1")).close();
     // verify for the files in ec zone
-    verifyErasureCodingInfo(src + "/child1");
+    verifyErasureCodingInfo(src + "/child1", usingSchema);
   }
 
-  private void verifyErasureCodingInfo(String src) throws IOException {
+  private void verifyErasureCodingInfo(
+      String src, ECSchema usingSchema) throws IOException {
     ECInfo ecInfo = fs.getClient().getErasureCodingInfo(src);
     assertNotNull("ECInfo should have been non-null", ecInfo);
     assertEquals(src, ecInfo.getSrc());
     ECSchema schema = ecInfo.getSchema();
     assertNotNull(schema);
-    assertEquals("Default schema should be returned", "RS-6-3",
-        schema.getSchemaName());
-    assertEquals("Default codec(rs) should be returned", "rs",
-        schema.getCodecName());
-    assertEquals("Default numDataUnits should be used", 6,
-        schema.getNumDataUnits());
-    assertEquals("Default numParityUnits should be used", 3,
-        schema.getNumParityUnits());
-    assertEquals("Default chunkSize should be used",
-        ECSchema.DEFAULT_CHUNK_SIZE, schema.getChunkSize());
+    assertEquals("Actually used schema should be equal with target schema",
+        usingSchema, schema);
   }
 }


[42/50] hadoop git commit: HDFS-8334. Erasure coding: rename DFSStripedInputStream related test classes. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-8334. Erasure coding: rename DFSStripedInputStream related test classes. Contributed by Zhe Zhang.


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

Branch: refs/heads/HDFS-7285
Commit: 2a98cee8ed5293bcac7d902cb0eaba79a5492969
Parents: 97691e5
Author: Zhe Zhang <zh...@apache.org>
Authored: Wed May 6 15:34:37 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:40:57 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   5 +
 .../hadoop/hdfs/TestDFSStripedInputStream.java  | 365 ++++++++-----------
 .../apache/hadoop/hdfs/TestReadStripedFile.java | 218 -----------
 .../hadoop/hdfs/TestWriteReadStripedFile.java   | 261 +++++++++++++
 4 files changed, 427 insertions(+), 422 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a98cee8/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 0d2d448..8729f8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -178,3 +178,8 @@
 
     HDFS-7348. Erasure Coding: DataNode reconstruct striped blocks. 
     (Yi Liu via Zhe Zhang)
+
+    HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng)
+
+    HDFS-8334. Erasure coding: rename DFSStripedInputStream related test 
+    classes. (Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a98cee8/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 11cdf7b..a1f704d 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
@@ -17,245 +17,202 @@
  */
 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.FSDataInputStream;
-import org.apache.hadoop.fs.FileStatus;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+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.ECInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
 
 public class TestDFSStripedInputStream {
-  private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
-  private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
-
-
-  private static DistributedFileSystem fs;
-  private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
-  private final static int stripesPerBlock = 4;
-  static int blockSize = cellSize * stripesPerBlock;
-  static int numDNs = dataBlocks + parityBlocks + 2;
-
-  private static MiniDFSCluster cluster;
 
-  @BeforeClass
-  public static void setup() throws IOException {
-    Configuration conf = new Configuration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+  public static final Log LOG = LogFactory.getLog(TestDFSStripedInputStream.class);
+
+  private MiniDFSCluster cluster;
+  private Configuration conf = new Configuration();
+  private DistributedFileSystem fs;
+  private final Path dirPath = new Path("/striped");
+  private Path filePath = new Path(dirPath, "file");
+  private ECInfo info = new ECInfo(filePath.toString(),
+      ECSchemaManager.getSystemDefaultSchema());
+  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 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;
+
+  @Before
+  public void setup() throws IOException {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE);
+    SimulatedFSDataset.setFactory(conf);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
+        DATA_BLK_NUM + PARITY_BLK_NUM).build();
+    cluster.waitActive();
     fs = cluster.getFileSystem();
+    fs.mkdirs(dirPath);
+    fs.getClient().createErasureCodingZone(dirPath.toString(), null);
   }
 
-  @AfterClass
-  public static void tearDown() {
+  @After
+  public void tearDown() {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
 
+  /**
+   * Test {@link DFSStripedInputStream#getBlockAt(long)}
+   */
   @Test
-  public void testFileEmpty() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/EmptyFile", 0);
-  }
-
-  @Test
-  public void testFileSmallerThanOneCell1() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1);
-  }
-
-  @Test
-  public void testFileSmallerThanOneCell2() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1);
-  }
-
-  @Test
-  public void testFileEqualsWithOneCell() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize);
-  }
-
-  @Test
-  public void testFileSmallerThanOneStripe1() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe",
-        cellSize * dataBlocks - 1);
-  }
-
-  @Test
-  public void testFileSmallerThanOneStripe2() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe",
-        cellSize + 123);
-  }
-
-  @Test
-  public void testFileEqualsWithOneStripe() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe",
-        cellSize * dataBlocks);
-  }
-
-  @Test
-  public void testFileMoreThanOneStripe1() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1",
-        cellSize * dataBlocks + 123);
-  }
-
-  @Test
-  public void testFileMoreThanOneStripe2() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2",
-        cellSize * dataBlocks + cellSize * dataBlocks + 123);
-  }
-
-  @Test
-  public void testLessThanFullBlockGroup() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup",
-        cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize);
-  }
-
-  @Test
-  public void testFileFullBlockGroup() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/FullBlockGroup",
-        blockSize * dataBlocks);
+  public void testGetBlock() throws Exception {
+    final int numBlocks = 4;
+    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
+    final DFSStripedInputStream in =
+        new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, info);
+
+    List<LocatedBlock> lbList = lbs.getLocatedBlocks();
+    for (LocatedBlock aLbList : lbList) {
+      LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList;
+      LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb,
+          CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM);
+      for (int j = 0; j < DATA_BLK_NUM; j++) {
+        LocatedBlock refreshed = in.getBlockAt(blks[j].getStartOffset());
+        assertEquals(blks[j].getBlock(), refreshed.getBlock());
+        assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset());
+        assertArrayEquals(blks[j].getLocations(), refreshed.getLocations());
+      }
+    }
   }
 
   @Test
-  public void testFileMoreThanABlockGroup1() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1",
-        blockSize * dataBlocks + 123);
-  }
+  public void testPread() throws Exception {
+    final int numBlocks = 2;
+    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCK_GROUP_SIZE);
+
+    assert lbs.get(0) instanceof LocatedStripedBlock;
+    LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0));
+    for (int i = 0; i < DATA_BLK_NUM; i++) {
+      Block blk = new Block(bg.getBlock().getBlockId() + i,
+          NUM_STRIPE_PER_BLOCK * CELLSIZE,
+          bg.getBlock().getGenerationStamp());
+      blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
+      cluster.injectBlocks(i, Arrays.asList(blk),
+          bg.getBlock().getBlockPoolId());
+    }
+    DFSStripedInputStream in =
+        new DFSStripedInputStream(fs.getClient(),
+            filePath.toString(), false, info);
+    int readSize = BLOCK_GROUP_SIZE;
+    byte[] readBuffer = new byte[readSize];
+    int ret = in.read(0, readBuffer, 0, readSize);
 
-  @Test
-  public void testFileMoreThanABlockGroup2() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2",
-        blockSize * dataBlocks + cellSize+ 123);
+    assertEquals(readSize, ret);
+    // TODO: verify read results with patterned data from HDFS-8117
   }
 
-
   @Test
-  public void testFileMoreThanABlockGroup3() throws IOException {
-    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3",
-        blockSize * dataBlocks * 3 + cellSize * dataBlocks
-            + cellSize + 123);
-  }
-
-  private byte[] generateBytes(int cnt) {
-    byte[] bytes = new byte[cnt];
-    for (int i = 0; i < cnt; i++) {
-      bytes[i] = getByte(i);
+  public void testStatefulRead() throws Exception {
+    testStatefulRead(false, false);
+    testStatefulRead(true, false);
+    testStatefulRead(true, true);
+  }
+
+  private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket)
+      throws Exception {
+    final int numBlocks = 2;
+    final int fileSize = numBlocks * BLOCK_GROUP_SIZE;
+    if (cellMisalignPacket) {
+      conf.setInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT + 1);
+      tearDown();
+      setup();
     }
-    return bytes;
-  }
-
-  private byte getByte(long pos) {
-    final int mod = 29;
-    return (byte) (pos % mod + 1);
-  }
-
-  private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes)
-      throws IOException {
-    Path testPath = new Path(src);
-    final byte[] bytes = generateBytes(writeBytes);
-    DFSTestUtil.writeFile(fs, testPath, new String(bytes));
-
-    //check file length
-    FileStatus status = fs.getFileStatus(testPath);
-    long fileLength = status.getLen();
-    Assert.assertEquals("File length should be the same",
-        writeBytes, fileLength);
-
-    // pread
-    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
-      byte[] buf = new byte[writeBytes + 100];
-      int readLen = fsdis.read(0, buf, 0, buf.length);
-      readLen = readLen >= 0 ? readLen : 0;
-      Assert.assertEquals("The length of file should be the same to write size",
-          writeBytes, readLen);
-      for (int i = 0; i < writeBytes; i++) {
-        Assert.assertEquals("Byte at " + i + " should be the same", getByte(i),
-            buf[i]);
+    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, fileSize);
+
+    assert lbs.getLocatedBlocks().size() == numBlocks;
+    for (LocatedBlock lb : lbs.getLocatedBlocks()) {
+      assert lb instanceof LocatedStripedBlock;
+      LocatedStripedBlock bg = (LocatedStripedBlock)(lb);
+      for (int i = 0; i < DATA_BLK_NUM; i++) {
+        Block blk = new Block(bg.getBlock().getBlockId() + i,
+            NUM_STRIPE_PER_BLOCK * CELLSIZE,
+            bg.getBlock().getGenerationStamp());
+        blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
+        cluster.injectBlocks(i, Arrays.asList(blk),
+            bg.getBlock().getBlockPoolId());
       }
     }
 
-    // stateful read with byte array
-    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
-      byte[] buf = new byte[writeBytes + 100];
-      int readLen = 0;
-      int ret;
-      do {
-        ret = fsdis.read(buf, readLen, buf.length - readLen);
-        if (ret > 0) {
-          readLen += ret;
+    DFSStripedInputStream in =
+        new DFSStripedInputStream(fs.getClient(), filePath.toString(),
+            false, info);
+
+    byte[] expected = new byte[fileSize];
+
+    for (LocatedBlock bg : lbs.getLocatedBlocks()) {
+      /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
+      for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) {
+        for (int j = 0; j < DATA_BLK_NUM; j++) {
+          for (int k = 0; k < CELLSIZE; k++) {
+            int posInBlk = i * CELLSIZE + k;
+            int posInFile = (int) bg.getStartOffset() +
+                i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k;
+            expected[posInFile] = SimulatedFSDataset.simulatedByte(
+                new Block(bg.getBlock().getBlockId() + j), posInBlk);
+          }
         }
-      } while (ret >= 0);
-      readLen = readLen >= 0 ? readLen : 0;
-      Assert.assertEquals("The length of file should be the same to write size",
-          writeBytes, readLen);
-      for (int i = 0; i < writeBytes; i++) {
-        Assert.assertEquals("Byte at " + i + " should be the same", getByte(i),
-            buf[i]);
       }
     }
 
-    // stateful read with ByteBuffer
-    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
-      ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100);
-      int readLen = 0;
-      int ret;
-      do {
-        ret = fsdis.read(buf);
-        if (ret > 0) {
-          readLen += ret;
-        }
-      } while (ret >= 0);
-      readLen = readLen >= 0 ? readLen : 0;
-      Assert.assertEquals("The length of file should be the same to write size",
-          writeBytes, readLen);
-      for (int i = 0; i < writeBytes; i++) {
-        Assert.assertEquals("Byte at " + i + " should be the same", getByte(i),
-            buf.array()[i]);
+    if (useByteBuffer) {
+      ByteBuffer readBuffer = ByteBuffer.allocate(fileSize);
+      int done = 0;
+      while (done < fileSize) {
+        int ret = in.read(readBuffer);
+        assertTrue(ret > 0);
+        done += ret;
       }
+      assertArrayEquals(expected, readBuffer.array());
+    } else {
+      byte[] readBuffer = new byte[fileSize];
+      int done = 0;
+      while (done < fileSize) {
+        int ret = in.read(readBuffer, done, fileSize - done);
+        assertTrue(ret > 0);
+        done += ret;
+      }
+      assertArrayEquals(expected, readBuffer);
     }
-
-    // stateful read with 1KB size byte array
-    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
-      final byte[] result = new byte[writeBytes];
-      final byte[] buf = new byte[1024];
-      int readLen = 0;
-      int ret;
-      do {
-        ret = fsdis.read(buf, 0, buf.length);
-        if (ret > 0) {
-          System.arraycopy(buf, 0, result, readLen, ret);
-          readLen += ret;
-        }
-      } while (ret >= 0);
-      Assert.assertEquals("The length of file should be the same to write size",
-          writeBytes, readLen);
-      Assert.assertArrayEquals(bytes, result);
-    }
-
-    // stateful read using ByteBuffer with 1KB size
-    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
-      final ByteBuffer result = ByteBuffer.allocate(writeBytes);
-      final ByteBuffer buf = ByteBuffer.allocate(1024);
-      int readLen = 0;
-      int ret;
-      do {
-        ret = fsdis.read(buf);
-        if (ret > 0) {
-          readLen += ret;
-          buf.flip();
-          result.put(buf);
-          buf.clear();
-        }
-      } while (ret >= 0);
-      Assert.assertEquals("The length of file should be the same to write size",
-          writeBytes, readLen);
-      Assert.assertArrayEquals(bytes, result.array());
-    }
+    fs.delete(filePath, true);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a98cee8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
deleted file mode 100644
index 1ad480e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
+++ /dev/null
@@ -1,218 +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.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
-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.ECInfo;
-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 static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
-import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
-import org.apache.hadoop.hdfs.util.StripedBlockUtil;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.List;
-
-public class TestReadStripedFile {
-
-  public static final Log LOG = LogFactory.getLog(TestReadStripedFile.class);
-
-  private MiniDFSCluster cluster;
-  private Configuration conf = new Configuration();
-  private DistributedFileSystem fs;
-  private final Path dirPath = new Path("/striped");
-  private Path filePath = new Path(dirPath, "file");
-  private ECInfo info = new ECInfo(filePath.toString(),
-      ECSchemaManager.getSystemDefaultSchema());
-  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 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;
-
-  @Before
-  public void setup() throws IOException {
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE);
-    SimulatedFSDataset.setFactory(conf);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
-        DATA_BLK_NUM + PARITY_BLK_NUM).build();
-    cluster.waitActive();
-    fs = cluster.getFileSystem();
-    fs.mkdirs(dirPath);
-    fs.getClient().createErasureCodingZone(dirPath.toString(), null);
-  }
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * Test {@link DFSStripedInputStream#getBlockAt(long)}
-   */
-  @Test
-  public void testGetBlock() throws Exception {
-    final int numBlocks = 4;
-    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        NUM_STRIPE_PER_BLOCK, false);
-    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
-        filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
-    final DFSStripedInputStream in =
-        new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, info);
-
-    List<LocatedBlock> lbList = lbs.getLocatedBlocks();
-    for (LocatedBlock aLbList : lbList) {
-      LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList;
-      LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb,
-          CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM);
-      for (int j = 0; j < DATA_BLK_NUM; j++) {
-        LocatedBlock refreshed = in.getBlockAt(blks[j].getStartOffset());
-        assertEquals(blks[j].getBlock(), refreshed.getBlock());
-        assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset());
-        assertArrayEquals(blks[j].getLocations(), refreshed.getLocations());
-      }
-    }
-  }
-
-  @Test
-  public void testPread() throws Exception {
-    final int numBlocks = 2;
-    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        NUM_STRIPE_PER_BLOCK, false);
-    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
-        filePath.toString(), 0, BLOCK_GROUP_SIZE);
-
-    assert lbs.get(0) instanceof LocatedStripedBlock;
-    LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0));
-    for (int i = 0; i < DATA_BLK_NUM; i++) {
-      Block blk = new Block(bg.getBlock().getBlockId() + i,
-          NUM_STRIPE_PER_BLOCK * CELLSIZE,
-          bg.getBlock().getGenerationStamp());
-      blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
-      cluster.injectBlocks(i, Arrays.asList(blk),
-          bg.getBlock().getBlockPoolId());
-    }
-    DFSStripedInputStream in =
-        new DFSStripedInputStream(fs.getClient(),
-            filePath.toString(), false, info);
-    int readSize = BLOCK_GROUP_SIZE;
-    byte[] readBuffer = new byte[readSize];
-    int ret = in.read(0, readBuffer, 0, readSize);
-
-    assertEquals(readSize, ret);
-    // TODO: verify read results with patterned data from HDFS-8117
-  }
-
-  @Test
-  public void testStatefulRead() throws Exception {
-    testStatefulRead(false, false);
-    testStatefulRead(true, false);
-    testStatefulRead(true, true);
-  }
-
-  private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket)
-      throws Exception {
-    final int numBlocks = 2;
-    final int fileSize = numBlocks * BLOCK_GROUP_SIZE;
-    if (cellMisalignPacket) {
-      conf.setInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT + 1);
-      tearDown();
-      setup();
-    }
-    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
-        NUM_STRIPE_PER_BLOCK, false);
-    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
-        filePath.toString(), 0, fileSize);
-
-    assert lbs.getLocatedBlocks().size() == numBlocks;
-    for (LocatedBlock lb : lbs.getLocatedBlocks()) {
-      assert lb instanceof LocatedStripedBlock;
-      LocatedStripedBlock bg = (LocatedStripedBlock)(lb);
-      for (int i = 0; i < DATA_BLK_NUM; i++) {
-        Block blk = new Block(bg.getBlock().getBlockId() + i,
-            NUM_STRIPE_PER_BLOCK * CELLSIZE,
-            bg.getBlock().getGenerationStamp());
-        blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
-        cluster.injectBlocks(i, Arrays.asList(blk),
-            bg.getBlock().getBlockPoolId());
-      }
-    }
-
-    DFSStripedInputStream in =
-        new DFSStripedInputStream(fs.getClient(), filePath.toString(),
-            false, info);
-
-    byte[] expected = new byte[fileSize];
-
-    for (LocatedBlock bg : lbs.getLocatedBlocks()) {
-      /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
-      for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) {
-        for (int j = 0; j < DATA_BLK_NUM; j++) {
-          for (int k = 0; k < CELLSIZE; k++) {
-            int posInBlk = i * CELLSIZE + k;
-            int posInFile = (int) bg.getStartOffset() +
-                i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k;
-            expected[posInFile] = SimulatedFSDataset.simulatedByte(
-                new Block(bg.getBlock().getBlockId() + j), posInBlk);
-          }
-        }
-      }
-    }
-
-    if (useByteBuffer) {
-      ByteBuffer readBuffer = ByteBuffer.allocate(fileSize);
-      int done = 0;
-      while (done < fileSize) {
-        int ret = in.read(readBuffer);
-        assertTrue(ret > 0);
-        done += ret;
-      }
-      assertArrayEquals(expected, readBuffer.array());
-    } else {
-      byte[] readBuffer = new byte[fileSize];
-      int done = 0;
-      while (done < fileSize) {
-        int ret = in.read(readBuffer, done, fileSize - done);
-        assertTrue(ret > 0);
-        done += ret;
-      }
-      assertArrayEquals(expected, readBuffer);
-    }
-    fs.delete(filePath, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a98cee8/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
new file mode 100644
index 0000000..eacc6ed
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
@@ -0,0 +1,261 @@
+/**
+ * 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.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class TestWriteReadStripedFile {
+  private static int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
+  private static int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
+
+
+  private static DistributedFileSystem fs;
+  private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final static int stripesPerBlock = 4;
+  static int blockSize = cellSize * stripesPerBlock;
+  static int numDNs = dataBlocks + parityBlocks + 2;
+
+  private static MiniDFSCluster cluster;
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    fs = cluster.getFileSystem();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testFileEmpty() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/EmptyFile", 0);
+  }
+
+  @Test
+  public void testFileSmallerThanOneCell1() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1);
+  }
+
+  @Test
+  public void testFileSmallerThanOneCell2() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1);
+  }
+
+  @Test
+  public void testFileEqualsWithOneCell() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize);
+  }
+
+  @Test
+  public void testFileSmallerThanOneStripe1() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe",
+        cellSize * dataBlocks - 1);
+  }
+
+  @Test
+  public void testFileSmallerThanOneStripe2() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe",
+        cellSize + 123);
+  }
+
+  @Test
+  public void testFileEqualsWithOneStripe() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe",
+        cellSize * dataBlocks);
+  }
+
+  @Test
+  public void testFileMoreThanOneStripe1() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1",
+        cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void testFileMoreThanOneStripe2() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2",
+        cellSize * dataBlocks + cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void testLessThanFullBlockGroup() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup",
+        cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize);
+  }
+
+  @Test
+  public void testFileFullBlockGroup() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/FullBlockGroup",
+        blockSize * dataBlocks);
+  }
+
+  @Test
+  public void testFileMoreThanABlockGroup1() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1",
+        blockSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void testFileMoreThanABlockGroup2() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2",
+        blockSize * dataBlocks + cellSize+ 123);
+  }
+
+
+  @Test
+  public void testFileMoreThanABlockGroup3() throws IOException {
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3",
+        blockSize * dataBlocks * 3 + cellSize * dataBlocks
+            + 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) {
+    final int mod = 29;
+    return (byte) (pos % mod + 1);
+  }
+
+  private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes)
+      throws IOException {
+    Path testPath = new Path(src);
+    final byte[] bytes = generateBytes(writeBytes);
+    DFSTestUtil.writeFile(fs, testPath, new String(bytes));
+
+    //check file length
+    FileStatus status = fs.getFileStatus(testPath);
+    long fileLength = status.getLen();
+    Assert.assertEquals("File length should be the same",
+        writeBytes, fileLength);
+
+    // pread
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
+      byte[] buf = new byte[writeBytes + 100];
+      int readLen = fsdis.read(0, buf, 0, buf.length);
+      readLen = readLen >= 0 ? readLen : 0;
+      Assert.assertEquals("The length of file should be the same to write size",
+          writeBytes, readLen);
+      for (int i = 0; i < writeBytes; i++) {
+        Assert.assertEquals("Byte at " + i + " should be the same", getByte(i),
+            buf[i]);
+      }
+    }
+
+    // stateful read with byte array
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
+      byte[] buf = new byte[writeBytes + 100];
+      int readLen = 0;
+      int ret;
+      do {
+        ret = fsdis.read(buf, readLen, buf.length - readLen);
+        if (ret > 0) {
+          readLen += ret;
+        }
+      } while (ret >= 0);
+      readLen = readLen >= 0 ? readLen : 0;
+      Assert.assertEquals("The length of file should be the same to write size",
+          writeBytes, readLen);
+      for (int i = 0; i < writeBytes; i++) {
+        Assert.assertEquals("Byte at " + i + " should be the same", getByte(i),
+            buf[i]);
+      }
+    }
+
+    // stateful read with ByteBuffer
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
+      ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100);
+      int readLen = 0;
+      int ret;
+      do {
+        ret = fsdis.read(buf);
+        if (ret > 0) {
+          readLen += ret;
+        }
+      } while (ret >= 0);
+      readLen = readLen >= 0 ? readLen : 0;
+      Assert.assertEquals("The length of file should be the same to write size",
+          writeBytes, readLen);
+      for (int i = 0; i < writeBytes; i++) {
+        Assert.assertEquals("Byte at " + i + " should be the same", getByte(i),
+            buf.array()[i]);
+      }
+    }
+
+    // stateful read with 1KB size byte array
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
+      final byte[] result = new byte[writeBytes];
+      final byte[] buf = new byte[1024];
+      int readLen = 0;
+      int ret;
+      do {
+        ret = fsdis.read(buf, 0, buf.length);
+        if (ret > 0) {
+          System.arraycopy(buf, 0, result, readLen, ret);
+          readLen += ret;
+        }
+      } while (ret >= 0);
+      Assert.assertEquals("The length of file should be the same to write size",
+          writeBytes, readLen);
+      Assert.assertArrayEquals(bytes, result);
+    }
+
+    // stateful read using ByteBuffer with 1KB size
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
+      final ByteBuffer result = ByteBuffer.allocate(writeBytes);
+      final ByteBuffer buf = ByteBuffer.allocate(1024);
+      int readLen = 0;
+      int ret;
+      do {
+        ret = fsdis.read(buf);
+        if (ret > 0) {
+          readLen += ret;
+          buf.flip();
+          result.put(buf);
+          buf.clear();
+        }
+      } while (ret >= 0);
+      Assert.assertEquals("The length of file should be the same to write size",
+          writeBytes, readLen);
+      Assert.assertArrayEquals(bytes, result.array());
+    }
+  }
+}


[28/50] hadoop git commit: HDFS-8235. Erasure Coding: Create DFSStripedInputStream in DFSClient#open. Contributed by Kai Sasaki.

Posted by zh...@apache.org.
HDFS-8235. Erasure Coding: Create DFSStripedInputStream in DFSClient#open. Contributed by Kai Sasaki.


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

Branch: refs/heads/HDFS-7285
Commit: 453e39079c0b46b2b940fe51d778e56730cf4689
Parents: 61923aa
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Apr 28 13:42:24 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:21 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt            |  5 ++++-
 .../main/java/org/apache/hadoop/hdfs/DFSClient.java |  7 ++++++-
 .../apache/hadoop/hdfs/DFSStripedInputStream.java   |  5 +++--
 .../hadoop/hdfs/TestDFSStripedInputStream.java      | 16 +++++++---------
 .../org/apache/hadoop/hdfs/TestReadStripedFile.java | 11 ++++++++---
 5 files changed, 28 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/453e3907/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 6c5d7ce..9b4bf24 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -139,4 +139,7 @@
     commands from standbynode if any (vinayakumarb)
 
     HDFS-8189. ClientProtocol#createErasureCodingZone API was wrongly annotated
-    as Idempotent (vinayakumarb)
\ No newline at end of file
+    as Idempotent (vinayakumarb)
+
+    HDFS-8235. Erasure Coding: Create DFSStripedInputStream in DFSClient#open.
+    (Kai Sasaki via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/453e3907/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 111398f..7682459 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
@@ -1188,7 +1188,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     //    Get block info from namenode
     TraceScope scope = getPathTraceScope("newDFSInputStream", src);
     try {
-      return new DFSInputStream(this, src, verifyChecksum);
+      ECInfo info = getErasureCodingInfo(src);
+      if (info != null) {
+        return new DFSStripedInputStream(this, src, verifyChecksum, info);
+      } else {
+        return new DFSInputStream(this, src, verifyChecksum);
+      }
     } finally {
       scope.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/453e3907/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 fe9e101..f6f7ed2 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
@@ -134,11 +134,12 @@ public class DFSStripedInputStream extends DFSInputStream {
   private final short parityBlkNum;
   private final ECInfo ecInfo;
 
-  DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum)
+  DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, ECInfo info)
       throws IOException {
     super(dfsClient, src, verifyChecksum);
     // ECInfo is restored from NN just before reading striped file.
-    ecInfo = dfsClient.getErasureCodingInfo(src);
+    assert info != null;
+    ecInfo = info;
     cellSize = ecInfo.getSchema().getChunkSize();
     dataBlkNum = (short)ecInfo.getSchema().getNumDataUnits();
     parityBlkNum = (short)ecInfo.getSchema().getNumParityUnits();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/453e3907/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 cf10981..bcfc74b 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -167,10 +168,9 @@ public class TestDFSStripedInputStream {
         writeBytes, fileLength);
 
     // pread
-    try (DFSStripedInputStream dis =
-             new DFSStripedInputStream(fs.getClient(), src, true)) {
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
       byte[] buf = new byte[writeBytes + 100];
-      int readLen = dis.read(0, buf, 0, buf.length);
+      int readLen = fsdis.read(0, buf, 0, buf.length);
       readLen = readLen >= 0 ? readLen : 0;
       Assert.assertEquals("The length of file should be the same to write size",
           writeBytes, readLen);
@@ -180,13 +180,12 @@ public class TestDFSStripedInputStream {
     }
 
     // stateful read with byte array
-    try (DFSStripedInputStream dis =
-             new DFSStripedInputStream(fs.getClient(), src, true)) {
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
       byte[] buf = new byte[writeBytes + 100];
       int readLen = 0;
       int ret;
       do {
-        ret = dis.read(buf, readLen, buf.length - readLen);
+        ret = fsdis.read(buf, readLen, buf.length - readLen);
         if (ret > 0) {
           readLen += ret;
         }
@@ -201,13 +200,12 @@ public class TestDFSStripedInputStream {
     }
 
     // stateful read with ByteBuffer
-    try (DFSStripedInputStream dis =
-             new DFSStripedInputStream(fs.getClient(), src, true)) {
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
       ByteBuffer buf = ByteBuffer.allocate(writeBytes + 100);
       int readLen = 0;
       int ret;
       do {
-        ret = dis.read(buf);
+        ret = fsdis.read(buf);
         if (ret > 0) {
           readLen += ret;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/453e3907/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
index d980bd6..1ad480e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
@@ -24,6 +24,7 @@ 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.ECInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -33,6 +34,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -52,6 +54,8 @@ public class TestReadStripedFile {
   private DistributedFileSystem fs;
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
+  private ECInfo info = new ECInfo(filePath.toString(),
+      ECSchemaManager.getSystemDefaultSchema());
   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;
@@ -89,7 +93,7 @@ public class TestReadStripedFile {
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
     final DFSStripedInputStream in =
-        new DFSStripedInputStream(fs.getClient(), filePath.toString(), false);
+        new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, info);
 
     List<LocatedBlock> lbList = lbs.getLocatedBlocks();
     for (LocatedBlock aLbList : lbList) {
@@ -124,7 +128,8 @@ public class TestReadStripedFile {
           bg.getBlock().getBlockPoolId());
     }
     DFSStripedInputStream in =
-        new DFSStripedInputStream(fs.getClient(), filePath.toString(), false);
+        new DFSStripedInputStream(fs.getClient(),
+            filePath.toString(), false, info);
     int readSize = BLOCK_GROUP_SIZE;
     byte[] readBuffer = new byte[readSize];
     int ret = in.read(0, readBuffer, 0, readSize);
@@ -170,7 +175,7 @@ public class TestReadStripedFile {
 
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(),
-            false);
+            false, info);
 
     byte[] expected = new byte[fileSize];
 


[36/50] hadoop git commit: HDFS-8137. Send the EC schema to DataNode via EC encoding/recovering command. Contributed by Uma Maheswara Rao G

Posted by zh...@apache.org.
HDFS-8137. Send the EC schema to DataNode via EC encoding/recovering command. 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/38092610
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/38092610
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/38092610

Branch: refs/heads/HDFS-7285
Commit: 38092610039e9390ab41a840ee08817efb53ad66
Parents: 3db0fe6
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Tue May 5 11:22:52 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:24 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  2 +
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  6 ++-
 .../server/blockmanagement/BlockManager.java    | 22 +++++++++-
 .../blockmanagement/DatanodeDescriptor.java     | 16 ++++----
 .../hdfs/server/namenode/FSNamesystem.java      | 43 +++++++++++---------
 .../hadoop/hdfs/server/namenode/Namesystem.java | 14 ++++++-
 .../server/protocol/BlockECRecoveryCommand.java | 14 ++++++-
 .../src/main/proto/erasurecoding.proto          |  1 +
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    | 21 ++++++++--
 9 files changed, 102 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/38092610/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 77272e7..faec023 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -164,3 +164,5 @@
 
     HDFS-8281. Erasure Coding: implement parallel stateful reading for striped layout.
     (jing9)
+
+    HDFS-8137. Send the EC schema to DataNode via EC encoding/recovering command(umamahesh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38092610/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 3cd3e03..e230232 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
@@ -3191,8 +3191,10 @@ public class PBHelper {
       liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue();
     }
 
+    ECSchema ecSchema = convertECSchema(blockEcRecoveryInfoProto.getEcSchema());
+
     return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos,
-        targetStorageUuids, convertStorageTypes, liveBlkIndices);
+        targetStorageUuids, convertStorageTypes, liveBlkIndices, ecSchema);
   }
 
   public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo(
@@ -3217,6 +3219,8 @@ public class PBHelper {
     short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices();
     builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices));
 
+    builder.setEcSchema(convertECSchema(blockEcRecoveryInfo.getECSchema()));
+
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38092610/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 5d998c8..0307792 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
@@ -65,7 +65,6 @@ import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
 import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
-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.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -83,7 +82,10 @@ 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.io.erasurecode.ECSchema;
+
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
+
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Daemon;
@@ -93,6 +95,7 @@ import org.apache.hadoop.util.Time;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -1552,10 +1555,25 @@ public class BlockManager {
           if (block.isStriped()) {
             assert rw instanceof ErasureCodingWork;
             assert rw.targets.length > 0;
+            String src = block.getBlockCollection().getName();
+            ECSchema ecSchema = null;
+            try {
+              ecSchema = namesystem.getECSchemaForPath(src);
+            } catch (IOException e) {
+              blockLog
+                  .warn("Failed to get the EC schema for the file {} ", src);
+            }
+            if (ecSchema == null) {
+              blockLog.warn("No EC schema 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);
+                ((ErasureCodingWork) rw).liveBlockIndicies, ecSchema);
           } else {
             rw.srcNodes[0].addBlockToBeReplicated(block, targets);
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38092610/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 35cc31b..83d3303 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
@@ -29,7 +29,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
-import java.util.Arrays;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
@@ -51,6 +50,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 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.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.Time;
 
@@ -608,15 +608,15 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Store block erasure coding work.
    */
-  void addBlockToBeErasureCoded(ExtendedBlock block, DatanodeDescriptor[] sources,
-      DatanodeStorageInfo[] targets, short[] liveBlockIndices) {
-    assert(block != null && sources != null && sources.length > 0);
+  void addBlockToBeErasureCoded(ExtendedBlock block,
+      DatanodeDescriptor[] sources, DatanodeStorageInfo[] targets,
+      short[] liveBlockIndices, ECSchema ecSchema) {
+    assert (block != null && sources != null && sources.length > 0);
     BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets,
-        liveBlockIndices);
+        liveBlockIndices, ecSchema);
     erasurecodeBlocks.offer(task);
-    BlockManager.LOG.debug("Adding block recovery task " + task +
-        "to " + getName() + ", current queue size is " +
-        erasurecodeBlocks.size());
+    BlockManager.LOG.debug("Adding block recovery task " + task + "to "
+        + getName() + ", current queue size is " + erasurecodeBlocks.size());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38092610/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 86745ec..9e6f74a 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
@@ -8172,25 +8172,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   ECInfo getErasureCodingInfo(String src) throws AccessControlException,
       UnresolvedLinkException, IOException {
-    checkOperation(OperationCategory.READ);
-    final byte[][] pathComponents = FSDirectory
-        .getPathComponentsForReservedPath(src);
-    final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
-    try {
-      checkOperation(OperationCategory.READ);
-      src = dir.resolvePath(pc, src, pathComponents);
-      final INodesInPath iip = dir.getINodesInPath(src, true);
-      if (isPermissionEnabled) {
-        dir.checkPathAccess(pc, iip, FsAction.READ);
-      }
-      // Get schema set for the zone
-      ECSchema schema = dir.getECSchema(iip);
-      if (schema != null) {
-        return new ECInfo(src, schema);
-      }
-    } finally {
-      readUnlock();
+    ECSchema schema = getECSchemaForPath(src);
+    if (schema != null) {
+      return new ECInfo(src, schema);
     }
     return null;
   }
@@ -8425,5 +8409,26 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
+  @Override
+  public ECSchema getECSchemaForPath(String src) throws IOException {
+    checkOperation(OperationCategory.READ);
+    final byte[][] pathComponents = FSDirectory
+        .getPathComponentsForReservedPath(src);
+    final FSPermissionChecker pc = getPermissionChecker();
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      src = dir.resolvePath(pc, src, pathComponents);
+      final INodesInPath iip = dir.getINodesInPath(src, true);
+      if (isPermissionEnabled) {
+        dir.checkPathAccess(pc, iip, FsAction.READ);
+      }
+      // Get schema set for the zone
+      return dir.getECSchema(iip);
+    } finally {
+      readUnlock();
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38092610/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 4695c3f..e6c7fc0 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
@@ -17,12 +17,14 @@
  */
 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.server.blockmanagement.BlockCollection;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.security.AccessControlException;
 
@@ -47,4 +49,14 @@ public interface Namesystem extends RwLock, SafeMode {
   public void checkOperation(OperationCategory read) throws StandbyException;
 
   public boolean isInSnapshot(BlockCollection bc);
+
+  /**
+   * Gets the ECSchema for the specified path
+   * 
+   * @param src
+   *          - path
+   * @return ECSchema
+   * @throws IOException
+   */
+  public ECSchema getECSchemaForPath(String src) throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38092610/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
index 9a387dd..61e49e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -76,9 +77,11 @@ public class BlockECRecoveryCommand extends DatanodeCommand {
     private String[] targetStorageIDs;
     private StorageType[] targetStorageTypes;
     private final short[] liveBlockIndices;
+    private final ECSchema ecSchema;
 
     public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources,
-        DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices) {
+        DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices,
+        ECSchema ecSchema) {
       this.block = block;
       this.sources = sources;
       this.targets = DatanodeStorageInfo.toDatanodeInfos(targetDnStorageInfo);
@@ -87,17 +90,20 @@ public class BlockECRecoveryCommand extends DatanodeCommand {
       this.targetStorageTypes = DatanodeStorageInfo
           .toStorageTypes(targetDnStorageInfo);
       this.liveBlockIndices = liveBlockIndices;
+      this.ecSchema = ecSchema;
     }
     
     public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources,
         DatanodeInfo[] targets, String[] targetStorageIDs,
-        StorageType[] targetStorageTypes, short[] liveBlockIndices) {
+        StorageType[] targetStorageTypes, short[] liveBlockIndices,
+        ECSchema ecSchema) {
       this.block = block;
       this.sources = sources;
       this.targets = targets;
       this.targetStorageIDs = targetStorageIDs;
       this.targetStorageTypes = targetStorageTypes;
       this.liveBlockIndices = liveBlockIndices;
+      this.ecSchema = ecSchema;
     }
 
     public ExtendedBlock getExtendedBlock() {
@@ -123,6 +129,10 @@ public class BlockECRecoveryCommand extends DatanodeCommand {
     public short[] getLiveBlockIndices() {
       return liveBlockIndices;
     }
+    
+    public ECSchema getECSchema() {
+      return ecSchema;
+    }
 
     @Override
     public String toString() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38092610/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
index 59bd949..702f6fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -99,4 +99,5 @@ message BlockECRecoveryInfoProto {
   required StorageUuidsProto targetStorageUuids = 4;
   required StorageTypesProto targetStorageTypes = 5;
   repeated uint32 liveBlockIndices = 6;
+  required ECSchemaProto ecSchema = 7;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38092610/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 4ec4ea5..f580cbb 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
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 
@@ -71,8 +70,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
@@ -88,6 +87,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 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;
@@ -663,7 +663,7 @@ public class TestPBHelper {
     short[] liveBlkIndices0 = new short[2];
     BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo(
         new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0,
-        liveBlkIndices0);
+        liveBlkIndices0, ECSchemaManager.getSystemDefaultSchema());
     DatanodeInfo[] dnInfos1 = new DatanodeInfo[] {
         DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
     DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil
@@ -677,7 +677,7 @@ public class TestPBHelper {
     short[] liveBlkIndices1 = new short[2];
     BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo(
         new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1,
-        liveBlkIndices1);
+        liveBlkIndices1, ECSchemaManager.getSystemDefaultSchema());
     List<BlockECRecoveryInfo> blkRecoveryInfosList = new ArrayList<BlockECRecoveryInfo>();
     blkRecoveryInfosList.add(blkECRecoveryInfo0);
     blkRecoveryInfosList.add(blkECRecoveryInfo1);
@@ -718,6 +718,19 @@ public class TestPBHelper {
     for (int i = 0; i < liveBlockIndices1.length; i++) {
       assertEquals(liveBlockIndices1[i], liveBlockIndices2[i]);
     }
+    
+    ECSchema ecSchema1 = blkECRecoveryInfo1.getECSchema();
+    ECSchema ecSchema2 = blkECRecoveryInfo2.getECSchema();
+    // Compare ECSchemas same as default ECSchema as we used system default
+    // ECSchema used in this test
+    compareECSchemas(ECSchemaManager.getSystemDefaultSchema(), ecSchema1);
+    compareECSchemas(ECSchemaManager.getSystemDefaultSchema(), ecSchema2);
+  }
+
+  private void compareECSchemas(ECSchema ecSchema1, ECSchema ecSchema2) {
+    assertEquals(ecSchema1.getSchemaName(), ecSchema2.getSchemaName());
+    assertEquals(ecSchema1.getNumDataUnits(), ecSchema2.getNumDataUnits());
+    assertEquals(ecSchema1.getNumParityUnits(), ecSchema2.getNumParityUnits());
   }
 
   private void assertDnInfosEqual(DatanodeInfo[] dnInfos1,


[33/50] hadoop git commit: HDFS-8281. Erasure Coding: implement parallel stateful reading for striped layout. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-8281. Erasure Coding: implement parallel stateful reading for striped layout. 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/3db0fe6c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3db0fe6c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3db0fe6c

Branch: refs/heads/HDFS-7285
Commit: 3db0fe6c5c0b55146eb7e6520bff6c18ad582c93
Parents: 1e34881
Author: Jing Zhao <ji...@apache.org>
Authored: Mon May 4 14:44:58 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:23 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  26 +++
 .../hadoop/hdfs/DFSStripedInputStream.java      | 217 +++++++++++++------
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  34 ++-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |  50 ++++-
 .../hadoop/hdfs/TestPlanReadPortions.java       |   4 +-
 6 files changed, 246 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3db0fe6c/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 e30b2ed..77272e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -161,3 +161,6 @@
 
     HDFS-8316. Erasure coding: refactor EC constants to be consistent with HDFS-8249.
     (Zhe Zhang via jing9)
+
+    HDFS-8281. Erasure Coding: implement parallel stateful reading for striped layout.
+    (jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3db0fe6c/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 bef4da0..ca799fa 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
@@ -716,6 +716,16 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   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,
@@ -749,6 +759,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       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;
+    }
   }
 
   /**
@@ -782,6 +799,15 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         }
       } 
     }
+
+    @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.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3db0fe6c/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 0dc98fd..13c4743 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.*;
@@ -37,6 +38,7 @@ import java.util.Set;
 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.ExecutionException;
 import java.util.concurrent.CancellationException;
@@ -62,7 +64,7 @@ import java.util.concurrent.Future;
  * +------+  <- A cell contains {@link #cellSize} bytes of data
  *
  * Three styles of read will eventually be supported:
- *   1. Stateful read: TODO: HDFS-8033
+ *   1. Stateful read
  *   2. pread without decode support
  *     This is implemented by calculating the portion of read from each block and
  *     issuing requests to each DataNode in parallel.
@@ -91,12 +93,38 @@ public class DFSStripedInputStream extends DFSInputStream {
     }
   }
 
+  /** 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 final short groupSize = HdfsConstants.NUM_DATA_BLOCKS;
   private final BlockReader[] blockReaders = new BlockReader[groupSize];
   private final DatanodeInfo[] currentNodes = new DatanodeInfo[groupSize];
   private final int cellSize;
   private final short dataBlkNum;
   private final short parityBlkNum;
+  /** the buffer for a complete stripe */
+  private ByteBuffer curStripeBuf;
+  /**
+   * indicate the start/end offset of the current buffered stripe in the
+   * block group
+   */
+  private StripeRange curStripeRange;
+  private final CompletionService<Integer> readingService;
 
   DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
       ECInfo ecInfo) throws IOException {
@@ -106,7 +134,20 @@ public class DFSStripedInputStream extends DFSInputStream {
     cellSize = ecInfo.getSchema().getChunkSize();
     dataBlkNum = (short) ecInfo.getSchema().getNumDataUnits();
     parityBlkNum = (short) ecInfo.getSchema().getNumParityUnits();
-    DFSClient.LOG.debug("Creating an striped input stream for file " + src);
+    curStripeRange = new StripeRange(0, 0);
+    readingService =
+        new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool());
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("Creating an striped input stream for file " + src);
+    }
+  }
+
+  private void resetCurStripeBuffer() {
+    if (curStripeBuf == null) {
+      curStripeBuf = ByteBuffer.allocateDirect(cellSize * dataBlkNum);
+    }
+    curStripeBuf.clear();
+    curStripeRange = new StripeRange(0, 0);
   }
 
   @Override
@@ -141,7 +182,7 @@ public class DFSStripedInputStream extends DFSInputStream {
         targetBlockGroup.getBlockSize() - 1;
     currentLocatedBlock = targetBlockGroup;
 
-    long offsetIntoBlockGroup = target - targetBlockGroup.getStartOffset();
+    final long offsetIntoBlockGroup = getOffsetInBlockGroup();
     LocatedBlock[] targetBlocks = StripedBlockUtil.parseStripedBlockGroup(
         targetBlockGroup, cellSize, dataBlkNum, parityBlkNum);
     // The purpose is to get start offset into each block
@@ -156,8 +197,8 @@ public class DFSStripedInputStream extends DFSInputStream {
         if (retval != null) {
           currentNodes[i] = retval.info;
           blockReaders[i] = getBlockReaderWithRetry(targetBlock,
-              readPortions[i].startOffsetInBlock,
-              targetBlock.getBlockSize() - readPortions[i].startOffsetInBlock,
+              readPortions[i].getStartOffsetInBlock(),
+              targetBlock.getBlockSize() - readPortions[i].getStartOffsetInBlock(),
               retval.addr, retval.storageType, retval.info, target, retry);
         }
       }
@@ -203,6 +244,7 @@ public class DFSStripedInputStream extends DFSInputStream {
    */
   @Override
   protected void closeCurrentBlockReaders() {
+    resetCurStripeBuffer();
     if (blockReaders ==  null || blockReaders.length == 0) {
       return;
     }
@@ -220,6 +262,73 @@ public class DFSStripedInputStream extends DFSInputStream {
     blockEnd = -1;
   }
 
+  private long getOffsetInBlockGroup() {
+    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;
+    int stripeIndex = (int) (offsetInBlockGroup / stripeLen);
+    curStripeRange = new StripeRange(stripeIndex * stripeLen,
+        Math.min(currentLocatedBlock.getBlockSize() - (stripeIndex * stripeLen),
+            stripeLen));
+    final int numCell = (int) ((curStripeRange.length - 1) / cellSize + 1);
+
+    // read the whole stripe in parallel
+    Map<Future<Integer>, Integer> futures = new HashMap<>();
+    for (int i = 0; i < numCell; i++) {
+      curStripeBuf.position(cellSize * i);
+      curStripeBuf.limit((int) Math.min(cellSize * (i + 1),
+          curStripeRange.length));
+      ByteBuffer buf = curStripeBuf.slice();
+      ByteBufferStrategy strategy = new ByteBufferStrategy(buf);
+      final int targetLength = buf.remaining();
+      Callable<Integer> readCallable = readCell(blockReaders[i],
+          currentNodes[i], strategy, targetLength, corruptedBlockMap);
+      Future<Integer> request = readingService.submit(readCallable);
+      futures.put(request, i);
+    }
+    while (!futures.isEmpty()) {
+      try {
+        waitNextCompletion(readingService, futures);
+        // TODO: decode and record bad reader if necessary
+      } catch (InterruptedException ignored) {
+        // ignore and retry
+      }
+    }
+  }
+
+  private Callable<Integer> readCell(final BlockReader reader,
+      final DatanodeInfo datanode, final ByteBufferStrategy strategy,
+      final int targetLength,
+      final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
+    return new Callable<Integer>() {
+      @Override
+      public Integer call() throws Exception {
+        int result = 0;
+        while (result < targetLength) {
+          int ret = readBuffer(reader, datanode, strategy, corruptedBlockMap);
+          if (ret < 0) {
+            throw new IOException("Unexpected EOS from the reader");
+          }
+          result += ret;
+        }
+        updateReadStatistics(readStatistics, targetLength, reader);
+        return result;
+      }
+    };
+  }
+
   @Override
   protected synchronized int readWithStrategy(ReaderStrategy strategy,
       int off, int len) throws IOException {
@@ -227,11 +336,10 @@ public class DFSStripedInputStream extends DFSInputStream {
     if (closed.get()) {
       throw new IOException("Stream closed");
     }
-    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
+    Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap =
+        new ConcurrentHashMap<>();
     failures = 0;
     if (pos < getFileLength()) {
-      /** Index of the target block in a stripe to read from */
-      int idxInGroup = (int) ((pos / cellSize) % dataBlkNum);
       try {
         if (pos > blockEnd) {
           blockSeekTo(pos);
@@ -247,40 +355,13 @@ public class DFSStripedInputStream extends DFSInputStream {
         /** Number of bytes already read into buffer */
         int result = 0;
         while (result < realLen) {
-          /**
-           * Temporary position into the file; {@link pos} might not proceed
-           * to this temporary position in case of exceptions.
-           */
-          long tmpPos = pos + result;
-          /** Start and end offsets of a cell in the file */
-          long cellStart = (tmpPos / cellSize) * cellSize;
-          long cellEnd = cellStart + cellSize - 1;
-
-          /** Number of bytes to read from the current cell */
-          int realLenInCell = (int) Math.min(realLen - result,
-              cellEnd - tmpPos + 1L);
-          assert realLenInCell > 0 : "Temporary position shouldn't be "
-              + "after cellEnd";
-
-          // Read from one blockReader up to cell boundary
-          int cellRet = readBuffer(blockReaders[idxInGroup],
-              currentNodes[idxInGroup], strategy, off + result, realLenInCell,
-              corruptedBlockMap);
-          if (cellRet >= 0) {
-            result += cellRet;
-            if (cellRet < realLenInCell) {
-              // A short read indicates the current blockReader buffer is
-              // already drained. Should return the read call. Otherwise
-              // should proceed to the next cell.
-              break;
-            }
-          } else {
-            // got a EOS from reader though we expect more data on it.
-            throw new IOException("Unexpected EOS from the reader");
+          if (!curStripeRange.include(getOffsetInBlockGroup())) {
+            readOneStripe(corruptedBlockMap);
           }
-          idxInGroup = (idxInGroup + 1) % dataBlkNum;
+          int ret = copy(strategy, off + result, realLen - result);
+          result += ret;
+          pos += ret;
         }
-        pos += result;
         if (dfsClient.stats != null) {
           dfsClient.stats.incrementBytesRead(result);
         }
@@ -295,11 +376,11 @@ public class DFSStripedInputStream extends DFSInputStream {
     return -1;
   }
 
-  private synchronized int readBuffer(BlockReader blockReader,
-      DatanodeInfo currentNode, ReaderStrategy readerStrategy, int off, int len,
+  private int readBuffer(BlockReader blockReader,
+      DatanodeInfo currentNode, ByteBufferStrategy readerStrategy,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
     try {
-      return readerStrategy.doRead(blockReader, off, len);
+      return readerStrategy.doRead(blockReader, 0, 0);
     } catch ( ChecksumException ce ) {
       DFSClient.LOG.warn("Found Checksum error for "
           + getCurrentBlock() + " from " + currentNode
@@ -312,26 +393,25 @@ public class DFSStripedInputStream extends DFSInputStream {
           + getCurrentBlock() + " of " + src + " from "
           + currentNode, e);
     }
-    // TODO: this should trigger decoding logic (HDFS-7678)
     return -1;
   }
 
-  protected class ByteBufferStrategy extends DFSInputStream.ByteBufferStrategy {
-    ByteBufferStrategy(ByteBuffer buf) {
-      super(buf);
-    }
-
-    @Override
-    public int doRead(BlockReader blockReader, int off, int len)
-        throws IOException {
-      int oldlimit = buf.limit();
-      if (buf.remaining() > len) {
-        buf.limit(buf.position() + len);
-      }
-      int ret = super.doRead(blockReader, off, len);
-      buf.limit(oldlimit);
-      return ret;
-    }
+  /**
+   * 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 copy(ReaderStrategy strategy, int offset, int length) {
+    final long stripeLen = cellSize * dataBlkNum;
+    final long offsetInBlk = pos - currentLocatedBlock.getStartOffset();
+    // compute the position in the curStripeBuf based on "pos"
+    int bufOffset = (int) (offsetInBlk % stripeLen);
+    curStripeBuf.position(bufOffset);
+    return strategy.copyFrom(curStripeBuf, offset,
+        Math.min(length, curStripeBuf.remaining()));
   }
 
   /**
@@ -366,8 +446,7 @@ public class DFSStripedInputStream extends DFSInputStream {
       DFSClient.LOG.debug("getBlockAt for striped blocks, offset="
           + blkStartOffset + ". Obtained block " + lb + ", idx=" + idx);
     }
-    return StripedBlockUtil.constructInternalBlock(lsb, i, cellSize,
-        dataBlkNum, idx);
+    return StripedBlockUtil.constructInternalBlock(lsb, i, cellSize, dataBlkNum, idx);
   }
 
   private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException {
@@ -404,7 +483,7 @@ public class DFSStripedInputStream extends DFSInputStream {
 
     for (short i = 0; i < dataBlkNum; i++) {
       ReadPortion rp = readPortions[i];
-      if (rp.readLength <= 0) {
+      if (rp.getReadLength() <= 0) {
         continue;
       }
       DatanodeInfo loc = blks[i].getLocations()[0];
@@ -413,8 +492,8 @@ public class DFSStripedInputStream extends DFSInputStream {
           loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())),
           type);
       Callable<Void> readCallable = getFromOneDataNode(dnAddr,
-          blks[i].getStartOffset(), rp.startOffsetInBlock,
-          rp.startOffsetInBlock + rp.readLength - 1, buf,
+          blks[i].getStartOffset(), rp.getStartOffsetInBlock(),
+          rp.getStartOffsetInBlock() + rp.getReadLength() - 1, buf,
           rp.getOffsets(), rp.getLengths(), corruptedBlockMap, i);
       Future<Void> getFromDNRequest = stripedReadsService.submit(readCallable);
       DFSClient.LOG.debug("Submitting striped read request for " + blks[i]);
@@ -451,14 +530,14 @@ public class DFSStripedInputStream extends DFSInputStream {
     };
   }
 
-  private void waitNextCompletion(CompletionService<Void> stripedReadsService,
-      Map<Future<Void>, Integer> futures) throws InterruptedException {
+  private <T> void waitNextCompletion(CompletionService<T> service,
+      Map<Future<T>, Integer> futures) throws InterruptedException {
     if (futures.isEmpty()) {
       throw new InterruptedException("Futures already empty");
     }
-    Future<Void> future = null;
+    Future<T> future = null;
     try {
-      future = stripedReadsService.take();
+      future = service.take();
       future.get();
       futures.remove(future);
     } catch (ExecutionException | CancellationException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3db0fe6c/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 b18e36f..24d4bfb 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
@@ -169,22 +169,22 @@ public class StripedBlockUtil {
     // blkIdxInGroup is the index of the block in the striped block group
     // E.g., blk_2 is the 3rd block in the group
     final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum);
-    results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk +
-        startInBlk % cellSize;
+    results[blkIdxInGroup].setStartOffsetInBlock(cellSize * cellIdxInBlk +
+        startInBlk % cellSize);
     boolean crossStripe = false;
     for (int i = 1; i < dataBlkNum; i++) {
       if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) {
         cellIdxInBlk++;
         crossStripe = true;
       }
-      results[(blkIdxInGroup + i) % dataBlkNum].startOffsetInBlock =
-          cellSize * cellIdxInBlk;
+      results[(blkIdxInGroup + i) % dataBlkNum].setStartOffsetInBlock(
+          cellSize * cellIdxInBlk);
     }
 
     int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len);
     results[blkIdxInGroup].offsetsInBuf.add(bufOffset);
     results[blkIdxInGroup].lengths.add(firstCellLen);
-    results[blkIdxInGroup].readLength += firstCellLen;
+    results[blkIdxInGroup].addReadLength(firstCellLen);
 
     int i = (blkIdxInGroup + 1) % dataBlkNum;
     for (int done = firstCellLen; done < len; done += cellSize) {
@@ -192,7 +192,7 @@ public class StripedBlockUtil {
       rp.offsetsInBuf.add(done + bufOffset);
       final int readLen = Math.min(len - done, cellSize);
       rp.lengths.add(readLen);
-      rp.readLength += readLen;
+      rp.addReadLength(readLen);
       i = (i + 1) % dataBlkNum;
     }
     return results;
@@ -274,8 +274,8 @@ public class StripedBlockUtil {
      * |  (partial)  |    (from blk_1 and blk_2)   |          |
      * +------------------------------------------------------+
      */
-    public long startOffsetInBlock = 0;
-    public int readLength = 0;
+    private long startOffsetInBlock = 0;
+    private int readLength = 0;
     public final List<Integer> offsetsInBuf = new ArrayList<>();
     public final List<Integer> lengths = new ArrayList<>();
 
@@ -295,10 +295,20 @@ public class StripedBlockUtil {
       return lens;
     }
 
-    public boolean containsReadPortion(ReadPortion rp) {
-      long end = startOffsetInBlock + readLength;
-      return startOffsetInBlock <= rp.startOffsetInBlock && end >=
-          rp.startOffsetInBlock + rp.readLength;
+    public long getStartOffsetInBlock() {
+      return startOffsetInBlock;
+    }
+
+    public int getReadLength() {
+      return readLength;
+    }
+
+    public void setStartOffsetInBlock(long startOffsetInBlock) {
+      this.startOffsetInBlock = startOffsetInBlock;
+    }
+
+    void addReadLength(int extraLength) {
+      this.readLength += extraLength;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3db0fe6c/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 bcfc74b..11cdf7b 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
@@ -158,7 +158,7 @@ public class TestDFSStripedInputStream {
   private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes)
       throws IOException {
     Path testPath = new Path(src);
-    byte[] bytes = generateBytes(writeBytes);
+    final byte[] bytes = generateBytes(writeBytes);
     DFSTestUtil.writeFile(fs, testPath, new String(bytes));
 
     //check file length
@@ -175,7 +175,8 @@ public class TestDFSStripedInputStream {
       Assert.assertEquals("The length of file should be the same to write size",
           writeBytes, readLen);
       for (int i = 0; i < writeBytes; i++) {
-        Assert.assertEquals("Byte at i should be the same", getByte(i), buf[i]);
+        Assert.assertEquals("Byte at " + i + " should be the same", getByte(i),
+            buf[i]);
       }
     }
 
@@ -190,12 +191,12 @@ public class TestDFSStripedInputStream {
           readLen += ret;
         }
       } while (ret >= 0);
-
       readLen = readLen >= 0 ? readLen : 0;
       Assert.assertEquals("The length of file should be the same to write size",
           writeBytes, readLen);
       for (int i = 0; i < writeBytes; i++) {
-        Assert.assertEquals("Byte at i should be the same", getByte(i), buf[i]);
+        Assert.assertEquals("Byte at " + i + " should be the same", getByte(i),
+            buf[i]);
       }
     }
 
@@ -214,8 +215,47 @@ public class TestDFSStripedInputStream {
       Assert.assertEquals("The length of file should be the same to write size",
           writeBytes, readLen);
       for (int i = 0; i < writeBytes; i++) {
-        Assert.assertEquals("Byte at i should be the same", getByte(i), buf.array()[i]);
+        Assert.assertEquals("Byte at " + i + " should be the same", getByte(i),
+            buf.array()[i]);
       }
     }
+
+    // stateful read with 1KB size byte array
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
+      final byte[] result = new byte[writeBytes];
+      final byte[] buf = new byte[1024];
+      int readLen = 0;
+      int ret;
+      do {
+        ret = fsdis.read(buf, 0, buf.length);
+        if (ret > 0) {
+          System.arraycopy(buf, 0, result, readLen, ret);
+          readLen += ret;
+        }
+      } while (ret >= 0);
+      Assert.assertEquals("The length of file should be the same to write size",
+          writeBytes, readLen);
+      Assert.assertArrayEquals(bytes, result);
+    }
+
+    // stateful read using ByteBuffer with 1KB size
+    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
+      final ByteBuffer result = ByteBuffer.allocate(writeBytes);
+      final ByteBuffer buf = ByteBuffer.allocate(1024);
+      int readLen = 0;
+      int ret;
+      do {
+        ret = fsdis.read(buf);
+        if (ret > 0) {
+          readLen += ret;
+          buf.flip();
+          result.put(buf);
+          buf.clear();
+        }
+      } while (ret >= 0);
+      Assert.assertEquals("The length of file should be the same to write size",
+          writeBytes, readLen);
+      Assert.assertArrayEquals(bytes, result.array());
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3db0fe6c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
index 3b5787a..75d0587 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
@@ -38,8 +38,8 @@ public class TestPlanReadPortions {
     assertEquals(GROUP_SIZE, results.length);
 
     for (int i = 0; i < GROUP_SIZE; i++) {
-      assertEquals(readLengths[i], results[i].readLength);
-      assertEquals(offsetsInBlock[i], results[i].startOffsetInBlock);
+      assertEquals(readLengths[i], results[i].getReadLength());
+      assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock());
       final int[] bOffsets = results[i].getOffsets();
       assertArrayEquals(bufferOffsets[i], bOffsets);
       final int[] bLengths = results[i].getLengths();


[14/50] hadoop git commit: HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to create BlockReader. Contributed by Tsz Wo Nicholas Sze.

Posted by zh...@apache.org.
HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to create BlockReader. 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/d387b82d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d387b82d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d387b82d

Branch: refs/heads/HDFS-7285
Commit: d387b82d5665bdc08e60dbcf1018136587b6fdf0
Parents: b59a0ba
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Apr 21 20:56:39 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:36:17 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +
 .../apache/hadoop/hdfs/BlockReaderTestUtil.java |  7 +--
 .../hadoop/hdfs/TestBlockReaderFactory.java     | 16 +++---
 .../hadoop/hdfs/TestDFSStripedOutputStream.java | 58 ++------------------
 4 files changed, 20 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d387b82d/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 8f28285..d8f2e9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -107,3 +107,6 @@
 
     HDFS-8190. StripedBlockUtil.getInternalBlockLength may have overflow error.
     (szetszwo)
+
+    HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to 
+    create BlockReader. (szetszwo via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d387b82d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
index 88b7f37..829cf03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
@@ -165,20 +165,19 @@ public class BlockReaderTestUtil {
    */
   public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToRead)
       throws IOException {
-    return getBlockReader(cluster, testBlock, offset, lenToRead);
+    return getBlockReader(cluster.getFileSystem(), testBlock, offset, lenToRead);
   }
 
   /**
    * Get a BlockReader for the given block.
    */
-  public static BlockReader getBlockReader(MiniDFSCluster cluster,
-      LocatedBlock testBlock, int offset, int lenToRead) throws IOException {
+  public static BlockReader getBlockReader(final DistributedFileSystem fs,
+      LocatedBlock testBlock, int offset, long lenToRead) throws IOException {
     InetSocketAddress targetAddr = null;
     ExtendedBlock block = testBlock.getBlock();
     DatanodeInfo[] nodes = testBlock.getLocations();
     targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr());
 
-    final DistributedFileSystem fs = cluster.getFileSystem();
     return new BlockReaderFactory(fs.getClient().getConf()).
       setInetSocketAddress(targetAddr).
       setBlock(block).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d387b82d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
index d8aceff..1a767c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
@@ -250,8 +250,8 @@ public class TestBlockReaderFactory {
           LocatedBlock lblock = locatedBlocks.get(0); // first block
           BlockReader blockReader = null;
           try {
-            blockReader = BlockReaderTestUtil.
-                getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
+            blockReader = BlockReaderTestUtil.getBlockReader(
+                cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN);
             Assert.fail("expected getBlockReader to fail the first time.");
           } catch (Throwable t) { 
             Assert.assertTrue("expected to see 'TCP reads were disabled " +
@@ -265,8 +265,8 @@ public class TestBlockReaderFactory {
 
           // Second time should succeed.
           try {
-            blockReader = BlockReaderTestUtil.
-                getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
+            blockReader = BlockReaderTestUtil.getBlockReader(
+                cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN);
           } catch (Throwable t) { 
             LOG.error("error trying to retrieve a block reader " +
                 "the second time.", t);
@@ -474,8 +474,8 @@ public class TestBlockReaderFactory {
           while (true) {
             BlockReader blockReader = null;
             try {
-              blockReader = BlockReaderTestUtil.
-                  getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
+              blockReader = BlockReaderTestUtil.getBlockReader(
+                  cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN);
               sem.release();
               try {
                 blockReader.readAll(buf, 0, TEST_FILE_LEN);
@@ -514,8 +514,8 @@ public class TestBlockReaderFactory {
     // getting a ClosedChannelException.
     BlockReader blockReader = null;
     try {
-      blockReader = BlockReaderTestUtil.
-          getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
+      blockReader = BlockReaderTestUtil.getBlockReader(
+          cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN);
       blockReader.readFully(buf, 0, TEST_FILE_LEN);
     } finally {
       if (blockReader != null) blockReader.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d387b82d/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 160b190..c213183 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,8 +18,6 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -29,25 +27,14 @@ 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.client.impl.DfsClientConf;
-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.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.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.token.Token;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -59,7 +46,6 @@ public class TestDFSStripedOutputStream {
   private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
 
   private MiniDFSCluster cluster;
-  private Configuration conf = new Configuration();
   private DistributedFileSystem fs;
   private final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
   private final int stripesPerBlock = 4;
@@ -173,7 +159,11 @@ public class TestDFSStripedOutputStream {
     // 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);
 
@@ -199,11 +189,7 @@ public class TestDFSStripedOutputStream {
         if (lblock == null) {
           continue;
         }
-        DatanodeInfo[] nodes = lblock.getLocations();
         ExtendedBlock block = lblock.getBlock();
-        InetSocketAddress targetAddr = NetUtils.createSocketAddr(
-            nodes[0].getXferAddr());
-
         byte[] blockBytes = new byte[(int)block.getNumBytes()];
         if (i < dataBlocks) {
           dataBlockBytes[i] = blockBytes;
@@ -215,40 +201,8 @@ public class TestDFSStripedOutputStream {
           continue;
         }
 
-        BlockReader blockReader = new BlockReaderFactory(new DfsClientConf(conf)).
-            setFileName(src).
-            setBlock(block).
-            setBlockToken(lblock.getBlockToken()).
-            setInetSocketAddress(targetAddr).
-            setStartOffset(0).
-            setLength(block.getNumBytes()).
-            setVerifyChecksum(true).
-            setClientName("TestStripeLayoutWrite").
-            setDatanodeInfo(nodes[0]).
-            setCachingStrategy(CachingStrategy.newDefaultStrategy()).
-            setClientCacheContext(ClientContext.getFromConf(conf)).
-            setConfiguration(conf).
-            setRemotePeerFactory(new RemotePeerFactory() {
-              @Override
-              public Peer newConnectedPeer(InetSocketAddress addr,
-                                           Token<BlockTokenIdentifier> blockToken,
-                                           DatanodeID datanodeId)
-                  throws IOException {
-                Peer peer = null;
-                Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
-                try {
-                  sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
-                  sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
-                  peer = TcpPeerServer.peerFromSocket(sock);
-                } finally {
-                  if (peer == null) {
-                    IOUtils.closeSocket(sock);
-                  }
-                }
-                return peer;
-              }
-            }).build();
-
+        final BlockReader blockReader = BlockReaderTestUtil.getBlockReader(
+            fs, lblock, 0, block.getNumBytes());
         blockReader.readAll(blockBytes, 0, (int) block.getNumBytes());
         blockReader.close();
       }


[45/50] hadoop git commit: Revert mistakenly committing of HADOOP-11920 patch

Posted by zh...@apache.org.
Revert mistakenly committing of HADOOP-11920 patch


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

Branch: refs/heads/HDFS-7285
Commit: e367ca4e52b30e40348dfff24501e21d319eafe9
Parents: 2a98cee
Author: Kai Zheng <ka...@intel.com>
Authored: Thu May 7 17:02:14 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:40:58 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  2 -
 .../hadoop/fs/CommonConfigurationKeys.java      |  4 ++
 .../apache/hadoop/io/erasurecode/ECChunk.java   | 20 ++------
 .../erasurecode/coder/AbstractErasureCoder.java |  6 ++-
 .../io/erasurecode/coder/RSErasureDecoder.java  | 40 ++++++++++++++-
 .../rawcoder/AbstractRawErasureCoder.java       | 35 +-------------
 .../rawcoder/AbstractRawErasureDecoder.java     | 51 ++++++++------------
 .../rawcoder/AbstractRawErasureEncoder.java     | 48 +++++++++---------
 .../erasurecode/rawcoder/RawErasureCoder.java   |  8 +--
 .../erasurecode/rawcoder/RawErasureDecoder.java | 24 ++++-----
 .../io/erasurecode/rawcoder/XORRawDecoder.java  | 24 ++++++---
 .../io/erasurecode/rawcoder/XORRawEncoder.java  |  6 +--
 .../hadoop/io/erasurecode/TestCoderBase.java    |  4 +-
 .../erasurecode/coder/TestRSErasureCoder.java   |  4 +-
 14 files changed, 130 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index 7a344a8..9749270 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -44,5 +44,3 @@
     HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng)
 
     HADOOP-11841. Remove unused ecschema-def.xml files.  (szetszwo)
-
-    HADOOP-11921 Enhance tests for erasure coders. (Kai Zheng)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 3f2871b..bd2a24b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -143,6 +143,10 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   /** Supported erasure codec classes */
   public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs";
 
+  /** Use XOR raw coder when possible for the RS codec */
+  public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY =
+      "io.erasurecode.codec.rs.usexor";
+
   /** Raw coder factory for the RS codec */
   public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY =
       "io.erasurecode.codec.rs.rawcoder";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
index 34dd90b..01e8f35 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
@@ -58,15 +58,8 @@ public class ECChunk {
   public static ByteBuffer[] toBuffers(ECChunk[] chunks) {
     ByteBuffer[] buffers = new ByteBuffer[chunks.length];
 
-    ECChunk chunk;
     for (int i = 0; i < chunks.length; i++) {
-      chunk = chunks[i];
-      if (chunk == null) {
-        buffers[i] = null;
-        continue;
-      }
-
-      buffers[i] = chunk.getBuffer();
+      buffers[i] = chunks[i].getBuffer();
     }
 
     return buffers;
@@ -78,19 +71,12 @@ public class ECChunk {
    * @param chunks
    * @return an array of byte array
    */
-  public static byte[][] toArrays(ECChunk[] chunks) {
+  public static byte[][] toArray(ECChunk[] chunks) {
     byte[][] bytesArr = new byte[chunks.length][];
 
     ByteBuffer buffer;
-    ECChunk chunk;
     for (int i = 0; i < chunks.length; i++) {
-      chunk = chunks[i];
-      if (chunk == null) {
-        bytesArr[i] = null;
-        continue;
-      }
-
-      buffer = chunk.getBuffer();
+      buffer = chunks[i].getBuffer();
       if (buffer.hasArray()) {
         bytesArr[i] = buffer.array();
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
index d491570..7403e35 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -90,7 +90,11 @@ public abstract class AbstractErasureCoder
       throw new RuntimeException("Failed to create raw coder", e);
     }
 
-    return isEncoder ? fact.createEncoder() : fact.createDecoder();
+    if (fact != null) {
+      return isEncoder ? fact.createEncoder() : fact.createDecoder();
+    }
+
+    return null;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
index ec7cbb5..fc664a5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -17,11 +17,13 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 
 /**
  * Reed-Solomon erasure decoder that decodes a block group.
@@ -30,14 +32,38 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
  */
 public class RSErasureDecoder extends AbstractErasureDecoder {
   private RawErasureDecoder rsRawDecoder;
+  private RawErasureDecoder xorRawDecoder;
+  private boolean useXorWhenPossible = true;
 
   @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+
+    if (conf != null) {
+      this.useXorWhenPossible = conf.getBoolean(
+          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, true);
+    }
+  }
+
+    @Override
   protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
 
+    RawErasureDecoder rawDecoder;
+
     ECBlock[] inputBlocks = getInputBlocks(blockGroup);
     ECBlock[] outputBlocks = getOutputBlocks(blockGroup);
 
-    RawErasureDecoder rawDecoder = checkCreateRSRawDecoder();
+    /**
+     * Optimization: according to some benchmark, when only one block is erased
+     * and to be recovering, the most simple XOR scheme can be much efficient.
+     * We will have benchmark tests to verify this opt is effect or not.
+     */
+    if (outputBlocks.length == 1 && useXorWhenPossible) {
+      rawDecoder = checkCreateXorRawDecoder();
+    } else {
+      rawDecoder = checkCreateRSRawDecoder();
+    }
+
     return new ErasureDecodingStep(inputBlocks,
         getErasedIndexes(inputBlocks), outputBlocks, rawDecoder);
   }
@@ -55,9 +81,19 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
     return rsRawDecoder;
   }
 
+  private RawErasureDecoder checkCreateXorRawDecoder() {
+    if (xorRawDecoder == null) {
+      xorRawDecoder = new XORRawDecoder();
+      xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize());
+    }
+    return xorRawDecoder;
+  }
+
   @Override
   public void release() {
-    if (rsRawDecoder != null) {
+    if (xorRawDecoder != null) {
+      xorRawDecoder.release();
+    } else if (rsRawDecoder != null) {
       rsRawDecoder.release();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
index eed9035..e6f3d92 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
@@ -19,9 +19,6 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 
 import org.apache.hadoop.conf.Configured;
 
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
 /**
  * A common class of basic facilities to be shared by encoder and decoder
  *
@@ -30,9 +27,6 @@ import java.util.Arrays;
 public abstract class AbstractRawErasureCoder
     extends Configured implements RawErasureCoder {
 
-  // Hope to reset coding buffers a little faster using it
-  protected byte[] zeroChunkBytes;
-
   private int numDataUnits;
   private int numParityUnits;
   private int chunkSize;
@@ -43,8 +37,6 @@ public abstract class AbstractRawErasureCoder
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
     this.chunkSize = chunkSize;
-
-    zeroChunkBytes = new byte[chunkSize]; // With ZERO by default
   }
 
   @Override
@@ -63,7 +55,7 @@ public abstract class AbstractRawErasureCoder
   }
 
   @Override
-  public boolean preferDirectBuffer() {
+  public boolean preferNativeBuffer() {
     return false;
   }
 
@@ -71,29 +63,4 @@ public abstract class AbstractRawErasureCoder
   public void release() {
     // Nothing to do by default
   }
-  /**
-   * Ensure the buffer (either input or output) ready to read or write with ZERO
-   * bytes fully in chunkSize.
-   * @param buffer
-   * @return the buffer itself
-   */
-  protected ByteBuffer resetDirectBuffer(ByteBuffer buffer) {
-    buffer.clear();
-    buffer.put(zeroChunkBytes);
-    buffer.position(0);
-
-    return buffer;
-  }
-
-  /**
-   * Ensure the buffer (either input or output) ready to read or write with ZERO
-   * bytes fully in chunkSize.
-   * @param buffer bytes array buffer
-   * @return the buffer itself
-   */
-  protected byte[] resetArrayBuffer(byte[] buffer) {
-    System.arraycopy(zeroChunkBytes, 0, buffer, 0, buffer.length);
-
-    return buffer;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
index 17f867a..4613b25 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
@@ -32,7 +32,9 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   @Override
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs) {
-    checkParameters(inputs, erasedIndexes, outputs);
+    if (erasedIndexes.length == 0) {
+      return;
+    }
 
     doDecode(inputs, erasedIndexes, outputs);
   }
@@ -48,7 +50,9 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
 
   @Override
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) {
-    checkParameters(inputs, erasedIndexes, outputs);
+    if (erasedIndexes.length == 0) {
+      return;
+    }
 
     doDecode(inputs, erasedIndexes, outputs);
   }
@@ -65,42 +69,25 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   @Override
   public void decode(ECChunk[] inputs, int[] erasedIndexes,
                      ECChunk[] outputs) {
-    checkParameters(inputs, erasedIndexes, outputs);
-
-    boolean hasArray = inputs[0].getBuffer().hasArray();
-    if (hasArray) {
-      byte[][] inputBytesArr = ECChunk.toArrays(inputs);
-      byte[][] outputBytesArr = ECChunk.toArrays(outputs);
-      doDecode(inputBytesArr, erasedIndexes, outputBytesArr);
-    } else {
-      ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
-      ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
-      doDecode(inputBuffers, erasedIndexes, outputBuffers);
-    }
+    doDecode(inputs, erasedIndexes, outputs);
   }
+
   /**
-   * Check and validate decoding parameters, throw exception accordingly. The
-   * checking assumes it's a MDS code. Other code  can override this.
+   * Perform the real decoding using chunks
    * @param inputs
    * @param erasedIndexes
    * @param outputs
    */
-  protected void checkParameters(Object[] inputs, int[] erasedIndexes,
-                                 Object[] outputs) {
-    if (inputs.length != getNumParityUnits() + getNumDataUnits()) {
-      throw new IllegalArgumentException("Invalid inputs length");
-    }
-
-    if (erasedIndexes.length != outputs.length) {
-      throw new IllegalArgumentException(
-          "erasedIndexes and outputs mismatch in length");
-    }
-
-    if (erasedIndexes.length > getNumParityUnits()) {
-      throw new IllegalArgumentException(
-          "Too many erased, not recoverable");
+  protected void doDecode(ECChunk[] inputs, int[] erasedIndexes,
+                          ECChunk[] outputs) {
+    if (inputs[0].getBuffer().hasArray()) {
+      byte[][] inputBytesArr = ECChunk.toArray(inputs);
+      byte[][] outputBytesArr = ECChunk.toArray(outputs);
+      doDecode(inputBytesArr, erasedIndexes, outputBytesArr);
+    } else {
+      ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
+      ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
+      doDecode(inputBuffers, erasedIndexes, outputBuffers);
     }
-
-
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
index 51695e8..4feaf39 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
@@ -31,7 +31,8 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
 
   @Override
   public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    checkParameters(inputs, outputs);
+    assert (inputs.length == getNumDataUnits());
+    assert (outputs.length == getNumParityUnits());
 
     doEncode(inputs, outputs);
   }
@@ -45,7 +46,8 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
 
   @Override
   public void encode(byte[][] inputs, byte[][] outputs) {
-    checkParameters(inputs, outputs);
+    assert (inputs.length == getNumDataUnits());
+    assert (outputs.length == getNumParityUnits());
 
     doEncode(inputs, outputs);
   }
@@ -59,37 +61,33 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
 
   @Override
   public void encode(ECChunk[] inputs, ECChunk[] outputs) {
-    checkParameters(inputs, outputs);
+    assert (inputs.length == getNumDataUnits());
+    assert (outputs.length == getNumParityUnits());
 
+    doEncode(inputs, outputs);
+  }
+
+  /**
+   * Perform the real encoding work using chunks.
+   * @param inputs
+   * @param outputs
+   */
+  protected void doEncode(ECChunk[] inputs, ECChunk[] outputs) {
     /**
-     * Note callers may pass byte array, or direct buffer via ECChunk according
+     * Note callers may pass byte array, or ByteBuffer via ECChunk according
      * to how ECChunk is created. Some implementations of coder use byte array
-     * (ex: pure Java), some use direct buffer (ex: ISA-L), all for the better
-     * performance.
+     * (ex: pure Java), some use native ByteBuffer (ex: ISA-L), all for the
+     * better performance.
      */
-    boolean hasArray = inputs[0].getBuffer().hasArray();
-    if (hasArray) {
-      byte[][] inputBytesArr = ECChunk.toArrays(inputs);
-      byte[][] outputBytesArr = ECChunk.toArrays(outputs);
-      encode(inputBytesArr, outputBytesArr);
+    if (inputs[0].getBuffer().hasArray()) {
+      byte[][] inputBytesArr = ECChunk.toArray(inputs);
+      byte[][] outputBytesArr = ECChunk.toArray(outputs);
+      doEncode(inputBytesArr, outputBytesArr);
     } else {
       ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
       ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
-      encode(inputBuffers, outputBuffers);
+      doEncode(inputBuffers, outputBuffers);
     }
   }
 
-  /**
-   * Check and validate decoding parameters, throw exception accordingly.
-   * @param inputs
-   * @param outputs
-   */
-  protected void checkParameters(Object[] inputs, Object[] outputs) {
-    if (inputs.length != getNumDataUnits()) {
-      throw new IllegalArgumentException("Invalid inputs length");
-    }
-    if (outputs.length != getNumParityUnits()) {
-      throw new IllegalArgumentException("Invalid outputs length");
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
index 3fb211f..9af5b6c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -64,13 +64,13 @@ public interface RawErasureCoder extends Configurable {
   public int getChunkSize();
 
   /**
-   * Tell if direct buffer is preferred or not. It's for callers to
-   * decide how to allocate coding chunk buffers, using DirectByteBuffer or
-   * bytes array. It will return false by default.
+   * Tell if native or off-heap buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, either on heap or off heap.
+   * It will return false by default.
    * @return true if native buffer is preferred for performance consideration,
    * otherwise false.
    */
-  public boolean preferDirectBuffer();
+  public boolean preferNativeBuffer();
 
   /**
    * Should be called when release this coder. Good chance to release encoding

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
index 1807da7..1358b7d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
@@ -31,30 +31,24 @@ import java.nio.ByteBuffer;
 public interface RawErasureDecoder extends RawErasureCoder {
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs.
-   * @param inputs inputs to read data from
-   * @param erasedIndexes indexes of erased units in the inputs array
-   * @param outputs outputs to write into for data generated according to
-   *                erasedIndexes
+   * Decode with inputs and erasedIndexes, generates outputs
+   * @param inputs
+   * @param outputs
    */
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs);
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs.
-   * @param inputs inputs to read data from
-   * @param erasedIndexes indexes of erased units in the inputs array
-   * @param outputs outputs to write into for data generated according to
-   *                erasedIndexes
+   * Decode with inputs and erasedIndexes, generates outputs
+   * @param inputs
+   * @param outputs
    */
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs);
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs.
-   * @param inputs inputs to read data from
-   * @param erasedIndexes indexes of erased units in the inputs array
-   * @param outputs outputs to write into for data generated according to
-   *                erasedIndexes
+   * Decode with inputs and erasedIndexes, generates outputs
+   * @param inputs
+   * @param outputs
    */
   public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
index 7c11681..b6b1633 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
@@ -27,11 +27,17 @@ public class XORRawDecoder extends AbstractRawErasureDecoder {
   @Override
   protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
                           ByteBuffer[] outputs) {
-    resetDirectBuffer(outputs[0]);
+    assert(erasedIndexes.length == outputs.length);
+    assert(erasedIndexes.length <= 1);
 
-    int bufSize = getChunkSize();
+    int bufSize = inputs[0].remaining();
     int erasedIdx = erasedIndexes[0];
 
+    // Set the output to zeros.
+    for (int j = 0; j < bufSize; j++) {
+      outputs[0].put(j, (byte) 0);
+    }
+
     // Process the inputs.
     for (int i = 0; i < inputs.length; i++) {
       // Skip the erased location.
@@ -46,13 +52,19 @@ public class XORRawDecoder extends AbstractRawErasureDecoder {
   }
 
   @Override
-  protected void doDecode(byte[][] inputs,
-                          int[] erasedIndexes, byte[][] outputs) {
-    resetArrayBuffer(outputs[0]);
+  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
+                          byte[][] outputs) {
+    assert(erasedIndexes.length == outputs.length);
+    assert(erasedIndexes.length <= 1);
 
-    int bufSize = getChunkSize();
+    int bufSize = inputs[0].length;
     int erasedIdx = erasedIndexes[0];
 
+    // Set the output to zeros.
+    for (int j = 0; j < bufSize; j++) {
+      outputs[0][j] = 0;
+    }
+
     // Process the inputs.
     for (int i = 0; i < inputs.length; i++) {
       // Skip the erased location.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
index c751a73..dbfab5d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
@@ -26,9 +26,8 @@ public class XORRawEncoder extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    resetDirectBuffer(outputs[0]);
+    int bufSize = inputs[0].remaining();
 
-    int bufSize = getChunkSize();
     // Get the first buffer's data.
     for (int j = 0; j < bufSize; j++) {
       outputs[0].put(j, inputs[0].get(j));
@@ -44,9 +43,8 @@ public class XORRawEncoder extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(byte[][] inputs, byte[][] outputs) {
-    resetArrayBuffer(outputs[0]);
+    int bufSize = inputs[0].length;
 
-    int bufSize = getChunkSize();
     // Get the first buffer's data.
     for (int j = 0; j < bufSize; j++) {
       outputs[0][j] = inputs[0][j];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
index 9c88665..22fd98d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -75,8 +75,8 @@ public abstract class TestCoderBase {
    */
   protected void compareAndVerify(ECChunk[] erasedChunks,
                                   ECChunk[] recoveredChunks) {
-    byte[][] erased = ECChunk.toArrays(erasedChunks);
-    byte[][] recovered = ECChunk.toArrays(recoveredChunks);
+    byte[][] erased = ECChunk.toArray(erasedChunks);
+    byte[][] recovered = ECChunk.toArray(recoveredChunks);
     boolean result = Arrays.deepEquals(erased, recovered);
     assertTrue("Decoding and comparing failed.", result);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e367ca4e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
index 6b3e742..3507dd2 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
@@ -59,9 +59,9 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
         RSRawErasureCoderFactory.class.getCanonicalName());
-
+    conf.setBoolean(
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false);
     prepare(conf, 10, 4, null);
-
     testCoding(true);
   }
 


[41/50] hadoop git commit: HADOOP-11921. Enhance tests for erasure coders. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11921. Enhance tests for erasure coders. Contributed by Kai Zheng


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

Branch: refs/heads/HDFS-7285
Commit: 97691e5b27d6c6f347ac9471ce8e5b0332f3a866
Parents: bdb8947
Author: Kai Zheng <ka...@intel.com>
Authored: Thu May 7 06:07:51 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon May 11 11:40:51 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  2 +
 .../hadoop/fs/CommonConfigurationKeys.java      |  4 --
 .../apache/hadoop/io/erasurecode/ECChunk.java   | 20 ++++++--
 .../erasurecode/coder/AbstractErasureCoder.java |  6 +--
 .../io/erasurecode/coder/RSErasureDecoder.java  | 40 +--------------
 .../rawcoder/AbstractRawErasureCoder.java       | 35 +++++++++++++-
 .../rawcoder/AbstractRawErasureDecoder.java     | 51 ++++++++++++--------
 .../rawcoder/AbstractRawErasureEncoder.java     | 48 +++++++++---------
 .../erasurecode/rawcoder/RawErasureCoder.java   |  8 +--
 .../erasurecode/rawcoder/RawErasureDecoder.java | 24 +++++----
 .../io/erasurecode/rawcoder/XORRawDecoder.java  | 24 +++------
 .../io/erasurecode/rawcoder/XORRawEncoder.java  |  6 ++-
 .../hadoop/io/erasurecode/TestCoderBase.java    |  4 +-
 .../erasurecode/coder/TestRSErasureCoder.java   |  4 +-
 14 files changed, 146 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index 9749270..7a344a8 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -44,3 +44,5 @@
     HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng)
 
     HADOOP-11841. Remove unused ecschema-def.xml files.  (szetszwo)
+
+    HADOOP-11921 Enhance tests for erasure coders. (Kai Zheng)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index bd2a24b..3f2871b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -143,10 +143,6 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   /** Supported erasure codec classes */
   public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs";
 
-  /** Use XOR raw coder when possible for the RS codec */
-  public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY =
-      "io.erasurecode.codec.rs.usexor";
-
   /** Raw coder factory for the RS codec */
   public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY =
       "io.erasurecode.codec.rs.rawcoder";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
index 01e8f35..34dd90b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
@@ -58,8 +58,15 @@ public class ECChunk {
   public static ByteBuffer[] toBuffers(ECChunk[] chunks) {
     ByteBuffer[] buffers = new ByteBuffer[chunks.length];
 
+    ECChunk chunk;
     for (int i = 0; i < chunks.length; i++) {
-      buffers[i] = chunks[i].getBuffer();
+      chunk = chunks[i];
+      if (chunk == null) {
+        buffers[i] = null;
+        continue;
+      }
+
+      buffers[i] = chunk.getBuffer();
     }
 
     return buffers;
@@ -71,12 +78,19 @@ public class ECChunk {
    * @param chunks
    * @return an array of byte array
    */
-  public static byte[][] toArray(ECChunk[] chunks) {
+  public static byte[][] toArrays(ECChunk[] chunks) {
     byte[][] bytesArr = new byte[chunks.length][];
 
     ByteBuffer buffer;
+    ECChunk chunk;
     for (int i = 0; i < chunks.length; i++) {
-      buffer = chunks[i].getBuffer();
+      chunk = chunks[i];
+      if (chunk == null) {
+        bytesArr[i] = null;
+        continue;
+      }
+
+      buffer = chunk.getBuffer();
       if (buffer.hasArray()) {
         bytesArr[i] = buffer.array();
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
index 7403e35..d491570 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -90,11 +90,7 @@ public abstract class AbstractErasureCoder
       throw new RuntimeException("Failed to create raw coder", e);
     }
 
-    if (fact != null) {
-      return isEncoder ? fact.createEncoder() : fact.createDecoder();
-    }
-
-    return null;
+    return isEncoder ? fact.createEncoder() : fact.createDecoder();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
index fc664a5..ec7cbb5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -17,13 +17,11 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
-import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 
 /**
  * Reed-Solomon erasure decoder that decodes a block group.
@@ -32,38 +30,14 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
  */
 public class RSErasureDecoder extends AbstractErasureDecoder {
   private RawErasureDecoder rsRawDecoder;
-  private RawErasureDecoder xorRawDecoder;
-  private boolean useXorWhenPossible = true;
 
   @Override
-  public void setConf(Configuration conf) {
-    super.setConf(conf);
-
-    if (conf != null) {
-      this.useXorWhenPossible = conf.getBoolean(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, true);
-    }
-  }
-
-    @Override
   protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
 
-    RawErasureDecoder rawDecoder;
-
     ECBlock[] inputBlocks = getInputBlocks(blockGroup);
     ECBlock[] outputBlocks = getOutputBlocks(blockGroup);
 
-    /**
-     * Optimization: according to some benchmark, when only one block is erased
-     * and to be recovering, the most simple XOR scheme can be much efficient.
-     * We will have benchmark tests to verify this opt is effect or not.
-     */
-    if (outputBlocks.length == 1 && useXorWhenPossible) {
-      rawDecoder = checkCreateXorRawDecoder();
-    } else {
-      rawDecoder = checkCreateRSRawDecoder();
-    }
-
+    RawErasureDecoder rawDecoder = checkCreateRSRawDecoder();
     return new ErasureDecodingStep(inputBlocks,
         getErasedIndexes(inputBlocks), outputBlocks, rawDecoder);
   }
@@ -81,19 +55,9 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
     return rsRawDecoder;
   }
 
-  private RawErasureDecoder checkCreateXorRawDecoder() {
-    if (xorRawDecoder == null) {
-      xorRawDecoder = new XORRawDecoder();
-      xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize());
-    }
-    return xorRawDecoder;
-  }
-
   @Override
   public void release() {
-    if (xorRawDecoder != null) {
-      xorRawDecoder.release();
-    } else if (rsRawDecoder != null) {
+    if (rsRawDecoder != null) {
       rsRawDecoder.release();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
index e6f3d92..eed9035 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
@@ -19,6 +19,9 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 
 import org.apache.hadoop.conf.Configured;
 
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
 /**
  * A common class of basic facilities to be shared by encoder and decoder
  *
@@ -27,6 +30,9 @@ import org.apache.hadoop.conf.Configured;
 public abstract class AbstractRawErasureCoder
     extends Configured implements RawErasureCoder {
 
+  // Hope to reset coding buffers a little faster using it
+  protected byte[] zeroChunkBytes;
+
   private int numDataUnits;
   private int numParityUnits;
   private int chunkSize;
@@ -37,6 +43,8 @@ public abstract class AbstractRawErasureCoder
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
     this.chunkSize = chunkSize;
+
+    zeroChunkBytes = new byte[chunkSize]; // With ZERO by default
   }
 
   @Override
@@ -55,7 +63,7 @@ public abstract class AbstractRawErasureCoder
   }
 
   @Override
-  public boolean preferNativeBuffer() {
+  public boolean preferDirectBuffer() {
     return false;
   }
 
@@ -63,4 +71,29 @@ public abstract class AbstractRawErasureCoder
   public void release() {
     // Nothing to do by default
   }
+  /**
+   * Ensure the buffer (either input or output) ready to read or write with ZERO
+   * bytes fully in chunkSize.
+   * @param buffer
+   * @return the buffer itself
+   */
+  protected ByteBuffer resetDirectBuffer(ByteBuffer buffer) {
+    buffer.clear();
+    buffer.put(zeroChunkBytes);
+    buffer.position(0);
+
+    return buffer;
+  }
+
+  /**
+   * Ensure the buffer (either input or output) ready to read or write with ZERO
+   * bytes fully in chunkSize.
+   * @param buffer bytes array buffer
+   * @return the buffer itself
+   */
+  protected byte[] resetArrayBuffer(byte[] buffer) {
+    System.arraycopy(zeroChunkBytes, 0, buffer, 0, buffer.length);
+
+    return buffer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
index 4613b25..17f867a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
@@ -32,9 +32,7 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   @Override
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs) {
-    if (erasedIndexes.length == 0) {
-      return;
-    }
+    checkParameters(inputs, erasedIndexes, outputs);
 
     doDecode(inputs, erasedIndexes, outputs);
   }
@@ -50,9 +48,7 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
 
   @Override
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) {
-    if (erasedIndexes.length == 0) {
-      return;
-    }
+    checkParameters(inputs, erasedIndexes, outputs);
 
     doDecode(inputs, erasedIndexes, outputs);
   }
@@ -69,20 +65,12 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   @Override
   public void decode(ECChunk[] inputs, int[] erasedIndexes,
                      ECChunk[] outputs) {
-    doDecode(inputs, erasedIndexes, outputs);
-  }
+    checkParameters(inputs, erasedIndexes, outputs);
 
-  /**
-   * Perform the real decoding using chunks
-   * @param inputs
-   * @param erasedIndexes
-   * @param outputs
-   */
-  protected void doDecode(ECChunk[] inputs, int[] erasedIndexes,
-                          ECChunk[] outputs) {
-    if (inputs[0].getBuffer().hasArray()) {
-      byte[][] inputBytesArr = ECChunk.toArray(inputs);
-      byte[][] outputBytesArr = ECChunk.toArray(outputs);
+    boolean hasArray = inputs[0].getBuffer().hasArray();
+    if (hasArray) {
+      byte[][] inputBytesArr = ECChunk.toArrays(inputs);
+      byte[][] outputBytesArr = ECChunk.toArrays(outputs);
       doDecode(inputBytesArr, erasedIndexes, outputBytesArr);
     } else {
       ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
@@ -90,4 +78,29 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
       doDecode(inputBuffers, erasedIndexes, outputBuffers);
     }
   }
+  /**
+   * Check and validate decoding parameters, throw exception accordingly. The
+   * checking assumes it's a MDS code. Other code  can override this.
+   * @param inputs
+   * @param erasedIndexes
+   * @param outputs
+   */
+  protected void checkParameters(Object[] inputs, int[] erasedIndexes,
+                                 Object[] outputs) {
+    if (inputs.length != getNumParityUnits() + getNumDataUnits()) {
+      throw new IllegalArgumentException("Invalid inputs length");
+    }
+
+    if (erasedIndexes.length != outputs.length) {
+      throw new IllegalArgumentException(
+          "erasedIndexes and outputs mismatch in length");
+    }
+
+    if (erasedIndexes.length > getNumParityUnits()) {
+      throw new IllegalArgumentException(
+          "Too many erased, not recoverable");
+    }
+
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
index 4feaf39..51695e8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
@@ -31,8 +31,7 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
 
   @Override
   public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    assert (inputs.length == getNumDataUnits());
-    assert (outputs.length == getNumParityUnits());
+    checkParameters(inputs, outputs);
 
     doEncode(inputs, outputs);
   }
@@ -46,8 +45,7 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
 
   @Override
   public void encode(byte[][] inputs, byte[][] outputs) {
-    assert (inputs.length == getNumDataUnits());
-    assert (outputs.length == getNumParityUnits());
+    checkParameters(inputs, outputs);
 
     doEncode(inputs, outputs);
   }
@@ -61,33 +59,37 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
 
   @Override
   public void encode(ECChunk[] inputs, ECChunk[] outputs) {
-    assert (inputs.length == getNumDataUnits());
-    assert (outputs.length == getNumParityUnits());
+    checkParameters(inputs, outputs);
 
-    doEncode(inputs, outputs);
-  }
-
-  /**
-   * Perform the real encoding work using chunks.
-   * @param inputs
-   * @param outputs
-   */
-  protected void doEncode(ECChunk[] inputs, ECChunk[] outputs) {
     /**
-     * Note callers may pass byte array, or ByteBuffer via ECChunk according
+     * Note callers may pass byte array, or direct buffer via ECChunk according
      * to how ECChunk is created. Some implementations of coder use byte array
-     * (ex: pure Java), some use native ByteBuffer (ex: ISA-L), all for the
-     * better performance.
+     * (ex: pure Java), some use direct buffer (ex: ISA-L), all for the better
+     * performance.
      */
-    if (inputs[0].getBuffer().hasArray()) {
-      byte[][] inputBytesArr = ECChunk.toArray(inputs);
-      byte[][] outputBytesArr = ECChunk.toArray(outputs);
-      doEncode(inputBytesArr, outputBytesArr);
+    boolean hasArray = inputs[0].getBuffer().hasArray();
+    if (hasArray) {
+      byte[][] inputBytesArr = ECChunk.toArrays(inputs);
+      byte[][] outputBytesArr = ECChunk.toArrays(outputs);
+      encode(inputBytesArr, outputBytesArr);
     } else {
       ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
       ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
-      doEncode(inputBuffers, outputBuffers);
+      encode(inputBuffers, outputBuffers);
     }
   }
 
+  /**
+   * Check and validate decoding parameters, throw exception accordingly.
+   * @param inputs
+   * @param outputs
+   */
+  protected void checkParameters(Object[] inputs, Object[] outputs) {
+    if (inputs.length != getNumDataUnits()) {
+      throw new IllegalArgumentException("Invalid inputs length");
+    }
+    if (outputs.length != getNumParityUnits()) {
+      throw new IllegalArgumentException("Invalid outputs length");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
index 9af5b6c..3fb211f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -64,13 +64,13 @@ public interface RawErasureCoder extends Configurable {
   public int getChunkSize();
 
   /**
-   * Tell if native or off-heap buffer is preferred or not. It's for callers to
-   * decide how to allocate coding chunk buffers, either on heap or off heap.
-   * It will return false by default.
+   * Tell if direct buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, using DirectByteBuffer or
+   * bytes array. It will return false by default.
    * @return true if native buffer is preferred for performance consideration,
    * otherwise false.
    */
-  public boolean preferNativeBuffer();
+  public boolean preferDirectBuffer();
 
   /**
    * Should be called when release this coder. Good chance to release encoding

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
index 1358b7d..1807da7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
@@ -31,24 +31,30 @@ import java.nio.ByteBuffer;
 public interface RawErasureDecoder extends RawErasureCoder {
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs
-   * @param inputs
-   * @param outputs
+   * Decode with inputs and erasedIndexes, generates outputs.
+   * @param inputs inputs to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes
    */
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs);
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs
-   * @param inputs
-   * @param outputs
+   * Decode with inputs and erasedIndexes, generates outputs.
+   * @param inputs inputs to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes
    */
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs);
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs
-   * @param inputs
-   * @param outputs
+   * Decode with inputs and erasedIndexes, generates outputs.
+   * @param inputs inputs to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes
    */
   public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
index b6b1633..7c11681 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
@@ -27,17 +27,11 @@ public class XORRawDecoder extends AbstractRawErasureDecoder {
   @Override
   protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
                           ByteBuffer[] outputs) {
-    assert(erasedIndexes.length == outputs.length);
-    assert(erasedIndexes.length <= 1);
+    resetDirectBuffer(outputs[0]);
 
-    int bufSize = inputs[0].remaining();
+    int bufSize = getChunkSize();
     int erasedIdx = erasedIndexes[0];
 
-    // Set the output to zeros.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0].put(j, (byte) 0);
-    }
-
     // Process the inputs.
     for (int i = 0; i < inputs.length; i++) {
       // Skip the erased location.
@@ -52,19 +46,13 @@ public class XORRawDecoder extends AbstractRawErasureDecoder {
   }
 
   @Override
-  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
-                          byte[][] outputs) {
-    assert(erasedIndexes.length == outputs.length);
-    assert(erasedIndexes.length <= 1);
+  protected void doDecode(byte[][] inputs,
+                          int[] erasedIndexes, byte[][] outputs) {
+    resetArrayBuffer(outputs[0]);
 
-    int bufSize = inputs[0].length;
+    int bufSize = getChunkSize();
     int erasedIdx = erasedIndexes[0];
 
-    // Set the output to zeros.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0][j] = 0;
-    }
-
     // Process the inputs.
     for (int i = 0; i < inputs.length; i++) {
       // Skip the erased location.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
index dbfab5d..c751a73 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
@@ -26,8 +26,9 @@ public class XORRawEncoder extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    int bufSize = inputs[0].remaining();
+    resetDirectBuffer(outputs[0]);
 
+    int bufSize = getChunkSize();
     // Get the first buffer's data.
     for (int j = 0; j < bufSize; j++) {
       outputs[0].put(j, inputs[0].get(j));
@@ -43,8 +44,9 @@ public class XORRawEncoder extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(byte[][] inputs, byte[][] outputs) {
-    int bufSize = inputs[0].length;
+    resetArrayBuffer(outputs[0]);
 
+    int bufSize = getChunkSize();
     // Get the first buffer's data.
     for (int j = 0; j < bufSize; j++) {
       outputs[0][j] = inputs[0][j];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
index 22fd98d..9c88665 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -75,8 +75,8 @@ public abstract class TestCoderBase {
    */
   protected void compareAndVerify(ECChunk[] erasedChunks,
                                   ECChunk[] recoveredChunks) {
-    byte[][] erased = ECChunk.toArray(erasedChunks);
-    byte[][] recovered = ECChunk.toArray(recoveredChunks);
+    byte[][] erased = ECChunk.toArrays(erasedChunks);
+    byte[][] recovered = ECChunk.toArrays(recoveredChunks);
     boolean result = Arrays.deepEquals(erased, recovered);
     assertTrue("Decoding and comparing failed.", result);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97691e5b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
index 3507dd2..6b3e742 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
@@ -59,9 +59,9 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
         RSRawErasureCoderFactory.class.getCanonicalName());
-    conf.setBoolean(
-        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false);
+
     prepare(conf, 10, 4, null);
+
     testCoding(true);
   }