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/26 21:34:31 UTC

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

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 48513749a -> c9e026821 (forced update)


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/5059958b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5059958b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5059958b

Branch: refs/heads/HDFS-7285
Commit: 5059958bf4f5541926bae3b145bceb2fd583e4ed
Parents: 1a31f1c
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Apr 30 19:42:29 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 11:59:56 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/5059958b/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/5059958b/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/5059958b/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 a209142..3fa8818 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
@@ -7529,9 +7529,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/5059958b/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/5059958b/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/5059958b/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


[30/50] [abbrv] hadoop git commit: HDFS-8320. Erasure coding: consolidate striping-related terminologies. Contributed by Zhe Zhang and Jing Zhao.

Posted by zh...@apache.org.
HDFS-8320. Erasure coding: consolidate striping-related terminologies. Contributed by Zhe Zhang and Jing Zhao.


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

Branch: refs/heads/HDFS-7285
Commit: 7434c44b16765a49168c8f5cb01a3d0b817d158b
Parents: b008348
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon May 18 22:55:27 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:32 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   2 +
 .../hadoop/hdfs/DFSStripedInputStream.java      |  46 +-
 .../erasurecode/ErasureCodingWorker.java        |   4 +-
 .../hadoop/hdfs/util/StripedBlockUtil.java      | 419 +++++++++----------
 .../hadoop/hdfs/TestPlanReadPortions.java       | 143 -------
 .../hadoop/hdfs/TestRecoverStripedFile.java     |   4 +-
 .../hadoop/hdfs/util/TestStripedBlockUtil.java  | 196 ++++++++-
 7 files changed, 400 insertions(+), 414 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7434c44b/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 1549930..3170e9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -223,3 +223,5 @@
 
     HDFS-8418. Fix the isNeededReplication calculation for Striped block in NN.
     (Yi Liu via jing9)
+
+    HDFS-8320. Erasure coding: consolidate striping-related terminologies. (zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7434c44b/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 8f15eda..744d586 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
@@ -23,19 +23,18 @@ import org.apache.hadoop.fs.ReadOption;
 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.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.ByteBufferPool;
 
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.planReadPortions;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.divideByteRangeIntoStripes;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.initDecodeInputs;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.decodeAndFillBuffer;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getNextCompletedStripedRead;
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getStartOffsetsForInternalBlocks;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.parseStripedBlockGroup;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
@@ -65,30 +64,9 @@ import java.util.concurrent.CancellationException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Future;
 
-/******************************************************************************
- * DFSStripedInputStream reads from striped block groups, illustrated below:
- *
- * | <- Striped Block Group -> |
- *  blk_0      blk_1       blk_2   <- A striped block group has
- *    |          |           |          {@link #dataBlkNum} blocks
- *    v          v           v
- * +------+   +------+   +------+
- * |cell_0|   |cell_1|   |cell_2|  <- The logical read order should be
- * +------+   +------+   +------+       cell_0, cell_1, ...
- * |cell_3|   |cell_4|   |cell_5|
- * +------+   +------+   +------+
- * |cell_6|   |cell_7|   |cell_8|
- * +------+   +------+   +------+
- * |cell_9|
- * +------+  <- A cell contains {@link #cellSize} bytes of data
- *
- * Three styles of read will eventually be supported:
- *   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.
- *   3. pread with decode support: TODO: will be supported after HDFS-7678
- *****************************************************************************/
+/**
+ * DFSStripedInputStream reads from striped block groups
+ */
 public class DFSStripedInputStream extends DFSInputStream {
 
   private static class ReaderRetryPolicy {
@@ -207,22 +185,24 @@ public class DFSStripedInputStream extends DFSInputStream {
     currentLocatedBlock = targetBlockGroup;
 
     final long offsetIntoBlockGroup = getOffsetInBlockGroup();
-    LocatedBlock[] targetBlocks = StripedBlockUtil.parseStripedBlockGroup(
+    LocatedBlock[] targetBlocks = parseStripedBlockGroup(
         targetBlockGroup, cellSize, dataBlkNum, parityBlkNum);
-    // The purpose is to get start offset into each block
-    ReadPortion[] readPortions = planReadPortions(groupSize, cellSize,
-        offsetIntoBlockGroup, 0, 0);
+    // The purpose is to get start offset into each block.
+    long[] offsetsForInternalBlocks = getStartOffsetsForInternalBlocks(schema,
+        targetBlockGroup, offsetIntoBlockGroup);
+    Preconditions.checkNotNull(offsetsForInternalBlocks);
 
     final ReaderRetryPolicy retry = new ReaderRetryPolicy();
     for (int i = 0; i < groupSize; i++) {
       LocatedBlock targetBlock = targetBlocks[i];
       if (targetBlock != null) {
+        long offsetInBlock = offsetsForInternalBlocks[i] < 0 ?
+            0 : offsetsForInternalBlocks[i];
         DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null);
         if (retval != null) {
           currentNodes[i] = retval.info;
           blockReaders[i] = getBlockReaderWithRetry(targetBlock,
-              readPortions[i].getStartOffsetInBlock(),
-              targetBlock.getBlockSize() - readPortions[i].getStartOffsetInBlock(),
+              offsetInBlock, targetBlock.getBlockSize() - offsetInBlock,
               retval.addr, retval.storageType, retval.info, target, retry);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7434c44b/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 7b3c24d..a1c0f72 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
@@ -301,12 +301,12 @@ public final class ErasureCodingWorker {
     }
 
     private ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) {
-      return StripedBlockUtil.constructStripedBlock(blockGroup, cellSize,
+      return StripedBlockUtil.constructInternalBlock(blockGroup, cellSize,
           dataBlkNum, i);
     }
 
     private long getBlockLen(ExtendedBlock blockGroup, int i) { 
-      return StripedBlockUtil.getStripedBlockLength(blockGroup.getNumBytes(),
+      return StripedBlockUtil.getInternalBlockLength(blockGroup.getNumBytes(),
           cellSize, dataBlkNum, i);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7434c44b/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 81c0c95..2fa3fdf 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
@@ -41,7 +41,28 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 /**
- * Utility class for analyzing striped block groups
+ * When accessing a file in striped layout, operations on logical byte ranges
+ * in the file need to be mapped to physical byte ranges on block files stored
+ * on DataNodes. This utility class facilities this mapping by defining and
+ * exposing a number of striping-related concepts. The most basic ones are
+ * illustrated in the following diagram. Unless otherwise specified, all
+ * range-related calculations are inclusive (the end offset of the previous
+ * range should be 1 byte lower than the start offset of the next one).
+ *
+ *  | <----  Block Group ----> |   <- Block Group: logical unit composing
+ *  |                          |        striped HDFS files.
+ *  blk_0      blk_1       blk_2   <- Internal Blocks: each internal block
+ *    |          |           |          represents a physically stored local
+ *    v          v           v          block file
+ * +------+   +------+   +------+
+ * |cell_0|   |cell_1|   |cell_2|  <- {@link StripingCell} represents the
+ * +------+   +------+   +------+       logical order that a Block Group should
+ * |cell_3|   |cell_4|   |cell_5|       be accessed: cell_0, cell_1, ...
+ * +------+   +------+   +------+
+ * |cell_6|   |cell_7|   |cell_8|
+ * +------+   +------+   +------+
+ * |cell_9|
+ * +------+  <- A cell contains cellSize bytes of data
  */
 @InterfaceAudience.Private
 public class StripedBlockUtil {
@@ -103,31 +124,6 @@ public class StripedBlockUtil {
         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
@@ -157,7 +153,7 @@ public class StripedBlockUtil {
     return (numStripes - 1L)*cellSize
         + lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i);
   }
-  
+
   private static int lastCellSize(int size, int cellSize, int numDataBlocks,
       int i) {
     if (i < numDataBlocks) {
@@ -184,60 +180,6 @@ public class StripedBlockUtil {
   }
 
   /**
-   * 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].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].setStartOffsetInBlock(
-          cellSize * cellIdxInBlk);
-    }
-
-    int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len);
-    results[blkIdxInGroup].offsetsInBuf.add(bufOffset);
-    results[blkIdxInGroup].lengths.add(firstCellLen);
-    results[blkIdxInGroup].addReadLength(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.addReadLength(readLen);
-      i = (i + 1) % dataBlkNum;
-    }
-    return results;
-  }
-
-  /**
    * Get the next completed striped read task
    *
    * @return {@link StripingChunkReadResult} indicating the status of the read task
@@ -360,84 +302,167 @@ public class StripedBlockUtil {
   }
 
   /**
-   * This method divides a requested byte range into an array of
-   * {@link AlignedStripe}
-   *
+   * This method divides a requested byte range into an array of inclusive
+   * {@link AlignedStripe}.
+   * @param ecSchema The codec schema for the file, which carries the numbers
+   *                 of data / parity blocks, as well as cell size
+   * @param blockGroup The striped block group
+   * @param rangeStartInBlockGroup The byte range's start offset in block group
+   * @param rangeEndInBlockGroup The byte range's end offset in block group
+   * @param buf Destination buffer of the read operation for the byte range
+   * @param offsetInBuf Start offset into the destination buffer
    *
-   * At most 5 stripes will be generated from each logical range
-   * TODO: cleanup and get rid of planReadPortions
+   * At most 5 stripes will be generated from each logical range, as
+   * demonstrated in the header of {@link AlignedStripe}.
    */
   public static AlignedStripe[] divideByteRangeIntoStripes (
-      ECSchema ecSchema, LocatedStripedBlock blockGroup, long start, long end,
-      byte[] buf, int offsetInBuf) {
+      ECSchema ecSchema, LocatedStripedBlock blockGroup,
+      long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf,
+      int offsetInBuf) {
     // TODO: change ECSchema naming to use cell size instead of chunk size
 
     // Step 0: analyze range and calculate basic parameters
     int cellSize = ecSchema.getChunkSize();
     int dataBlkNum = ecSchema.getNumDataUnits();
-    int len = (int) (end - start + 1);
-    int firstCellIdxInBG = (int) (start / cellSize);
-    int lastCellIdxInBG = (int) (end / cellSize);
-    int firstCellSize = Math.min(cellSize - (int) (start % cellSize), len);
-    long firstCellOffsetInBlk = firstCellIdxInBG / dataBlkNum * cellSize +
-        start % cellSize;
-    int lastCellSize = lastCellIdxInBG == firstCellIdxInBG ?
-        firstCellSize : (int) (end % cellSize) + 1;
-
-    // Step 1: get the unmerged ranges on each internal block
-    // TODO: StripingCell should carry info on size and start offset (HDFS-8320)
-    VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema,
-        firstCellIdxInBG, lastCellIdxInBG, firstCellSize, firstCellOffsetInBlk,
-        lastCellSize);
-
-    // Step 2: merge into at most 5 stripes
+
+    // Step 1: map the byte range to StripingCells
+    StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, blockGroup,
+        rangeStartInBlockGroup, rangeEndInBlockGroup);
+
+    // Step 2: get the unmerged ranges on each internal block
+    VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cells);
+
+    // Step 3: merge into at most 5 stripes
     AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges);
 
-    // Step 3: calculate each chunk's position in destination buffer
-    calcualteChunkPositionsInBuf(ecSchema, blockGroup, buf, offsetInBuf,
-        firstCellIdxInBG, lastCellIdxInBG, firstCellSize, firstCellOffsetInBlk,
-        lastCellSize, stripes);
+    // Step 4: calculate each chunk's position in destination buffer
+    calcualteChunkPositionsInBuf(ecSchema, stripes, cells, buf, offsetInBuf);
 
-    // Step 4: prepare ALLZERO blocks
+    // Step 5: prepare ALLZERO blocks
     prepareAllZeroChunks(blockGroup, buf, stripes, cellSize, dataBlkNum);
 
     return stripes;
   }
 
-  private static VerticalRange[] getRangesForInternalBlocks (ECSchema ecSchema,
-      int firstCellIdxInBG, int lastCellIdxInBG, int firstCellSize,
-      long firstCellOffsetInBlk, int lastCellSize) {
+  /**
+   * Map the logical byte range to a set of inclusive {@link StripingCell}
+   * instances, each representing the overlap of the byte range to a cell
+   * used by {@link DFSStripedOutputStream} in encoding
+   */
+  @VisibleForTesting
+  private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema,
+      LocatedStripedBlock blockGroup,
+      long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
+    Preconditions.checkArgument(
+        rangeStartInBlockGroup <= rangeEndInBlockGroup &&
+            rangeEndInBlockGroup < blockGroup.getBlockSize());
     int cellSize = ecSchema.getChunkSize();
-    int dataBlkNum = ecSchema.getNumDataUnits();
+    int len = (int) (rangeEndInBlockGroup - rangeStartInBlockGroup + 1);
+    int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
+    int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize);
+    int numCells = lastCellIdxInBG - firstCellIdxInBG + 1;
+    StripingCell[] cells = new StripingCell[numCells];
+    cells[0] = new StripingCell(ecSchema, firstCellIdxInBG);
+    cells[numCells - 1] = new StripingCell(ecSchema, lastCellIdxInBG);
+
+    cells[0].offset = (int) (rangeStartInBlockGroup % cellSize);
+    cells[0].size =
+        Math.min(cellSize - (int) (rangeStartInBlockGroup % cellSize), len);
+    if (lastCellIdxInBG != firstCellIdxInBG) {
+      cells[numCells - 1].size = (int) (rangeEndInBlockGroup % cellSize) + 1;
+    }
+
+    for (int i = 1; i < numCells - 1; i++) {
+      cells[i] = new StripingCell(ecSchema, i + firstCellIdxInBG);
+    }
+
+    return cells;
+  }
 
+  /**
+   * Given a logical start offset in a block group, calculate the physical
+   * start offset into each stored internal block.
+   */
+  public static long[] getStartOffsetsForInternalBlocks(
+      ECSchema ecSchema, LocatedStripedBlock blockGroup,
+      long rangeStartInBlockGroup) {
+    Preconditions.checkArgument(
+        rangeStartInBlockGroup < blockGroup.getBlockSize());
+    int dataBlkNum = ecSchema.getNumDataUnits();
+    int parityBlkNum = ecSchema.getNumParityUnits();
+    int cellSize = ecSchema.getChunkSize();
+    long[] startOffsets = new long[dataBlkNum + parityBlkNum];
+    Arrays.fill(startOffsets, -1L);
+    int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
     StripingCell firstCell = new StripingCell(ecSchema, firstCellIdxInBG);
-    StripingCell lastCell = new StripingCell(ecSchema, lastCellIdxInBG);
+    firstCell.offset = (int) (rangeStartInBlockGroup % cellSize);
+    startOffsets[firstCell.idxInStripe] =
+        firstCell.idxInInternalBlk * cellSize + firstCell.offset;
+    long earliestStart = startOffsets[firstCell.idxInStripe];
+    for (int i = 1; i < dataBlkNum; i++) {
+      int idx = firstCellIdxInBG + i;
+      if (idx * cellSize >= blockGroup.getBlockSize()) {
+        break;
+      }
+      StripingCell cell = new StripingCell(ecSchema, idx);
+      startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * cellSize;
+      if (startOffsets[cell.idxInStripe] < earliestStart) {
+        earliestStart = startOffsets[cell.idxInStripe];
+      }
+    }
+    for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) {
+      startOffsets[i] = earliestStart;
+    }
+    return startOffsets;
+  }
 
-    VerticalRange ranges[] = new VerticalRange[dataBlkNum];
-    ranges[firstCell.idxInStripe] =
-        new VerticalRange(firstCellOffsetInBlk, firstCellSize);
-    for (int i = firstCellIdxInBG + 1; i < lastCellIdxInBG; i++) {
+  /**
+   * Given a logical byte range, mapped to each {@link StripingCell}, calculate
+   * the physical byte range (inclusive) on each stored internal block.
+   */
+  @VisibleForTesting
+  private static VerticalRange[] getRangesForInternalBlocks(ECSchema ecSchema,
+      StripingCell[] cells) {
+    int cellSize = ecSchema.getChunkSize();
+    int dataBlkNum = ecSchema.getNumDataUnits();
+    int parityBlkNum = ecSchema.getNumParityUnits();
+
+    VerticalRange ranges[] = new VerticalRange[dataBlkNum + parityBlkNum];
+
+    long earliestStart = Long.MAX_VALUE;
+    long latestEnd = -1;
+    for (StripingCell cell : cells) {
       // iterate through all cells and update the list of StripeRanges
-      StripingCell cell = new StripingCell(ecSchema, i);
       if (ranges[cell.idxInStripe] == null) {
         ranges[cell.idxInStripe] = new VerticalRange(
-            cell.idxInInternalBlk * cellSize, cellSize);
+            cell.idxInInternalBlk * cellSize + cell.offset, cell.size);
       } else {
-        ranges[cell.idxInStripe].spanInBlock += cellSize;
+        ranges[cell.idxInStripe].spanInBlock += cell.size;
+      }
+      VerticalRange range = ranges[cell.idxInStripe];
+      if (range.offsetInBlock < earliestStart) {
+        earliestStart = range.offsetInBlock;
+      }
+      if (range.offsetInBlock + range.spanInBlock - 1 > latestEnd) {
+        latestEnd = range.offsetInBlock + range.spanInBlock - 1;
       }
     }
-    if (ranges[lastCell.idxInStripe] == null) {
-      ranges[lastCell.idxInStripe] = new VerticalRange(
-          lastCell.idxInInternalBlk * cellSize, lastCellSize);
-    } else if (lastCell.idxInBlkGroup != firstCell.idxInBlkGroup) {
-      ranges[lastCell.idxInStripe].spanInBlock += lastCellSize;
+
+    // Each parity block should be fetched at maximum range of all data blocks
+    for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) {
+      ranges[i] = new VerticalRange(earliestStart,
+          latestEnd - earliestStart + 1);
     }
 
     return ranges;
   }
 
-  private static AlignedStripe[] mergeRangesForInternalBlocks(ECSchema ecSchema,
-      VerticalRange[] ranges) {
+  /**
+   * Merge byte ranges on each internal block into a set of inclusive
+   * {@link AlignedStripe} instances.
+   */
+  private static AlignedStripe[] mergeRangesForInternalBlocks(
+      ECSchema ecSchema, VerticalRange[] ranges) {
     int dataBlkNum = ecSchema.getNumDataUnits();
     int parityBlkNum = ecSchema.getNumParityUnits();
     List<AlignedStripe> stripes = new ArrayList<>();
@@ -461,12 +486,8 @@ public class StripedBlockUtil {
   }
 
   private static void calcualteChunkPositionsInBuf(ECSchema ecSchema,
-      LocatedStripedBlock blockGroup, byte[] buf, int offsetInBuf,
-      int firstCellIdxInBG, int lastCellIdxInBG, int firstCellSize,
-      long firstCellOffsetInBlk, int lastCellSize, AlignedStripe[] stripes) {
-    int cellSize = ecSchema.getChunkSize();
-    int dataBlkNum = ecSchema.getNumDataUnits();
-    // Step 3: calculate each chunk's position in destination buffer
+      AlignedStripe[] stripes, StripingCell[] cells, byte[] buf,
+      int offsetInBuf) {
     /**
      *     | <--------------- AlignedStripe --------------->|
      *
@@ -484,20 +505,11 @@ public class StripedBlockUtil {
      *
      * Cell indexing convention defined in {@link StripingCell}
      */
+    int cellSize = ecSchema.getChunkSize();
     int done = 0;
-    for (int i = firstCellIdxInBG; i <= lastCellIdxInBG; i++) {
-      StripingCell cell  = new StripingCell(ecSchema, i);
-      long cellStart = i == firstCellIdxInBG ?
-          firstCellOffsetInBlk : cell.idxInInternalBlk * cellSize;
-      int cellLen;
-      if (i == firstCellIdxInBG) {
-        cellLen = firstCellSize;
-      } else if (i == lastCellIdxInBG) {
-        cellLen = lastCellSize;
-      } else {
-        cellLen = cellSize;
-      }
-      long cellEnd = cellStart + cellLen - 1;
+    for (StripingCell cell : cells) {
+      long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
+      long cellEnd = cellStart + cell.size - 1;
       for (AlignedStripe s : stripes) {
         long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1;
         long overlapStart = Math.max(cellStart, s.getOffsetInBlock());
@@ -514,10 +526,14 @@ public class StripedBlockUtil {
             add((int)(offsetInBuf + done + overlapStart - cellStart));
         s.chunks[cell.idxInStripe].lengthsInBuf.add(overLapLen);
       }
-      done += cellLen;
+      done += cell.size;
     }
   }
 
+  /**
+   * If a {@link StripingChunk} maps to a byte range beyond an internal block's
+   * size, the chunk should be treated as zero bytes in decoding.
+   */
   private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup,
       byte[] buf, AlignedStripe[] stripes, int cellSize, int dataBlkNum) {
     for (AlignedStripe s : stripes) {
@@ -534,51 +550,13 @@ public class StripedBlockUtil {
   }
 
   /**
-   * This class represents the portion of I/O associated with each block in the
-   * striped block group.
-   * TODO: consolidate ReadPortion with AlignedStripe
-   */
-  public static class ReadPortion {
-    private long startOffsetInBlock = 0;
-    private 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 long getStartOffsetInBlock() {
-      return startOffsetInBlock;
-    }
-
-    public int getReadLength() {
-      return readLength;
-    }
-
-    public void setStartOffsetInBlock(long startOffsetInBlock) {
-      this.startOffsetInBlock = startOffsetInBlock;
-    }
-
-    void addReadLength(int extraLength) {
-      this.readLength += extraLength;
-    }
-  }
-
-  /**
-   * The unit of encoding used in {@link DFSStripedOutputStream}
+   * Cell is the unit of encoding used in {@link DFSStripedOutputStream}. This
+   * size impacts how a logical offset in the file or block group translates
+   * to physical byte offset in a stored internal block. The StripingCell util
+   * class facilitates this calculation. Each StripingCell is inclusive with
+   * its start and end offsets -- e.g., the end logical offset of cell_0_0_0
+   * should be 1 byte lower than the start logical offset of cell_1_0_1.
+   *
    *  | <------- Striped Block Group -------> |
    *    blk_0          blk_1          blk_2
    *      |              |              |
@@ -586,43 +564,57 @@ public class StripedBlockUtil {
    * +----------+   +----------+   +----------+
    * |cell_0_0_0|   |cell_1_0_1|   |cell_2_0_2|
    * +----------+   +----------+   +----------+
-   * |cell_3_1_0|   |cell_4_1_1|   |cell_5_1_2| <- {@link idxInBlkGroup} = 5
-   * +----------+   +----------+   +----------+    {@link idxInInternalBlk} = 1
-   *                                               {@link idxInStripe} = 2
+   * |cell_3_1_0|   |cell_4_1_1|   |cell_5_1_2| <- {@link #idxInBlkGroup} = 5
+   * +----------+   +----------+   +----------+    {@link #idxInInternalBlk} = 1
+   *                                               {@link #idxInStripe} = 2
    * A StripingCell is a special instance of {@link StripingChunk} whose offset
    * and size align with the cell used when writing data.
    * TODO: consider parity cells
    */
-  public static class StripingCell {
+  @VisibleForTesting
+  static class StripingCell {
     public final ECSchema schema;
     /** Logical order in a block group, used when doing I/O to a block group */
-    public final int idxInBlkGroup;
-    public final int idxInInternalBlk;
-    public final int idxInStripe;
+    final int idxInBlkGroup;
+    final int idxInInternalBlk;
+    final int idxInStripe;
+    /**
+     * When a logical byte range is mapped to a set of cells, it might
+     * partially overlap with the first and last cells. This field and the
+     * {@link #size} variable represent the start offset and size of the
+     * overlap.
+     */
+    int offset;
+    int size;
 
-    public StripingCell(ECSchema ecSchema, int idxInBlkGroup) {
+    StripingCell(ECSchema ecSchema, int idxInBlkGroup) {
       this.schema = ecSchema;
       this.idxInBlkGroup = idxInBlkGroup;
       this.idxInInternalBlk = idxInBlkGroup / ecSchema.getNumDataUnits();
       this.idxInStripe = idxInBlkGroup -
           this.idxInInternalBlk * ecSchema.getNumDataUnits();
+      this.offset = 0;
+      this.size = ecSchema.getChunkSize();
     }
 
-    public StripingCell(ECSchema ecSchema, int idxInInternalBlk,
+    StripingCell(ECSchema ecSchema, int idxInInternalBlk,
         int idxInStripe) {
       this.schema = ecSchema;
       this.idxInInternalBlk = idxInInternalBlk;
       this.idxInStripe = idxInStripe;
       this.idxInBlkGroup =
           idxInInternalBlk * ecSchema.getNumDataUnits() + idxInStripe;
+      this.offset = 0;
+      this.size = ecSchema.getChunkSize();
     }
   }
 
   /**
    * Given a requested byte range on a striped block group, an AlignedStripe
-   * represents a {@link VerticalRange} that is aligned with both the byte range
-   * and boundaries of all internal blocks. As illustrated in the diagram, any
-   * given byte range on a block group leads to 1~5 AlignedStripe's.
+   * represents an inclusive {@link VerticalRange} that is aligned with both
+   * the byte range and boundaries of all internal blocks. As illustrated in
+   * the diagram, any given byte range on a block group leads to 1~5
+   * AlignedStripe's.
    *
    * |<-------- Striped Block Group -------->|
    * blk_0   blk_1   blk_2      blk_3   blk_4
@@ -648,6 +640,7 @@ public class StripedBlockUtil {
    *
    * The coverage of an AlignedStripe on an internal block is represented as a
    * {@link StripingChunk}.
+   *
    * To simplify the logic of reading a logical byte range from a block group,
    * a StripingChunk is either completely in the requested byte range or
    * completely outside the requested byte range.
@@ -692,19 +685,19 @@ public class StripedBlockUtil {
 
   /**
    * A simple utility class representing an arbitrary vertical inclusive range
-   * starting at {@link offsetInBlock} and lasting for {@link length} bytes in
-   * an internal block. Note that VerticalRange doesn't necessarily align with
-   * {@link StripingCell}.
+   * starting at {@link #offsetInBlock} and lasting for {@link #spanInBlock}
+   * bytes in an internal block. Note that VerticalRange doesn't necessarily
+   * align with {@link StripingCell}.
    *
    * |<- Striped Block Group ->|
    *  blk_0
    *    |
    *    v
    * +-----+
-   * |~~~~~| <-- {@link offsetInBlock}
+   * |~~~~~| <-- {@link #offsetInBlock}
    * |     |  ^
    * |     |  |
-   * |     |  | {@link spanInBlock}
+   * |     |  | {@link #spanInBlock}
    * |     |  v
    * |~~~~~| ---
    * |     |
@@ -743,9 +736,9 @@ public class StripedBlockUtil {
    * +---------+  +---------+               |  +----+  +----+
    * <----------- data blocks ------------> | <--- parity --->
    *
-   * The class also carries {@link buf}, {@link offsetsInBuf}, and
-   * {@link lengthsInBuf} to define how read task for this chunk should deliver
-   * the returned data.
+   * The class also carries {@link #buf}, {@link #offsetsInBuf}, and
+   * {@link #lengthsInBuf} to define how read task for this chunk should
+   * deliver the returned data.
    */
   public static class StripingChunk {
     /** Chunk has been successfully fetched */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7434c44b/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
deleted file mode 100644
index 75d0587..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import org.junit.Test;
-
-import org.apache.hadoop.hdfs.util.StripedBlockUtil;
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion;
-import static org.junit.Assert.*;
-
-public class TestPlanReadPortions {
-
-  // We only support this as num of data blocks. It might be good enough for now
-  // for the purpose, even not flexible yet for any number in a schema.
-  private final short GROUP_SIZE = 3;
-  private final int CELLSIZE = 128 * 1024;
-
-  private void testPlanReadPortions(int startInBlk, int length,
-      int bufferOffset, int[] readLengths, int[] offsetsInBlock,
-      int[][] bufferOffsets, int[][] bufferLengths) {
-    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());
-      final int[] bOffsets = results[i].getOffsets();
-      assertArrayEquals(bufferOffsets[i], bOffsets);
-      final int[] bLengths = results[i].getLengths();
-      assertArrayEquals(bufferLengths[i], bLengths);
-    }
-  }
-
-  /**
-   * Test {@link StripedBlockUtil#planReadPortions}
-   */
-  @Test
-  public void testPlanReadPortions() {
-    /**
-     * start block offset is 0, read cellSize - 10
-     */
-    testPlanReadPortions(0, CELLSIZE - 10, 0,
-        new int[]{CELLSIZE - 10, 0, 0}, new int[]{0, 0, 0},
-        new int[][]{new int[]{0}, new int[]{}, new int[]{}},
-        new int[][]{new int[]{CELLSIZE - 10}, new int[]{}, new int[]{}});
-
-    /**
-     * start block offset is 0, read 3 * cellSize
-     */
-    testPlanReadPortions(0, GROUP_SIZE * CELLSIZE, 0,
-        new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, new int[]{0, 0, 0},
-        new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{CELLSIZE * 2}},
-        new int[][]{new int[]{CELLSIZE}, new int[]{CELLSIZE}, new int[]{CELLSIZE}});
-
-    /**
-     * start block offset is 0, read cellSize + 10
-     */
-    testPlanReadPortions(0, CELLSIZE + 10, 0,
-        new int[]{CELLSIZE, 10, 0}, new int[]{0, 0, 0},
-        new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{}},
-        new int[][]{new int[]{CELLSIZE}, new int[]{10}, new int[]{}});
-
-    /**
-     * start block offset is 0, read 5 * cellSize + 10, buffer start offset is 100
-     */
-    testPlanReadPortions(0, 5 * CELLSIZE + 10, 100,
-        new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE + 10}, new int[]{0, 0, 0},
-        new int[][]{new int[]{100, 100 + CELLSIZE * GROUP_SIZE},
-            new int[]{100 + CELLSIZE, 100 + CELLSIZE * 4},
-            new int[]{100 + CELLSIZE * 2, 100 + CELLSIZE * 5}},
-        new int[][]{new int[]{CELLSIZE, CELLSIZE},
-            new int[]{CELLSIZE, CELLSIZE},
-            new int[]{CELLSIZE, 10}});
-
-    /**
-     * start block offset is 2, read 3 * cellSize
-     */
-    testPlanReadPortions(2, GROUP_SIZE * CELLSIZE, 100,
-        new int[]{CELLSIZE, CELLSIZE, CELLSIZE},
-        new int[]{2, 0, 0},
-        new int[][]{new int[]{100, 100 + GROUP_SIZE * CELLSIZE - 2},
-            new int[]{100 + CELLSIZE - 2},
-            new int[]{100 + CELLSIZE * 2 - 2}},
-        new int[][]{new int[]{CELLSIZE - 2, 2},
-            new int[]{CELLSIZE},
-            new int[]{CELLSIZE}});
-
-    /**
-     * start block offset is 2, read 3 * cellSize + 10
-     */
-    testPlanReadPortions(2, GROUP_SIZE * CELLSIZE + 10, 0,
-        new int[]{CELLSIZE + 10, CELLSIZE, CELLSIZE},
-        new int[]{2, 0, 0},
-        new int[][]{new int[]{0, GROUP_SIZE * CELLSIZE - 2},
-            new int[]{CELLSIZE - 2},
-            new int[]{CELLSIZE * 2 - 2}},
-        new int[][]{new int[]{CELLSIZE - 2, 12},
-            new int[]{CELLSIZE},
-            new int[]{CELLSIZE}});
-
-    /**
-     * start block offset is cellSize * 2 - 1, read 5 * cellSize + 10
-     */
-    testPlanReadPortions(CELLSIZE * 2 - 1, 5 * CELLSIZE + 10, 0,
-        new int[]{CELLSIZE * 2, CELLSIZE + 10, CELLSIZE * 2},
-        new int[]{CELLSIZE, CELLSIZE - 1, 0},
-        new int[][]{new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1},
-            new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1},
-            new int[]{1, 3 * CELLSIZE + 1}},
-        new int[][]{new int[]{CELLSIZE, CELLSIZE},
-            new int[]{1, CELLSIZE, 9},
-            new int[]{CELLSIZE, CELLSIZE}});
-
-    /**
-     * start block offset is cellSize * 6 - 1, read 7 * cellSize + 10
-     */
-    testPlanReadPortions(CELLSIZE * 6 - 1, 7 * CELLSIZE + 10, 0,
-        new int[]{CELLSIZE * 3, CELLSIZE * 2 + 9, CELLSIZE * 2 + 1},
-        new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE * 2 - 1},
-        new int[][]{new int[]{1, 3 * CELLSIZE + 1, 6 * CELLSIZE + 1},
-            new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1, 7 * CELLSIZE + 1},
-            new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}},
-        new int[][]{new int[]{CELLSIZE, CELLSIZE, CELLSIZE},
-            new int[]{CELLSIZE, CELLSIZE, 9},
-            new int[]{1, CELLSIZE, CELLSIZE}});
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7434c44b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
index b4f05d4..dfdcee2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
@@ -189,13 +189,13 @@ public class TestRecoverStripedFile {
       deadDnIndices[i] = dnMap.get(dataDNs[i]);
       
       // Check the block replica file on deadDn before it dead.
-      blocks[i] = StripedBlockUtil.constructStripedBlock(
+      blocks[i] = StripedBlockUtil.constructInternalBlock(
           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(
+          StripedBlockUtil.getInternalBlockLength(
           lastBlock.getBlockSize(), cellSize, dataBlkNum, indices[toDead[i]]));
       assertTrue(metadatas[i].getName().
           endsWith(blocks[i].getGenerationStamp() + ".meta"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7434c44b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
index ec0b1bb..6f29d69 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs.util;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -26,26 +27,107 @@ 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 static org.apache.hadoop.hdfs.util.StripedBlockUtil.*;
+
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.junit.Before;
 import org.junit.Test;
 
+import java.util.Random;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
+/**
+ * Need to cover the following combinations:
+ * 1. Block group size:
+ *  1.1 One byte
+ *  1.2 Smaller than cell
+ *  1.3 One full cell
+ *  1.4 x full cells, where x is smaller than number of data blocks
+ *  1.5 x full cells plus a partial cell
+ *  1.6 One full stripe
+ *  1.7 One full stripe plus a partial cell
+ *  1.8 One full stripe plus x full cells
+ *  1.9 One full stripe plus x full cells plus a partial cell
+ *  1.10 y full stripes, but smaller than full block group size
+ *  1.11 Full block group size
+ *
+ * 2. Byte range start
+ *  2.1 Zero
+ *  2.2 Within first cell
+ *  2.3 End of first cell
+ *  2.4 Start of a middle* cell in the first stripe (* neither first or last)
+ *  2.5 End of middle cell in the first stripe
+ *  2.6 Within a middle cell in the first stripe
+ *  2.7 Start of the last cell in the first stripe
+ *  2.8 Within the last cell in the first stripe
+ *  2.9 End of the last cell in the first stripe
+ *  2.10 Start of a middle stripe
+ *  2.11 Within a middle stripe
+ *  2.12 End of a middle stripe
+ *  2.13 Start of the last stripe
+ *  2.14 Within the last stripe
+ *  2.15 End of the last stripe (last byte)
+ *
+ * 3. Byte range length: same settings as block group size
+ *
+ * We should test in total 11 x 15 x 11 = 1815 combinations
+ * TODO: test parity block logic
+ */
 public class TestStripedBlockUtil {
   private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
   private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
-  private final short BLK_GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM;
+  private final short BLK_GROUP_WIDTH = DATA_BLK_NUM + PARITY_BLK_NUM;
   private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final int FULL_STRIPE_SIZE = DATA_BLK_NUM * CELLSIZE;
+  /** number of full stripes in a full block group */
+  private final int BLK_GROUP_STRIPE_NUM = 16;
+  private final ECSchema SCEHMA = ErasureCodingSchemaManager.
+      getSystemDefaultSchema();
+  private final Random random = new Random();
+
+  private int[] blockGroupSizes;
+  private int[] byteRangeStartOffsets;
+  private int[] byteRangeSizes;
+
+  @Before
+  public void setup(){
+    blockGroupSizes = new int[]{1, getDelta(CELLSIZE), CELLSIZE,
+        getDelta(DATA_BLK_NUM) * CELLSIZE,
+        getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE),
+        FULL_STRIPE_SIZE, FULL_STRIPE_SIZE + getDelta(CELLSIZE),
+        FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE,
+        FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE),
+        getDelta(BLK_GROUP_STRIPE_NUM) * FULL_STRIPE_SIZE,
+        BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE};
+    byteRangeStartOffsets = new int[] {0, getDelta(CELLSIZE), CELLSIZE - 1};
+    byteRangeSizes = new int[]{1, getDelta(CELLSIZE), CELLSIZE,
+        getDelta(DATA_BLK_NUM) * CELLSIZE,
+        getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE),
+        FULL_STRIPE_SIZE, FULL_STRIPE_SIZE + getDelta(CELLSIZE),
+        FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE,
+        FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE),
+        getDelta(BLK_GROUP_STRIPE_NUM) * FULL_STRIPE_SIZE,
+        BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE};
+  }
 
-  private LocatedStripedBlock createDummyLocatedBlock() {
+  private int getDelta(int size) {
+    return 1 + random.nextInt(size - 2);
+  }
+  private byte hashIntToByte(int i) {
+    int BYTE_MASK = 0xff;
+    return (byte) (((i + 13) * 29) & BYTE_MASK);
+  }
+
+  private LocatedStripedBlock createDummyLocatedBlock(int bgSize) {
     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++) {
+    DatanodeInfo[] locs = new DatanodeInfo[BLK_GROUP_WIDTH];
+    String[] storageIDs = new String[BLK_GROUP_WIDTH];
+    StorageType[] storageTypes = new StorageType[BLK_GROUP_WIDTH];
+    int[] indices = new int[BLK_GROUP_WIDTH];
+    for (int i = 0; i < BLK_GROUP_WIDTH; i++) {
       indices[i] = (i + 2) % DATA_BLK_NUM;
       // Location port always equal to logical index of a block,
       // for easier verification
@@ -53,13 +135,40 @@ public class TestStripedBlockUtil {
       storageIDs[i] = locs[i].getDatanodeUuid();
       storageTypes[i] = StorageType.DISK;
     }
-    return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID),
-        locs, storageIDs, storageTypes, indices, 0, false, null);
+    return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID,
+        bgSize, 1001), locs, storageIDs, storageTypes, indices, 0, false,
+        null);
+  }
+
+  private byte[][] createInternalBlkBuffers(int bgSize) {
+    byte[][] bufs = new byte[DATA_BLK_NUM + PARITY_BLK_NUM][];
+    int[] pos = new int[DATA_BLK_NUM + PARITY_BLK_NUM];
+    for (int i = 0; i < DATA_BLK_NUM + PARITY_BLK_NUM; i++) {
+      int bufSize = (int) getInternalBlockLength(
+          bgSize, CELLSIZE, DATA_BLK_NUM, i);
+      bufs[i] = new byte[bufSize];
+      pos[i] = 0;
+    }
+    int done = 0;
+    while (done < bgSize) {
+      Preconditions.checkState(done % CELLSIZE == 0);
+      StripingCell cell = new StripingCell(SCEHMA, done / CELLSIZE);
+      int idxInStripe = cell.idxInStripe;
+      int size = Math.min(CELLSIZE, bgSize - done);
+      for (int i = 0; i < size; i++) {
+        bufs[idxInStripe][pos[idxInStripe] + i] = hashIntToByte(done + i);
+      }
+      done += size;
+      pos[idxInStripe] += size;
+    }
+
+    return bufs;
   }
 
   @Test
   public void testParseDummyStripedBlock() {
-    LocatedStripedBlock lsb = createDummyLocatedBlock();
+    LocatedStripedBlock lsb = createDummyLocatedBlock(
+        BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE);
     LocatedBlock[] blocks = parseStripedBlockGroup(
         lsb, CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM);
     assertEquals(DATA_BLK_NUM + PARITY_BLK_NUM, blocks.length);
@@ -68,14 +177,15 @@ public class TestStripedBlockUtil {
       assertEquals(i,
           BlockIdManager.getBlockIndex(blocks[i].getBlock().getLocalBlock()));
       assertEquals(i * CELLSIZE, blocks[i].getStartOffset());
+      /** TODO: properly define {@link LocatedBlock#offset} for internal blocks */
       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++) {
+  private void verifyInternalBlocks (int numBytesInGroup, int[] expected) {
+    for (int i = 1; i < BLK_GROUP_WIDTH; i++) {
       assertEquals(expected[i],
           getInternalBlockLength(numBytesInGroup, CELLSIZE, DATA_BLK_NUM, i));
     }
@@ -85,41 +195,85 @@ public class TestStripedBlockUtil {
   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,
+        new int[] {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,
+        new int[] {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,
+        new int[] {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,
+        new int[] {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,
+        new int[] {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,
+        new int[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
             2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE - delta,
             2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE});
   }
 
+  /**
+   * Test dividing a byte range into aligned stripes and verify the aligned
+   * ranges can be translated back to the byte range.
+   */
+  @Test
+  public void testDivideByteRangeIntoStripes() {
+    byte[] assembled = new byte[BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE];
+    for (int bgSize : blockGroupSizes) {
+      LocatedStripedBlock blockGroup = createDummyLocatedBlock(bgSize);
+      byte[][] internalBlkBufs = createInternalBlkBuffers(bgSize);
+      for (int brStart : byteRangeStartOffsets) {
+        for (int brSize : byteRangeSizes) {
+          if (brStart + brSize > bgSize) {
+            continue;
+          }
+          AlignedStripe[] stripes = divideByteRangeIntoStripes(SCEHMA,
+              blockGroup, brStart, brStart + brSize - 1, assembled, 0);
+
+          for (AlignedStripe stripe : stripes) {
+            for (int i = 0; i < DATA_BLK_NUM; i++) {
+              StripingChunk chunk = stripe.chunks[i];
+              if (chunk == null || chunk.state != StripingChunk.REQUESTED) {
+                continue;
+              }
+              int done = 0;
+              for (int j = 0; j < chunk.getLengths().length; j++) {
+                System.arraycopy(internalBlkBufs[i],
+                    (int) stripe.getOffsetInBlock() + done, assembled,
+                    chunk.getOffsets()[j], chunk.getLengths()[j]);
+                done += chunk.getLengths()[j];
+              }
+            }
+          }
+          for (int i = 0; i < brSize; i++) {
+            if (hashIntToByte(brStart + i) != assembled[i]) {
+              System.out.println("Oops");
+            }
+            assertEquals("Byte at " + (brStart + i) + " should be the same",
+                hashIntToByte(brStart + i), assembled[i]);
+          }
+        }
+      }
+    }
+  }
+
 }


[36/50] [abbrv] hadoop git commit: HDFS-8378. Erasure Coding: Few improvements for the erasure coding worker. Contributed by Rakesh R.

Posted by zh...@apache.org.
HDFS-8378. Erasure Coding: Few improvements for the erasure coding worker. 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/91458093
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/91458093
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/91458093

Branch: refs/heads/HDFS-7285
Commit: 914580934c566cd18019035b244f82006868bd7b
Parents: 8c95673
Author: Walter Su <wa...@apache.org>
Authored: Tue May 19 14:59:23 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:33 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  5 ++
 .../hdfs/server/datanode/BPOfferService.java    |  1 +
 .../erasurecode/ErasureCodingWorker.java        | 59 ++++++++++----------
 3 files changed, 35 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/91458093/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 939ba89..1e7dbea 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -218,6 +218,8 @@
 
     HDFS-8367. BlockInfoStriped uses EC schema. (Kai Sasaki via Kai Zheng)
 
+    HDFS-8352. Erasure Coding: test webhdfs read write stripe file. (waltersu4549)
+
     HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe.
     (Walter Su via jing9)
 
@@ -228,3 +230,6 @@
 	
 	HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue 
 	configurable in DFSStripedOutputStream. (Li Bo)
+
+    HDFS-8378. Erasure Coding: Few improvements for the erasure coding worker.
+    (Rakesh R via waltersu4549)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91458093/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 6606d0b..d77b36d 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
@@ -728,6 +728,7 @@ class BPOfferService {
       LOG.info("DatanodeCommand action: DNA_ERASURE_CODING_RECOVERY");
       Collection<BlockECRecoveryInfo> ecTasks = ((BlockECRecoveryCommand) cmd).getECTasks();
       dn.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
+      break;
     default:
       LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91458093/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 a1c0f72..4723e9f 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
@@ -62,7 +62,6 @@ 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;
@@ -88,12 +87,12 @@ import com.google.common.base.Preconditions;
  * commands.
  */
 public final class ErasureCodingWorker {
-  private final Log LOG = DataNode.LOG;
+  private static final Log LOG = DataNode.LOG;
   
   private final DataNode datanode; 
-  private Configuration conf;
+  private final Configuration conf;
 
-  private ThreadPoolExecutor STRIPED_READ_TRHEAD_POOL;
+  private ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
   private final int STRIPED_READ_THRESHOLD_MILLIS;
   private final int STRIPED_READ_BUFFER_SIZE;
 
@@ -121,7 +120,10 @@ public final class ErasureCodingWorker {
   }
 
   private void initializeStripedReadThreadPool(int num) {
-    STRIPED_READ_TRHEAD_POOL = new ThreadPoolExecutor(1, num, 60,
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using striped reads; pool threads=" + num);
+    }
+    STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
         TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
         new Daemon.DaemonFactory() {
       private final AtomicInteger threadIndex = new AtomicInteger(0);
@@ -141,7 +143,7 @@ public final class ErasureCodingWorker {
         super.rejectedExecution(runnable, e);
       }
     });
-    STRIPED_READ_TRHEAD_POOL.allowCoreThreadTimeOut(true);
+    STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
   }
 
   /**
@@ -231,23 +233,23 @@ public final class ErasureCodingWorker {
 
     // sources
     private final short[] liveIndices;
-    private DatanodeInfo[] sources;
+    private final DatanodeInfo[] sources;
 
-    private List<StripedReader> stripedReaders;
+    private final List<StripedReader> stripedReaders;
 
     // targets
-    private DatanodeInfo[] targets;
-    private StorageType[] targetStorageTypes;
+    private final DatanodeInfo[] targets;
+    private final StorageType[] targetStorageTypes;
 
-    private short[] targetIndices;
-    private ByteBuffer[] targetBuffers;
+    private final short[] targetIndices;
+    private final ByteBuffer[] targetBuffers;
 
-    private Socket[] targetSockets;
-    private DataOutputStream[] targetOutputStreams;
-    private DataInputStream[] targetInputStreams;
+    private final Socket[] targetSockets;
+    private final DataOutputStream[] targetOutputStreams;
+    private final DataInputStream[] targetInputStreams;
 
-    private long[] blockOffset4Targets;
-    private long[] seqNo4Targets;
+    private final long[] blockOffset4Targets;
+    private final long[] seqNo4Targets;
 
     private final int WRITE_PACKET_SIZE = 64 * 1024;
     private DataChecksum checksum;
@@ -257,11 +259,11 @@ public final class ErasureCodingWorker {
     private int bytesPerChecksum;
     private int checksumSize;
 
-    private CachingStrategy cachingStrategy;
+    private final CachingStrategy cachingStrategy;
 
-    private Map<Future<Void>, Integer> futures = new HashMap<>();
-    private CompletionService<Void> readService =
-        new ExecutorCompletionService<>(STRIPED_READ_TRHEAD_POOL);
+    private final Map<Future<Void>, Integer> futures = new HashMap<>();
+    private final CompletionService<Void> readService =
+        new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL);
 
     ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) {
       ECSchema schema = recoveryInfo.getECSchema();
@@ -277,7 +279,8 @@ public final class ErasureCodingWorker {
 
       Preconditions.checkArgument(liveIndices.length >= dataBlkNum,
           "No enough live striped blocks.");
-      Preconditions.checkArgument(liveIndices.length == sources.length);
+      Preconditions.checkArgument(liveIndices.length == sources.length,
+          "liveBlockIndices and source dns should match");
 
       targets = recoveryInfo.getTargetDnInfos();
       targetStorageTypes = recoveryInfo.getTargetStorageTypes();
@@ -336,7 +339,6 @@ public final class ErasureCodingWorker {
         if (nsuccess < dataBlkNum) {
           String error = "Can't find minimum sources required by "
               + "recovery, block id: " + blockGroup.getBlockId();
-          LOG.warn(error);
           throw new IOException(error);
         }
 
@@ -358,7 +360,6 @@ public final class ErasureCodingWorker {
         boolean[] targetsStatus = new boolean[targets.length];
         if (initTargetStreams(targetsStatus) == 0) {
           String error = "All targets are failed.";
-          LOG.warn(error);
           throw new IOException(error);
         }
 
@@ -372,7 +373,6 @@ public final class ErasureCodingWorker {
           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);
           }
 
@@ -385,7 +385,6 @@ public final class ErasureCodingWorker {
           // step3: transfer data
           if (transferData2Targets(targetsStatus) == 0) {
             String error = "Transfer failed for all targets.";
-            LOG.warn(error);
             throw new IOException(error);
           }
 
@@ -906,11 +905,11 @@ public final class ErasureCodingWorker {
   }
 
   private class StripedReader {
-    short index;
-    BlockReader blockReader;
-    ByteBuffer buffer;
+    private final short index;
+    private BlockReader blockReader;
+    private ByteBuffer buffer;
 
-    public StripedReader(short index) {
+    private StripedReader(short index) {
       this.index = index;
     }
   }


[34/50] [abbrv] hadoop git commit: HDFS-8375. Add cellSize as an XAttr to ECZone. Contributed by Vinayakumar B.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
index 86fcb88..9c585a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
@@ -66,7 +66,7 @@ public class TestQuotaWithStripedBlocks {
     dfs = cluster.getFileSystem();
 
     dfs.mkdirs(ecDir);
-    dfs.getClient().createErasureCodingZone(ecDir.toString(), ecSchema);
+    dfs.getClient().createErasureCodingZone(ecDir.toString(), ecSchema, 0);
     dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA);
     dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA);
     dfs.setStoragePolicy(ecDir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME);

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
index 6f29d69..5a1c3fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
@@ -152,7 +152,7 @@ public class TestStripedBlockUtil {
     int done = 0;
     while (done < bgSize) {
       Preconditions.checkState(done % CELLSIZE == 0);
-      StripingCell cell = new StripingCell(SCEHMA, done / CELLSIZE);
+      StripingCell cell = new StripingCell(SCEHMA, CELLSIZE, done / CELLSIZE);
       int idxInStripe = cell.idxInStripe;
       int size = Math.min(CELLSIZE, bgSize - done);
       for (int i = 0; i < size; i++) {
@@ -247,7 +247,7 @@ public class TestStripedBlockUtil {
             continue;
           }
           AlignedStripe[] stripes = divideByteRangeIntoStripes(SCEHMA,
-              blockGroup, brStart, brStart + brSize - 1, assembled, 0);
+              CELLSIZE, blockGroup, brStart, brStart + brSize - 1, assembled, 0);
 
           for (AlignedStripe stripe : stripes) {
             for (int i = 0; i < DATA_BLK_NUM; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 8947c5b..303d063 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, null);
+        HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null, 0);
     final FileStatus fstatus = toFileStatus(status, parent);
     System.out.println("status  = " + status);
     System.out.println("fstatus = " + fstatus);


[31/50] [abbrv] hadoop git commit: HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue configurable in DFSStripedOutputStream. Contributed by Li Bo

Posted by zh...@apache.org.
HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue configurable in DFSStripedOutputStream. Contributed by Li Bo


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

Branch: refs/heads/HDFS-7285
Commit: 8c95673db47763650c4bb9f9bf39ef770aa73cdf
Parents: 7434c44
Author: boli2 <bo.b.li@intel.com (file://bo.b.li@intel.com/)>
Authored: Tue May 19 02:14:46 2015 -0400
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:32 2015 -0700

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       | 10 ++++++++
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +++
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 19 ++++++++++++----
 .../hadoop/hdfs/client/impl/DfsClientConf.java  | 24 ++++++++++++++++++++
 4 files changed, 51 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c95673d/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 6006d71..9373e98 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
@@ -189,6 +189,16 @@ public interface HdfsClientConfigKeys {
     int     THREADPOOL_SIZE_DEFAULT = 18;
   }
 
+  /** dfs.client.write.striped configuration properties */
+  interface StripedWrite {
+    String PREFIX = Write.PREFIX + "striped.";
+
+    String  MAX_SECONDS_GET_STRIPED_BLOCK_KEY = PREFIX + "max-seconds-get-striped-block";
+    int     MAX_SECONDS_GET_STRIPED_BLOCK_DEFAULT = 90;
+    String  MAX_SECONDS_GET_ENDED_BLOCK_KEY = PREFIX + "max-seconds-get-ended-block";
+    int     MAX_SECONDS_GET_ENDED_BLOCK_DEFAULT = 60;
+  }
+
   /** dfs.http.client configuration properties */
   interface HttpClient {
     String  PREFIX = "dfs.http.client.";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c95673d/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
old mode 100644
new mode 100755
index 3170e9b..939ba89
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -225,3 +225,6 @@
     (Yi Liu via jing9)
 
     HDFS-8320. Erasure coding: consolidate striping-related terminologies. (zhz)
+	
+	HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue 
+	configurable in DFSStripedOutputStream. (Li Bo)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c95673d/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 b99afab..a648023 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
@@ -33,6 +33,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -61,11 +63,14 @@ import com.google.common.base.Preconditions;
 public class DFSStripedOutputStream extends DFSOutputStream {
   /** Coordinate the communication between the streamers. */
   static class Coordinator {
+    private final DfsClientConf conf;
     private final List<BlockingQueue<ExtendedBlock>> endBlocks;
     private final List<BlockingQueue<LocatedBlock>> stripedBlocks;
     private volatile boolean shouldLocateFollowingBlock = false;
 
-    Coordinator(final int numDataBlocks, final int numAllBlocks) {
+    Coordinator(final DfsClientConf conf, final int numDataBlocks,
+                final int numAllBlocks) {
+      this.conf = conf;
       endBlocks = new ArrayList<>(numDataBlocks);
       for (int i = 0; i < numDataBlocks; i++) {
         endBlocks.add(new LinkedBlockingQueue<ExtendedBlock>(1));
@@ -91,7 +96,9 @@ public class DFSStripedOutputStream extends DFSOutputStream {
 
     ExtendedBlock getEndBlock(int i) throws InterruptedIOException {
       try {
-        return endBlocks.get(i).poll(30, TimeUnit.SECONDS);
+        return endBlocks.get(i).poll(
+            conf.getStripedWriteMaxSecondsGetEndedBlock(),
+            TimeUnit.SECONDS);
       } catch (InterruptedException e) {
         throw DFSUtil.toInterruptedIOException(
             "getEndBlock interrupted, i=" + i, e);
@@ -121,7 +128,9 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     LocatedBlock getStripedBlock(int i) throws IOException {
       final LocatedBlock lb;
       try {
-        lb = stripedBlocks.get(i).poll(90, TimeUnit.SECONDS);
+        lb = stripedBlocks.get(i).poll(
+            conf.getStripedWriteMaxSecondsGetStripedBlock(),
+            TimeUnit.SECONDS);
       } catch (InterruptedException e) {
         throw DFSUtil.toInterruptedIOException("getStripedBlock interrupted", e);
       }
@@ -133,7 +142,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     }
   }
 
-  /** Buffers for writing the data and parity cells of a strip. */
+  /** Buffers for writing the data and parity cells of a stripe. */
   class CellBuffers {
     private final ByteBuffer[] buffers;
     private final byte[][] checksumArrays;
@@ -228,7 +237,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     encoder = new RSRawEncoder();
     encoder.initialize(numDataBlocks, numParityBlocks, cellSize);
 
-    coordinator = new Coordinator(numDataBlocks, numAllBlocks);
+    coordinator = new Coordinator(dfsClient.getConf(), numDataBlocks, numAllBlocks);
     try {
       cellBuffers = new CellBuffers(numParityBlocks);
     } catch (InterruptedException ie) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c95673d/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 32a3da0..34ec06d 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
@@ -103,6 +103,9 @@ public class DfsClientConf {
   private final int hedgedReadThreadpoolSize;
 
   private final int stripedReadThreadpoolSize;
+  private final int stripedWriteMaxSecondsGetStripedBlock;
+  private final int stripedWriteMaxSecondsGetEndedBlock;
+
 
   public DfsClientConf(Configuration conf) {
     // The hdfsTimeout is currently the same as the ipc timeout 
@@ -225,6 +228,13 @@ public class DfsClientConf {
     Preconditions.checkArgument(stripedReadThreadpoolSize > 0, "The value of " +
         HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY +
         " must be greater than 0.");
+
+    stripedWriteMaxSecondsGetStripedBlock = conf.getInt(
+        HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_STRIPED_BLOCK_KEY,
+        HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_STRIPED_BLOCK_DEFAULT);
+    stripedWriteMaxSecondsGetEndedBlock = conf.getInt(
+        HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_ENDED_BLOCK_KEY,
+        HdfsClientConfigKeys.StripedWrite.MAX_SECONDS_GET_ENDED_BLOCK_DEFAULT);
   }
 
   private DataChecksum.Type getChecksumType(Configuration conf) {
@@ -509,6 +519,20 @@ public class DfsClientConf {
   }
 
   /**
+   * @return stripedWriteMaxSecondsGetStripedBlock
+   */
+  public int getStripedWriteMaxSecondsGetStripedBlock() {
+    return stripedWriteMaxSecondsGetStripedBlock;
+  }
+
+  /**
+   * @return stripedWriteMaxSecondsGetEndedBlock
+   */
+  public int getStripedWriteMaxSecondsGetEndedBlock() {
+    return stripedWriteMaxSecondsGetEndedBlock;
+  }
+
+  /**
    * @return the shortCircuitConf
    */
   public ShortCircuitConf getShortCircuitConf() {


[50/50] [abbrv] hadoop git commit: Addendum fix for HDFS-7912.

Posted by zh...@apache.org.
Addendum fix for HDFS-7912.


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

Branch: refs/heads/HDFS-7285
Commit: c9e0268216584f1df1a7c6cd25d2cfb2bc6d1d3c
Parents: 4ad4848
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Tue May 26 12:32:16 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:32:16 2015 -0700

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/BlockManager.java |  2 +-
 .../hadoop/hdfs/TestRecoverStripedFile.java       |  3 ++-
 .../blockmanagement/TestPendingReplication.java   |  5 +++--
 .../blockmanagement/TestReplicationPolicy.java    | 18 ++++--------------
 4 files changed, 10 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e02682/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 8f1f6b7..32757f9 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
@@ -1819,7 +1819,7 @@ public class BlockManager {
            * Use the blockinfo from the blocksmap to be certain we're working
            * with the most up-to-date block information (e.g. genstamp).
            */
-          BlockInfoContiguous bi = blocksMap.getStoredBlock(timedOutItems[i]);
+          BlockInfo bi = blocksMap.getStoredBlock(timedOutItems[i]);
           if (bi == null) {
             continue;
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e02682/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
index e5e324c..9e44761 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
@@ -50,6 +50,7 @@ import java.util.BitSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
 
 public class TestRecoverStripedFile {
   public static final Log LOG = LogFactory.getLog(TestRecoverStripedFile.class);
@@ -340,7 +341,7 @@ public class TestRecoverStripedFile {
   private void testCreateStripedFile(Path file, int dataLen)
       throws IOException {
     final byte[] data = new byte[dataLen];
-    DFSUtil.getRandom().nextBytes(data);
+    ThreadLocalRandom.current().nextBytes(data);
     writeContents(file, data);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e02682/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
index b714afa..c00037b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
@@ -186,7 +186,7 @@ public class TestPendingReplication {
       block = new Block(1, 1, 0);
       blockInfo = new BlockInfoContiguous(block, (short) 3);
 
-      pendingReplications.increment(block,
+      pendingReplications.increment(blockInfo,
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(1)));
       BlockCollection bc = Mockito.mock(BlockCollection.class);
@@ -201,7 +201,8 @@ public class TestPendingReplication {
       // Add a second block to pendingReplications that has no
       // corresponding entry in blocksmap
       block = new Block(2, 2, 0);
-      pendingReplications.increment(block,
+      blockInfo = new BlockInfoContiguous(block, (short) 3);
+      pendingReplications.increment(blockInfo,
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(1)));
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9e02682/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 1916b3c..6e92264 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -1209,13 +1209,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-<<<<<<< HEAD
-    Block block1 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block2 = new Block(ThreadLocalRandom.current().nextLong());
-=======
-    BlockInfo block1 = genBlockInfo(random.nextLong());
-    BlockInfo block2 = genBlockInfo(random.nextLong());
->>>>>>> 3e6f458... HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. Contributed by Jing Zhao.
+    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1277,13 +1272,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-<<<<<<< HEAD
-    Block block1 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block2 = new Block(ThreadLocalRandom.current().nextLong());
-=======
-    BlockInfo block1 = genBlockInfo(random.nextLong());
-    BlockInfo block2 = genBlockInfo(random.nextLong());
->>>>>>> 3e6f458... HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. Contributed by Jing Zhao.
+    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);


[23/50] [abbrv] hadoop git commit: Merge HDFS-8394 from trunk: Move getAdditionalBlock() and related functionalities into a separate class.

Posted by zh...@apache.org.
Merge HDFS-8394 from trunk: Move getAdditionalBlock() and related functionalities into a separate class.


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

Branch: refs/heads/HDFS-7285
Commit: d8ea443af0b1c8289a1dd738945831ff8be0e9c1
Parents: c99c337
Author: Jing Zhao <ji...@apache.org>
Authored: Sat May 16 16:57:12 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:26 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   2 +-
 .../blockmanagement/BlockInfoContiguous.java    |   2 +-
 .../server/blockmanagement/BlockManager.java    |   8 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  | 111 +++++++++++--------
 .../hdfs/server/namenode/FSNamesystem.java      |   6 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   8 --
 .../hadoop/hdfs/util/StripedBlockUtil.java      |   2 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   2 +-
 8 files changed, 78 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8ea443a/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 2e2ecfd..c06a435 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
@@ -1514,7 +1514,7 @@ public class DFSUtil {
   public static int getSmallBufferSize(Configuration conf) {
     return Math.min(getIoFileBufferSize(conf) / 2, 512);
   }
-  
+
   /**
    * Probe for HDFS Encryption being enabled; this uses the value of
    * the option {@link DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI},

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8ea443a/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 eeab076..416091f 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
@@ -42,7 +42,7 @@ public class BlockInfoContiguous extends BlockInfo {
    * @param from BlockReplicationInfo to copy from.
    */
   protected BlockInfoContiguous(BlockInfoContiguous from) {
-    this(from, from.getBlockCollection().getBlockReplication());
+    this(from, from.getBlockCollection().getPreferredBlockReplication());
     this.triplets = new Object[from.triplets.length];
     this.setBlockCollection(from.getBlockCollection());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8ea443a/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 5b876f9..8b51448 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
@@ -3567,6 +3567,11 @@ public class BlockManager {
     return storages;
   }
 
+  /** @return an iterator of the datanodes. */
+  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
+    return blocksMap.getStorages(block);
+  }
+
   public int getTotalBlocks() {
     return blocksMap.size();
   }
@@ -3958,7 +3963,7 @@ public class BlockManager {
         null);
   }
 
-  public LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info,
+  public static LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info,
       DatanodeStorageInfo[] locs, long offset) throws IOException {
     final LocatedBlock lb;
     if (info.isStriped()) {
@@ -3968,7 +3973,6 @@ public class BlockManager {
     } else {
       lb = newLocatedBlock(eb, locs, offset, false);
     }
-    setBlockToken(lb, BlockTokenIdentifier.AccessMode.WRITE);
     return lb;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8ea443a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index fa17d9d..81a2fa7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -45,10 +45,13 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.util.ChunkedArrayList;
@@ -74,7 +77,7 @@ class FSDirWriteFileOp {
       Block block) throws IOException {
     // modify file-> block and blocksMap
     // fileNode should be under construction
-    BlockInfoContiguousUnderConstruction uc = fileNode.removeLastBlock(block);
+    BlockInfoUnderConstruction uc = fileNode.removeLastBlock(block);
     if (uc == null) {
       return false;
     }
@@ -88,7 +91,7 @@ class FSDirWriteFileOp {
 
     // update space consumed
     fsd.updateCount(iip, 0, -fileNode.getPreferredBlockSize(),
-                    fileNode.getPreferredBlockReplication(), true);
+        fileNode.getPreferredBlockReplication(), true);
     return true;
   }
 
@@ -168,7 +171,7 @@ class FSDirWriteFileOp {
       String src, long fileId, String clientName,
       ExtendedBlock previous, LocatedBlock[] onRetryBlock) throws IOException {
     final long blockSize;
-    final int replication;
+    final short numTargets;
     final byte storagePolicyID;
     String clientMachine;
 
@@ -196,18 +199,21 @@ class FSDirWriteFileOp {
     blockSize = pendingFile.getPreferredBlockSize();
     clientMachine = pendingFile.getFileUnderConstructionFeature()
         .getClientMachine();
-    replication = pendingFile.getFileReplication();
+    boolean isStriped = pendingFile.isStriped();
+    numTargets = isStriped ?
+        HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS :
+        pendingFile.getFileReplication();
     storagePolicyID = pendingFile.getStoragePolicyID();
-    return new ValidateAddBlockResult(blockSize, replication, storagePolicyID,
-                                    clientMachine);
+    return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
+                                      clientMachine);
   }
 
-  static LocatedBlock makeLocatedBlock(FSNamesystem fsn, Block blk,
+  static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
       DatanodeStorageInfo[] locs, long offset) throws IOException {
     LocatedBlock lBlk = BlockManager.newLocatedBlock(fsn.getExtendedBlock(blk),
-                                                     locs, offset, false);
+        blk, locs, offset);
     fsn.getBlockManager().setBlockToken(lBlk,
-                                        BlockTokenIdentifier.AccessMode.WRITE);
+        BlockTokenIdentifier.AccessMode.WRITE);
     return lBlk;
   }
 
@@ -236,9 +242,10 @@ class FSDirWriteFileOp {
         return onRetryBlock[0];
       } else {
         // add new chosen targets to already allocated block and return
-        BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
-        ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
-            .setExpectedLocations(targets);
+        BlockInfo lastBlockInFile = pendingFile.getLastBlock();
+        final BlockInfoUnderConstruction uc
+            = (BlockInfoUnderConstruction)lastBlockInFile;
+        uc.setExpectedLocations(targets);
         offset = pendingFile.computeFileSize();
         return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
       }
@@ -249,15 +256,17 @@ class FSDirWriteFileOp {
                                   ExtendedBlock.getLocalBlock(previous));
 
     // allocate new block, record block locations in INode.
-    Block newBlock = fsn.createNewBlock();
+    final boolean isStriped = pendingFile.isStriped();
+    // allocate new block, record block locations in INode.
+    Block newBlock = fsn.createNewBlock(isStriped);
     INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
-    saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets);
+    saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets, isStriped);
 
     persistNewBlock(fsn, src, pendingFile);
     offset = pendingFile.computeFileSize();
 
     // Return located block
-    return makeLocatedBlock(fsn, newBlock, targets, offset);
+    return makeLocatedBlock(fsn, fsn.getStoredBlock(newBlock), targets, offset);
   }
 
   static DatanodeStorageInfo[] chooseTargetForNewBlock(
@@ -278,7 +287,7 @@ class FSDirWriteFileOp {
         : Arrays.asList(favoredNodes);
 
     // choose targets for the new block to be allocated.
-    return bm.chooseTarget4NewBlock(src, r.replication, clientNode,
+    return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
                                     excludedNodesSet, r.blockSize,
                                     favoredNodesList, r.storagePolicyID);
   }
@@ -504,25 +513,38 @@ class FSDirWriteFileOp {
   /**
    * Add a block to the file. Returns a reference to the added block.
    */
-  private static BlockInfoContiguous addBlock(
-      FSDirectory fsd, String path, INodesInPath inodesInPath, Block block,
-      DatanodeStorageInfo[] targets) throws IOException {
+  private static BlockInfo addBlock(FSDirectory fsd, String path,
+      INodesInPath inodesInPath, Block block, DatanodeStorageInfo[] targets,
+      boolean isStriped) throws IOException {
     fsd.writeLock();
     try {
       final INodeFile fileINode = inodesInPath.getLastINode().asFile();
       Preconditions.checkState(fileINode.isUnderConstruction());
 
-      // check quota limits and updated space consumed
-      fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
-          fileINode.getPreferredBlockReplication(), true);
-
       // associate new last block for the file
-      BlockInfoContiguousUnderConstruction blockInfo =
-        new BlockInfoContiguousUnderConstruction(
-            block,
-            fileINode.getFileReplication(),
-            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+      final BlockInfo blockInfo;
+      if (isStriped) {
+        ECSchema ecSchema = fsd.getECSchema(inodesInPath);
+        short numDataUnits = (short) ecSchema.getNumDataUnits();
+        short numParityUnits = (short) ecSchema.getNumParityUnits();
+        short numLocations = (short) (numDataUnits + numParityUnits);
+
+        // check quota limits and updated space consumed
+        fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
+            numLocations, true);
+        blockInfo = new BlockInfoStripedUnderConstruction(block, numDataUnits,
+            numParityUnits, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
             targets);
+      } else {
+        // check quota limits and updated space consumed
+        fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
+            fileINode.getPreferredBlockReplication(), true);
+
+        short numLocations = fileINode.getFileReplication();
+        blockInfo = new BlockInfoContiguousUnderConstruction(block,
+            numLocations, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+            targets);
+      }
       fsd.getBlockManager().addBlockCollection(blockInfo, fileINode);
       fileINode.addBlock(blockInfo);
 
@@ -576,7 +598,7 @@ class FSDirWriteFileOp {
   private static FileState analyzeFileState(
       FSNamesystem fsn, String src, long fileId, String clientName,
       ExtendedBlock previous, LocatedBlock[] onRetryBlock)
-          throws IOException  {
+      throws IOException {
     assert fsn.hasReadLock();
 
     checkBlock(fsn, previous);
@@ -659,8 +681,8 @@ class FSDirWriteFileOp {
             "allocation of a new block in " + src + ". Returning previously" +
             " allocated block " + lastBlockInFile);
         long offset = file.computeFileSize();
-        BlockInfoContiguousUnderConstruction lastBlockUC =
-            (BlockInfoContiguousUnderConstruction) lastBlockInFile;
+        BlockInfoUnderConstruction lastBlockUC =
+            (BlockInfoUnderConstruction) lastBlockInFile;
         onRetryBlock[0] = makeLocatedBlock(fsn, lastBlockInFile,
             lastBlockUC.getExpectedStorageLocations(), offset);
         return new FileState(file, src, iip);
@@ -685,14 +707,8 @@ class FSDirWriteFileOp {
     checkBlock(fsn, last);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsn.dir.resolvePath(pc, src, pathComponents);
-    boolean success = completeFileInternal(fsn, src, holder,
-                                           ExtendedBlock.getLocalBlock(last),
-                                           fileId);
-    if (success) {
-      NameNode.stateChangeLog.info("DIR* completeFile: " + srcArg
-                                       + " is closed by " + holder);
-    }
-    return success;
+    return completeFileInternal(fsn, src, holder,
+        ExtendedBlock.getLocalBlock(last), fileId);
   }
 
   private static boolean completeFileInternal(
@@ -794,13 +810,12 @@ class FSDirWriteFileOp {
    * @param targets target datanodes where replicas of the new block is placed
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
-  private static void saveAllocatedBlock(
-      FSNamesystem fsn, String src, INodesInPath inodesInPath, Block newBlock,
-      DatanodeStorageInfo[] targets)
-      throws IOException {
+  private static void saveAllocatedBlock(FSNamesystem fsn, String src,
+      INodesInPath inodesInPath, Block newBlock, DatanodeStorageInfo[] targets,
+      boolean isStriped) throws IOException {
     assert fsn.hasWriteLock();
-    BlockInfoContiguous b = addBlock(fsn.dir, src, inodesInPath, newBlock,
-                                     targets);
+    BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets,
+        isStriped);
     NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
   }
@@ -849,15 +864,15 @@ class FSDirWriteFileOp {
 
   static class ValidateAddBlockResult {
     final long blockSize;
-    final int replication;
+    final int numTargets;
     final byte storagePolicyID;
     final String clientMachine;
 
     ValidateAddBlockResult(
-        long blockSize, int replication, byte storagePolicyID,
+        long blockSize, int numTargets, byte storagePolicyID,
         String clientMachine) {
       this.blockSize = blockSize;
-      this.replication = replication;
+      this.numTargets = numTargets;
       this.storagePolicyID = storagePolicyID;
       this.clientMachine = clientMachine;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8ea443a/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 3c2c9cf..71a3dcf 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
@@ -3073,6 +3073,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       writeUnlock();
     }
     getEditLog().logSync();
+    if (success) {
+      NameNode.stateChangeLog.info("DIR* completeFile: " + src
+          + " is closed by " + holder);
+    }
     return success;
   }
 
@@ -3080,7 +3084,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Create new block with a unique block id and a new generation stamp.
    * @param isStriped is the file under striping or contiguous layout?
    */
-  Block createNewBlock() throws IOException {
+  Block createNewBlock(boolean isStriped) throws IOException {
     assert hasWriteLock();
     Block b = new Block(nextBlockId(isStriped), 0, 0);
     // Increment the generation stamp for every new block.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8ea443a/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 154198c..41287e8 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
@@ -905,14 +905,6 @@ public class INodeFile extends INodeWithAdditionalFields
     return counts;
   }
 
-  public final short getReplication(int lastSnapshotId) {
-    if (lastSnapshotId != CURRENT_STATE_ID) {
-      return getFileReplication(lastSnapshotId);
-    } else {
-      return getBlockReplication();
-    }
-  }
-
   /**
    * Return the penultimate allocated block for this file.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8ea443a/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 f7ae88a..c95f0b4 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
@@ -87,7 +87,7 @@ public class StripedBlockUtil {
         new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
         new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
         new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
-        bg.getStartOffset() + idxInBlockGroup, bg.isCorrupt(),
+        bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(),
         null);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8ea443a/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 0165189..9f106cf 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
@@ -1915,7 +1915,7 @@ public class DFSTestUtil {
         fileNode.getId(), null);
 
     final BlockInfo lastBlock = fileNode.getLastBlock();
-    final int groupSize = fileNode.getBlockReplication();
+    final int groupSize = fileNode.getPreferredBlockReplication();
     assert dataNodes.size() >= groupSize;
     // 1. RECEIVING_BLOCK IBR
     for (int i = 0; i < groupSize; i++) {


[38/50] [abbrv] hadoop git commit: HDFS-8427. Remove dataBlockNum and parityBlockNum from BlockInfoStriped. Contributed by Kai Sasaki.

Posted by zh...@apache.org.
HDFS-8427. Remove dataBlockNum and parityBlockNum from BlockInfoStriped. 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/744ef177
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/744ef177
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/744ef177

Branch: refs/heads/HDFS-7285
Commit: 744ef1779202621b2034b97926ad7ebcc67f433a
Parents: 45db1a0
Author: Jing Zhao <ji...@apache.org>
Authored: Wed May 20 14:52:51 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:34 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt             |  3 +++
 .../server/blockmanagement/BlockInfoStriped.java     | 15 ++++++---------
 2 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/744ef177/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 b608b10..610a5eb 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -241,3 +241,6 @@
 
     HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream.
     (Tsz Wo Nicholas Sze via jing9)
+
+    HDFS-8427. Remove dataBlockNum and parityBlockNum from BlockInfoStriped.
+    (Kai Sasaki via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/744ef177/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 d7a48a0..3898a49 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,7 +19,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 
@@ -39,8 +38,6 @@ import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_S
  * array to record the block index for each triplet.
  */
 public class BlockInfoStriped extends BlockInfo {
-  private final short dataBlockNum;
-  private final short parityBlockNum;
   private final ECSchema schema;
   /**
    * Always the same size with triplets. Record the block index for each triplet
@@ -54,8 +51,6 @@ public class BlockInfoStriped extends BlockInfo {
     indices = new byte[schema.getNumDataUnits() + schema.getNumParityUnits()];
     initIndices();
     this.schema = schema;
-    this.dataBlockNum = (short)schema.getNumDataUnits();
-    this.parityBlockNum = (short)schema.getNumParityUnits();
   }
 
   BlockInfoStriped(BlockInfoStriped b) {
@@ -64,15 +59,16 @@ public class BlockInfoStriped extends BlockInfo {
   }
 
   public short getTotalBlockNum() {
-    return (short) (dataBlockNum + parityBlockNum);
+    return (short) (this.schema.getNumDataUnits()
+        + this.schema.getNumParityUnits());
   }
 
   public short getDataBlockNum() {
-    return dataBlockNum;
+    return (short) this.schema.getNumDataUnits();
   }
 
   public short getParityBlockNum() {
-    return parityBlockNum;
+    return (short) this.schema.getNumParityUnits();
   }
 
   public ECSchema getSchema() {
@@ -210,7 +206,8 @@ public class BlockInfoStriped extends BlockInfo {
     // be the total of data blocks and parity blocks because
     // `getNumBytes` is the total of actual data block size.
     return StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(),
-        dataBlockNum, parityBlockNum, BLOCK_STRIPED_CELL_SIZE);
+        this.schema.getNumDataUnits(), this.schema.getNumParityUnits(),
+        BLOCK_STRIPED_CELL_SIZE);
     }
 
   @Override


[24/50] [abbrv] hadoop git commit: HDFS-8352. Erasure Coding: test webhdfs read write stripe file. (waltersu4549)

Posted by zh...@apache.org.
HDFS-8352. Erasure Coding: test webhdfs read write stripe file. (waltersu4549)


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

Branch: refs/heads/HDFS-7285
Commit: a9197269143f9d976a1565bbf4e383fac6e1326f
Parents: c9103e9
Author: waltersu4549 <wa...@apache.org>
Authored: Mon May 18 19:10:37 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:30 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/TestWriteReadStripedFile.java   | 267 ++++++++++---------
 1 file changed, 148 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9197269/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 57d6eb9..f78fb7a 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
@@ -21,9 +21,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.web.ByteRangeInputStream;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -33,23 +37,26 @@ import org.junit.Test;
 import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Random;
 
 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;
+  private static Configuration conf;
+  private static FileSystem fs;
+
+  private static Random r= new Random();
 
   @BeforeClass
   public static void setup() throws IOException {
-    Configuration conf = new 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);
@@ -134,7 +141,7 @@ public class TestWriteReadStripedFile {
   @Test
   public void testFileMoreThanABlockGroup2() throws IOException {
     testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2",
-        blockSize * dataBlocks + cellSize+ 123);
+        blockSize * dataBlocks + cellSize + 123);
   }
 
 
@@ -171,7 +178,7 @@ public class TestWriteReadStripedFile {
   }
 
   private void assertSeekAndRead(FSDataInputStream fsdis, int pos,
-      int writeBytes) throws IOException {
+                                 int writeBytes) throws IOException {
     fsdis.seek(pos);
     byte[] buf = new byte[writeBytes];
     int readLen = readAll(fsdis, buf);
@@ -182,147 +189,169 @@ public class TestWriteReadStripedFile {
     }
   }
 
-  private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes)
+  private void testOneFileUsingDFSStripedInputStream(String src, int fileLength)
       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();
+    final byte[] expected = generateBytes(fileLength);
+    Path srcPath = new Path(src);
+    DFSTestUtil.writeFile(fs, srcPath, new String(expected));
+
+    verifyLength(fs, srcPath, fileLength);
+
+    byte[] smallBuf = new byte[1024];
+    byte[] largeBuf = new byte[fileLength + 100];
+    verifyPread(fs, srcPath, fileLength, expected, largeBuf);
+
+    verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf);
+    verifySeek(fs, srcPath, fileLength);
+    verifyStatefulRead(fs, srcPath, fileLength, expected,
+        ByteBuffer.allocate(fileLength + 100));
+    verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf);
+    verifyStatefulRead(fs, srcPath, fileLength, expected,
+        ByteBuffer.allocate(1024));
+  }
+
+  @Test
+  public void testWriteReadUsingWebHdfs() throws Exception {
+    int fileLength = blockSize * dataBlocks + cellSize + 123;
+
+    final byte[] expected = generateBytes(fileLength);
+    FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+    Path srcPath = new Path("/testWriteReadUsingWebHdfs_stripe");
+    DFSTestUtil.writeFile(fs, srcPath, new String(expected));
+
+    verifyLength(fs, srcPath, fileLength);
+
+    byte[] smallBuf = new byte[1024];
+    byte[] largeBuf = new byte[fileLength + 100];
+    verifyPread(fs, srcPath, fileLength, expected, largeBuf);
+
+    verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf);
+    verifySeek(fs, srcPath, fileLength);
+    verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf);
+    //webhdfs doesn't support bytebuffer read
+
+  }
+
+  void verifyLength(FileSystem fs, Path srcPath, int fileLength)
+      throws IOException {
+    FileStatus status = fs.getFileStatus(srcPath);
     Assert.assertEquals("File length should be the same",
-        writeBytes, fileLength);
+        fileLength, status.getLen());
+  }
 
-    // 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]);
+  void verifyPread(FileSystem fs, Path srcPath,  int fileLength,
+                   byte[] expected, byte[] buf) throws IOException {
+    FSDataInputStream in = fs.open(srcPath);
+    int[] startOffsets = {0, 1, cellSize - 102, cellSize, cellSize + 102,
+        cellSize * (dataBlocks - 1), cellSize * (dataBlocks - 1) + 102,
+        cellSize * dataBlocks, fileLength - 102, fileLength - 1};
+    for (int startOffset : startOffsets) {
+      startOffset = Math.max(0, Math.min(startOffset, fileLength - 1));
+      int remaining = fileLength - startOffset;
+      in.readFully(startOffset, buf, 0, remaining);
+      for (int i = 0; i < remaining; i++) {
+        Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " +
+                "same",
+            expected[startOffset + i], buf[i]);
       }
     }
+    in.close();
+  }
 
-    // stateful read with byte array
-    try (FSDataInputStream fsdis = fs.open(new Path(src))) {
-      byte[] buf = new byte[writeBytes + 100];
-      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++) {
-        Assert.assertEquals("Byte at " + i + " should be the same", getByte(i),
-            buf[i]);
+  void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength,
+                          byte[] expected, byte[] buf) throws IOException {
+    FSDataInputStream in = fs.open(srcPath);
+    final byte[] result = new byte[fileLength];
+    int readLen = 0;
+    int ret;
+    do {
+      ret = in.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",
+        fileLength, readLen);
+    Assert.assertArrayEquals(expected, result);
+    in.close();
+  }
 
-    // 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);
+  void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength,
+                          byte[] expected, ByteBuffer buf) throws IOException {
+    FSDataInputStream in = fs.open(srcPath);
+    ByteBuffer result = ByteBuffer.allocate(fileLength);
+    int readLen = 0;
+    int ret;
+    do {
+      ret = in.read(buf);
+      if (ret > 0) {
+        readLen += ret;
+        buf.flip();
+        result.put(buf);
+        buf.clear();
+      }
+    } while (ret >= 0);
+    readLen = readLen >= 0 ? readLen : 0;
+    Assert.assertEquals("The length of file should be the same to write size",
+        fileLength, readLen);
+    Assert.assertArrayEquals(expected, result.array());
+    in.close();
+  }
 
-      // seek to 0 pos
-      pos = 0;
-      assertSeekAndRead(fsdis, pos, writeBytes);
 
-      if (writeBytes > cellSize) {
-        // seek to cellSize boundary
-        pos = cellSize -1;
-        assertSeekAndRead(fsdis, pos, writeBytes);
-      }
+  void verifySeek(FileSystem fs, Path srcPath, int fileLength)
+      throws IOException {
+    FSDataInputStream in = fs.open(srcPath);
+    // seek to 1/2 of content
+    int pos = fileLength / 2;
+    assertSeekAndRead(in, pos, fileLength);
+
+    // seek to 1/3 of content
+    pos = fileLength / 3;
+    assertSeekAndRead(in, pos, fileLength);
+
+    // seek to 0 pos
+    pos = 0;
+    assertSeekAndRead(in, pos, fileLength);
+
+    if (fileLength > cellSize) {
+      // seek to cellSize boundary
+      pos = cellSize - 1;
+      assertSeekAndRead(in, pos, fileLength);
+    }
 
-      if (writeBytes > cellSize * dataBlocks) {
-        // seek to striped cell group boundary
-        pos = cellSize * dataBlocks - 1;
-        assertSeekAndRead(fsdis, pos, writeBytes);
-      }
+    if (fileLength > cellSize * dataBlocks) {
+      // seek to striped cell group boundary
+      pos = cellSize * dataBlocks - 1;
+      assertSeekAndRead(in, pos, fileLength);
+    }
 
-      if (writeBytes > blockSize * dataBlocks) {
-        // seek to striped block group boundary
-        pos = blockSize * dataBlocks - 1;
-        assertSeekAndRead(fsdis, pos, writeBytes);
-      }
+    if (fileLength > blockSize * dataBlocks) {
+      // seek to striped block group boundary
+      pos = blockSize * dataBlocks - 1;
+      assertSeekAndRead(in, pos, fileLength);
+    }
 
+    if(!(in.getWrappedStream() instanceof ByteRangeInputStream)){
       try {
-        fsdis.seek(-1);
+        in.seek(-1);
         Assert.fail("Should be failed if seek to negative offset");
       } catch (EOFException e) {
         // expected
       }
 
       try {
-        fsdis.seek(writeBytes + 1);
+        in.seek(fileLength + 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);
-      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());
-    }
+    in.close();
   }
 
   @Test


[09/50] [abbrv] 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/6616de24
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6616de24
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6616de24

Branch: refs/heads/HDFS-7285
Commit: 6616de24cb14f1c2d0d6568fd4382062618834bd
Parents: 220ca96
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue May 5 16:33:56 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:47 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         |  31 +-
 .../hadoop/hdfs/TestRecoverStripedFile.java     | 356 ++++++++
 14 files changed, 1377 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6616de24/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/6616de24/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/6616de24/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/6616de24/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/6616de24/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 6bc005b..d5d3095 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
@@ -369,6 +369,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/6616de24/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/6616de24/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/6616de24/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/6616de24/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/6616de24/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/6616de24/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/6616de24/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/6616de24/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 7f0730b..f802128 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
@@ -2312,11 +2312,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>
@@ -2354,4 +2354,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/6616de24/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();
+    }
+  }
+}


[16/50] [abbrv] 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/9da92754
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9da92754
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9da92754

Branch: refs/heads/HDFS-7285
Commit: 9da927540f0ea6698388a4e79ef32c4dc51495ea
Parents: a17cedb
Author: Jing Zhao <ji...@apache.org>
Authored: Thu May 7 11:52:49 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:49 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/9da92754/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/9da92754/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/9da92754/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/9da92754/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/9da92754/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 8f250fd..9155b4d 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
@@ -1193,9 +1193,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/9da92754/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/9da92754/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/9da92754/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/9da92754/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/9da92754/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/9da92754/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/9da92754/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/9da92754/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/9da92754/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/9da92754/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/9da92754/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/9da92754/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 e0bd6f4..6e2ec5e 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/9da92754/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/9da92754/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/9da92754/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 eabd0c8..7870827 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/9da92754/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);


[39/50] [abbrv] hadoop git commit: HDFS-8294. Erasure Coding: Fix Findbug warnings present in erasure coding. Contributed by Rakesh R.

Posted by zh...@apache.org.
HDFS-8294. Erasure Coding: Fix Findbug warnings present in erasure coding. 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/7af05a3d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7af05a3d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7af05a3d

Branch: refs/heads/HDFS-7285
Commit: 7af05a3db4c731eca5e674b3d9e3b7abbf82ccd5
Parents: e53fa76
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Thu May 21 14:40:14 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:34 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt                |  3 +++
 .../org/apache/hadoop/hdfs/DFSStripedOutputStream.java  | 12 ++++++------
 .../BlockInfoStripedUnderConstruction.java              |  3 +++
 .../datanode/erasurecode/ErasureCodingWorker.java       |  4 ++--
 .../hdfs/server/namenode/ErasureCodingZoneManager.java  |  4 ++--
 .../org/apache/hadoop/hdfs/util/StripedBlockUtil.java   |  6 +++---
 6 files changed, 19 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7af05a3d/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 3bdff6f..c986f19 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -247,3 +247,6 @@
 
     HDFS-8186. Erasure coding: Make block placement policy for EC file configurable.
     (Walter Su via zhz)
+
+    HDFS-8294. Erasure Coding: Fix Findbug warnings present in erasure coding.
+    (Rakesh R via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7af05a3d/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 8eed6ad..515ce0c 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
@@ -276,11 +276,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     return getCurrentStreamer().getIndex();
   }
 
-  StripedDataStreamer getCurrentStreamer() {
+  private synchronized StripedDataStreamer getCurrentStreamer() {
     return (StripedDataStreamer)streamer;
   }
 
-  private StripedDataStreamer setCurrentStreamer(int i) {
+  private synchronized StripedDataStreamer setCurrentStreamer(int i) {
     streamer = streamers.get(i);
     return getCurrentStreamer();
   }
@@ -344,8 +344,8 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     int ckOff = 0;
     while (byteBuffer.remaining() > 0) {
       DFSPacket p = createPacket(packetSize, chunksPerPacket,
-          streamer.getBytesCurBlock(),
-          streamer.getAndIncCurrentSeqno(), false);
+          getCurrentStreamer().getBytesCurBlock(),
+          getCurrentStreamer().getAndIncCurrentSeqno(), false);
       int maxBytesToPacket = p.getMaxChunks() * bytesPerChecksum;
       int toWrite = byteBuffer.remaining() > maxBytesToPacket ?
           maxBytesToPacket: byteBuffer.remaining();
@@ -353,7 +353,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       p.writeChecksum(checksumBuf, ckOff, ckLen);
       ckOff += ckLen;
       p.writeData(byteBuffer, toWrite);
-      streamer.incBytesCurBlock(toWrite);
+      getCurrentStreamer().incBytesCurBlock(toWrite);
       packets.add(p);
     }
     return packets;
@@ -529,7 +529,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     if (!current.isFailed()) {
       try {
         for (DFSPacket p : generatePackets(buffer, checksumBuf)) {
-          streamer.waitAndQueuePacket(p);
+          getCurrentStreamer().waitAndQueuePacket(p);
         }
         endBlock();
       } catch(Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7af05a3d/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 40f880f..76d7920 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
@@ -189,6 +189,9 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
       NameNode.blockStateChangeLog.warn("BLOCK*" +
           " BlockInfoStripedUnderConstruction.initLeaseRecovery:" +
           " No blocks found, lease removed.");
+      // sets primary node index and return.
+      primaryNodeIndex = -1;
+      return;
     }
     boolean allLiveReplicasTriedAsPrimary = true;
     for (ReplicaUnderConstruction replica : replicas) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7af05a3d/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 d227de8..ded51eb 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
@@ -251,7 +251,7 @@ public final class ErasureCodingWorker {
     private final long[] blockOffset4Targets;
     private final long[] seqNo4Targets;
 
-    private final int WRITE_PACKET_SIZE = 64 * 1024;
+    private final static int WRITE_PACKET_SIZE = 64 * 1024;
     private DataChecksum checksum;
     private int maxChunksPerPacket;
     private byte[] packetBuf;
@@ -904,7 +904,7 @@ public final class ErasureCodingWorker {
     }
   }
 
-  private class StripedReader {
+  private static class StripedReader {
     private final short index;
     private BlockReader blockReader;
     private ByteBuffer buffer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7af05a3d/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 371b8ac..89fecc6 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
@@ -92,8 +92,8 @@ public class ErasureCodingZoneManager {
           String schemaName = WritableUtils.readString(dIn);
           ECSchema schema = dir.getFSNamesystem().getECSchemaManager()
               .getSchema(schemaName);
-          return new ErasureCodingZoneInfo(inode.getFullPathName(), schema,
-              cellSize);
+          return new ErasureCodingZoneInfo(dir.getInode(inode.getId())
+              .getFullPathName(), schema, cellSize);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7af05a3d/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 0b09f37..38dc61a 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
@@ -105,7 +105,7 @@ public class StripedBlockUtil {
     final ExtendedBlock blk = constructInternalBlock(
         bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup);
 
-    final long offset = bg.getStartOffset() + idxInBlockGroup * cellSize;
+    final long offset = bg.getStartOffset() + idxInBlockGroup * (long) cellSize;
     if (idxInReturnedLocs < bg.getLocations().length) {
       return new LocatedBlock(blk,
           new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
@@ -406,11 +406,11 @@ public class StripedBlockUtil {
     long earliestStart = startOffsets[firstCell.idxInStripe];
     for (int i = 1; i < dataBlkNum; i++) {
       int idx = firstCellIdxInBG + i;
-      if (idx * cellSize >= blockGroup.getBlockSize()) {
+      if (idx * (long) cellSize >= blockGroup.getBlockSize()) {
         break;
       }
       StripingCell cell = new StripingCell(ecSchema, cellSize, idx);
-      startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * cellSize;
+      startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * (long) cellSize;
       if (startOffsets[cell.idxInStripe] < earliestStart) {
         earliestStart = startOffsets[cell.idxInStripe];
       }


[07/50] [abbrv] 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/43923255
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/43923255
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/43923255

Branch: refs/heads/HDFS-7285
Commit: 4392325546a49d29720fe705381447a0fdd5826e
Parents: 436c148
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue May 5 11:54:30 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 11:59:58 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/43923255/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/43923255/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/43923255/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/43923255/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/43923255/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/43923255/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/43923255/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>


[28/50] [abbrv] 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/09c3a375
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/09c3a375
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/09c3a375

Branch: refs/heads/HDFS-7285
Commit: 09c3a375bafa481e88d1317388a73c46950164c9
Parents: 9c7a78c
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon May 18 10:09:57 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:31 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       | 63 +++++++++++++++++++-
 .../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, 156 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c3a375/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 c10ffbd..a152e31 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 via Zhe Zhang)
+
+    HADOOP-11920. Refactor some codes for erasure coders. (Kai Zheng via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c3a375/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/09c3a375/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/09c3a375/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/09c3a375/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/09c3a375/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..2400313 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
+  private 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,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/09c3a375/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/09c3a375/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/09c3a375/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/09c3a375/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/09c3a375/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/09c3a375/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/09c3a375/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/09c3a375/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


[37/50] [abbrv] hadoop git commit: HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file. Contributed by Yi Liu.

Posted by zh...@apache.org.
HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file. 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/47ef869f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/47ef869f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/47ef869f

Branch: refs/heads/HDFS-7285
Commit: 47ef869fa790dd096b576697c4245d2f3a3193fa
Parents: 91c81fd
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Tue May 19 17:25:27 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:33 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt    | 9 ++++++---
 .../hadoop/hdfs/server/blockmanagement/BlockManager.java    | 2 +-
 2 files changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/47ef869f/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 d6c9dba..48bc9d6 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -59,7 +59,7 @@
 
     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.
@@ -110,7 +110,7 @@
 
     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)
 
@@ -172,7 +172,7 @@
 
     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)
 
@@ -235,3 +235,6 @@
     (Rakesh R via waltersu4549)
 
     HDFS-8375. Add cellSize as an XAttr to ECZone. ( Vinayakumar B via zhz).
+
+    HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file.
+    (Yi Liu via zhz).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47ef869f/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 28e7b89..fc13965 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
@@ -3396,7 +3396,7 @@ public class BlockManager {
     for (ReceivedDeletedBlockInfo rdbi : srdb.getBlocks()) {
       switch (rdbi.getStatus()) {
       case DELETED_BLOCK:
-        removeStoredBlock(storageInfo, getStoredBlock(rdbi.getBlock()), node);
+        removeStoredBlock(storageInfo, rdbi.getBlock(), node);
         deleted++;
         break;
       case RECEIVED_BLOCK:


[04/50] [abbrv] 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/6dea01f1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6dea01f1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6dea01f1

Branch: refs/heads/HDFS-7285
Commit: 6dea01f1ee5ea3bf6e146e1b68616c2f43ba4792
Parents: ea6c66e
Author: Jing Zhao <ji...@apache.org>
Authored: Mon May 4 14:44:58 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 11:59:57 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/6dea01f1/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/6dea01f1/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 6e58cd6..0d51a57 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
@@ -717,6 +717,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,
@@ -750,6 +760,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;
+    }
   }
 
   /**
@@ -783,6 +800,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/6dea01f1/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/6dea01f1/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/6dea01f1/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/6dea01f1/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();


[33/50] [abbrv] hadoop git commit: HDFS-8418. Fix the isNeededReplication calculation for Striped block in NN. Contributed by Yi Liu.

Posted by zh...@apache.org.
HDFS-8418. Fix the isNeededReplication calculation for Striped block in NN. 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/b008348d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b008348d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b008348d

Branch: refs/heads/HDFS-7285
Commit: b008348dbf9bdd5070930be5d182116c5d370f6b
Parents: 6c310db
Author: Jing Zhao <ji...@apache.org>
Authored: Mon May 18 19:06:34 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:32 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 ++
 .../server/blockmanagement/BlockManager.java    | 54 ++++++++++++--------
 .../blockmanagement/DecommissionManager.java    | 11 ++--
 .../hdfs/server/namenode/NamenodeFsck.java      |  2 +-
 4 files changed, 43 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b008348d/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 e016ba0..1549930 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -220,3 +220,6 @@
 
     HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe.
     (Walter Su via jing9)
+
+    HDFS-8418. Fix the isNeededReplication calculation for Striped block in NN.
+    (Yi Liu via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b008348d/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 8b51448..d296aa8 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
@@ -43,7 +43,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -85,6 +84,7 @@ 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.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
 
 import org.apache.hadoop.net.Node;
@@ -603,16 +603,7 @@ public class BlockManager {
 
   public short getMinStorageNum(BlockInfo block) {
     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;
+      return getStripedDataBlockNum(block);
     } else {
       return minReplication;
     }
@@ -1258,7 +1249,7 @@ public class BlockManager {
       return;
     } 
     short expectedReplicas =
-        b.stored.getBlockCollection().getPreferredBlockReplication();
+        getExpectedReplicaNum(b.stored.getBlockCollection(), b.stored);
 
     // Add replica to the data-node if it is not already there
     if (storageInfo != null) {
@@ -1437,7 +1428,7 @@ public class BlockManager {
               continue;
             }
 
-            requiredReplication = bc.getPreferredBlockReplication();
+            requiredReplication = getExpectedReplicaNum(bc, block);
 
             // get a source data-node
             containingNodes = new ArrayList<>();
@@ -1537,7 +1528,7 @@ public class BlockManager {
             rw.targets = null;
             continue;
           }
-          requiredReplication = bc.getPreferredBlockReplication();
+          requiredReplication = getExpectedReplicaNum(bc, block);
 
           // do not schedule more if enough replicas is already pending
           NumberReplicas numReplicas = countNodes(block);
@@ -2539,7 +2530,7 @@ public class BlockManager {
           int reportedBlkIdx = BlockIdManager.getBlockIndex(reported);
           wrongSize = reported.getNumBytes() !=
               getInternalBlockLength(stripedBlock.getNumBytes(),
-                  HdfsConstants.BLOCK_STRIPED_CELL_SIZE,
+                  BLOCK_STRIPED_CELL_SIZE,
                   stripedBlock.getDataBlockNum(), reportedBlkIdx);
         } else {
           wrongSize = storedBlock.getNumBytes() != reported.getNumBytes();
@@ -2763,7 +2754,7 @@ public class BlockManager {
     }
 
     // handle underReplication/overReplication
-    short fileReplication = bc.getPreferredBlockReplication();
+    short fileReplication = getExpectedReplicaNum(bc, storedBlock);
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
       neededReplications.remove(storedBlock, numCurrentReplica,
           num.decommissionedAndDecommissioning(), fileReplication);
@@ -3003,7 +2994,7 @@ public class BlockManager {
     }
     // calculate current replication
     short expectedReplication =
-        block.getBlockCollection().getPreferredBlockReplication();
+        getExpectedReplicaNum(block.getBlockCollection(), block);
     NumberReplicas num = countNodes(block);
     int numCurrentReplica = num.liveReplicas();
     // add to under-replicated queue if need to be
@@ -3638,8 +3629,8 @@ public class BlockManager {
    * process it as an over replicated block.
    */
   public void checkReplication(BlockCollection bc) {
-    final short expected = bc.getPreferredBlockReplication();
     for (BlockInfo block : bc.getBlocks()) {
+      short expected = getExpectedReplicaNum(bc, block);
       final NumberReplicas n = countNodes(block);
       if (isNeededReplication(block, expected, n.liveReplicas())) { 
         neededReplications.add(block, n.liveReplicas(),
@@ -3674,9 +3665,9 @@ public class BlockManager {
    * @return 0 if the block is not found;
    *         otherwise, return the replication factor of the block.
    */
-  private int getReplication(Block block) {
+  private int getReplication(BlockInfo block) {
     final BlockCollection bc = blocksMap.getBlockCollection(block);
-    return bc == null? 0: bc.getPreferredBlockReplication();
+    return bc == null? 0: getExpectedReplicaNum(bc, block);
   }
 
 
@@ -3759,6 +3750,29 @@ public class BlockManager {
     return current < expected || !blockHasEnoughRacks(storedBlock, expected);
   }
   
+  public short getExpectedReplicaNum(BlockCollection bc, BlockInfo block) {
+    if (block.isStriped()) {
+      return (short) (getStripedDataBlockNum(block) + 
+          ((BlockInfoStriped) block).getParityBlockNum());
+    } else {
+      return bc.getPreferredBlockReplication();
+    }
+  }
+  
+  short getStripedDataBlockNum(BlockInfo block) {
+    assert 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) / BLOCK_STRIPED_CELL_SIZE + 1);
+    }
+    return dataBlockNum;
+  }
+  
   public long getMissingBlocksCount() {
     // not locking
     return this.neededReplications.getCorruptBlockSize();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b008348d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 37ce8e3..b1cc9bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -36,7 +36,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
@@ -251,7 +250,7 @@ public class DecommissionManager {
    */
   private boolean isSufficient(BlockInfo block, BlockCollection bc,
       NumberReplicas numberReplicas) {
-    final int numExpected = bc.getPreferredBlockReplication();
+    final int numExpected = blockManager.getExpectedReplicaNum(bc, block);
     final int numLive = numberReplicas.liveReplicas();
     if (!blockManager.isNeededReplication(block, numExpected, numLive)) {
       // Block doesn't need replication. Skip.
@@ -285,11 +284,11 @@ public class DecommissionManager {
     return false;
   }
 
-  private static void logBlockReplicationInfo(Block block, BlockCollection bc,
+  private void logBlockReplicationInfo(BlockInfo block, BlockCollection bc,
       DatanodeDescriptor srcNode, NumberReplicas num,
       Iterable<DatanodeStorageInfo> storages) {
     int curReplicas = num.liveReplicas();
-    int curExpectedReplicas = bc.getPreferredBlockReplication();
+    int curExpectedReplicas = blockManager.getExpectedReplicaNum(bc, block);
     StringBuilder nodeList = new StringBuilder();
     for (DatanodeStorageInfo storage : storages) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
@@ -561,7 +560,7 @@ public class DecommissionManager {
         // Schedule under-replicated blocks for replication if not already
         // pending
         if (blockManager.isNeededReplication(block,
-            bc.getPreferredBlockReplication(), liveReplicas)) {
+            blockManager.getExpectedReplicaNum(bc, block), liveReplicas)) {
           if (!blockManager.neededReplications.contains(block) &&
               blockManager.pendingReplications.getNumReplicas(block) == 0 &&
               namesystem.isPopulatingReplQueues()) {
@@ -569,7 +568,7 @@ public class DecommissionManager {
             blockManager.neededReplications.add(block,
                 liveReplicas,
                 num.decommissionedAndDecommissioning(),
-                bc.getPreferredBlockReplication());
+                blockManager.getExpectedReplicaNum(bc, block));
           }
         }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b008348d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 24a38e5..fccef17 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -256,7 +256,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       out.println("Block Id: " + blockId);
       out.println("Block belongs to: "+iNode.getFullPathName());
       out.println("No. of Expected Replica: " +
-          bc.getPreferredBlockReplication());
+          bm.getExpectedReplicaNum(bc, blockInfo));
       out.println("No. of live Replica: " + numberReplicas.liveReplicas());
       out.println("No. of excess Replica: " + numberReplicas.excessReplicas());
       out.println("No. of stale Replica: " +


[18/50] [abbrv] hadoop git commit: HDFS-8195. Erasure coding: Fix file quota change when we complete/commit the striped blocks. Contributed by Takuya Fukudome.

Posted by zh...@apache.org.
HDFS-8195. Erasure coding: Fix file quota change when we complete/commit the striped blocks. Contributed by Takuya Fukudome.


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

Branch: refs/heads/HDFS-7285
Commit: 9798065cbb5cf5de94fe8e17ac22388f70e12dd6
Parents: 8da9e18
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue May 12 23:10:25 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:50 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../hdfs/server/namenode/FSDirectory.java       |   2 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  25 +++-
 .../namenode/TestQuotaWithStripedBlocks.java    | 125 +++++++++++++++++++
 4 files changed, 151 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9798065c/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 0a2bb9e..0945d72 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -206,3 +206,6 @@
     handled properly (Rakesh R via zhz)
 
     HDFS-8363. Erasure Coding: DFSStripedInputStream#seekToNewSource. (yliu)
+
+    HDFS-8195. Erasure coding: Fix file quota change when we complete/commit 
+    the striped blocks. (Takuya Fukudome via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9798065c/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 b830157..7989ccb 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
@@ -520,7 +520,7 @@ public class FSDirectory implements Closeable {
     final INodeFile fileINode = iip.getLastINode().asFile();
     EnumCounters<StorageType> typeSpaceDeltas =
       getStorageTypeDeltas(fileINode.getStoragePolicyID(), ssDelta,
-          replication, replication);;
+          replication, replication);
     updateCount(iip, iip.length() - 1,
       new QuotaCounts.Builder().nameSpace(nsDelta).storageSpace(ssDelta * replication).
           typeSpaces(typeSpaceDeltas).build(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9798065c/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 3179b93..3c2c9cf 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
@@ -3656,11 +3656,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // Adjust disk space consumption if required
-    // TODO: support EC files
-    final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();    
+    final long diff;
+    final short replicationFactor;
+    if (fileINode.isStriped()) {
+      final ECSchema ecSchema = dir.getECSchema(iip);
+      final short numDataUnits = (short) ecSchema.getNumDataUnits();
+      final short numParityUnits = (short) ecSchema.getNumParityUnits();
+
+      final long numBlocks = numDataUnits + numParityUnits;
+      final long fullBlockGroupSize =
+          fileINode.getPreferredBlockSize() * numBlocks;
+
+      final BlockInfoStriped striped = new BlockInfoStriped(commitBlock,
+          numDataUnits, numParityUnits);
+      final long actualBlockGroupSize = striped.spaceConsumed();
+
+      diff = fullBlockGroupSize - actualBlockGroupSize;
+      replicationFactor = (short) 1;
+    } else {
+      diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();
+      replicationFactor = fileINode.getFileReplication();
+    }
     if (diff > 0) {
       try {
-        dir.updateSpaceConsumed(iip, 0, -diff, fileINode.getFileReplication());
+        dir.updateSpaceConsumed(iip, 0, -diff, replicationFactor);
       } catch (IOException e) {
         LOG.warn("Unexpected exception while updating disk space.", e);
       }

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


[42/50] [abbrv] hadoop git commit: HDFS-8441. Erasure Coding: make condition check earlier for setReplication. (waltersu4549)

Posted by zh...@apache.org.
HDFS-8441. Erasure Coding: make condition check earlier for setReplication. (waltersu4549)


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

Branch: refs/heads/HDFS-7285
Commit: 3d734df24cba53ec56074b4d28e3bcdce7d2894e
Parents: 7af05a3
Author: Walter Su <wa...@apache.org>
Authored: Fri May 22 10:25:53 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:07:09 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +++
 .../hdfs/server/namenode/FSDirAttrOp.java       |  7 +++++-
 .../hdfs/server/namenode/FSNamesystem.java      | 20 +++++++++++++++-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  2 --
 .../hadoop/hdfs/TestErasureCodingZones.java     | 24 +++++++++++++++++++-
 5 files changed, 51 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d734df2/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 c986f19..d71b9c3 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -250,3 +250,6 @@
 
     HDFS-8294. Erasure Coding: Fix Findbug warnings present in erasure coding.
     (Rakesh R via zhz)
+
+    HDFS-8441. Erasure Coding: make condition check earlier for setReplication.
+    (waltersu4549)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d734df2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 9abb9fa..d34a0fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -380,7 +380,7 @@ public class FSDirAttrOp {
   static BlockInfoContiguous[] unprotectedSetReplication(
       FSDirectory fsd, String src, short replication, short[] blockRepls)
       throws QuotaExceededException, UnresolvedLinkException,
-             SnapshotAccessControlException {
+      SnapshotAccessControlException, UnsupportedActionException {
     assert fsd.hasWriteLock();
 
     final INodesInPath iip = fsd.getINodesInPath4Write(src, true);
@@ -389,6 +389,11 @@ public class FSDirAttrOp {
       return null;
     }
     INodeFile file = inode.asFile();
+    if (file.isStriped()) {
+      throw new UnsupportedActionException(
+          "Cannot set replication to a file with striped blocks");
+    }
+
     final short oldBR = file.getPreferredBlockReplication();
 
     // before setFileReplication, check for increasing block replication.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d734df2/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 e9bb2f7..a53cd61 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
@@ -2404,7 +2404,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if (!DFSUtil.isValidName(src)) {
       throw new InvalidPathException(src);
     }
-    blockManager.verifyReplication(src, replication, clientMachine);
+
+    checkOperation(OperationCategory.READ);
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      if (!isInECZone(src)) {
+        blockManager.verifyReplication(src, replication, clientMachine);
+      }
+    } finally {
+      readUnlock();
+    }
+    
     checkOperation(OperationCategory.WRITE);
     if (blockSize < minBlockSize) {
       throw new IOException("Specified block size is less than configured" +
@@ -7605,6 +7616,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     logAuditEvent(success, "createErasureCodingZone", srcArg, null, resultingStat);
   }
 
+  private boolean isInECZone(String src) throws IOException {
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    src = FSDirectory.resolvePath(src, pathComponents, dir);
+    final INodesInPath iip = dir.getINodesInPath(src, true);
+    return dir.isInECZone(iip);
+  }
+
   /**
    * Get the erasure coding information for specified src
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d734df2/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 41287e8..4688001 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
@@ -429,8 +429,6 @@ public class INodeFile extends INodeWithAdditionalFields
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication,
       int latestSnapshotId) throws QuotaExceededException {
-    Preconditions.checkState(!isStriped(),
-        "Cannot set replication to a file with striped blocks");
     recordModification(latestSnapshotId);
     setFileReplication(replication);
     return this;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d734df2/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 a7c3cd4..480791e 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
@@ -35,6 +36,7 @@ import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.*;
 
 public class TestErasureCodingZones {
+  private Configuration conf;
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
   private static final int BLOCK_SIZE = 1024;
@@ -42,7 +44,7 @@ public class TestErasureCodingZones {
 
   @Before
   public void setupCluster() throws IOException {
-    Configuration conf = new HdfsConfiguration();
+    conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     cluster = new MiniDFSCluster.Builder(conf).
         numDataNodes(1).build();
@@ -150,6 +152,26 @@ public class TestErasureCodingZones {
   }
 
   @Test
+  public void testReplication() throws IOException {
+    final Path testDir = new Path("/ec");
+    fs.mkdir(testDir, FsPermission.getDirDefault());
+    fs.createErasureCodingZone(testDir, null, 0);
+    final Path fooFile = new Path(testDir, "foo");
+    // create ec file with replication=0
+    fs.create(fooFile, FsPermission.getFileDefault(), true,
+        conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short)0, fs.getDefaultBlockSize(fooFile), null);
+
+    try {
+      fs.setReplication(fooFile, (short) 3);
+      fail("Shouldn't allow to set replication to a file with striped blocks");
+    } catch (IOException e) {
+      assertExceptionContains(
+          "Cannot set replication to a file with striped blocks", e);
+    }
+  }
+
+  @Test
   public void testGetErasureCodingInfoWithSystemDefaultSchema() throws Exception {
     String src = "/ec";
     final Path ecDir = new Path(src);


[29/50] [abbrv] hadoop git commit: HADOOP-11566. Add tests and fix for erasure coders to recover erased parity units. Contributed by Kai Zheng.

Posted by zh...@apache.org.
HADOOP-11566. Add tests and fix for erasure coders to recover erased parity units. 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/b64f6745
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b64f6745
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b64f6745

Branch: refs/heads/HDFS-7285
Commit: b64f6745a45754dcf79c9c2626f3db7db2f33858
Parents: 09c3a37
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon May 18 10:13:03 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:31 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  3 ++
 .../apache/hadoop/io/erasurecode/ECChunk.java   | 17 ++++++-
 .../coder/AbstractErasureDecoder.java           | 13 ++++--
 .../hadoop/io/erasurecode/TestCoderBase.java    | 37 +++++++++++----
 .../erasurecode/coder/TestErasureCoderBase.java | 37 +++++++--------
 .../erasurecode/coder/TestRSErasureCoder.java   | 48 +++++++++++---------
 .../io/erasurecode/coder/TestXORCoder.java      |  6 +--
 .../io/erasurecode/rawcoder/TestRSRawCoder.java | 37 +++++++++------
 .../erasurecode/rawcoder/TestRawCoderBase.java  |  2 +-
 .../erasurecode/rawcoder/TestXORRawCoder.java   | 11 ++++-
 10 files changed, 134 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b64f6745/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 a152e31..34dfc9e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -48,3 +48,6 @@
     HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng via Zhe Zhang)
 
     HADOOP-11920. Refactor some codes for erasure coders. (Kai Zheng via Zhe Zhang)
+
+    HADOOP-11566. Add tests and fix for erasure coders to recover erased parity 
+    units. (Kai Zheng via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b64f6745/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 436e13e..69a8343 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,14 @@ 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;
+      } else {
+        buffers[i] = chunk.getBuffer();
+      }
     }
 
     return buffers;
@@ -75,8 +81,15 @@ public class ECChunk {
     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/b64f6745/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
index cd31294..6437236 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
@@ -60,16 +60,21 @@ public abstract class AbstractErasureDecoder extends AbstractErasureCoder {
   }
 
   /**
-   * Which blocks were erased ? We only care data blocks here. Sub-classes can
-   * override this behavior.
+   * Which blocks were erased ?
    * @param blockGroup
    * @return output blocks to recover
    */
   protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
-    ECBlock[] outputBlocks = new ECBlock[
-        getNumErasedBlocks(blockGroup.getDataBlocks())];
+    ECBlock[] outputBlocks = new ECBlock[getNumErasedBlocks(blockGroup)];
 
     int idx = 0;
+
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      if (blockGroup.getParityBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getParityBlocks()[i];
+      }
+    }
+
     for (int i = 0; i < getNumDataUnits(); i++) {
       if (blockGroup.getDataBlocks()[i].isErased()) {
         outputBlocks[idx++] = blockGroup.getDataBlocks()[i];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b64f6745/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 704b3f0..769427d 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
@@ -37,10 +37,12 @@ public abstract class TestCoderBase {
   protected int numParityUnits;
   protected int chunkSize = 16 * 1024;
 
-  // Indexes of erased data units. Will also support test of erasing
-  // parity units
+  // Indexes of erased data units.
   protected int[] erasedDataIndexes = new int[] {0};
 
+  // Indexes of erased parity units.
+  protected int[] erasedParityIndexes = new int[] {0};
+
   // Data buffers are either direct or on-heap, for performance the two cases
   // may go to different coding implementations.
   protected boolean usingDirectBuffer = true;
@@ -52,12 +54,15 @@ public abstract class TestCoderBase {
    * @param erasedDataIndexes
    */
   protected void prepare(Configuration conf, int numDataUnits,
-                         int numParityUnits, int[] erasedDataIndexes) {
+                         int numParityUnits, int[] erasedDataIndexes,
+                         int[] erasedParityIndexes) {
     this.conf = conf;
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
     this.erasedDataIndexes = erasedDataIndexes != null ?
         erasedDataIndexes : new int[] {0};
+    this.erasedParityIndexes = erasedParityIndexes != null ?
+        erasedParityIndexes : new int[] {0};
   }
 
   /**
@@ -87,10 +92,15 @@ public abstract class TestCoderBase {
    * @return erased indexes altogether
    */
   protected int[] getErasedIndexesForDecoding() {
-    int[] erasedIndexesForDecoding = new int[erasedDataIndexes.length];
+    int[] erasedIndexesForDecoding =
+        new int[erasedParityIndexes.length + erasedDataIndexes.length];
 
     int idx = 0;
 
+    for (int i = 0; i < erasedParityIndexes.length; i++) {
+      erasedIndexesForDecoding[idx ++] = erasedParityIndexes[i];
+    }
+
     for (int i = 0; i < erasedDataIndexes.length; i++) {
       erasedIndexesForDecoding[idx ++] = erasedDataIndexes[i] + numParityUnits;
     }
@@ -123,15 +133,25 @@ public abstract class TestCoderBase {
    * Erase chunks to test the recovering of them. Before erasure clone them
    * first so could return them.
    * @param dataChunks
+   * @param parityChunks
    * @return clone of erased chunks
    */
-  protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks) {
-    ECChunk[] toEraseChunks = new ECChunk[erasedDataIndexes.length];
+  protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks,
+                                      ECChunk[] parityChunks) {
+    ECChunk[] toEraseChunks = new ECChunk[erasedParityIndexes.length +
+        erasedDataIndexes.length];
 
     int idx = 0;
+    ECChunk chunk;
+
+    for (int i = 0; i < erasedParityIndexes.length; i++) {
+      chunk = parityChunks[erasedParityIndexes[i]];
+      toEraseChunks[idx ++] = cloneChunkWithData(chunk);
+      eraseDataFromChunk(chunk);
+    }
 
     for (int i = 0; i < erasedDataIndexes.length; i++) {
-      ECChunk chunk = dataChunks[erasedDataIndexes[i]];
+      chunk = dataChunks[erasedDataIndexes[i]];
       toEraseChunks[idx ++] = cloneChunkWithData(chunk);
       eraseDataFromChunk(chunk);
     }
@@ -273,7 +293,8 @@ public abstract class TestCoderBase {
    * @return
    */
   protected ECChunk[] prepareOutputChunksForDecoding() {
-    ECChunk[] chunks = new ECChunk[erasedDataIndexes.length];
+    ECChunk[] chunks = new ECChunk[erasedDataIndexes.length +
+        erasedParityIndexes.length];
 
     for (int i = 0; i < chunks.length; i++) {
       chunks[i] = allocateOutputChunk();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b64f6745/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 fdd0b50..f30323b 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
@@ -63,13 +63,15 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     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());
+    TestBlock[] clonedDataBlocks =
+        cloneBlocksWithData((TestBlock[]) blockGroup.getDataBlocks());
+    TestBlock[] parityBlocks = (TestBlock[]) blockGroup.getParityBlocks();
 
     ErasureCodingStep codingStep;
     codingStep = encoder.calculateCoding(blockGroup);
     performCodingStep(codingStep);
     // Erase specified sources but return copies of them for later comparing
-    TestBlock[] backupBlocks = backupAndEraseBlocks(clonedDataBlocks);
+    TestBlock[] backupBlocks = backupAndEraseBlocks(clonedDataBlocks, parityBlocks);
 
     // Decode
     blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
@@ -207,34 +209,27 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
    * @param dataBlocks
    * @return clone of erased dataBlocks
    */
-  protected TestBlock[] backupAndEraseBlocks(TestBlock[] dataBlocks) {
-    TestBlock[] toEraseBlocks = new TestBlock[erasedDataIndexes.length];
+  protected TestBlock[] backupAndEraseBlocks(TestBlock[] dataBlocks,
+                                             TestBlock[] parityBlocks) {
+    TestBlock[] toEraseBlocks = new TestBlock[erasedDataIndexes.length +
+                                          erasedParityIndexes.length];
 
     int idx = 0;
+    TestBlock block;
 
-    for (int i = 0; i < erasedDataIndexes.length; i++) {
-      TestBlock block = dataBlocks[erasedDataIndexes[i]];
+    for (int i = 0; i < erasedParityIndexes.length; i++) {
+      block = parityBlocks[erasedParityIndexes[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
-   * @return
-   */
-  protected TestBlock[] copyDataBlocksToErase(TestBlock[] dataBlocks) {
-    TestBlock[] copiedBlocks = new TestBlock[erasedDataIndexes.length];
-
-    for (int i = 0; i < erasedDataIndexes.length; ++i) {
-      copiedBlocks[i] = cloneBlockWithData(dataBlocks[erasedDataIndexes[i]]);
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
+      block = dataBlocks[erasedDataIndexes[i]];
+      toEraseBlocks[idx ++] = cloneBlockWithData(block);
+      eraseDataFromBlock(block);
     }
 
-    return copiedBlocks;
+    return toEraseBlocks;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b64f6745/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 3ae6a93..94f77db 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,8 +40,8 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
   }
 
   @Test
-  public void testCodingNoDirectBuffer_10x4_erasing_d0() {
-    prepare(null, 10, 4, new int[] {0});
+  public void testCodingNoDirectBuffer_10x4_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, 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.
@@ -53,34 +53,41 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
   @Test
   public void testCodingDirectBufferWithConf_10x4_erasing_d0() {
     /**
-     * This tests if the two configuration items work or not.
+     * This tests if the configuration items work or not.
      */
     Configuration conf = new Configuration();
-    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, RSRawErasureCoderFactory.class.getCanonicalName());
-
-    prepare(conf, 10, 4, new int[]{0});
+    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
+        RSRawErasureCoderFactory.class.getCanonicalName());
+    prepare(conf, 10, 4, new int[]{0}, new int[0]);
 
     testCoding(true);
     testCoding(true);
   }
-
+  
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_p1() {
+    prepare(null, 10, 4, new int[]{}, new int[]{1});
+    testCoding(true);
+    testCoding(true);
+  }
+  
   @Test
   public void testCodingDirectBuffer_10x4_erasing_d2() {
-    prepare(null, 10, 4, new int[] {2});
+    prepare(null, 10, 4, new int[] {2}, new int[] {});
     testCoding(true);
     testCoding(true);
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasing_d0() {
-    prepare(null, 10, 4, new int[] {0});
+  public void testCodingDirectBuffer_10x4_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
     testCoding(true);
     testCoding(true);
   }
 
   @Test
-  public void testCodingBothBuffers_10x4_erasing_d0() {
-    prepare(null, 10, 4, new int[] {0});
+  public void testCodingBothBuffers_10x4_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
 
     /**
      * Doing in mixed buffer usage model to test if the coders can be repeatedly
@@ -94,27 +101,26 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasure_of_d2_d4() {
-    prepare(null, 10, 4, new int[] {2, 4});
+  public void testCodingDirectBuffer_10x4_erasure_of_d2_d4_p0() {
+    prepare(null, 10, 4, new int[] {2, 4}, new int[] {0});
     testCoding(true);
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasing_d0_d1() {
-    prepare(null, 10, 4, new int[] {0, 1});
+  public void testCodingDirectBuffer_10x4_erasing_d0_d1_p0_p1() {
+    prepare(null, 10, 4, new int[] {0, 1}, new int[] {0, 1});
     testCoding(true);
   }
 
   @Test
-  public void testCodingNoDirectBuffer_3x3_erasing_d0() {
-    prepare(null, 3, 3, new int[] {0});
+  public void testCodingNoDirectBuffer_3x3_erasing_d0_p0() {
+    prepare(null, 3, 3, new int[] {0}, new int[] {0});
     testCoding(false);
   }
 
   @Test
-  public void testCodingDirectBuffer_3x3_erasing_d0() {
-    prepare(null, 3, 3, new int[] {0});
+  public void testCodingDirectBuffer_3x3_erasing_d0_p0() {
+    prepare(null, 3, 3, new int[] {0}, new int[] {0});
     testCoding(true);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b64f6745/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 87aa656..06e0087 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
@@ -36,8 +36,8 @@ public class TestXORCoder extends TestErasureCoderBase {
   }
 
   @Test
-  public void testCodingNoDirectBuffer_erasing_d0() {
-    prepare(null, 10, 1, new int[] {0});
+  public void testCodingNoDirectBuffer_erasing_p0() {
+    prepare(null, 10, 1, new int[0], new int[] {0});
 
     /**
      * Doing twice to test if the coders can be repeatedly reused. This matters
@@ -49,7 +49,7 @@ public class TestXORCoder extends TestErasureCoderBase {
 
   @Test
   public void testCodingBothBuffers_erasing_d5() {
-    prepare(null, 10, 1, new int[]{5});
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
 
     /**
      * Doing in mixed buffer usage model to test if the coders can be repeatedly

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b64f6745/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 9ba3e88..84bad92 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
@@ -32,8 +32,8 @@ public class TestRSRawCoder extends TestRSRawCoderBase {
   }
 
   @Test
-  public void testCodingNoDirectBuffer_10x4_erasing_d0() {
-    prepare(null, 10, 4, new int[] {0});
+  public void testCodingNoDirectBuffer_10x4_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, 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.
@@ -43,22 +43,29 @@ public class TestRSRawCoder extends TestRSRawCoderBase {
   }
 
   @Test
+  public void testCodingDirectBuffer_10x4_erasing_p1() {
+    prepare(null, 10, 4, new int[] {}, new int[] {1});
+    testCoding(true);
+    testCoding(true);
+  }
+
+  @Test
   public void testCodingDirectBuffer_10x4_erasing_d2() {
-    prepare(null, 10, 4, new int[] {2});
+    prepare(null, 10, 4, new int[] {2}, new int[] {});
     testCoding(true);
     testCoding(true);
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasing_d0() {
-    prepare(null, 10, 4, new int[] {0});
+  public void testCodingDirectBuffer_10x4_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
     testCoding(true);
     testCoding(true);
   }
 
   @Test
-  public void testCodingBothBuffers_10x4_erasing_d0() {
-    prepare(null, 10, 4, new int[] {0});
+  public void testCodingBothBuffers_10x4_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
 
     /**
      * Doing in mixed buffer usage model to test if the coders can be repeatedly
@@ -72,26 +79,26 @@ public class TestRSRawCoder extends TestRSRawCoderBase {
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasure_of_d2_d4() {
-    prepare(null, 10, 4, new int[] {2, 4});
+  public void testCodingDirectBuffer_10x4_erasure_of_d2_d4_p0() {
+    prepare(null, 10, 4, new int[] {2, 4}, new int[] {0});
     testCoding(true);
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasing_d0_d1() {
-    prepare(null, 10, 4, new int[] {0, 1});
+  public void testCodingDirectBuffer_10x4_erasing_d0_d1_p0_p1() {
+    prepare(null, 10, 4, new int[] {0, 1}, new int[] {0, 1});
     testCoding(true);
   }
 
   @Test
-  public void testCodingNoDirectBuffer_3x3_erasing_d0() {
-    prepare(null, 3, 3, new int[] {0});
+  public void testCodingNoDirectBuffer_3x3_erasing_d0_p0() {
+    prepare(null, 3, 3, new int[] {0}, new int[] {0});
     testCoding(false);
   }
 
   @Test
-  public void testCodingDirectBuffer_3x3_erasing_d0() {
-    prepare(null, 3, 3, new int[] {0});
+  public void testCodingDirectBuffer_3x3_erasing_d0_p0() {
+    prepare(null, 3, 3, new int[] {0}, new int[] {0});
     testCoding(true);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b64f6745/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 7ba320a..8543c4d 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
@@ -52,7 +52,7 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     encoder.encode(dataChunks, parityChunks);
 
     // Backup and erase some chunks
-    ECChunk[] backupChunks = backupAndEraseChunks(clonedDataChunks);
+    ECChunk[] backupChunks = backupAndEraseChunks(clonedDataChunks, parityChunks);
 
     // Decode
     ECChunk[] inputChunks = prepareInputChunksForDecoding(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b64f6745/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 62ce4fb..b8912a9 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
@@ -36,7 +36,7 @@ public class TestXORRawCoder extends TestRawCoderBase {
 
   @Test
   public void testCodingNoDirectBuffer_erasing_d0() {
-    prepare(null, 10, 1, new int[] {0});
+    prepare(null, 10, 1, new int[] {0}, new int[0]);
 
     /**
      * Doing twice to test if the coders can be repeatedly reused. This matters
@@ -47,8 +47,15 @@ public class TestXORRawCoder extends TestRawCoderBase {
   }
 
   @Test
+  public void testCodingDirectBuffer_erasing_p0() {
+    prepare(null, 10, 1, new int[0], new int[] {0});
+    testCoding(true);
+    testCoding(true);
+  }
+
+  @Test
   public void testCodingBothBuffers_erasing_d5() {
-    prepare(null, 10, 1, new int[]{5});
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
 
     /**
      * Doing in mixed buffer usage model to test if the coders can be repeatedly


[49/50] [abbrv] hadoop git commit: HADOOP-11847 Enhance raw coder allowing to read least required inputs in decoding. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11847 Enhance raw coder allowing to read least required inputs in decoding. 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/4ad48488
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4ad48488
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4ad48488

Branch: refs/heads/HDFS-7285
Commit: 4ad484883f773c702a1874fc12816ef1a4a54136
Parents: 5a391e1
Author: Kai Zheng <ka...@intel.com>
Authored: Tue May 26 22:45:19 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:07:16 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |   3 +
 .../rawcoder/AbstractRawErasureCoder.java       |  27 +++-
 .../rawcoder/AbstractRawErasureDecoder.java     |  75 +++++++--
 .../rawcoder/AbstractRawErasureEncoder.java     |   8 +-
 .../io/erasurecode/rawcoder/RSRawDecoder.java   | 162 ++++++++++++++++++-
 .../erasurecode/rawcoder/RawErasureDecoder.java |  20 ++-
 .../io/erasurecode/rawcoder/XORRawDecoder.java  |   2 +-
 .../io/erasurecode/rawcoder/XORRawEncoder.java  |   2 +-
 .../erasurecode/rawcoder/util/GaloisField.java  |  12 +-
 .../hadoop/io/erasurecode/TestCoderBase.java    |  39 ++---
 .../erasurecode/coder/TestErasureCoderBase.java |   1 -
 .../io/erasurecode/rawcoder/TestRSRawCoder.java | 101 ++++++------
 .../erasurecode/rawcoder/TestRawCoderBase.java  |  54 +++++++
 .../erasurecode/rawcoder/TestXORRawCoder.java   |  45 ++----
 14 files changed, 397 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 c9b80d3..0c24473 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -59,3 +59,6 @@
 
     HADOOP-12029. Remove chunkSize from ECSchema as its not required for coders
     (vinayakumarb)
+
+    HADOOP-11847. Enhance raw coder allowing to read least required inputs in decoding.
+    (Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 06ae660..e6a1542 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
@@ -60,12 +60,13 @@ public abstract class AbstractRawErasureCoder
   }
 
   /**
-   * Ensure output buffer filled with ZERO bytes fully in chunkSize.
-   * @param buffer a buffer ready to write chunk size bytes
+   * Ensure a buffer filled with ZERO bytes from current readable/writable
+   * position.
+   * @param buffer a buffer ready to read / write certain size bytes
    * @return the buffer itself, with ZERO bytes written, the position and limit
    *         are not changed after the call
    */
-  protected ByteBuffer resetOutputBuffer(ByteBuffer buffer) {
+  protected ByteBuffer resetBuffer(ByteBuffer buffer) {
     int pos = buffer.position();
     for (int i = pos; i < buffer.limit(); ++i) {
       buffer.put((byte) 0);
@@ -77,7 +78,7 @@ public abstract class AbstractRawErasureCoder
 
   /**
    * Ensure the buffer (either input or output) ready to read or write with ZERO
-   * bytes fully in chunkSize.
+   * bytes fully in specified length of len.
    * @param buffer bytes array buffer
    * @return the buffer itself
    */
@@ -92,11 +93,16 @@ public abstract class AbstractRawErasureCoder
   /**
    * Check and ensure the buffers are of the length specified by dataLen.
    * @param buffers
+   * @param allowNull
    * @param dataLen
    */
-  protected void ensureLength(ByteBuffer[] buffers, int dataLen) {
+  protected void ensureLength(ByteBuffer[] buffers,
+                              boolean allowNull, int dataLen) {
     for (int i = 0; i < buffers.length; ++i) {
-      if (buffers[i].remaining() != dataLen) {
+      if (buffers[i] == null && !allowNull) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer found, not allowing null");
+      } else if (buffers[i] != null && buffers[i].remaining() != dataLen) {
         throw new HadoopIllegalArgumentException(
             "Invalid buffer, not of length " + dataLen);
       }
@@ -106,11 +112,16 @@ public abstract class AbstractRawErasureCoder
   /**
    * Check and ensure the buffers are of the length specified by dataLen.
    * @param buffers
+   * @param allowNull
    * @param dataLen
    */
-  protected void ensureLength(byte[][] buffers, int dataLen) {
+  protected void ensureLength(byte[][] buffers,
+                              boolean allowNull, int dataLen) {
     for (int i = 0; i < buffers.length; ++i) {
-      if (buffers[i].length != dataLen) {
+      if (buffers[i] == null && !allowNull) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer found, not allowing null");
+      } else if (buffers[i] != null && buffers[i].length != dataLen) {
         throw new HadoopIllegalArgumentException(
             "Invalid buffer not of length " + dataLen);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 0c1f80f..c6105b0 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
@@ -21,6 +21,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.io.erasurecode.ECChunk;
 
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 
 /**
  * An abstract raw erasure decoder that's to be inherited by new decoders.
@@ -38,14 +39,16 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs) {
     checkParameters(inputs, erasedIndexes, outputs);
-    int dataLen = inputs[0].remaining();
+
+    ByteBuffer validInput = findFirstValidInput(inputs);
+    int dataLen = validInput.remaining();
     if (dataLen == 0) {
       return;
     }
-    ensureLength(inputs, dataLen);
-    ensureLength(outputs, dataLen);
+    ensureLength(inputs, true, dataLen);
+    ensureLength(outputs, false, dataLen);
 
-    boolean usingDirectBuffer = inputs[0].isDirect();
+    boolean usingDirectBuffer = validInput.isDirect();
     if (usingDirectBuffer) {
       doDecode(inputs, erasedIndexes, outputs);
       return;
@@ -59,8 +62,10 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
     ByteBuffer buffer;
     for (int i = 0; i < inputs.length; ++i) {
       buffer = inputs[i];
-      inputOffsets[i] = buffer.position();
-      newInputs[i] = buffer.array();
+      if (buffer != null) {
+        inputOffsets[i] = buffer.position();
+        newInputs[i] = buffer.array();
+      }
     }
 
     for (int i = 0; i < outputs.length; ++i) {
@@ -74,7 +79,10 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
 
     for (int i = 0; i < inputs.length; ++i) {
       buffer = inputs[i];
-      buffer.position(inputOffsets[i] + dataLen); // dataLen bytes consumed
+      if (buffer != null) {
+        // dataLen bytes consumed
+        buffer.position(inputOffsets[i] + dataLen);
+      }
     }
   }
 
@@ -90,12 +98,14 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   @Override
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) {
     checkParameters(inputs, erasedIndexes, outputs);
-    int dataLen = inputs[0].length;
+
+    byte[] validInput = findFirstValidInput(inputs);
+    int dataLen = validInput.length;
     if (dataLen == 0) {
       return;
     }
-    ensureLength(inputs, dataLen);
-    ensureLength(outputs, dataLen);
+    ensureLength(inputs, true, dataLen);
+    ensureLength(outputs, false, dataLen);
 
     int[] inputOffsets = new int[inputs.length]; // ALL ZERO
     int[] outputOffsets = new int[outputs.length]; // ALL ZERO
@@ -148,5 +158,50 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
       throw new HadoopIllegalArgumentException(
           "Too many erased, not recoverable");
     }
+
+    int validInputs = 0;
+    for (int i = 0; i < inputs.length; ++i) {
+      if (inputs[i] != null) {
+        validInputs += 1;
+      }
+    }
+
+    if (validInputs < getNumDataUnits()) {
+      throw new HadoopIllegalArgumentException(
+          "No enough valid inputs are provided, not recoverable");
+    }
+  }
+
+  /**
+   * Get indexes into inputs array for items marked as null, either erased or
+   * not to read.
+   * @return indexes into inputs array
+   */
+  protected int[] getErasedOrNotToReadIndexes(Object[] inputs) {
+    int[] invalidIndexes = new int[inputs.length];
+    int idx = 0;
+    for (int i = 0; i < inputs.length; i++) {
+      if (inputs[i] == null) {
+        invalidIndexes[idx++] = i;
+      }
+    }
+
+    return Arrays.copyOf(invalidIndexes, idx);
+  }
+
+  /**
+   * Find the valid input from all the inputs.
+   * @param inputs
+   * @return the first valid input
+   */
+  protected static <T> T findFirstValidInput(T[] inputs) {
+    for (int i = 0; i < inputs.length; i++) {
+      if (inputs[i] != null) {
+        return inputs[i];
+      }
+    }
+
+    throw new HadoopIllegalArgumentException(
+        "Invalid inputs are found, all being null");
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 c7a136b..d1faa8c 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
@@ -41,8 +41,8 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
     if (dataLen == 0) {
       return;
     }
-    ensureLength(inputs, dataLen);
-    ensureLength(outputs, dataLen);
+    ensureLength(inputs, false, dataLen);
+    ensureLength(outputs, false, dataLen);
 
     boolean usingDirectBuffer = inputs[0].isDirect();
     if (usingDirectBuffer) {
@@ -90,8 +90,8 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
     if (dataLen == 0) {
       return;
     }
-    ensureLength(inputs, dataLen);
-    ensureLength(outputs, dataLen);
+    ensureLength(inputs, false, dataLen);
+    ensureLength(outputs, false, dataLen);
 
     int[] inputOffsets = new int[inputs.length]; // ALL ZERO
     int[] outputOffsets = new int[outputs.length]; // ALL ZERO

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
index e265dce..57e6957 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
 
 import java.nio.ByteBuffer;
@@ -25,35 +26,64 @@ import java.nio.ByteBuffer;
  * A raw erasure decoder in RS code scheme in pure Java in case native one
  * isn't available in some environment. Please always use native implementations
  * when possible.
+ *
+ * Currently this implementation will compute and decode not to read units
+ * unnecessarily due to the underlying implementation limit in GF. This will be
+ * addressed in HADOOP-11871.
  */
 public class RSRawDecoder extends AbstractRawErasureDecoder {
   // To describe and calculate the needed Vandermonde matrix
   private int[] errSignature;
   private int[] primitivePower;
 
+  /**
+   * We need a set of reusable buffers either for the bytes array
+   * decoding version or direct buffer decoding version. Normally not both.
+   *
+   * For output, in addition to the valid buffers from the caller
+   * passed from above, we need to provide extra buffers for the internal
+   * decoding implementation. For output, the caller should provide no more
+   * than numParityUnits but at least one buffers. And the left buffers will be
+   * borrowed from either bytesArrayBuffers, for the bytes array version.
+   *
+   */
+  // Reused buffers for decoding with bytes arrays
+  private byte[][] bytesArrayBuffers = new byte[getNumParityUnits()][];
+  private byte[][] adjustedByteArrayOutputsParameter =
+      new byte[getNumParityUnits()][];
+  private int[] adjustedOutputOffsets = new int[getNumParityUnits()];
+
+  // Reused buffers for decoding with direct ByteBuffers
+  private ByteBuffer[] directBuffers = new ByteBuffer[getNumParityUnits()];
+  private ByteBuffer[] adjustedDirectBufferOutputsParameter =
+      new ByteBuffer[getNumParityUnits()];
+
   public RSRawDecoder(int numDataUnits, int numParityUnits) {
     super(numDataUnits, numParityUnits);
-    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
+    if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
+      throw new HadoopIllegalArgumentException(
+              "Invalid numDataUnits and numParityUnits");
+    }
 
     this.errSignature = new int[numParityUnits];
     this.primitivePower = RSUtil.getPrimitivePower(numDataUnits,
         numParityUnits);
   }
 
-  @Override
-  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+  private void doDecodeImpl(ByteBuffer[] inputs, int[] erasedIndexes,
                           ByteBuffer[] outputs) {
+    ByteBuffer valid = findFirstValidInput(inputs);
+    int dataLen = valid.remaining();
     for (int i = 0; i < erasedIndexes.length; i++) {
       errSignature[i] = primitivePower[erasedIndexes[i]];
-      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
+      RSUtil.GF.substitute(inputs, dataLen, outputs[i], primitivePower[i]);
     }
 
     RSUtil.GF.solveVandermondeSystem(errSignature,
         outputs, erasedIndexes.length);
   }
 
-  @Override
-  protected void doDecode(byte[][] inputs, int[] inputOffsets,
+  private void doDecodeImpl(byte[][] inputs, int[] inputOffsets,
                           int dataLen, int[] erasedIndexes,
                           byte[][] outputs, int[] outputOffsets) {
     for (int i = 0; i < erasedIndexes.length; i++) {
@@ -63,6 +93,124 @@ public class RSRawDecoder extends AbstractRawErasureDecoder {
     }
 
     RSUtil.GF.solveVandermondeSystem(errSignature, outputs, outputOffsets,
-        erasedIndexes.length, dataLen);
+            erasedIndexes.length, dataLen);
+  }
+
+  @Override
+  protected void doDecode(byte[][] inputs, int[] inputOffsets,
+                          int dataLen, int[] erasedIndexes,
+                          byte[][] outputs, int[] outputOffsets) {
+    /**
+     * As passed parameters are friendly to callers but not to the underlying
+     * implementations, so we have to adjust them before calling doDecodeImpl.
+     */
+
+    int[] erasedOrNotToReadIndexes = getErasedOrNotToReadIndexes(inputs);
+
+    // Prepare for adjustedOutputsParameter
+
+    // First reset the positions needed this time
+    for (int i = 0; i < erasedOrNotToReadIndexes.length; i++) {
+      adjustedByteArrayOutputsParameter[i] = null;
+      adjustedOutputOffsets[i] = 0;
+    }
+    // Use the caller passed buffers in erasedIndexes positions
+    for (int outputIdx = 0, i = 0; i < erasedIndexes.length; i++) {
+      boolean found = false;
+      for (int j = 0; j < erasedOrNotToReadIndexes.length; j++) {
+        // If this index is one requested by the caller via erasedIndexes, then
+        // we use the passed output buffer to avoid copying data thereafter.
+        if (erasedIndexes[i] == erasedOrNotToReadIndexes[j]) {
+          found = true;
+          adjustedByteArrayOutputsParameter[j] = resetBuffer(
+                  outputs[outputIdx], outputOffsets[outputIdx], dataLen);
+          adjustedOutputOffsets[j] = outputOffsets[outputIdx];
+          outputIdx++;
+        }
+      }
+      if (!found) {
+        throw new HadoopIllegalArgumentException(
+            "Inputs not fully corresponding to erasedIndexes in null places");
+      }
+    }
+    // Use shared buffers for other positions (not set yet)
+    for (int bufferIdx = 0, i = 0; i < erasedOrNotToReadIndexes.length; i++) {
+      if (adjustedByteArrayOutputsParameter[i] == null) {
+        adjustedByteArrayOutputsParameter[i] = resetBuffer(
+            checkGetBytesArrayBuffer(bufferIdx, dataLen), 0, dataLen);
+        adjustedOutputOffsets[i] = 0; // Always 0 for such temp output
+        bufferIdx++;
+      }
+    }
+
+    doDecodeImpl(inputs, inputOffsets, dataLen, erasedOrNotToReadIndexes,
+            adjustedByteArrayOutputsParameter, adjustedOutputOffsets);
+  }
+
+  @Override
+  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                          ByteBuffer[] outputs) {
+    ByteBuffer validInput = findFirstValidInput(inputs);
+    int dataLen = validInput.remaining();
+
+    /**
+     * As passed parameters are friendly to callers but not to the underlying
+     * implementations, so we have to adjust them before calling doDecodeImpl.
+     */
+
+    int[] erasedOrNotToReadIndexes = getErasedOrNotToReadIndexes(inputs);
+
+    // Prepare for adjustedDirectBufferOutputsParameter
+
+    // First reset the positions needed this time
+    for (int i = 0; i < erasedOrNotToReadIndexes.length; i++) {
+      adjustedDirectBufferOutputsParameter[i] = null;
+    }
+    // Use the caller passed buffers in erasedIndexes positions
+    for (int outputIdx = 0, i = 0; i < erasedIndexes.length; i++) {
+      boolean found = false;
+      for (int j = 0; j < erasedOrNotToReadIndexes.length; j++) {
+        // If this index is one requested by the caller via erasedIndexes, then
+        // we use the passed output buffer to avoid copying data thereafter.
+        if (erasedIndexes[i] == erasedOrNotToReadIndexes[j]) {
+          found = true;
+          adjustedDirectBufferOutputsParameter[j] =
+              resetBuffer(outputs[outputIdx++]);
+        }
+      }
+      if (!found) {
+        throw new HadoopIllegalArgumentException(
+            "Inputs not fully corresponding to erasedIndexes in null places");
+      }
+    }
+    // Use shared buffers for other positions (not set yet)
+    for (int bufferIdx = 0, i = 0; i < erasedOrNotToReadIndexes.length; i++) {
+      if (adjustedDirectBufferOutputsParameter[i] == null) {
+        ByteBuffer buffer = checkGetDirectBuffer(bufferIdx, dataLen);
+        buffer.position(0);
+        buffer.limit(dataLen);
+        adjustedDirectBufferOutputsParameter[i] = resetBuffer(buffer);
+        bufferIdx++;
+      }
+    }
+
+    doDecodeImpl(inputs, erasedOrNotToReadIndexes,
+        adjustedDirectBufferOutputsParameter);
+  }
+
+  private byte[] checkGetBytesArrayBuffer(int idx, int bufferLen) {
+    if (bytesArrayBuffers[idx] == null ||
+            bytesArrayBuffers[idx].length < bufferLen) {
+      bytesArrayBuffers[idx] = new byte[bufferLen];
+    }
+    return bytesArrayBuffers[idx];
+  }
+
+  private ByteBuffer checkGetDirectBuffer(int idx, int bufferLen) {
+    if (directBuffers[idx] == null ||
+        directBuffers[idx].capacity() < bufferLen) {
+      directBuffers[idx] = ByteBuffer.allocateDirect(bufferLen);
+    }
+    return directBuffers[idx];
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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..ad7f32d 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
@@ -32,6 +32,22 @@ public interface RawErasureDecoder extends RawErasureCoder {
 
   /**
    * Decode with inputs and erasedIndexes, generates outputs.
+   * How to prepare for inputs:
+   * 1. Create an array containing parity units + data units;
+   * 2. Set null in the array locations specified via erasedIndexes to indicate
+   *    they're erased and no data are to read from;
+   * 3. Set null in the array locations for extra redundant items, as they're
+   *    not necessary to read when decoding. For example in RS-6-3, if only 1
+   *    unit is really erased, then we have 2 extra items as redundant. They can
+   *    be set as null to indicate no data will be used from them.
+   *
+   * For an example using RS (6, 3), assuming sources (d0, d1, d2, d3, d4, d5)
+   * and parities (p0, p1, p2), d2 being erased. We can and may want to use only
+   * 6 units like (d1, d3, d4, d5, p0, p2) to recover d2. We will have:
+   *     inputs = [p0, null(p1), p2, null(d0), d1, null(d2), d3, d4, d5]
+   *     erasedIndexes = [5] // index of d2 into inputs array
+   *     outputs = [a-writable-buffer]
+   *
    * @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
@@ -41,7 +57,7 @@ public interface RawErasureDecoder extends RawErasureCoder {
                      ByteBuffer[] outputs);
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs.
+   * Decode with inputs and erasedIndexes, generates outputs. More see above.
    * @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
@@ -50,7 +66,7 @@ public interface RawErasureDecoder extends RawErasureCoder {
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs);
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs.
+   * Decode with inputs and erasedIndexes, generates outputs. More see above.
    * @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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 a09105c..e20e543 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
@@ -36,7 +36,7 @@ public class XORRawDecoder extends AbstractRawErasureDecoder {
   protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
                           ByteBuffer[] outputs) {
     ByteBuffer output = outputs[0];
-    resetOutputBuffer(output);
+    resetBuffer(output);
 
     int erasedIdx = erasedIndexes[0];
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 894f20c..f4d242e 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
@@ -34,7 +34,7 @@ public class XORRawEncoder extends AbstractRawErasureEncoder {
 
   protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
     ByteBuffer output = outputs[0];
-    resetOutputBuffer(output);
+    resetBuffer(output);
 
     // Get the first buffer's data.
     int iIdx, oIdx;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
index 62b22c9..03683b0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
@@ -423,7 +423,7 @@ public class GaloisField {
       byte[] pi = p[i];
       for (iIdx = offsets[i], oIdx = offset;
            iIdx < offsets[i] + len; iIdx++, oIdx++) {
-        int pij = pi[iIdx] & 0x000000FF;
+        int pij = pi != null ? pi[iIdx] & 0x000000FF : 0;
         q[oIdx] = (byte) (q[oIdx] ^ mulTable[pij][y]);
       }
       y = mulTable[x][y];
@@ -438,13 +438,15 @@ public class GaloisField {
    * @param q store the return result
    * @param x input field
    */
-  public void substitute(ByteBuffer[] p, ByteBuffer q, int x) {
+  public void substitute(ByteBuffer[] p, int len, ByteBuffer q, int x) {
     int y = 1, iIdx, oIdx;
     for (int i = 0; i < p.length; i++) {
       ByteBuffer pi = p[i];
-      for (iIdx = pi.position(), oIdx = q.position();
-           iIdx < pi.limit(); iIdx++, oIdx++) {
-        int pij = pi.get(iIdx) & 0x000000FF;
+      int pos = pi != null ? pi.position() : 0;
+      int limit = pi != null ? pi.limit() : len;
+      for (oIdx = q.position(), iIdx = pos;
+           iIdx < limit; iIdx++, oIdx++) {
+        int pij = pi != null ? pi.get(iIdx) & 0x000000FF : 0;
         q.put(oIdx, (byte) (q.get(oIdx) ^ mulTable[pij][y]));
       }
       y = mulTable[x][y];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 3686695..9f50f33 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
@@ -35,7 +35,7 @@ public abstract class TestCoderBase {
   private Configuration conf;
   protected int numDataUnits;
   protected int numParityUnits;
-  protected int baseChunkSize = 16 * 1024;
+  protected int baseChunkSize = 513;
   private int chunkSize = baseChunkSize;
 
   private byte[] zeroChunkBytes;
@@ -186,8 +186,9 @@ public abstract class TestCoderBase {
   }
 
   /**
-   * Erase chunks to test the recovering of them. Before erasure clone them
-   * first so could return them.
+   * Erase some data chunks to test the recovering of them. As they're erased,
+   * we don't need to read them and will not have the buffers at all, so just
+   * set them as null.
    * @param dataChunks
    * @param parityChunks
    * @return clone of erased chunks
@@ -198,51 +199,31 @@ public abstract class TestCoderBase {
         erasedDataIndexes.length];
 
     int idx = 0;
-    ECChunk chunk;
 
     for (int i = 0; i < erasedParityIndexes.length; i++) {
-      chunk = parityChunks[erasedParityIndexes[i]];
-      toEraseChunks[idx ++] = cloneChunkWithData(chunk);
-      eraseDataFromChunk(chunk);
+      toEraseChunks[idx ++] = parityChunks[erasedParityIndexes[i]];
+      parityChunks[erasedParityIndexes[i]] = null;
     }
 
     for (int i = 0; i < erasedDataIndexes.length; i++) {
-      chunk = dataChunks[erasedDataIndexes[i]];
-      toEraseChunks[idx ++] = cloneChunkWithData(chunk);
-      eraseDataFromChunk(chunk);
+      toEraseChunks[idx ++] = dataChunks[erasedDataIndexes[i]];
+      dataChunks[erasedDataIndexes[i]] = null;
     }
 
     return toEraseChunks;
   }
 
   /**
-   * Erase data from the specified chunks, putting ZERO bytes to the buffers.
+   * Erase data from the specified chunks, just setting them as null.
    * @param chunks
    */
   protected void eraseDataFromChunks(ECChunk[] chunks) {
     for (int i = 0; i < chunks.length; i++) {
-      eraseDataFromChunk(chunks[i]);
+      chunks[i] = null;
     }
   }
 
   /**
-   * Erase data from the specified chunk, putting ZERO bytes to the buffer.
-   * @param chunk with a buffer ready to read at the current position
-   */
-  protected void eraseDataFromChunk(ECChunk chunk) {
-    ByteBuffer chunkBuffer = chunk.getBuffer();
-    // Erase the data at the position, and restore the buffer ready for reading
-    // same many bytes but all ZERO.
-    int pos = chunkBuffer.position();
-    int len = chunkBuffer.remaining();
-    chunkBuffer.put(zeroChunkBytes, 0, len);
-    // Back to readable again after data erased
-    chunkBuffer.flip();
-    chunkBuffer.position(pos);
-    chunkBuffer.limit(pos + len);
-  }
-
-  /**
    * Clone chunks along with copying the associated data. It respects how the
    * chunk buffer is allocated, direct or non-direct. It avoids affecting the
    * original chunk buffers.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 f9666b6..98fa956 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
@@ -232,7 +232,6 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
                                              TestBlock[] parityBlocks) {
     TestBlock[] toEraseBlocks = new TestBlock[erasedDataIndexes.length +
                                           erasedParityIndexes.length];
-
     int idx = 0;
     TestBlock block;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 80ec04d..7b7ea42 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
@@ -32,89 +32,86 @@ public class TestRSRawCoder extends TestRSRawCoderBase {
   }
 
   @Test
-  public void testCodingNoDirectBuffer_10x4_erasing_d0_p0() {
-    prepare(null, 10, 4, new int[] {0}, 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);
+  public void testCoding_6x3_erasing_all_d() {
+    prepare(null, 6, 3, new int[]{0, 1, 2}, new int[0], true);
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasing_p1() {
-    prepare(null, 10, 4, new int[0], new int[] {1});
-    testCoding(true);
-    testCoding(true);
+  public void testCoding_6x3_erasing_d0_d2() {
+    prepare(null, 6, 3, new int[] {0, 2}, new int[]{});
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasing_d2() {
-    prepare(null, 10, 4, new int[] {2}, new int[] {});
-    testCoding(true);
-    testCoding(true);
+  public void testCoding_6x3_erasing_d0() {
+    prepare(null, 6, 3, new int[]{0}, new int[0]);
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasing_d0_p0() {
-    prepare(null, 10, 4, new int[] {0}, new int[] {0});
-    testCoding(true);
-    testCoding(true);
+  public void testCoding_6x3_erasing_d2() {
+    prepare(null, 6, 3, new int[]{2}, new int[]{});
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingBothBuffers_10x4_erasing_d0_p0() {
-    prepare(null, 10, 4, new int[] {0}, new int[] {0});
+  public void testCoding_6x3_erasing_d0_p0() {
+    prepare(null, 6, 3, new int[]{0}, new int[]{0});
+    testCodingDoMixAndTwice();
+  }
 
-    /**
-     * 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 testCoding_6x3_erasing_all_p() {
+    prepare(null, 6, 3, new int[0], new int[]{0, 1, 2});
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasure_of_d2_d4_p0() {
-    prepare(null, 10, 4, new int[]{2, 4}, new int[]{0});
-    testCoding(true);
+  public void testCoding_6x3_erasing_p0() {
+    prepare(null, 6, 3, new int[0], new int[]{0});
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingDirectBuffer_usingFixedData_10x4_erasure_of_d2_d4_p0() {
-    prepare(null, 10, 4, new int[] {2, 4}, new int[] {0}, true);
-    testCoding(true);
+  public void testCoding_6x3_erasing_p2() {
+    prepare(null, 6, 3, new int[0], new int[]{2});
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingDirectBuffer_10x4_erasing_d0_d1_p0_p1() {
-    prepare(null, 10, 4, new int[] {0, 1}, new int[] {0, 1});
-    testCoding(true);
+  public void testCoding_6x3_erasure_p0_p2() {
+    prepare(null, 6, 3, new int[0], new int[]{0, 2});
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingNoDirectBuffer_3x3_erasing_d0_p0() {
-    prepare(null, 3, 3, new int[] {0}, new int[] {0});
-    testCoding(false);
+  public void testCoding_6x3_erasing_d0_p0_p1() {
+    prepare(null, 6, 3, new int[]{0}, new int[]{0, 1});
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingDirectBuffer_3x3_erasing_d0_p0() {
-    prepare(null, 3, 3, new int[] {0}, new int[] {0});
-    testCoding(true);
+  public void testCoding_6x3_erasing_d0_d2_p2() {
+    prepare(null, 6, 3, new int[]{0, 2}, new int[]{2});
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingNegative_10x4_erasing_d2_d4() {
-    prepare(null, 10, 4, new int[]{2, 4}, new int[0]);
+  public void testCodingNegative_6x3_erasing_d2_d4() {
+    prepare(null, 6, 3, new int[]{2, 4}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
 
-    testCodingWithBadInput(true);
-    testCodingWithBadOutput(false);
-    testCodingWithBadInput(true);
-    testCodingWithBadOutput(false);
+  @Test
+  public void testCodingNegative_6x3_erasing_too_many() {
+    prepare(null, 6, 3, new int[]{2, 4}, new int[]{0, 1});
+    testCodingWithErasingTooMany();
+  }
+
+  @Test
+  public void testCoding_10x4_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
+    testCodingDoMixAndTwice();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 cfaa2c5..dd5452b 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 import org.apache.hadoop.io.erasurecode.ECChunk;
 import org.apache.hadoop.io.erasurecode.TestCoderBase;
 import org.junit.Assert;
+import org.junit.Test;
 
 import java.lang.reflect.Constructor;
 
@@ -33,6 +34,25 @@ public abstract class TestRawCoderBase extends TestCoderBase {
   private RawErasureDecoder decoder;
 
   /**
+   * Doing twice to test if the coders can be repeatedly reused. This matters
+   * as the underlying coding buffers are shared, which may have bugs.
+   */
+  protected void testCodingDoMixAndTwice() {
+    testCodingDoMixed();
+    testCodingDoMixed();
+  }
+
+  /**
+   * 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.
+   */
+  protected void testCodingDoMixed() {
+    testCoding(true);
+    testCoding(false);
+  }
+
+  /**
    * Generating source data, encoding, recovering and then verifying.
    * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
    * it supports two kinds of ByteBuffers, one is array backed, the other is
@@ -85,6 +105,23 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     }
   }
 
+  @Test
+  public void testCodingWithErasingTooMany() {
+    try {
+      testCoding(true);
+      Assert.fail("Decoding test erasing too many should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+
+    try {
+      testCoding(false);
+      Assert.fail("Decoding test erasing too many should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
   private void performTestCoding(int chunkSize,
                                  boolean useBadInput, boolean useBadOutput) {
     setChunkSize(chunkSize);
@@ -110,6 +147,9 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     ECChunk[] inputChunks = prepareInputChunksForDecoding(
         clonedDataChunks, parityChunks);
 
+    // Remove unnecessary chunks, allowing only least required chunks to be read.
+    ensureOnlyLeastRequiredChunks(inputChunks);
+
     ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
     if (useBadOutput) {
       corruptSomeChunk(recoveredChunks);
@@ -131,6 +171,20 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     }
   }
 
+  private void ensureOnlyLeastRequiredChunks(ECChunk[] inputChunks) {
+    int leastRequiredNum = numDataUnits;
+    int erasedNum = erasedDataIndexes.length + erasedParityIndexes.length;
+    int goodNum = inputChunks.length - erasedNum;
+    int redundantNum = goodNum - leastRequiredNum;
+
+    for (int i = 0; i < inputChunks.length && redundantNum > 0; i++) {
+      if (inputChunks[i] != null) {
+        inputChunks[i] = null; // Setting it null, not needing it actually
+        redundantNum--;
+      }
+    }
+  }
+
   /**
    * Create the raw erasure encoder to test
    * @return

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad48488/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 327174e..48463ad 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
@@ -29,58 +29,35 @@ public class TestXORRawCoder extends TestRawCoderBase {
   public void setup() {
     this.encoderClass = XORRawEncoder.class;
     this.decoderClass = XORRawDecoder.class;
-
-    this.numDataUnits = 10;
-    this.numParityUnits = 1;
   }
 
   @Test
-  public void testCodingNoDirectBuffer_erasing_d0() {
+  public void testCoding_10x1_erasing_d0() {
     prepare(null, 10, 1, new int[] {0}, 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);
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingDirectBuffer_erasing_p0() {
+  public void testCoding_10x1_erasing_p0() {
     prepare(null, 10, 1, new int[0], new int[] {0});
-
-    testCoding(true);
-    testCoding(true);
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingDirectBuffer_erasing_d0() {
-    prepare(null, 10, 1, new int[] {0}, new int[0]);
-
-    testCoding(true);
-    testCoding(true);
+  public void testCoding_10x1_erasing_d5() {
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
+    testCodingDoMixAndTwice();
   }
 
   @Test
-  public void testCodingBothBuffers_erasing_d5() {
-    prepare(null, 10, 1, new int[]{5}, 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);
+  public void testCodingNegative_10x1_erasing_too_many() {
+    prepare(null, 10, 1, new int[]{2}, new int[]{0});
+    testCodingWithErasingTooMany();
   }
 
   @Test
-  public void testCodingNegative_erasing_d5() {
+  public void testCodingNegative_10x1_erasing_d5() {
     prepare(null, 10, 1, new int[]{5}, new int[0]);
-
     testCodingWithBadInput(true);
     testCodingWithBadOutput(false);
     testCodingWithBadInput(true);


[35/50] [abbrv] hadoop git commit: HDFS-8375. Add cellSize as an XAttr to ECZone. Contributed by Vinayakumar B.

Posted by zh...@apache.org.
HDFS-8375. Add cellSize as an XAttr to ECZone. 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/91c81fdc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/91c81fdc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/91c81fdc

Branch: refs/heads/HDFS-7285
Commit: 91c81fdc24709b3caf1f6281c8879ffee08db956
Parents: 9145809
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue May 19 13:58:50 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:33 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsFileStatus.java    | 14 ++++--
 .../protocol/SnapshottableDirectoryStatus.java  |  2 +-
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |  2 +-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  8 ++--
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  7 +--
 .../hadoop/hdfs/DFSStripedInputStream.java      | 10 ++--
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  2 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |  9 ++--
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  2 +-
 .../hdfs/protocol/ErasureCodingZoneInfo.java    | 14 +++++-
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |  5 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |  3 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  5 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 15 ++++--
 .../server/blockmanagement/BlockManager.java    | 12 +++--
 .../blockmanagement/DatanodeDescriptor.java     |  4 +-
 .../erasurecode/ErasureCodingWorker.java        |  2 +-
 .../namenode/ErasureCodingZoneManager.java      | 40 +++++++++++++---
 .../server/namenode/FSDirStatAndListingOp.java  | 21 ++++++---
 .../hdfs/server/namenode/FSDirectory.java       |  4 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 49 +++++++-------------
 .../hdfs/server/namenode/NameNodeRpcServer.java |  5 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |  8 ++--
 .../server/protocol/BlockECRecoveryCommand.java | 26 ++++++-----
 .../hdfs/tools/erasurecode/ECCommand.java       | 14 +++++-
 .../hadoop/hdfs/util/StripedBlockUtil.java      | 49 +++++++++-----------
 .../src/main/proto/erasurecoding.proto          |  3 ++
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |  3 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  2 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |  4 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  | 10 ++--
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |  2 +-
 .../TestDFSStripedOutputStreamWithFailure.java  |  2 +-
 .../apache/hadoop/hdfs/TestEncryptionZones.java |  2 +-
 .../hadoop/hdfs/TestErasureCodingZones.java     | 18 +++----
 .../hadoop/hdfs/TestFileStatusWithECschema.java |  2 +-
 .../java/org/apache/hadoop/hdfs/TestLease.java  |  4 +-
 .../hadoop/hdfs/TestRecoverStripedFile.java     |  2 +-
 .../hadoop/hdfs/TestWriteReadStripedFile.java   |  2 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |  6 ++-
 .../server/namenode/TestAddStripedBlocks.java   |  2 +-
 .../server/namenode/TestFSEditLogLoader.java    |  4 +-
 .../hdfs/server/namenode/TestFSImage.java       |  4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  4 +-
 .../namenode/TestQuotaWithStripedBlocks.java    |  2 +-
 ...TestOfflineImageViewerWithStripedBlocks.java |  2 +-
 .../hadoop/hdfs/util/TestStripedBlockUtil.java  |  4 +-
 .../apache/hadoop/hdfs/web/TestJsonUtil.java    |  2 +-
 48 files changed, 244 insertions(+), 174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 f07973a..8c902b4 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
@@ -49,7 +49,8 @@ public class HdfsFileStatus {
 
   private final FileEncryptionInfo feInfo;
 
-  private final ECSchema schema;
+  private final ECSchema ecSchema;
+  private final int stripeCellSize;
   
   // Used by dir, not including dot and dotdot. Always zero for a regular file.
   private final int childrenNum;
@@ -76,7 +77,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, ECSchema schema) {
+      byte storagePolicy, ECSchema ecSchema, int stripeCellSize) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
@@ -96,7 +97,8 @@ public class HdfsFileStatus {
     this.childrenNum = childrenNum;
     this.feInfo = feInfo;
     this.storagePolicy = storagePolicy;
-    this.schema = schema;
+    this.ecSchema = ecSchema;
+    this.stripeCellSize = stripeCellSize;
   }
 
   /**
@@ -255,7 +257,11 @@ public class HdfsFileStatus {
   }
 
   public ECSchema getECSchema() {
-    return schema;
+    return ecSchema;
+  }
+
+  public int getStripeCellSize() {
+    return stripeCellSize;
   }
 
   public final int getChildrenNum() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 813ea26..a6c7b10 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, null);
+        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null, 0);
     this.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 62f679b..5635c1b 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, null);
+        storagePolicy, null, 0);
   }
 
   /** Convert a Json map to an ExtendedBlock object. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 1e7dbea..d6c9dba 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -227,9 +227,11 @@
     (Yi Liu via jing9)
 
     HDFS-8320. Erasure coding: consolidate striping-related terminologies. (zhz)
-	
-	HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue 
-	configurable in DFSStripedOutputStream. (Li Bo)
+
+    HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue 
+    configurable in DFSStripedOutputStream. (Li Bo)
 
     HDFS-8378. Erasure Coding: Few improvements for the erasure coding worker.
     (Rakesh R via waltersu4549)
+
+    HDFS-8375. Add cellSize as an XAttr to ECZone. ( Vinayakumar B via zhz).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 ffeb568..42f48f0 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
@@ -1197,7 +1197,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       if (fileInfo != null) {
         ECSchema schema = fileInfo.getECSchema();
         if (schema != null) {
-          return new DFSStripedInputStream(this, src, verifyChecksum, schema);
+          return new DFSStripedInputStream(this, src, verifyChecksum, schema,
+              fileInfo.getStripeCellSize());
         }
       }
       return new DFSInputStream(this, src, verifyChecksum);
@@ -3009,12 +3010,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return new EncryptionZoneIterator(namenode, traceSampler);
   }
 
-  public void createErasureCodingZone(String src, ECSchema schema)
+  public void createErasureCodingZone(String src, ECSchema schema, int cellSize)
       throws IOException {
     checkOpen();
     TraceScope scope = getPathTraceScope("createErasureCodingZone", src);
     try {
-      namenode.createErasureCodingZone(src, schema);
+      namenode.createErasureCodingZone(src, schema, cellSize);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
           SafeModeException.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 744d586..3b7eb58 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
@@ -125,12 +125,12 @@ public class DFSStripedInputStream extends DFSInputStream {
   private final CompletionService<Integer> readingService;
 
   DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
-      ECSchema schema) throws IOException {
+      ECSchema schema, int cellSize) throws IOException {
     super(dfsClient, src, verifyChecksum);
 
     assert schema != null;
     this.schema = schema;
-    cellSize = schema.getChunkSize();
+    this.cellSize = cellSize;
     dataBlkNum = (short) schema.getNumDataUnits();
     parityBlkNum = (short) schema.getNumParityUnits();
     groupSize = dataBlkNum;
@@ -189,7 +189,7 @@ public class DFSStripedInputStream extends DFSInputStream {
         targetBlockGroup, cellSize, dataBlkNum, parityBlkNum);
     // The purpose is to get start offset into each block.
     long[] offsetsForInternalBlocks = getStartOffsetsForInternalBlocks(schema,
-        targetBlockGroup, offsetIntoBlockGroup);
+        cellSize, targetBlockGroup, offsetIntoBlockGroup);
     Preconditions.checkNotNull(offsetsForInternalBlocks);
 
     final ReaderRetryPolicy retry = new ReaderRetryPolicy();
@@ -514,8 +514,8 @@ public class DFSStripedInputStream extends DFSInputStream {
     // Refresh the striped block group
     LocatedStripedBlock blockGroup = getBlockGroupAt(blockStartOffset);
 
-    AlignedStripe[] stripes = divideByteRangeIntoStripes(schema, blockGroup,
-        start, end, buf, offset);
+    AlignedStripe[] stripes = divideByteRangeIntoStripes(schema, cellSize,
+        blockGroup, start, end, buf, offset);
     for (AlignedStripe stripe : stripes) {
       fetchOneStripe(blockGroup, buf, stripe, corruptedBlockMap);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 a648023..4399a37 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
@@ -230,7 +230,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
 
     final ECSchema schema = stat.getECSchema();
     final int numParityBlocks = schema.getNumParityUnits();
-    cellSize = schema.getChunkSize();
+    cellSize = stat.getStripeCellSize();
     numDataBlocks = schema.getNumDataUnits();
     numAllBlocks = numDataBlocks + numParityBlocks;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 0cd8334..2e21372 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,16 +2281,17 @@ public class DistributedFileSystem extends FileSystem {
    * 
    * @param path Directory to create the ec zone
    * @param schema ECSchema for the zone. If not specified default will be used.
+   * @param cellSize Cellsize for the striped erasure coding
    * @throws IOException
    */
-  public void createErasureCodingZone(final Path path, final ECSchema schema)
-      throws IOException {
+  public void createErasureCodingZone(final Path path, final ECSchema schema,
+      final int cellSize) throws IOException {
     Path absF = fixRelativePart(path);
     new FileSystemLinkResolver<Void>() {
       @Override
       public Void doCall(final Path p) throws IOException,
           UnresolvedLinkException {
-        dfs.createErasureCodingZone(getPathName(p), schema);
+        dfs.createErasureCodingZone(getPathName(p), schema, cellSize);
         return null;
       }
 
@@ -2298,7 +2299,7 @@ public class DistributedFileSystem extends FileSystem {
       public Void next(final FileSystem fs, final Path p) throws IOException {
         if (fs instanceof DistributedFileSystem) {
           DistributedFileSystem myDfs = (DistributedFileSystem) fs;
-          myDfs.createErasureCodingZone(p, schema);
+          myDfs.createErasureCodingZone(p, schema, cellSize);
           return null;
         }
         throw new UnsupportedOperationException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 b0b457c..4f985ba 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
@@ -1463,7 +1463,7 @@ public interface ClientProtocol {
    * default
    */
   @AtMostOnce
-  public void createErasureCodingZone(String src, ECSchema schema)
+  public void createErasureCodingZone(String src, ECSchema schema, int cellSize)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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
index ec0efbd..282eeaf 100644
--- 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
@@ -25,10 +25,12 @@ public class ErasureCodingZoneInfo {
 
   private String dir;
   private ECSchema schema;
+  private int cellSize;
 
-  public ErasureCodingZoneInfo(String dir, ECSchema schema) {
+  public ErasureCodingZoneInfo(String dir, ECSchema schema, int cellSize) {
     this.dir = dir;
     this.schema = schema;
+    this.cellSize = cellSize;
   }
 
   /**
@@ -49,8 +51,16 @@ public class ErasureCodingZoneInfo {
     return schema;
   }
 
+  /**
+   * Get cellSize for the EC Zone
+   */
+  public int getCellSize() {
+    return cellSize;
+  }
+
   @Override
   public String toString() {
-    return "Dir: " + getDir() + ", Schema: " + schema;
+    return "Dir: " + getDir() + ", Schema: " + schema + ", cellSize: "
+        + cellSize;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 9194d26..4701538 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
@@ -59,10 +59,11 @@ 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, ECSchema schema) {
+      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy,
+      ECSchema schema, int stripeCellSize) {
     super(length, isdir, block_replication, blocksize, modification_time,
         access_time, permission, owner, group, symlink, path, fileId,
-        childrenNum, feInfo, storagePolicy, schema);
+        childrenNum, feInfo, storagePolicy, schema, stripeCellSize);
     this.locations = locations;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 bae753b..863b217 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
@@ -1408,7 +1408,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     try {
       ECSchema schema = req.hasSchema() ? PBHelper.convertECSchema(req
           .getSchema()) : null;
-      server.createErasureCodingZone(req.getSrc(), schema);
+      int cellSize = req.hasCellSize() ? req.getCellSize() : 0;
+      server.createErasureCodingZone(req.getSrc(), schema, cellSize);
       return CreateErasureCodingZoneResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 67b1457..336e3a2 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
@@ -1422,7 +1422,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void createErasureCodingZone(String src, ECSchema schema)
+  public void createErasureCodingZone(String src, ECSchema schema, int cellSize)
       throws IOException {
     final CreateErasureCodingZoneRequestProto.Builder builder =
         CreateErasureCodingZoneRequestProto.newBuilder();
@@ -1430,6 +1430,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
     if (schema != null) {
       builder.setSchema(PBHelper.convertECSchema(schema));
     }
+    if (cellSize > 0) {
+      builder.setCellSize(cellSize);
+    }
     CreateErasureCodingZoneRequestProto req = builder.build();
     try {
       rpcProxy.createErasureCodingZone(null, req);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 a6a356c..4d0f871 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
@@ -1506,7 +1506,8 @@ public class PBHelper {
         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
             : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-        fs.hasEcSchema() ? PBHelper.convertECSchema(fs.getEcSchema()) : null);
+        fs.hasEcSchema() ? PBHelper.convertECSchema(fs.getEcSchema()) : null,
+        fs.hasStripeCellSize() ? fs.getStripeCellSize() : 0);
   }
 
   public static SnapshottableDirectoryStatus convert(
@@ -1570,6 +1571,7 @@ public class PBHelper {
     if(fs.getECSchema() != null) {
       builder.setEcSchema(PBHelper.convertECSchema(fs.getECSchema()));
     }
+    builder.setStripeCellSize(fs.getStripeCellSize());
     return builder.build();
   }
   
@@ -3157,12 +3159,14 @@ public class PBHelper {
 
   public static ErasureCodingZoneInfoProto convertECZoneInfo(ErasureCodingZoneInfo ecZoneInfo) {
     return ErasureCodingZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir())
-        .setSchema(convertECSchema(ecZoneInfo.getSchema())).build();
+        .setSchema(convertECSchema(ecZoneInfo.getSchema()))
+        .setCellSize(ecZoneInfo.getCellSize()).build();
   }
 
   public static ErasureCodingZoneInfo convertECZoneInfo(ErasureCodingZoneInfoProto ecZoneInfoProto) {
     return new ErasureCodingZoneInfo(ecZoneInfoProto.getDir(),
-        convertECSchema(ecZoneInfoProto.getSchema()));
+        convertECSchema(ecZoneInfoProto.getSchema()),
+        ecZoneInfoProto.getCellSize());
   }
   
   public static BlockECRecoveryInfo convertBlockECRecoveryInfo(
@@ -3196,9 +3200,11 @@ public class PBHelper {
     }
 
     ECSchema ecSchema = convertECSchema(blockEcRecoveryInfoProto.getEcSchema());
+    int cellSize = blockEcRecoveryInfoProto.getCellSize();
 
     return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos,
-        targetStorageUuids, convertStorageTypes, liveBlkIndices, ecSchema);
+        targetStorageUuids, convertStorageTypes, liveBlkIndices, ecSchema,
+        cellSize);
   }
 
   public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo(
@@ -3224,6 +3230,7 @@ public class PBHelper {
     builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices));
 
     builder.setEcSchema(convertECSchema(blockEcRecoveryInfo.getECSchema()));
+    builder.setCellSize(blockEcRecoveryInfo.getCellSize());
 
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 d296aa8..28e7b89 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
@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -1560,14 +1561,14 @@ public class BlockManager {
             assert rw instanceof ErasureCodingWork;
             assert rw.targets.length > 0;
             String src = block.getBlockCollection().getName();
-            ECSchema ecSchema = null;
+            ErasureCodingZoneInfo ecZoneInfo = null;
             try {
-              ecSchema = namesystem.getECSchemaForPath(src);
+              ecZoneInfo = namesystem.getErasureCodingZoneInfoForPath(src);
             } catch (IOException e) {
               blockLog
-                  .warn("Failed to get the EC schema for the file {} ", src);
+                  .warn("Failed to get the EC zone info for the file {} ", src);
             }
-            if (ecSchema == null) {
+            if (ecZoneInfo == 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
@@ -1577,7 +1578,8 @@ public class BlockManager {
             rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
                 new ExtendedBlock(namesystem.getBlockPoolId(), block),
                 rw.srcNodes, rw.targets,
-                ((ErasureCodingWork) rw).liveBlockIndicies, ecSchema);
+                ((ErasureCodingWork) rw).liveBlockIndicies,
+                ecZoneInfo.getSchema(), ecZoneInfo.getCellSize());
           } else {
             rw.srcNodes[0].addBlockToBeReplicated(block, targets);
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 83d3303..47bc765 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
@@ -610,10 +610,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
    */
   void addBlockToBeErasureCoded(ExtendedBlock block,
       DatanodeDescriptor[] sources, DatanodeStorageInfo[] targets,
-      short[] liveBlockIndices, ECSchema ecSchema) {
+      short[] liveBlockIndices, ECSchema ecSchema, int cellSize) {
     assert (block != null && sources != null && sources.length > 0);
     BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets,
-        liveBlockIndices, ecSchema);
+        liveBlockIndices, ecSchema, cellSize);
     erasurecodeBlocks.offer(task);
     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/91c81fdc/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 4723e9f..d227de8 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
@@ -269,7 +269,7 @@ public final class ErasureCodingWorker {
       ECSchema schema = recoveryInfo.getECSchema();
       dataBlkNum = schema.getNumDataUnits();
       parityBlkNum = schema.getNumParityUnits();
-      cellSize = schema.getChunkSize();
+      cellSize = recoveryInfo.getCellSize();
 
       blockGroup = recoveryInfo.getExtendedBlock();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 6b6add7..371b8ac 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
@@ -19,12 +19,20 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.EnumSet;
@@ -78,17 +86,21 @@ public class ErasureCodingZoneManager {
           : inode.getXAttrFeature().getXAttrs();
       for (XAttr xAttr : xAttrs) {
         if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
-          String schemaName = new String(xAttr.getValue());
+          ByteArrayInputStream bIn=new ByteArrayInputStream(xAttr.getValue());
+          DataInputStream dIn=new DataInputStream(bIn);
+          int cellSize = WritableUtils.readVInt(dIn);
+          String schemaName = WritableUtils.readString(dIn);
           ECSchema schema = dir.getFSNamesystem().getECSchemaManager()
               .getSchema(schemaName);
-          return new ErasureCodingZoneInfo(inode.getFullPathName(), schema);
+          return new ErasureCodingZoneInfo(inode.getFullPathName(), schema,
+              cellSize);
         }
       }
     }
     return null;
   }
 
-  XAttr createErasureCodingZone(String src, ECSchema schema)
+  XAttr createErasureCodingZone(String src, ECSchema schema, int cellSize)
       throws IOException {
     assert dir.hasWriteLock();
     final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false);
@@ -113,10 +125,24 @@ public class ErasureCodingZoneManager {
       schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     }
 
-    // Now persist the schema name in xattr
-    byte[] schemaBytes = schema.getSchemaName().getBytes();
-    final XAttr ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE,
-        schemaBytes);
+    if (cellSize <= 0) {
+      cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+    }
+
+    // Write the cellsize first and then schema name
+    final XAttr ecXAttr;
+    DataOutputStream dOut = null;
+    try {
+      ByteArrayOutputStream bOut = new ByteArrayOutputStream();
+      dOut = new DataOutputStream(bOut);
+      WritableUtils.writeVInt(dOut, cellSize);
+      // Now persist the schema name in xattr
+      WritableUtils.writeString(dOut, schema.getSchemaName());
+      ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE,
+          bOut.toByteArray());
+    } finally {
+      IOUtils.closeStream(dOut);
+    }
     final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
     xattrs.add(ecXAttr);
     FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 7133cf1..eba5013 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+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.HdfsFileStatus;
@@ -317,7 +318,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, null);
+            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null, 0);
       }
       return null;
     }
@@ -385,8 +386,10 @@ class FSDirStatAndListingOp {
     final FileEncryptionInfo feInfo = isRawPath ? null :
         fsd.getFileEncryptionInfo(node, snapshot, iip);
     
-    final ECSchema schema = fsd.getECSchema(iip);
-    
+    final ErasureCodingZoneInfo ecZoneInfo = fsd.getECZoneInfo(iip);
+    final ECSchema schema = ecZoneInfo != null ? ecZoneInfo.getSchema() : null;
+    final int cellSize = ecZoneInfo != null ? ecZoneInfo.getCellSize() : 0;
+
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
       size = fileNode.computeFileSize(snapshot);
@@ -417,7 +420,8 @@ class FSDirStatAndListingOp {
         childrenNum,
         feInfo,
         storagePolicy,
-        schema);
+        schema,
+        cellSize);
   }
 
   private static INodeAttributes getINodeAttributes(
@@ -464,8 +468,10 @@ class FSDirStatAndListingOp {
     }
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
-    final ECSchema schema = fsd.getECSchema(iip);
-        
+    final ErasureCodingZoneInfo ecZoneInfo = fsd.getECZoneInfo(iip);
+    final ECSchema schema = ecZoneInfo != null ? ecZoneInfo.getSchema() : null;
+    final int cellSize = ecZoneInfo != null ? ecZoneInfo.getCellSize() : 0;
+
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
           blocksize, node.getModificationTime(snapshot),
@@ -473,7 +479,8 @@ class FSDirStatAndListingOp {
           getPermissionForFileStatus(nodeAttrs, isEncrypted),
           nodeAttrs.getUserName(), nodeAttrs.getGroupName(),
           node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
-          node.getId(), loc, childrenNum, feInfo, storagePolicy, schema);
+          node.getId(), loc, childrenNum, feInfo, storagePolicy, schema,
+          cellSize);
     // 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/91c81fdc/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 bb8282d..735ea5a 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
@@ -1230,11 +1230,11 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  XAttr createErasureCodingZone(String src, ECSchema schema)
+  XAttr createErasureCodingZone(String src, ECSchema schema, int cellSize)
       throws IOException {
     writeLock();
     try {
-      return ecZoneManager.createErasureCodingZone(src, schema);
+      return ecZoneManager.createErasureCodingZone(src, schema, cellSize);
     } finally {
       writeUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 6643dfe..60f86d6 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
@@ -7555,14 +7555,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @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
-   *
+   * @param cellSize Cell size of stripe 
    * @throws AccessControlException  if the caller is not the superuser.
    * @throws UnresolvedLinkException if the path can't be resolved.
    * @throws SafeModeException       if the Namenode is in safe mode.
    */
   void createErasureCodingZone(final String srcArg, final ECSchema schema,
-      final boolean logRetryCache) throws IOException, UnresolvedLinkException,
-      SafeModeException, AccessControlException {
+      int cellSize, final boolean logRetryCache) throws IOException,
+      UnresolvedLinkException, SafeModeException, AccessControlException {
     String src = srcArg;
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = null;
@@ -7585,7 +7585,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkNameNodeSafeMode("Cannot create erasure coding zone on " + src);
       src = dir.resolvePath(pc, src, pathComponents);
 
-      final XAttr ecXAttr = dir.createErasureCodingZone(src, schema);
+      final XAttr ecXAttr = dir.createErasureCodingZone(src, schema, cellSize);
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
       xAttrs.add(ecXAttr);
       getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
@@ -7604,9 +7604,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   ErasureCodingInfo getErasureCodingInfo(String src) throws AccessControlException,
       UnresolvedLinkException, IOException {
-    ECSchema schema = getECSchemaForPath(src);
-    if (schema != null) {
-      return new ErasureCodingInfo(src, schema);
+    ErasureCodingZoneInfo zoneInfo = getErasureCodingZoneInfo(src);
+    if (zoneInfo != null) {
+      return new ErasureCodingInfo(src, zoneInfo.getSchema());
     }
     return null;
   }
@@ -7614,21 +7614,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Get the erasure coding zone information for specified path
    */
-  ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws AccessControlException,
-      UnresolvedLinkException, IOException {
+  ErasureCodingZoneInfo 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);
+      return getErasureCodingZoneInfoForPath(src);
     } finally {
       readUnlock();
     }
@@ -7849,24 +7841,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override
-  public ECSchema getECSchemaForPath(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
+  public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src)
+      throws IOException {
     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();
+    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);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 5a69b2f..747f528 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
@@ -1824,7 +1824,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public void createErasureCodingZone(String src, ECSchema schema)
+  public void createErasureCodingZone(String src, ECSchema schema, int cellSize)
       throws IOException {
     checkNNStartup();
     final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
@@ -1833,7 +1833,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
     boolean success = false;
     try {
-      namesystem.createErasureCodingZone(src, schema, cacheEntry != null);
+      namesystem.createErasureCodingZone(src, schema, cellSize,
+          cacheEntry != null);
       success = true;
     } finally {
       RetryCache.setState(cacheEntry, success);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 e6c7fc0..a32e800 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
@@ -51,12 +52,13 @@ public interface Namesystem extends RwLock, SafeMode {
   public boolean isInSnapshot(BlockCollection bc);
 
   /**
-   * Gets the ECSchema for the specified path
+   * Gets the ECZone info for path
    * 
    * @param src
    *          - path
-   * @return ECSchema
+   * @return {@link ErasureCodingZoneInfo}
    * @throws IOException
    */
-  public ECSchema getECSchemaForPath(String src) throws IOException;
+  public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src)
+      throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 61e49e9..56a1546 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
@@ -78,25 +78,22 @@ public class BlockECRecoveryCommand extends DatanodeCommand {
     private StorageType[] targetStorageTypes;
     private final short[] liveBlockIndices;
     private final ECSchema ecSchema;
+    private final int cellSize;
 
     public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources,
         DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices,
-        ECSchema ecSchema) {
-      this.block = block;
-      this.sources = sources;
-      this.targets = DatanodeStorageInfo.toDatanodeInfos(targetDnStorageInfo);
-      this.targetStorageIDs = DatanodeStorageInfo
-          .toStorageIDs(targetDnStorageInfo);
-      this.targetStorageTypes = DatanodeStorageInfo
-          .toStorageTypes(targetDnStorageInfo);
-      this.liveBlockIndices = liveBlockIndices;
-      this.ecSchema = ecSchema;
+        ECSchema ecSchema, int cellSize) {
+      this(block, sources, DatanodeStorageInfo
+          .toDatanodeInfos(targetDnStorageInfo), DatanodeStorageInfo
+          .toStorageIDs(targetDnStorageInfo), DatanodeStorageInfo
+          .toStorageTypes(targetDnStorageInfo), liveBlockIndices, ecSchema,
+          cellSize);
     }
-    
+
     public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources,
         DatanodeInfo[] targets, String[] targetStorageIDs,
         StorageType[] targetStorageTypes, short[] liveBlockIndices,
-        ECSchema ecSchema) {
+        ECSchema ecSchema, int cellSize) {
       this.block = block;
       this.sources = sources;
       this.targets = targets;
@@ -104,6 +101,7 @@ public class BlockECRecoveryCommand extends DatanodeCommand {
       this.targetStorageTypes = targetStorageTypes;
       this.liveBlockIndices = liveBlockIndices;
       this.ecSchema = ecSchema;
+      this.cellSize = cellSize;
     }
 
     public ExtendedBlock getExtendedBlock() {
@@ -134,6 +132,10 @@ public class BlockECRecoveryCommand extends DatanodeCommand {
       return ecSchema;
     }
 
+    public int getCellSize() {
+      return cellSize;
+    }
+
     @Override
     public String toString() {
       return new StringBuilder().append("BlockECRecoveryInfo(\n  ")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 2b6a6a5..2d82208 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
@@ -31,6 +31,7 @@ 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.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.StringUtils;
@@ -88,14 +89,23 @@ public abstract class ECCommand extends Command {
         + "Options :\n"
         + "  -s <schemaName> : EC schema name to encode files. "
         + "If not passed default schema will be used\n"
+        + "  -c <cellSize> : cell size to use for striped encoding files."
+        + " If not passed default cellsize of "
+        + HdfsConstants.BLOCK_STRIPED_CELL_SIZE + " will be used\n"
         + "  <path>  : Path to an empty directory. Under this directory "
         + "files will be encoded using specified schema";
     private String schemaName;
+    private int cellSize = 0;
     private ECSchema schema = null;
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
       schemaName = StringUtils.popOptionWithArgument("-s", args);
+      String cellSizeStr = StringUtils.popOptionWithArgument("-c", args);
+      if (cellSizeStr != null) {
+        cellSize = (int) StringUtils.TraditionalBinaryPrefix
+            .string2long(cellSizeStr);
+      }
       if (args.isEmpty()) {
         throw new HadoopIllegalArgumentException("<path> is missing");
       }
@@ -131,7 +141,7 @@ public abstract class ECCommand extends Command {
             throw new HadoopIllegalArgumentException(sb.toString());
           }
         }
-        dfs.createErasureCodingZone(item.path, schema);
+        dfs.createErasureCodingZone(item.path, schema, cellSize);
         out.println("EC Zone created successfully at " + item.path);
       } catch (IOException e) {
         throw new IOException("Unable to create EC zone for the path "
@@ -213,4 +223,4 @@ public abstract class ECCommand extends Command {
       out.println(sb.toString());
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 2fa3fdf..6f7dcb1 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
@@ -306,6 +306,7 @@ public class StripedBlockUtil {
    * {@link AlignedStripe}.
    * @param ecSchema The codec schema for the file, which carries the numbers
    *                 of data / parity blocks, as well as cell size
+   * @param cellSize Cell size of stripe
    * @param blockGroup The striped block group
    * @param rangeStartInBlockGroup The byte range's start offset in block group
    * @param rangeEndInBlockGroup The byte range's end offset in block group
@@ -315,28 +316,29 @@ public class StripedBlockUtil {
    * At most 5 stripes will be generated from each logical range, as
    * demonstrated in the header of {@link AlignedStripe}.
    */
-  public static AlignedStripe[] divideByteRangeIntoStripes (
-      ECSchema ecSchema, LocatedStripedBlock blockGroup,
+  public static AlignedStripe[] divideByteRangeIntoStripes(ECSchema ecSchema,
+      int cellSize, LocatedStripedBlock blockGroup,
       long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf,
       int offsetInBuf) {
     // TODO: change ECSchema naming to use cell size instead of chunk size
 
     // Step 0: analyze range and calculate basic parameters
-    int cellSize = ecSchema.getChunkSize();
     int dataBlkNum = ecSchema.getNumDataUnits();
 
     // Step 1: map the byte range to StripingCells
-    StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, blockGroup,
-        rangeStartInBlockGroup, rangeEndInBlockGroup);
+    StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, cellSize,
+        blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup);
 
     // Step 2: get the unmerged ranges on each internal block
-    VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cells);
+    VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cellSize,
+        cells);
 
     // Step 3: merge into at most 5 stripes
     AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges);
 
     // Step 4: calculate each chunk's position in destination buffer
-    calcualteChunkPositionsInBuf(ecSchema, stripes, cells, buf, offsetInBuf);
+    calcualteChunkPositionsInBuf(ecSchema, cellSize, stripes, cells, buf,
+        offsetInBuf);
 
     // Step 5: prepare ALLZERO blocks
     prepareAllZeroChunks(blockGroup, buf, stripes, cellSize, dataBlkNum);
@@ -351,19 +353,18 @@ public class StripedBlockUtil {
    */
   @VisibleForTesting
   private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema,
-      LocatedStripedBlock blockGroup,
+      int cellSize, LocatedStripedBlock blockGroup,
       long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
     Preconditions.checkArgument(
         rangeStartInBlockGroup <= rangeEndInBlockGroup &&
             rangeEndInBlockGroup < blockGroup.getBlockSize());
-    int cellSize = ecSchema.getChunkSize();
     int len = (int) (rangeEndInBlockGroup - rangeStartInBlockGroup + 1);
     int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
     int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize);
     int numCells = lastCellIdxInBG - firstCellIdxInBG + 1;
     StripingCell[] cells = new StripingCell[numCells];
-    cells[0] = new StripingCell(ecSchema, firstCellIdxInBG);
-    cells[numCells - 1] = new StripingCell(ecSchema, lastCellIdxInBG);
+    cells[0] = new StripingCell(ecSchema, cellSize, firstCellIdxInBG);
+    cells[numCells - 1] = new StripingCell(ecSchema, cellSize, lastCellIdxInBG);
 
     cells[0].offset = (int) (rangeStartInBlockGroup % cellSize);
     cells[0].size =
@@ -373,7 +374,7 @@ public class StripedBlockUtil {
     }
 
     for (int i = 1; i < numCells - 1; i++) {
-      cells[i] = new StripingCell(ecSchema, i + firstCellIdxInBG);
+      cells[i] = new StripingCell(ecSchema, cellSize, i + firstCellIdxInBG);
     }
 
     return cells;
@@ -383,18 +384,16 @@ public class StripedBlockUtil {
    * Given a logical start offset in a block group, calculate the physical
    * start offset into each stored internal block.
    */
-  public static long[] getStartOffsetsForInternalBlocks(
-      ECSchema ecSchema, LocatedStripedBlock blockGroup,
-      long rangeStartInBlockGroup) {
+  public static long[] getStartOffsetsForInternalBlocks(ECSchema ecSchema,
+      int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup) {
     Preconditions.checkArgument(
         rangeStartInBlockGroup < blockGroup.getBlockSize());
     int dataBlkNum = ecSchema.getNumDataUnits();
     int parityBlkNum = ecSchema.getNumParityUnits();
-    int cellSize = ecSchema.getChunkSize();
     long[] startOffsets = new long[dataBlkNum + parityBlkNum];
     Arrays.fill(startOffsets, -1L);
     int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
-    StripingCell firstCell = new StripingCell(ecSchema, firstCellIdxInBG);
+    StripingCell firstCell = new StripingCell(ecSchema, cellSize, firstCellIdxInBG);
     firstCell.offset = (int) (rangeStartInBlockGroup % cellSize);
     startOffsets[firstCell.idxInStripe] =
         firstCell.idxInInternalBlk * cellSize + firstCell.offset;
@@ -404,7 +403,7 @@ public class StripedBlockUtil {
       if (idx * cellSize >= blockGroup.getBlockSize()) {
         break;
       }
-      StripingCell cell = new StripingCell(ecSchema, idx);
+      StripingCell cell = new StripingCell(ecSchema, cellSize, idx);
       startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * cellSize;
       if (startOffsets[cell.idxInStripe] < earliestStart) {
         earliestStart = startOffsets[cell.idxInStripe];
@@ -422,8 +421,7 @@ public class StripedBlockUtil {
    */
   @VisibleForTesting
   private static VerticalRange[] getRangesForInternalBlocks(ECSchema ecSchema,
-      StripingCell[] cells) {
-    int cellSize = ecSchema.getChunkSize();
+      int cellSize, StripingCell[] cells) {
     int dataBlkNum = ecSchema.getNumDataUnits();
     int parityBlkNum = ecSchema.getNumParityUnits();
 
@@ -486,7 +484,7 @@ public class StripedBlockUtil {
   }
 
   private static void calcualteChunkPositionsInBuf(ECSchema ecSchema,
-      AlignedStripe[] stripes, StripingCell[] cells, byte[] buf,
+      int cellSize, AlignedStripe[] stripes, StripingCell[] cells, byte[] buf,
       int offsetInBuf) {
     /**
      *     | <--------------- AlignedStripe --------------->|
@@ -505,7 +503,6 @@ public class StripedBlockUtil {
      *
      * Cell indexing convention defined in {@link StripingCell}
      */
-    int cellSize = ecSchema.getChunkSize();
     int done = 0;
     for (StripingCell cell : cells) {
       long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
@@ -587,17 +584,17 @@ public class StripedBlockUtil {
     int offset;
     int size;
 
-    StripingCell(ECSchema ecSchema, int idxInBlkGroup) {
+    StripingCell(ECSchema ecSchema, int cellSize, int idxInBlkGroup) {
       this.schema = ecSchema;
       this.idxInBlkGroup = idxInBlkGroup;
       this.idxInInternalBlk = idxInBlkGroup / ecSchema.getNumDataUnits();
       this.idxInStripe = idxInBlkGroup -
           this.idxInInternalBlk * ecSchema.getNumDataUnits();
       this.offset = 0;
-      this.size = ecSchema.getChunkSize();
+      this.size = cellSize;
     }
 
-    StripingCell(ECSchema ecSchema, int idxInInternalBlk,
+    StripingCell(ECSchema ecSchema, int cellSize, int idxInInternalBlk,
         int idxInStripe) {
       this.schema = ecSchema;
       this.idxInInternalBlk = idxInInternalBlk;
@@ -605,7 +602,7 @@ public class StripedBlockUtil {
       this.idxInBlkGroup =
           idxInInternalBlk * ecSchema.getNumDataUnits() + idxInStripe;
       this.offset = 0;
-      this.size = ecSchema.getChunkSize();
+      this.size = cellSize;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 2302d1d..058ed96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -37,11 +37,13 @@ message ErasureCodingInfoProto {
 message ErasureCodingZoneInfoProto {
   required string dir = 1;
   required ECSchemaProto schema = 2;
+  required uint32 cellSize = 3;
 }
 
 message CreateErasureCodingZoneRequestProto {
   required string src = 1;
   optional ECSchemaProto schema = 2;
+  optional uint32 cellSize = 3;
 }
 
 message CreateErasureCodingZoneResponseProto {
@@ -81,4 +83,5 @@ message BlockECRecoveryInfoProto {
   required StorageTypesProto targetStorageTypes = 5;
   repeated uint32 liveBlockIndices = 6;
   required ECSchemaProto ecSchema = 7;
+  required uint32 cellSize = 8;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 64030be..f64cf8f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -359,7 +359,8 @@ message HdfsFileStatusProto {
 
   // Optional field for erasure coding
   optional ECSchemaProto ecSchema = 17;
-} 
+  optional uint32 stripeCellSize = 18;
+}
 
 /**
  * Checksum algorithms/types used in HDFS

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 9f106cf..558c45d 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
@@ -1867,7 +1867,7 @@ public class DFSTestUtil {
       assert dir != null;
       dfs.mkdirs(dir);
       try {
-        dfs.getClient().createErasureCodingZone(dir.toString(), null);
+        dfs.getClient().createErasureCodingZone(dir.toString(), null, 0);
       } catch (IOException e) {
         if (!e.getMessage().contains("non-empty directory")) {
           throw e;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 ec88a54..4918dee 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, null)).when(mockNN).getFileInfo(anyString());
+                1010, 0, null, (byte) 0, null, 0)).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, null))
+                1010, 0, null, (byte) 0, null, 0))
         .when(mockNN)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 9032d09..a71441f 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
@@ -73,7 +73,7 @@ public class TestDFSStripedInputStream {
     cluster.waitActive();
     fs = cluster.getFileSystem();
     fs.mkdirs(dirPath);
-    fs.getClient().createErasureCodingZone(dirPath.toString(), null);
+    fs.getClient().createErasureCodingZone(dirPath.toString(), null, CELLSIZE);
   }
 
   @After
@@ -94,7 +94,7 @@ public class TestDFSStripedInputStream {
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
         filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
     final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
-        filePath.toString(), false, schema);
+        filePath.toString(), false, schema, CELLSIZE);
 
     List<LocatedBlock> lbList = lbs.getLocatedBlocks();
     for (LocatedBlock aLbList : lbList) {
@@ -146,7 +146,7 @@ public class TestDFSStripedInputStream {
       }
     }
     DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
-            filePath.toString(), false, schema);
+        filePath.toString(), false, schema, CELLSIZE);
 
     int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102,
         CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102,
@@ -188,7 +188,7 @@ public class TestDFSStripedInputStream {
     }
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(), false,
-            ErasureCodingSchemaManager.getSystemDefaultSchema());
+            ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE);
     int readSize = BLOCK_GROUP_SIZE;
     byte[] readBuffer = new byte[readSize];
     byte[] expected = new byte[readSize];
@@ -284,7 +284,7 @@ public class TestDFSStripedInputStream {
 
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(),
-            false, schema);
+            false, schema, CELLSIZE);
 
     byte[] expected = new byte[fileSize];
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 ec98e68..e795ea7 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
@@ -65,7 +65,7 @@ public class TestDFSStripedOutputStream {
     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);
+    cluster.getFileSystem().getClient().createErasureCodingZone("/", null, 0);
     fs = cluster.getFileSystem();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index c2e588a..4ad3b2e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -74,7 +74,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     cluster.waitActive();
     dfs = cluster.getFileSystem();
     dfs.mkdirs(dir);
-    dfs.createErasureCodingZone(dir, null);
+    dfs.createErasureCodingZone(dir, null, 0);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 6e2ec5e..1f317b8 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, null))
+            (byte) 0, null, 0))
         .when(mcp)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 59818c0..a7c3cd4 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
@@ -63,7 +63,7 @@ public class TestErasureCodingZones {
     fs.mkdir(testDir, FsPermission.getDirDefault());
 
     /* Normal creation of an erasure coding zone */
-    fs.getClient().createErasureCodingZone(testDir.toString(), null);
+    fs.getClient().createErasureCodingZone(testDir.toString(), null, 0);
 
     /* Verify files under the zone are striped */
     final Path ECFilePath = new Path(testDir, "foo");
@@ -76,7 +76,7 @@ public class TestErasureCodingZones {
     fs.mkdir(notEmpty, FsPermission.getDirDefault());
     fs.create(new Path(notEmpty, "foo"));
     try {
-      fs.getClient().createErasureCodingZone(notEmpty.toString(), null);
+      fs.getClient().createErasureCodingZone(notEmpty.toString(), null, 0);
       fail("Erasure coding zone on non-empty dir");
     } catch (IOException e) {
       assertExceptionContains("erasure coding zone for a non-empty directory", e);
@@ -86,10 +86,10 @@ public class TestErasureCodingZones {
     final Path zone1 = new Path("/zone1");
     final Path zone2 = new Path(zone1, "zone2");
     fs.mkdir(zone1, FsPermission.getDirDefault());
-    fs.getClient().createErasureCodingZone(zone1.toString(), null);
+    fs.getClient().createErasureCodingZone(zone1.toString(), null, 0);
     fs.mkdir(zone2, FsPermission.getDirDefault());
     try {
-      fs.getClient().createErasureCodingZone(zone2.toString(), null);
+      fs.getClient().createErasureCodingZone(zone2.toString(), null, 0);
       fail("Nested erasure coding zones");
     } catch (IOException e) {
       assertExceptionContains("already in an erasure coding zone", e);
@@ -99,7 +99,7 @@ public class TestErasureCodingZones {
     final Path fPath = new Path("/file");
     fs.create(fPath);
     try {
-      fs.getClient().createErasureCodingZone(fPath.toString(), null);
+      fs.getClient().createErasureCodingZone(fPath.toString(), null, 0);
       fail("Erasure coding zone on file");
     } catch (IOException e) {
       assertExceptionContains("erasure coding zone for a file", e);
@@ -112,8 +112,8 @@ public class TestErasureCodingZones {
     final Path dstECDir = new Path("/dstEC");
     fs.mkdir(srcECDir, FsPermission.getDirDefault());
     fs.mkdir(dstECDir, FsPermission.getDirDefault());
-    fs.getClient().createErasureCodingZone(srcECDir.toString(), null);
-    fs.getClient().createErasureCodingZone(dstECDir.toString(), null);
+    fs.getClient().createErasureCodingZone(srcECDir.toString(), null, 0);
+    fs.getClient().createErasureCodingZone(dstECDir.toString(), null, 0);
     final Path srcFile = new Path(srcECDir, "foo");
     fs.create(srcFile);
 
@@ -157,7 +157,7 @@ public class TestErasureCodingZones {
     // dir ECInfo before creating ec zone
     assertNull(fs.getClient().getErasureCodingInfo(src));
     // dir ECInfo after creating ec zone
-    fs.getClient().createErasureCodingZone(src, null); //Default one will be used.
+    fs.getClient().createErasureCodingZone(src, null, 0); //Default one will be used.
     ECSchema sysDefaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     verifyErasureCodingInfo(src, sysDefaultSchema);
     fs.create(new Path(ecDir, "/child1")).close();
@@ -178,7 +178,7 @@ public class TestErasureCodingZones {
     // dir ECInfo before creating ec zone
     assertNull(fs.getClient().getErasureCodingInfo(src));
     // dir ECInfo after creating ec zone
-    fs.getClient().createErasureCodingZone(src, usingSchema);
+    fs.getClient().createErasureCodingZone(src, usingSchema, 0);
     verifyErasureCodingInfo(src, usingSchema);
     fs.create(new Path(ecDir, "/child1")).close();
     // verify for the files in ec zone

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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
index f8c0667..3c400b7 100644
--- 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
@@ -50,7 +50,7 @@ public class TestFileStatusWithECschema {
 
     final ECSchema schema1 = ErasureCodingSchemaManager.getSystemDefaultSchema();
     // create EC zone on dir
-    fs.createErasureCodingZone(dir, schema1);
+    fs.createErasureCodingZone(dir, schema1, 0);
     final ECSchema schame2 = client.getFileInfo(dir.toUri().getPath()).getECSchema();
     assertNotNull(schame2);
     assertTrue(schema1.equals(schame2));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 b77ff3a..d0cd335 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, null)).when(mcp).getFileInfo(anyString());
+            1010, 0, null, (byte) 0, null, 0)).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, null))
+                1010, 0, null, (byte) 0, null, 0))
         .when(mcp)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 f78fb7a..1976dca 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
@@ -59,7 +59,7 @@ public class TestWriteReadStripedFile {
     conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
+    cluster.getFileSystem().getClient().createErasureCodingZone("/", null, cellSize);
     fs = cluster.getFileSystem();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 1be00b6..a0b2038 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
@@ -663,7 +663,8 @@ public class TestPBHelper {
     short[] liveBlkIndices0 = new short[2];
     BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo(
         new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0,
-        liveBlkIndices0, ErasureCodingSchemaManager.getSystemDefaultSchema());
+        liveBlkIndices0, ErasureCodingSchemaManager.getSystemDefaultSchema(),
+        64 * 1024);
     DatanodeInfo[] dnInfos1 = new DatanodeInfo[] {
         DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
     DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil
@@ -677,7 +678,8 @@ public class TestPBHelper {
     short[] liveBlkIndices1 = new short[2];
     BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo(
         new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1,
-        liveBlkIndices1, ErasureCodingSchemaManager.getSystemDefaultSchema());
+        liveBlkIndices1, ErasureCodingSchemaManager.getSystemDefaultSchema(),
+        64 * 1024);
     List<BlockECRecoveryInfo> blkRecoveryInfosList = new ArrayList<BlockECRecoveryInfo>();
     blkRecoveryInfosList.add(blkECRecoveryInfo0);
     blkRecoveryInfosList.add(blkECRecoveryInfo1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 ba76360..a35cbf4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -74,7 +74,7 @@ public class TestAddStripedBlocks {
         .numDataNodes(GROUP_SIZE).build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
-    dfs.getClient().createErasureCodingZone("/", null);
+    dfs.getClient().createErasureCodingZone("/", null, 0);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 2909423..abb9bf5 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
@@ -451,7 +451,7 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.getClient().getNamenode().createErasureCodingZone(testDir, null);
+      fs.getClient().getNamenode().createErasureCodingZone(testDir, null, 0);
 
       // Create a file with striped block
       Path p = new Path(testFilePath);
@@ -523,7 +523,7 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.getClient().getNamenode().createErasureCodingZone(testDir, null);
+      fs.getClient().getNamenode().createErasureCodingZone(testDir, null, 0);
 
       //create a file with striped blocks
       Path p = new Path(testFilePath);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 5e015bf..8fd0753 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
@@ -140,7 +140,7 @@ public class TestFSImage {
   private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf,
                                                boolean isUC) throws IOException{
     // contruct a INode with StripedBlock for saving and loading
-    fsn.createErasureCodingZone("/", null, false);
+    fsn.createErasureCodingZone("/", null, 0, false);
     long id = 123456789;
     byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
     PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
@@ -402,7 +402,7 @@ public class TestFSImage {
           .build();
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
-      fs.getClient().getNamenode().createErasureCodingZone("/", null);
+      fs.getClient().getNamenode().createErasureCodingZone("/", null, 0);
       Path file = new Path("/striped");
       FSDataOutputStream out = fs.create(file);
       byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91c81fdc/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 7870827..b36beb2 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, null);
+        path, fileId, numChildren, null, storagePolicy, null, 0);
     Result res = new Result(conf);
 
     try {
@@ -1629,4 +1629,4 @@ public class TestFsck {
       }
     }
   }
-}
\ No newline at end of file
+}


[27/50] [abbrv] 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/9c7a78c8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9c7a78c8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9c7a78c8

Branch: refs/heads/HDFS-7285
Commit: 9c7a78c8748a55dc94f7913b5d5b2c2bc27720d5
Parents: a919726
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon May 18 10:06:56 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:31 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/9c7a78c8/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..c10ffbd 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 via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c7a78c8/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/9c7a78c8/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/9c7a78c8/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/9c7a78c8/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/9c7a78c8/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/9c7a78c8/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/9c7a78c8/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/9c7a78c8/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);
   }
-
 }


[41/50] [abbrv] hadoop git commit: HDFS-8186. Erasure coding: Make block placement policy for EC file configurable. Contributed by Walter Su.

Posted by zh...@apache.org.
HDFS-8186. Erasure coding: Make block placement policy for EC file configurable. Contributed by Walter Su.


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

Branch: refs/heads/HDFS-7285
Commit: e53fa769c97416af69ea567aecd44f67e896688b
Parents: 744ef17
Author: Zhe Zhang <zh...@cloudera.com>
Authored: Wed May 20 15:37:50 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:34 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 ++
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  3 ++
 .../hadoop/hdfs/server/balancer/Balancer.java   |  6 ++-
 .../server/blockmanagement/BlockManager.java    | 32 ++++++------
 .../blockmanagement/BlockPlacementPolicies.java | 54 ++++++++++++++++++++
 .../blockmanagement/BlockPlacementPolicy.java   | 26 +---------
 .../hdfs/server/namenode/FSDirWriteFileOp.java  | 12 +++--
 .../hdfs/server/namenode/FSNamesystem.java      |  4 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |  9 ++--
 9 files changed, 95 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e53fa769/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 610a5eb..3bdff6f 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -244,3 +244,6 @@
 
     HDFS-8427. Remove dataBlockNum and parityBlockNum from BlockInfoStriped.
     (Kai Sasaki via jing9)
+
+    HDFS-8186. Erasure coding: Make block placement policy for EC file configurable.
+    (Walter Su via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e53fa769/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 d5d3095..8e6b9f0 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
@@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolarent;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
 import org.apache.hadoop.http.HttpConfig;
 
@@ -434,6 +435,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final Class<BlockPlacementPolicyDefault> DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT = BlockPlacementPolicyDefault.class;
   public static final String  DFS_REPLICATION_MAX_KEY = "dfs.replication.max";
   public static final int     DFS_REPLICATION_MAX_DEFAULT = 512;
+  public static final String DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY = "dfs.block.placement.ec.classname";
+  public static final Class<BlockPlacementPolicyRackFaultTolarent> DFS_BLOCK_PLACEMENT_EC_CLASSNAME_DEFAULT = BlockPlacementPolicyRackFaultTolarent.class;
 
   public static final String  DFS_DF_INTERVAL_KEY = "dfs.df.interval";
   public static final int     DFS_DF_INTERVAL_DEFAULT = 60000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e53fa769/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index bc7e448..d756f2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -47,8 +47,8 @@ import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Source;
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Task;
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Util;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -199,7 +199,9 @@ public class Balancer {
    */
   private static void checkReplicationPolicyCompatibility(Configuration conf
       ) throws UnsupportedActionException {
-    if (!(BlockPlacementPolicy.getInstance(conf, null, null, null) instanceof 
+    BlockPlacementPolicies placementPolicies =
+        new BlockPlacementPolicies(conf, null, null, null);
+    if (!(placementPolicies.getPolicy(false) instanceof
         BlockPlacementPolicyDefault)) {
       throw new UnsupportedActionException(
           "Balancer without BlockPlacementPolicyDefault");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e53fa769/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 fc13965..79cbcc6 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,7 +273,7 @@ public class BlockManager {
   private double replicationQueuesInitProgress = 0.0;
 
   /** for block replicas placement */
-  private BlockPlacementPolicy blockplacement;
+  private BlockPlacementPolicies placementPolicies;
   private final BlockStoragePolicySuite storagePolicySuite;
 
   /** Check whether name system is running before terminating */
@@ -297,7 +297,7 @@ public class BlockManager {
     // Compute the map capacity by allocating 2% of total memory
     blocksMap = new BlocksMap(
         LightWeightGSet.computeCapacity(2.0, "BlocksMap"));
-    blockplacement = BlockPlacementPolicy.getInstance(
+    placementPolicies = new BlockPlacementPolicies(
       conf, datanodeManager.getFSClusterStats(),
       datanodeManager.getNetworkTopology(),
       datanodeManager.getHost2DatanodeMap());
@@ -490,15 +490,7 @@ public class BlockManager {
 
   @VisibleForTesting
   public BlockPlacementPolicy getBlockPlacementPolicy() {
-    return blockplacement;
-  }
-
-  /** Set BlockPlacementPolicy */
-  public void setBlockPlacementPolicy(BlockPlacementPolicy newpolicy) {
-    if (newpolicy == null) {
-      throw new HadoopIllegalArgumentException("newpolicy == null");
-    }
-    this.blockplacement = newpolicy;
+    return placementPolicies.getPolicy(false);
   }
 
   /** Dump meta data to out. */
@@ -1504,7 +1496,9 @@ public class BlockManager {
       // choose replication targets: NOT HOLDING THE GLOBAL LOCK
       // It is costly to extract the filename for which chooseTargets is called,
       // so for now we pass in the block collection itself.
-      rw.chooseTargets(blockplacement, storagePolicySuite, excludedNodes);
+      final BlockPlacementPolicy placementPolicy =
+          placementPolicies.getPolicy(rw.block.isStriped());
+      rw.chooseTargets(placementPolicy, storagePolicySuite, excludedNodes);
     }
 
     // Step 3: add tasks to the DN
@@ -1630,7 +1624,7 @@ public class BlockManager {
   /** Choose target for WebHDFS redirection. */
   public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src,
       DatanodeDescriptor clientnode, Set<Node> excludes, long blocksize) {
-    return blockplacement.chooseTarget(src, 1, clientnode,
+    return placementPolicies.getPolicy(false).chooseTarget(src, 1, clientnode,
         Collections.<DatanodeStorageInfo>emptyList(), false, excludes,
         blocksize, storagePolicySuite.getDefaultPolicy());
   }
@@ -1642,9 +1636,10 @@ public class BlockManager {
       List<DatanodeStorageInfo> chosen,
       Set<Node> excludes,
       long blocksize,
-      byte storagePolicyID) {
-    
+      byte storagePolicyID,
+      boolean isStriped) {
     final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
+    final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped);
     return blockplacement.chooseTarget(src, numAdditionalNodes, clientnode,
         chosen, true, excludes, blocksize, storagePolicy);
   }
@@ -1662,10 +1657,12 @@ public class BlockManager {
       final Set<Node> excludedNodes,
       final long blocksize,
       final List<String> favoredNodes,
-      final byte storagePolicyID) throws IOException {
+      final byte storagePolicyID,
+      final boolean isStriped) throws IOException {
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
         getDatanodeDescriptors(favoredNodes);
     final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
+    final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped);
     final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
         numOfReplicas, client, excludedNodes, blocksize, 
         favoredDatanodeDescriptors, storagePolicy);
@@ -3088,7 +3085,7 @@ public class BlockManager {
       }
     }
     chooseExcessReplicates(nonExcess, block, replication, 
-        addedNode, delNodeHint, blockplacement);
+        addedNode, delNodeHint, placementPolicies.getPolicy(false));
   }
 
 
@@ -4126,4 +4123,5 @@ public class BlockManager {
     clearQueues();
     blocksMap.clear();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e53fa769/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java
new file mode 100644
index 0000000..622b258
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.util.ReflectionUtils;
+
+public class BlockPlacementPolicies{
+
+  private final BlockPlacementPolicy replicationPolicy;
+  private final BlockPlacementPolicy ecPolicy;
+
+  public BlockPlacementPolicies(Configuration conf, FSClusterStats stats,
+                                NetworkTopology clusterMap,
+                                Host2NodesMap host2datanodeMap){
+    final Class<? extends BlockPlacementPolicy> replicatorClass = conf
+        .getClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
+            DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT,
+            BlockPlacementPolicy.class);
+    replicationPolicy = ReflectionUtils.newInstance(replicatorClass, conf);
+    replicationPolicy.initialize(conf, stats, clusterMap, host2datanodeMap);
+    final Class<? extends BlockPlacementPolicy> blockPlacementECClass =
+        conf.getClass(DFSConfigKeys.DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY,
+            DFSConfigKeys.DFS_BLOCK_PLACEMENT_EC_CLASSNAME_DEFAULT,
+            BlockPlacementPolicy.class);
+    ecPolicy = ReflectionUtils.newInstance(blockPlacementECClass, conf);
+    ecPolicy.initialize(conf, stats, clusterMap, host2datanodeMap);
+  }
+
+  public BlockPlacementPolicy getPolicy(boolean isStriped){
+    if (isStriped) {
+      return ecPolicy;
+    } else {
+      return replicationPolicy;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e53fa769/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
index 9696179..86aaf79 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
@@ -145,31 +145,7 @@ public abstract class BlockPlacementPolicy {
   abstract protected void initialize(Configuration conf,  FSClusterStats stats, 
                                      NetworkTopology clusterMap, 
                                      Host2NodesMap host2datanodeMap);
-    
-  /**
-   * Get an instance of the configured Block Placement Policy based on the
-   * the configuration property
-   * {@link  DFSConfigKeys#DFS_BLOCK_REPLICATOR_CLASSNAME_KEY}.
-   * 
-   * @param conf the configuration to be used
-   * @param stats an object that is used to retrieve the load on the cluster
-   * @param clusterMap the network topology of the cluster
-   * @return an instance of BlockPlacementPolicy
-   */
-  public static BlockPlacementPolicy getInstance(Configuration conf, 
-                                                 FSClusterStats stats,
-                                                 NetworkTopology clusterMap,
-                                                 Host2NodesMap host2datanodeMap) {
-    final Class<? extends BlockPlacementPolicy> replicatorClass = conf.getClass(
-        DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
-        DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT,
-        BlockPlacementPolicy.class);
-    final BlockPlacementPolicy replicator = ReflectionUtils.newInstance(
-        replicatorClass, conf);
-    replicator.initialize(conf, stats, clusterMap, host2datanodeMap);
-    return replicator;
-  }
-  
+
   /**
    * Adjust rackmap, moreThanOne, and exactlyOne after removing replica on cur.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e53fa769/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 6137fd5..11b6d8f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -174,6 +174,7 @@ class FSDirWriteFileOp {
     final short numTargets;
     final byte storagePolicyID;
     String clientMachine;
+    final boolean isStriped;
 
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsn.dir.resolvePath(pc, src, pathComponents);
@@ -199,13 +200,13 @@ class FSDirWriteFileOp {
     blockSize = pendingFile.getPreferredBlockSize();
     clientMachine = pendingFile.getFileUnderConstructionFeature()
         .getClientMachine();
-    boolean isStriped = pendingFile.isStriped();
+    isStriped = pendingFile.isStriped();
     numTargets = isStriped ?
         HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS :
         pendingFile.getFileReplication();
     storagePolicyID = pendingFile.getStoragePolicyID();
     return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
-                                      clientMachine);
+                                      clientMachine, isStriped);
   }
 
   static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
@@ -289,7 +290,8 @@ class FSDirWriteFileOp {
     // choose targets for the new block to be allocated.
     return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
                                     excludedNodesSet, r.blockSize,
-                                    favoredNodesList, r.storagePolicyID);
+                                    favoredNodesList, r.storagePolicyID,
+                                    r.isStriped);
   }
 
   /**
@@ -867,14 +869,16 @@ class FSDirWriteFileOp {
     final int numTargets;
     final byte storagePolicyID;
     final String clientMachine;
+    final boolean isStriped;
 
     ValidateAddBlockResult(
         long blockSize, int numTargets, byte storagePolicyID,
-        String clientMachine) {
+        String clientMachine, boolean isStriped) {
       this.blockSize = blockSize;
       this.numTargets = numTargets;
       this.storagePolicyID = storagePolicyID;
       this.clientMachine = clientMachine;
+      this.isStriped = isStriped;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e53fa769/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 7e45e90..e9bb2f7 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
@@ -2935,6 +2935,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final long preferredblocksize;
     final byte storagePolicyID;
     final List<DatanodeStorageInfo> chosen;
+    final boolean isStriped;
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     FSPermissionChecker pc = getPermissionChecker();
@@ -2961,6 +2962,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
       preferredblocksize = file.getPreferredBlockSize();
       storagePolicyID = file.getStoragePolicyID();
+      isStriped = file.isStriped();
 
       //find datanode storages
       final DatanodeManager dm = blockManager.getDatanodeManager();
@@ -2976,7 +2978,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // choose new datanodes.
     final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
         src, numAdditionalNodes, clientnode, chosen, 
-        excludes, preferredblocksize, storagePolicyID);
+        excludes, preferredblocksize, storagePolicyID, isStriped);
     final LocatedBlock lb = BlockManager.newLocatedBlock(
         blk, targets, -1, false);
     blockManager.setBlockToken(lb, BlockTokenIdentifier.AccessMode.COPY);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e53fa769/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index fccef17..1067394 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -65,9 +65,8 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -171,7 +170,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   private final PrintWriter out;
   private List<String> snapshottableDirs = null;
 
-  private final BlockPlacementPolicy bpPolicy;
+  private final BlockPlacementPolicies bpPolicies;
   private StoragePolicySummary storageTypeSummary = null;
 
   /**
@@ -193,7 +192,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     this.out = out;
     this.totalDatanodes = totalDatanodes;
     this.remoteAddress = remoteAddress;
-    this.bpPolicy = BlockPlacementPolicy.getInstance(conf, null,
+    this.bpPolicies = new BlockPlacementPolicies(conf, null,
         networktopology,
         namenode.getNamesystem().getBlockManager().getDatanodeManager()
         .getHost2DatanodeMap());
@@ -601,7 +600,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       }
 
       // count mis replicated blocks
-      BlockPlacementStatus blockPlacementStatus = bpPolicy
+      BlockPlacementStatus blockPlacementStatus = bpPolicies.getPolicy(false)
           .verifyBlockPlacement(path, lBlk, targetFileReplication);
       if (!blockPlacementStatus.isPlacementPolicySatisfied()) {
         res.numMisReplicatedBlocks++;


[08/50] [abbrv] 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/220ca960
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/220ca960
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/220ca960

Branch: refs/heads/HDFS-7285
Commit: 220ca960bce970d5969b9af570a3ce43360b7e2b
Parents: e849be2
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue May 5 16:26:49 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:00:45 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    |  11 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  15 +-
 .../apache/hadoop/hdfs/StripedDataStreamer.java | 156 ++----
 .../hdfs/server/namenode/FSDirectory.java       |   2 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |   2 -
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |  18 +-
 .../TestDFSStripedOutputStreamWithFailure.java  | 323 ++++++++++++
 10 files changed, 765 insertions(+), 335 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/220ca960/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/220ca960/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/220ca960/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/220ca960/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 cae56c0..2e2ecfd 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.concurrent.ThreadLocalRandom;
 
 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
@@ -1513,7 +1514,7 @@ public class DFSUtil {
   public static int getSmallBufferSize(Configuration conf) {
     return Math.min(getIoFileBufferSize(conf) / 2, 512);
   }
-
+  
   /**
    * Probe for HDFS Encryption being enabled; this uses the value of
    * the option {@link DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI},
@@ -1527,4 +1528,10 @@ public class DFSUtil {
         DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "").isEmpty();
   }
 
+  public static InterruptedIOException toInterruptedIOException(String message,
+      InterruptedException e) {
+    final InterruptedIOException iioe = new InterruptedIOException(message);
+    iioe.initCause(e);
+    return iioe;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/220ca960/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 631f386..8f07341 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
@@ -575,7 +575,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();
@@ -593,10 +593,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()) {
@@ -1751,7 +1748,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();
     }
@@ -1901,4 +1898,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/220ca960/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/220ca960/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 7392552..8f843d5 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
@@ -390,7 +390,7 @@ public class FSDirectory implements Closeable {
   void disableQuotaChecks() {
     skipQuotaCheck = true;
   }
-  
+
   /**
    * This is a wrapper for resolvePath(). If the path passed
    * is prefixed with /.reserved/raw, then it checks to ensure that the caller

http://git-wip-us.apache.org/repos/asf/hadoop/blob/220ca960/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 fdbacdc..4ec9bf9 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/220ca960/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/220ca960/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);
+    }
+  }
+}


[14/50] [abbrv] 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/6bacaa9a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6bacaa9a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6bacaa9a

Branch: refs/heads/HDFS-7285
Commit: 6bacaa9a5233cbad7f311ccd9d8f8dc9375c732d
Parents: 51ea117
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon May 11 12:22:12 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:49 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/6bacaa9a/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/6bacaa9a/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 d3051a3..eeab076 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/6bacaa9a/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 aa0343a..5b876f9 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
@@ -2477,7 +2477,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(),
@@ -3222,7 +3222,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/6bacaa9a/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/6bacaa9a/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 f87a29c..cc18770 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
@@ -706,11 +706,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
@@ -979,11 +975,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/6bacaa9a/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/6bacaa9a/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


[15/50] [abbrv] 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/51ea117f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/51ea117f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/51ea117f

Branch: refs/heads/HDFS-7285
Commit: 51ea117f883f9c049de58987dc66e07e71a68ee4
Parents: 9da9275
Author: Jing Zhao <ji...@apache.org>
Authored: Fri May 8 13:56:56 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:49 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      | 20 ++---
 7 files changed, 95 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51ea117f/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/51ea117f/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/51ea117f/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/51ea117f/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/51ea117f/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/51ea117f/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 a3d75b5..aa0343a 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
@@ -637,13 +637,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;
   }
   
   /**
@@ -700,7 +706,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);
     
@@ -738,7 +747,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);
   }
 
@@ -2250,12 +2261,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
@@ -2617,7 +2629,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 &&
@@ -3945,6 +3958,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/51ea117f/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 7f82d2d..3179b93 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;
@@ -152,6 +152,7 @@ import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
 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;
@@ -160,10 +161,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;
@@ -172,6 +171,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;
@@ -180,15 +180,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;
@@ -209,7 +209,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;
@@ -217,7 +217,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;
@@ -3799,7 +3798,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


[46/50] [abbrv] hadoop git commit: HDFS-8408. Revisit and refactor ErasureCodingInfo (Contributed by Vinayakumar B) Added missed file

Posted by zh...@apache.org.
HDFS-8408. Revisit and refactor ErasureCodingInfo (Contributed by Vinayakumar B)
Added missed file


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

Branch: refs/heads/HDFS-7285
Commit: 80c56c2d3167f5195c72802071ea6dfb52abff1a
Parents: 9a18598
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon May 25 13:24:50 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:07:15 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/ErasureCodingZone.java | 66 ++++++++++++++++++++
 1 file changed, 66 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/80c56c2d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
new file mode 100644
index 0000000..655def3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZone.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+/**
+ * Information about the EC Zone at the specified path.
+ */
+public class ErasureCodingZone {
+
+  private String dir;
+  private ECSchema schema;
+  private int cellSize;
+
+  public ErasureCodingZone(String dir, ECSchema schema, int cellSize) {
+    this.dir = dir;
+    this.schema = schema;
+    this.cellSize = cellSize;
+  }
+
+  /**
+   * 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;
+  }
+
+  /**
+   * Get cellSize for the EC Zone
+   */
+  public int getCellSize() {
+    return cellSize;
+  }
+
+  @Override
+  public String toString() {
+    return "Dir: " + getDir() + ", Schema: " + schema + ", cellSize: "
+        + cellSize;
+  }
+}


[44/50] [abbrv] hadoop git commit: HDFS-8382 Remove chunkSize and initialize from erasure coder. Contributed by Kai Zheng

Posted by zh...@apache.org.
HDFS-8382 Remove chunkSize and initialize from erasure coder. 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/b30e96bf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b30e96bf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b30e96bf

Branch: refs/heads/HDFS-7285
Commit: b30e96bfb4b8ce5537671c97f0c9c56cd195bfdc
Parents: 0ed92e5
Author: Kai Zheng <ka...@intel.com>
Authored: Mon May 25 16:13:29 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:07:15 2015 -0700

----------------------------------------------------------------------
 .../erasurecode/codec/AbstractErasureCodec.java | 43 ++------------------
 .../io/erasurecode/codec/ErasureCodec.java      |  7 ----
 .../io/erasurecode/codec/RSErasureCodec.java    | 13 ++++--
 .../io/erasurecode/codec/XORErasureCodec.java   | 13 +++---
 .../erasurecode/coder/AbstractErasureCoder.java | 39 ++++++++----------
 .../coder/AbstractErasureDecoder.java           |  9 ++++
 .../coder/AbstractErasureEncoder.java           |  9 ++++
 .../io/erasurecode/coder/ErasureCoder.java      | 27 ++----------
 .../io/erasurecode/coder/RSErasureDecoder.java  | 16 ++++++--
 .../io/erasurecode/coder/RSErasureEncoder.java  | 16 ++++++--
 .../io/erasurecode/coder/XORErasureDecoder.java | 14 +++++--
 .../io/erasurecode/coder/XORErasureEncoder.java | 16 ++++++--
 .../rawcoder/AbstractRawErasureCoder.java       | 15 ++-----
 .../rawcoder/AbstractRawErasureDecoder.java     |  4 ++
 .../rawcoder/AbstractRawErasureEncoder.java     |  4 ++
 .../io/erasurecode/rawcoder/RSRawDecoder.java   |  5 +--
 .../io/erasurecode/rawcoder/RSRawEncoder.java   |  6 +--
 .../rawcoder/RSRawErasureCoderFactory.java      |  8 ++--
 .../erasurecode/rawcoder/RawErasureCoder.java   | 14 -------
 .../rawcoder/RawErasureCoderFactory.java        |  8 +++-
 .../io/erasurecode/rawcoder/XORRawDecoder.java  |  4 ++
 .../io/erasurecode/rawcoder/XORRawEncoder.java  |  4 ++
 .../rawcoder/XORRawErasureCoderFactory.java     |  8 ++--
 .../erasurecode/coder/TestErasureCoderBase.java | 42 +++++++++++--------
 .../erasurecode/rawcoder/TestRawCoderBase.java  | 15 ++++---
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  2 +
 .../hadoop/hdfs/DFSStripedOutputStream.java     |  3 +-
 .../erasurecode/ErasureCodingWorker.java        | 14 +++----
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  3 +-
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |  4 +-
 .../hadoop/hdfs/TestWriteReadStripedFile.java   |  3 +-
 31 files changed, 187 insertions(+), 201 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
index 9993786..0cacfbc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.io.erasurecode.codec;
 
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.erasurecode.ECSchema;
-import org.apache.hadoop.io.erasurecode.coder.*;
 import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper;
 
 /**
@@ -28,10 +27,9 @@ import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper;
 public abstract class AbstractErasureCodec extends Configured
     implements ErasureCodec {
 
-  private ECSchema schema;
+  private final ECSchema schema;
 
-  @Override
-  public void setSchema(ECSchema schema) {
+  public AbstractErasureCodec(ECSchema schema) {
     this.schema = schema;
   }
 
@@ -39,7 +37,7 @@ public abstract class AbstractErasureCodec extends Configured
     return schema.getCodecName();
   }
 
-  protected ECSchema getSchema() {
+  public ECSchema getSchema() {
     return schema;
   }
 
@@ -50,39 +48,4 @@ public abstract class AbstractErasureCodec extends Configured
 
     return blockGrouper;
   }
-
-  @Override
-  public ErasureCoder createEncoder() {
-    ErasureCoder encoder = doCreateEncoder();
-    prepareErasureCoder(encoder);
-    return encoder;
-  }
-
-  /**
-   * Create a new encoder instance to be initialized afterwards.
-   * @return encoder
-   */
-  protected abstract ErasureCoder doCreateEncoder();
-
-  @Override
-  public ErasureCoder createDecoder() {
-    ErasureCoder decoder = doCreateDecoder();
-    prepareErasureCoder(decoder);
-    return decoder;
-  }
-
-  /**
-   * Create a new decoder instance to be initialized afterwards.
-   * @return decoder
-   */
-  protected abstract ErasureCoder doCreateDecoder();
-
-  private void prepareErasureCoder(ErasureCoder erasureCoder) {
-    if (getSchema() == null) {
-      throw new RuntimeException("No schema been set yet");
-    }
-
-    erasureCoder.setConf(getConf());
-    erasureCoder.initialize(getSchema());
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
index e639484..9aa3db2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.io.erasurecode.codec;
 
 import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
 import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper;
 
@@ -30,12 +29,6 @@ import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper;
 public interface ErasureCodec extends Configurable {
 
   /**
-   * Set EC schema to be used by this codec.
-   * @param schema
-   */
-  public void setSchema(ECSchema schema);
-
-  /**
    * Create block grouper
    * @return block grouper
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
index 9e91b60..6edd638 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.io.erasurecode.codec;
 
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
 import org.apache.hadoop.io.erasurecode.coder.RSErasureDecoder;
 import org.apache.hadoop.io.erasurecode.coder.RSErasureEncoder;
@@ -26,13 +27,17 @@ import org.apache.hadoop.io.erasurecode.coder.RSErasureEncoder;
  */
 public class RSErasureCodec extends AbstractErasureCodec {
 
+  public RSErasureCodec(ECSchema schema) {
+    super(schema);
+  }
+
   @Override
-  protected ErasureCoder doCreateEncoder() {
-    return new RSErasureEncoder();
+  public ErasureCoder createEncoder() {
+    return new RSErasureEncoder(getSchema());
   }
 
   @Override
-  protected ErasureCoder doCreateDecoder() {
-    return new RSErasureDecoder();
+  public ErasureCoder createDecoder() {
+    return new RSErasureDecoder(getSchema());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
index 0f726d7..e2dcfa7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
@@ -27,19 +27,18 @@ import org.apache.hadoop.io.erasurecode.coder.XORErasureEncoder;
  */
 public class XORErasureCodec extends AbstractErasureCodec {
 
-  @Override
-  public void setSchema(ECSchema schema) {
-    super.setSchema(schema);
+  public XORErasureCodec(ECSchema schema) {
+    super(schema);
     assert(schema.getNumParityUnits() == 1);
   }
 
   @Override
-  protected ErasureCoder doCreateEncoder() {
-    return new XORErasureEncoder();
+  public ErasureCoder createEncoder() {
+    return new XORErasureEncoder(getSchema());
   }
 
   @Override
-  protected ErasureCoder doCreateDecoder() {
-    return new XORErasureDecoder();
+  public ErasureCoder createDecoder() {
+    return new XORErasureDecoder(getSchema());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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..c572bad 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
@@ -33,18 +33,18 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 public abstract class AbstractErasureCoder
     extends Configured implements ErasureCoder {
 
-  private int numDataUnits;
-  private int numParityUnits;
-  private int chunkSize;
+  private final int numDataUnits;
+  private final int numParityUnits;
 
   /**
    * Create raw decoder using the factory specified by rawCoderFactoryKey
    * @param rawCoderFactoryKey
    * @return raw decoder
    */
-  protected RawErasureDecoder createRawDecoder(String rawCoderFactoryKey) {
+  protected RawErasureDecoder createRawDecoder(
+          String rawCoderFactoryKey, int dataUnitsCount, int parityUnitsCount) {
     RawErasureCoder rawCoder = createRawCoder(getConf(),
-        rawCoderFactoryKey, false);
+        rawCoderFactoryKey, false, dataUnitsCount, parityUnitsCount);
     return (RawErasureDecoder) rawCoder;
   }
 
@@ -53,9 +53,10 @@ public abstract class AbstractErasureCoder
    * @param rawCoderFactoryKey
    * @return raw encoder
    */
-  protected RawErasureEncoder createRawEncoder(String rawCoderFactoryKey) {
+  protected RawErasureEncoder createRawEncoder(
+          String rawCoderFactoryKey, int dataUnitsCount, int parityUnitsCount) {
     RawErasureCoder rawCoder = createRawCoder(getConf(),
-        rawCoderFactoryKey, true);
+        rawCoderFactoryKey, true, dataUnitsCount, parityUnitsCount);
     return (RawErasureEncoder) rawCoder;
   }
 
@@ -67,7 +68,8 @@ public abstract class AbstractErasureCoder
    * @return raw coder
    */
   public static RawErasureCoder createRawCoder(Configuration conf,
-      String rawCoderFactoryKey, boolean isEncoder) {
+      String rawCoderFactoryKey, boolean isEncoder, int numDataUnits,
+                                               int numParityUnits) {
 
     if (conf == null) {
       return null;
@@ -90,21 +92,17 @@ public abstract class AbstractErasureCoder
       throw new RuntimeException("Failed to create raw coder", e);
     }
 
-    return isEncoder ? fact.createEncoder() : fact.createDecoder();
+    return isEncoder ? fact.createEncoder(numDataUnits, numParityUnits) :
+            fact.createDecoder(numDataUnits, numParityUnits);
   }
 
-  @Override
-  public void initialize(int numDataUnits, int numParityUnits,
-                         int chunkSize) {
+  public AbstractErasureCoder(int numDataUnits, int numParityUnits) {
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
-    this.chunkSize = chunkSize;
   }
 
-  @Override
-  public void initialize(ECSchema schema) {
-      initialize(schema.getNumDataUnits(), schema.getNumParityUnits(),
-          schema.getChunkSize());
+  public AbstractErasureCoder(ECSchema schema) {
+      this(schema.getNumDataUnits(), schema.getNumParityUnits());
   }
 
   @Override
@@ -118,12 +116,7 @@ public abstract class AbstractErasureCoder
   }
 
   @Override
-  public int getChunkSize() {
-    return chunkSize;
-  }
-
-  @Override
-  public boolean preferNativeBuffer() {
+  public boolean preferDirectBuffer() {
     return false;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
index 6437236..3ea9311 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 /**
  * An abstract erasure decoder that's to be inherited by new decoders.
@@ -27,6 +28,14 @@ import org.apache.hadoop.io.erasurecode.ECBlockGroup;
  */
 public abstract class AbstractErasureDecoder extends AbstractErasureCoder {
 
+  public AbstractErasureDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public AbstractErasureDecoder(ECSchema schema) {
+    super(schema);
+  }
+
   @Override
   public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
     // We may have more than this when considering complicate cases. HADOOP-11550

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
index a836b75..7c887e8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 /**
  * An abstract erasure encoder that's to be inherited by new encoders.
@@ -27,6 +28,14 @@ import org.apache.hadoop.io.erasurecode.ECBlockGroup;
  */
 public abstract class AbstractErasureEncoder extends AbstractErasureCoder {
 
+  public AbstractErasureEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public AbstractErasureEncoder(ECSchema schema) {
+    super(schema);
+  }
+
   @Override
   public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
     // We may have more than this when considering complicate cases. HADOOP-11550

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
index 64a82ea..f05ea41 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 
 /**
  * An erasure coder to perform encoding or decoding given a group. Generally it
@@ -38,20 +37,6 @@ import org.apache.hadoop.io.erasurecode.ECSchema;
 public interface ErasureCoder extends Configurable {
 
   /**
-   * Initialize with the important parameters for the code.
-   * @param numDataUnits how many data inputs for the coding
-   * @param numParityUnits how many parity outputs the coding generates
-   * @param chunkSize the size of the input/output buffer
-   */
-  public void initialize(int numDataUnits, int numParityUnits, int chunkSize);
-
-  /**
-   * Initialize with an EC schema.
-   * @param schema
-   */
-  public void initialize(ECSchema schema);
-
-  /**
    * The number of data input units for the coding. A unit can be a byte,
    * chunk or buffer or even a block.
    * @return count of data input units
@@ -66,12 +51,6 @@ public interface ErasureCoder extends Configurable {
   public int getNumParityUnits();
 
   /**
-   * Chunk buffer size for the input/output
-   * @return chunk buffer size
-   */
-  public int getChunkSize();
-
-  /**
    * Calculate the encoding or decoding steps given a block blockGroup.
    *
    * Note, currently only one coding step is supported. Will support complex
@@ -83,13 +62,13 @@ public interface ErasureCoder extends Configurable {
   public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup);
 
   /**
-   * Tell if native or off-heap buffer is preferred or not. It's for callers to
+   * Tell if direct 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,
+   * @return true if direct buffer is preferred for performance consideration,
    * otherwise false.
    */
-  public boolean preferNativeBuffer();
+  public boolean preferDirectBuffer();
 
   /**
    * Release the resources if any. Good chance to invoke RawErasureCoder#release.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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..57f4373 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 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.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 
@@ -31,6 +32,14 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 public class RSErasureDecoder extends AbstractErasureDecoder {
   private RawErasureDecoder rsRawDecoder;
 
+  public RSErasureDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public RSErasureDecoder(ECSchema schema) {
+    super(schema);
+  }
+
   @Override
   protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
 
@@ -45,12 +54,11 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
   private RawErasureDecoder checkCreateRSRawDecoder() {
     if (rsRawDecoder == null) {
       rsRawDecoder = createRawDecoder(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
+          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
+              getNumDataUnits(), getNumParityUnits());
       if (rsRawDecoder == null) {
-        rsRawDecoder = new RSRawDecoder();
+        rsRawDecoder = new RSRawDecoder(getNumDataUnits(), getNumParityUnits());
       }
-      rsRawDecoder.initialize(getNumDataUnits(),
-          getNumParityUnits(), getChunkSize());
     }
     return rsRawDecoder;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
index 18ca5ac..ab23474 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 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.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 
@@ -31,6 +32,14 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 public class RSErasureEncoder extends AbstractErasureEncoder {
   private RawErasureEncoder rawEncoder;
 
+  public RSErasureEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public RSErasureEncoder(ECSchema schema) {
+    super(schema);
+  }
+
   @Override
   protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
 
@@ -45,12 +54,11 @@ public class RSErasureEncoder extends AbstractErasureEncoder {
   private RawErasureEncoder checkCreateRSRawEncoder() {
     if (rawEncoder == null) {
       rawEncoder = createRawEncoder(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
+          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
+              getNumDataUnits(), getNumParityUnits());
       if (rawEncoder == null) {
-        rawEncoder = new RSRawEncoder();
+        rawEncoder = new RSRawEncoder(getNumDataUnits(), getNumParityUnits());
       }
-      rawEncoder.initialize(getNumDataUnits(),
-          getNumParityUnits(), getChunkSize());
     }
     return rawEncoder;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
index 0672549..3fe8d1b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 
@@ -29,12 +30,19 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
  */
 public class XORErasureDecoder extends AbstractErasureDecoder {
 
+  public XORErasureDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public XORErasureDecoder(ECSchema schema) {
+    super(schema);
+  }
+
   @Override
   protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
     // May be configured
-    RawErasureDecoder rawDecoder = new XORRawDecoder();
-    rawDecoder.initialize(getNumDataUnits(),
-        getNumParityUnits(), getChunkSize());
+    RawErasureDecoder rawDecoder = new XORRawDecoder(
+            getNumDataUnits(), getNumParityUnits());
 
     ECBlock[] inputBlocks = getInputBlocks(blockGroup);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
index 9011857..5020896 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
@@ -19,22 +19,30 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.XORRawEncoder;
 
 /**
  * Xor erasure encoder that encodes a block group.
  *
- * It implements {@link ErasureEncoder}.
+ * It implements {@link ErasureCoder}.
  */
 public class XORErasureEncoder extends AbstractErasureEncoder {
 
+  public XORErasureEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public XORErasureEncoder(ECSchema schema) {
+    super(schema);
+  }
+
   @Override
   protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
     // May be configured
-    RawErasureEncoder rawEncoder = new XORRawEncoder();
-    rawEncoder.initialize(getNumDataUnits(),
-        getNumParityUnits(), getChunkSize());
+    RawErasureEncoder rawEncoder = new XORRawEncoder(
+            getNumDataUnits(), getNumParityUnits());
 
     ECBlock[] inputBlocks = getInputBlocks(blockGroup);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 5268962..06ae660 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
@@ -31,16 +31,12 @@ import java.util.Arrays;
 public abstract class AbstractRawErasureCoder
     extends Configured implements RawErasureCoder {
 
-  private int numDataUnits;
-  private int numParityUnits;
-  private int chunkSize;
+  private final int numDataUnits;
+  private final int numParityUnits;
 
-  @Override
-  public void initialize(int numDataUnits, int numParityUnits,
-                         int chunkSize) {
+  public AbstractRawErasureCoder(int numDataUnits, int numParityUnits) {
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
-    this.chunkSize = chunkSize;
   }
 
   @Override
@@ -54,11 +50,6 @@ public abstract class AbstractRawErasureCoder
   }
 
   @Override
-  public int getChunkSize() {
-    return chunkSize;
-  }
-
-  @Override
   public boolean preferDirectBuffer() {
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 31f4fb8..0c1f80f 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
@@ -30,6 +30,10 @@ import java.nio.ByteBuffer;
 public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
     implements RawErasureDecoder {
 
+  public AbstractRawErasureDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
   @Override
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 0ae54c5..c7a136b 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
@@ -30,6 +30,10 @@ import java.nio.ByteBuffer;
 public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
     implements RawErasureEncoder {
 
+  public AbstractRawErasureEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
   @Override
   public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
     checkParameters(inputs, outputs);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
index ff1162f..e265dce 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
@@ -31,9 +31,8 @@ public class RSRawDecoder extends AbstractRawErasureDecoder {
   private int[] errSignature;
   private int[] primitivePower;
 
-  @Override
-  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
-    super.initialize(numDataUnits, numParityUnits, chunkSize);
+  public RSRawDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
     assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
 
     this.errSignature = new int[numParityUnits];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
index 9136331..efeee90 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
@@ -29,9 +29,9 @@ import java.nio.ByteBuffer;
 public class RSRawEncoder extends AbstractRawErasureEncoder {
   private int[] generatingPolynomial;
 
-  @Override
-  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
-    super.initialize(numDataUnits, numParityUnits, chunkSize);
+  public RSRawEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+
     assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
 
     int[] primitivePower = RSUtil.getPrimitivePower(numDataUnits,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
index 19a95af..5db49e3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
@@ -23,12 +23,12 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 public class RSRawErasureCoderFactory implements RawErasureCoderFactory {
 
   @Override
-  public RawErasureEncoder createEncoder() {
-    return new RSRawEncoder();
+  public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits) {
+    return new RSRawEncoder(numDataUnits, numParityUnits);
   }
 
   @Override
-  public RawErasureDecoder createDecoder() {
-    return new RSRawDecoder();
+  public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) {
+    return new RSRawDecoder(numDataUnits, numParityUnits);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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..0ddb460 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
@@ -36,14 +36,6 @@ import org.apache.hadoop.conf.Configurable;
 public interface RawErasureCoder extends Configurable {
 
   /**
-   * Initialize with the important parameters for the code.
-   * @param numDataUnits how many data inputs for the coding
-   * @param numParityUnits how many parity outputs the coding generates
-   * @param chunkSize the size of the input/output buffer
-   */
-  public void initialize(int numDataUnits, int numParityUnits, int chunkSize);
-
-  /**
    * The number of data input units for the coding. A unit can be a byte,
    * chunk or buffer or even a block.
    * @return count of data input units
@@ -58,12 +50,6 @@ public interface RawErasureCoder extends Configurable {
   public int getNumParityUnits();
 
   /**
-   * Chunk buffer size for the input/output
-   * @return chunk buffer size
-   */
-  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.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
index 95a1cfe..26eddfc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
@@ -26,13 +26,17 @@ public interface RawErasureCoderFactory {
 
   /**
    * Create raw erasure encoder.
+   * @param numDataUnits
+   * @param numParityUnits
    * @return raw erasure encoder
    */
-  public RawErasureEncoder createEncoder();
+  public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits);
 
   /**
    * Create raw erasure decoder.
+   * @param numDataUnits
+   * @param numParityUnits
    * @return raw erasure decoder
    */
-  public RawErasureDecoder createDecoder();
+  public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 bf6e894..a09105c 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
@@ -28,6 +28,10 @@ import java.nio.ByteBuffer;
  */
 public class XORRawDecoder extends AbstractRawErasureDecoder {
 
+  public XORRawDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
   @Override
   protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
                           ByteBuffer[] outputs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 feffbbf..894f20c 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
@@ -28,6 +28,10 @@ import java.nio.ByteBuffer;
  */
 public class XORRawEncoder extends AbstractRawErasureEncoder {
 
+  public XORRawEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
   protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
     ByteBuffer output = outputs[0];
     resetOutputBuffer(output);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java
index 67f45c1..de20c95 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java
@@ -23,12 +23,12 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 public class XORRawErasureCoderFactory implements RawErasureCoderFactory {
 
   @Override
-  public RawErasureEncoder createEncoder() {
-    return new XORRawEncoder();
+  public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits) {
+    return new XORRawEncoder(numDataUnits, numParityUnits);
   }
 
   @Override
-  public RawErasureDecoder createDecoder() {
-    return new XORRawDecoder();
+  public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) {
+    return new XORRawDecoder(numDataUnits, numParityUnits);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 154ec18..f9666b6 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
@@ -22,6 +22,8 @@ import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 import org.apache.hadoop.io.erasurecode.ECChunk;
 import org.apache.hadoop.io.erasurecode.TestCoderBase;
 
+import java.lang.reflect.Constructor;
+
 /**
  * Erasure coder test base with utilities.
  */
@@ -139,46 +141,50 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
   }
 
+  private void prepareCoders() {
+    if (encoder == null) {
+      encoder = createEncoder();
+    }
+
+    if (decoder == null) {
+      decoder = createDecoder();
+    }
+  }
+
   /**
-   * Create erasure encoder for test.
+   * Create the raw erasure encoder to test
    * @return
    */
-  private ErasureCoder createEncoder() {
+  protected ErasureCoder createEncoder() {
     ErasureCoder encoder;
     try {
-      encoder = encoderClass.newInstance();
+      Constructor<? extends ErasureCoder> constructor =
+          (Constructor<? extends ErasureCoder>)
+              encoderClass.getConstructor(int.class, int.class);
+      encoder = constructor.newInstance(numDataUnits, numParityUnits);
     } catch (Exception e) {
       throw new RuntimeException("Failed to create encoder", e);
     }
 
-    encoder.initialize(numDataUnits, numParityUnits, getChunkSize());
     encoder.setConf(getConf());
     return encoder;
   }
 
-  private void prepareCoders() {
-    if (encoder == null) {
-      encoder = createEncoder();
-    }
-
-    if (decoder == null) {
-      decoder = createDecoder();
-    }
-  }
-
   /**
-   * Create the erasure decoder for the test.
+   * create the raw erasure decoder to test
    * @return
    */
-  private ErasureCoder createDecoder() {
+  protected ErasureCoder createDecoder() {
     ErasureCoder decoder;
     try {
-      decoder = decoderClass.newInstance();
+      Constructor<? extends ErasureCoder> constructor =
+          (Constructor<? extends ErasureCoder>)
+              decoderClass.getConstructor(int.class, int.class);
+      decoder = constructor.newInstance(numDataUnits, numParityUnits);
     } catch (Exception e) {
       throw new RuntimeException("Failed to create decoder", e);
     }
 
-    decoder.initialize(numDataUnits, numParityUnits, getChunkSize());
     decoder.setConf(getConf());
     return decoder;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 4582317..cfaa2c5 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
@@ -21,6 +21,8 @@ import org.apache.hadoop.io.erasurecode.ECChunk;
 import org.apache.hadoop.io.erasurecode.TestCoderBase;
 import org.junit.Assert;
 
+import java.lang.reflect.Constructor;
+
 /**
  * Raw coder test base with utilities.
  */
@@ -136,12 +138,14 @@ public abstract class TestRawCoderBase extends TestCoderBase {
   protected RawErasureEncoder createEncoder() {
     RawErasureEncoder encoder;
     try {
-      encoder = encoderClass.newInstance();
+      Constructor<? extends RawErasureEncoder> constructor =
+              (Constructor<? extends RawErasureEncoder>)
+                      encoderClass.getConstructor(int.class, int.class);
+      encoder = constructor.newInstance(numDataUnits, numParityUnits);
     } catch (Exception e) {
       throw new RuntimeException("Failed to create encoder", e);
     }
 
-    encoder.initialize(numDataUnits, numParityUnits, getChunkSize());
     encoder.setConf(getConf());
     return encoder;
   }
@@ -153,14 +157,15 @@ public abstract class TestRawCoderBase extends TestCoderBase {
   protected RawErasureDecoder createDecoder() {
     RawErasureDecoder decoder;
     try {
-      decoder = decoderClass.newInstance();
+      Constructor<? extends RawErasureDecoder> constructor =
+              (Constructor<? extends RawErasureDecoder>)
+                      decoderClass.getConstructor(int.class, int.class);
+      decoder = constructor.newInstance(numDataUnits, numParityUnits);
     } catch (Exception e) {
       throw new RuntimeException("Failed to create decoder", e);
     }
 
-    decoder.initialize(numDataUnits, numParityUnits, getChunkSize());
     decoder.setConf(getConf());
     return decoder;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 efbadaf..4609fb6 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -255,3 +255,5 @@
     (waltersu4549)
 
     HDFS-7768. Change fsck to support EC files.  (Takanobu Asanuma via szetszwo)
+
+    HDFS-8382. Remove chunkSize and initialize from erasure coder. (Kai Zheng)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 515ce0c..0935d5c 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
@@ -247,8 +247,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     numDataBlocks = schema.getNumDataUnits();
     numAllBlocks = numDataBlocks + numParityBlocks;
 
-    encoder = new RSRawEncoder();
-    encoder.initialize(numDataBlocks, numParityBlocks, cellSize);
+    encoder = new RSRawEncoder(numDataBlocks, numParityBlocks);
 
     coordinator = new Coordinator(dfsClient.getConf(), numDataBlocks, numAllBlocks);
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 ded51eb..00cf0fd 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
@@ -111,12 +111,12 @@ public final class ErasureCodingWorker {
         DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT);
   }
 
-  private RawErasureEncoder newEncoder() {
-    return new RSRawEncoder();
+  private RawErasureEncoder newEncoder(int numDataUnits, int numParityUnits) {
+    return new RSRawEncoder(numDataUnits, numParityUnits);
   }
   
-  private RawErasureDecoder newDecoder() {
-    return new RSRawDecoder();
+  private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) {
+    return new RSRawDecoder(numDataUnits, numParityUnits);
   }
 
   private void initializeStripedReadThreadPool(int num) {
@@ -517,16 +517,14 @@ public final class ErasureCodingWorker {
     // Initialize encoder
     private void initEncoderIfNecessary() {
       if (encoder == null) {
-        encoder = newEncoder();
-        encoder.initialize(dataBlkNum, parityBlkNum, bufferSize);
+        encoder = newEncoder(dataBlkNum, parityBlkNum);
       }
     }
     
     // Initialize decoder
     private void initDecoderIfNecessary() {
       if (decoder == null) {
-        decoder = newDecoder();
-        decoder.initialize(dataBlkNum, parityBlkNum, bufferSize);
+        decoder = newDecoder(dataBlkNum, parityBlkNum);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 38dc61a..8f63236 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
@@ -287,8 +287,7 @@ public class StripedBlockUtil {
     }
 
     byte[][] outputs = new byte[parityBlkNum][(int) alignedStripe.getSpanInBlock()];
-    RSRawDecoder rsRawDecoder = new RSRawDecoder();
-    rsRawDecoder.initialize(dataBlkNum, parityBlkNum, (int) alignedStripe.getSpanInBlock());
+    RSRawDecoder rsRawDecoder = new RSRawDecoder(dataBlkNum, parityBlkNum);
     rsRawDecoder.decode(decodeInputs, decodeIndices, outputs);
 
     for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 e795ea7..e041dbe 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
@@ -274,8 +274,8 @@ public class TestDFSStripedOutputStream {
         System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length);
       }
     }
-    final RawErasureEncoder encoder = new RSRawEncoder();
-    encoder.initialize(dataBytes.length, parityBytes.length, cellSize);
+    final RawErasureEncoder encoder =
+            new RSRawEncoder(dataBytes.length, parityBytes.length);
     encoder.encode(dataBytes, expectedParityBytes);
     for (int i = 0; i < parityBytes.length; i++) {
       if (i != killedDnIndex) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b30e96bf/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 1976dca..70802fb 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
@@ -382,8 +382,7 @@ public class TestWriteReadStripedFile {
       Assert.assertEquals("The length of file should be the same to write size",
           length - startOffsetInFile, readLen);
 
-      RSRawDecoder rsRawDecoder = new RSRawDecoder();
-      rsRawDecoder.initialize(dataBlocks, parityBlocks, 1);
+      RSRawDecoder rsRawDecoder = new RSRawDecoder(dataBlocks, parityBlocks);
       byte[] expected = new byte[readLen];
       for (int i = startOffsetInFile; i < length; i++) {
         //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938


[10/50] [abbrv] 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/a17cedb4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a17cedb4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a17cedb4

Branch: refs/heads/HDFS-7285
Commit: a17cedb44c2a98d863407fab95c2b7f0893d0727
Parents: ac97edd
Author: Jing Zhao <ji...@apache.org>
Authored: Thu May 7 11:06:40 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:48 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/a17cedb4/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/a17cedb4/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/a17cedb4/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);


[02/50] [abbrv] 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/1a31f1c3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1a31f1c3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1a31f1c3

Branch: refs/heads/HDFS-7285
Commit: 1a31f1c303e8f541f0b22eccf85f3b789a220f21
Parents: f062828
Author: Zhe Zhang <zh...@apache.org>
Authored: Thu Apr 30 00:13:32 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 11:59:56 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/1a31f1c3/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/1a31f1c3/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");
+    }
   }
 
   /**


[11/50] [abbrv] 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/ac97edd1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ac97edd1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ac97edd1

Branch: refs/heads/HDFS-7285
Commit: ac97edd1abcca2be93aa3a8dcdc642734c7c00ab
Parents: cea46f7
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Thu May 7 16:26:01 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:48 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/ac97edd1/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/ac97edd1/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 84e7871..8f250fd 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,8 +119,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;
@@ -1193,7 +1193,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 {
@@ -3134,7 +3134,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 {
@@ -3356,7 +3356,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/ac97edd1/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/ac97edd1/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 d471271..0cd8334 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
@@ -75,7 +75,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;
@@ -2315,18 +2315,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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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 8f843d5..b830157 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
@@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 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;
@@ -1250,7 +1250,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/ac97edd1/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 79dd3d7..7f82d2d 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
@@ -180,8 +180,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;
@@ -428,7 +428,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;
@@ -608,7 +608,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     leaseManager.removeAllLeases();
     snapshotManager.clearSnapshottableDirs();
     cacheManager.clear();
-    schemaManager.clear();
+    ecSchemaManager.clear();
     setImageLoaded(false);
     blockManager.clear();
   }
@@ -848,7 +848,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);
@@ -6632,8 +6632,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /** @return the schema manager. */
-  public ECSchemaManager getSchemaManager() {
-    return schemaManager;
+  public ErasureCodingSchemaManager getECSchemaManager() {
+    return ecSchemaManager;
   }
 
   @Override  // NameNodeMXBean
@@ -7579,11 +7579,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;
   }
@@ -7591,7 +7591,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
@@ -7620,7 +7620,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return schemaManager.getSchemas();
+      return ecSchemaManager.getSchemas();
     } finally {
       readUnlock();
     }
@@ -7635,7 +7635,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/ac97edd1/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 dcf0607..5a69b2f 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;
@@ -2035,7 +2035,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);
   }
@@ -2047,7 +2047,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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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/ac97edd1/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);


[32/50] [abbrv] hadoop git commit: HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe. Contributed by Walter Su.

Posted by zh...@apache.org.
HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe. Contributed by Walter Su.


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

Branch: refs/heads/HDFS-7285
Commit: 6c310db15939bf5199894060607ec10ecfbe0877
Parents: 343c0e7
Author: Jing Zhao <ji...@apache.org>
Authored: Mon May 18 15:08:30 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:32 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 +
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  3 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  | 72 ++++++++++++--------
 .../server/datanode/SimulatedFSDataset.java     |  2 +-
 4 files changed, 50 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c310db1/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 333d85f..e016ba0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -217,3 +217,6 @@
     assigning new tasks. (umamahesh)
 
     HDFS-8367. BlockInfoStriped uses EC schema. (Kai Sasaki via Kai Zheng)
+
+    HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe.
+    (Walter Su via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c310db1/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 c95f0b4..81c0c95 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
@@ -379,7 +379,8 @@ public class StripedBlockUtil {
     int firstCellIdxInBG = (int) (start / cellSize);
     int lastCellIdxInBG = (int) (end / cellSize);
     int firstCellSize = Math.min(cellSize - (int) (start % cellSize), len);
-    long firstCellOffsetInBlk = start % cellSize;
+    long firstCellOffsetInBlk = firstCellIdxInBG / dataBlkNum * cellSize +
+        start % cellSize;
     int lastCellSize = lastCellIdxInBG == firstCellIdxInBG ?
         firstCellSize : (int) (end % cellSize) + 1;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c310db1/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 452cc2b..9032d09 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -115,40 +116,55 @@ public class TestDFSStripedInputStream {
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
         NUM_STRIPE_PER_BLOCK, false);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
-        filePath.toString(), 0, BLOCK_GROUP_SIZE);
+        filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
+    int fileLen = BLOCK_GROUP_SIZE * numBlocks;
 
-    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, schema);
-    int readSize = BLOCK_GROUP_SIZE;
-    byte[] readBuffer = new byte[readSize];
-    int ret = in.read(0, readBuffer, 0, readSize);
+    byte[] expected = new byte[fileLen];
+    assertEquals(numBlocks, lbs.getLocatedBlocks().size());
+    for (int bgIdx = 0; bgIdx < numBlocks; bgIdx++) {
+      LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(bgIdx));
+      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());
+      }
 
-    byte[] expected = new byte[readSize];
-    /** 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 = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k;
-          expected[posInFile] = SimulatedFSDataset.simulatedByte(
-              new Block(bg.getBlock().getBlockId() + j), posInBlk);
+      /** 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 = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k;
+            expected[bgIdx*BLOCK_GROUP_SIZE + posInFile] =
+                SimulatedFSDataset.simulatedByte(
+                    new Block(bg.getBlock().getBlockId() + j), posInBlk);
+          }
         }
       }
     }
+    DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
+            filePath.toString(), false, schema);
 
-    assertEquals(readSize, ret);
-    assertArrayEquals(expected, readBuffer);
+    int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102,
+        CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102,
+        BLOCK_GROUP_SIZE - 102, BLOCK_GROUP_SIZE, BLOCK_GROUP_SIZE + 102,
+        fileLen - 1};
+    for (int startOffset : startOffsets) {
+      startOffset = Math.max(0, Math.min(startOffset, fileLen - 1));
+      int remaining = fileLen - startOffset;
+      byte[] buf = new byte[fileLen];
+      int ret = in.read(startOffset, buf, 0, fileLen);
+      assertEquals(remaining, ret);
+      for (int i = 0; i < remaining; i++) {
+        Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " +
+                "same",
+            expected[startOffset + i], buf[i]);
+      }
+    }
+    in.close();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c310db1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 778dd28..dd6b600 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -100,7 +100,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   public static byte simulatedByte(Block b, long offsetInBlk) {
     byte firstByte = (byte) (b.getBlockId() & BYTE_MASK);
-    return (byte) ((firstByte + offsetInBlk) & BYTE_MASK);
+    return (byte) ((firstByte + offsetInBlk % 29) & BYTE_MASK);
   }
   
   public static final String CONFIG_PROPERTY_CAPACITY =


[40/50] [abbrv] hadoop git commit: HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream. Contributed by Tsz Wo Nicholas Sze.

Posted by zh...@apache.org.
HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream. 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/45db1a0b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/45db1a0b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/45db1a0b

Branch: refs/heads/HDFS-7285
Commit: 45db1a0b727965ceb27411ea5567b3a76c63b181
Parents: 47ef869
Author: Jing Zhao <ji...@apache.org>
Authored: Tue May 19 21:19:51 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:34 2015 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/LocatedStripedBlock.java      |  12 ++-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 106 +++++++++++--------
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  15 ++-
 .../apache/hadoop/hdfs/StripedDataStreamer.java |  67 +++++++++---
 .../blockmanagement/DatanodeStorageInfo.java    |  15 +--
 .../hdfs/server/namenode/FSNamesystem.java      |  38 ++++---
 .../hdfs/server/namenode/NameNodeRpcServer.java |   2 +-
 .../hadoop/hdfs/util/StripedBlockUtil.java      |  40 +++++--
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   8 +-
 .../TestDFSStripedOutputStreamWithFailure.java  |  69 +++++++-----
 .../server/namenode/TestAddStripedBlocks.java   |  12 ++-
 12 files changed, 258 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/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
index 93a5948..dc5a77f 100644
--- 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
@@ -31,15 +31,21 @@ import java.util.Arrays;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class LocatedStripedBlock extends LocatedBlock {
+  private static final int[] EMPTY_INDICES = {};
+
   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);
+
+    if (indices == null) {
+      this.blockIndices = EMPTY_INDICES;
+    } else {
+      this.blockIndices = new int[indices.length];
+      System.arraycopy(indices, 0, blockIndices, 0, indices.length);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/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 48bc9d6..b608b10 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -238,3 +238,6 @@
 
     HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file.
     (Yi Liu via zhz).
+
+    HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream.
+    (Tsz Wo Nicholas Sze via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/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 4399a37..8eed6ad 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
@@ -33,7 +33,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -61,52 +60,72 @@ import com.google.common.base.Preconditions;
 
 @InterfaceAudience.Private
 public class DFSStripedOutputStream extends DFSOutputStream {
-  /** Coordinate the communication between the streamers. */
-  static class Coordinator {
-    private final DfsClientConf conf;
-    private final List<BlockingQueue<ExtendedBlock>> endBlocks;
-    private final List<BlockingQueue<LocatedBlock>> stripedBlocks;
-    private volatile boolean shouldLocateFollowingBlock = false;
-
-    Coordinator(final DfsClientConf conf, final int numDataBlocks,
-                final int numAllBlocks) {
-      this.conf = conf;
-      endBlocks = new ArrayList<>(numDataBlocks);
-      for (int i = 0; i < numDataBlocks; i++) {
-        endBlocks.add(new LinkedBlockingQueue<ExtendedBlock>(1));
+  static class MultipleBlockingQueue<T> {
+    private final int pullTimeout;
+    private final List<BlockingQueue<T>> queues;
+
+    MultipleBlockingQueue(int numQueue, int queueSize, int pullTimeout) {
+      queues = new ArrayList<>(numQueue);
+      for (int i = 0; i < numQueue; i++) {
+        queues.add(new LinkedBlockingQueue<T>(queueSize));
       }
 
-      stripedBlocks = new ArrayList<>(numAllBlocks);
-      for (int i = 0; i < numAllBlocks; i++) {
-        stripedBlocks.add(new LinkedBlockingQueue<LocatedBlock>(1));
+      this.pullTimeout = pullTimeout;
+    }
+
+    void offer(int i, T object) {
+      final boolean b = queues.get(i).offer(object);
+      Preconditions.checkState(b, "Failed to offer " + object
+          + " to queue, i=" + i);
+    }
+
+    T poll(int i) throws InterruptedIOException {
+      try {
+        return queues.get(i).poll(pullTimeout, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        throw DFSUtil.toInterruptedIOException("poll interrupted, i=" + i, e);
       }
     }
 
-    boolean shouldLocateFollowingBlock() {
-      return shouldLocateFollowingBlock;
+    T peek(int i) {
+      return queues.get(i).peek();
     }
+  }
 
-    void putEndBlock(int i, ExtendedBlock block) {
-      shouldLocateFollowingBlock = true;
+  /** Coordinate the communication between the streamers. */
+  static class Coordinator {
+    private final MultipleBlockingQueue<LocatedBlock> stripedBlocks;
+    private final MultipleBlockingQueue<ExtendedBlock> endBlocks;
+    private final MultipleBlockingQueue<ExtendedBlock> updateBlocks;
 
-      final boolean b = endBlocks.get(i).offer(block);
-      Preconditions.checkState(b, "Failed to add " + block
-          + " to endBlocks queue, i=" + i);
+    Coordinator(final DfsClientConf conf, final int numDataBlocks,
+        final int numAllBlocks) {
+      stripedBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1,
+          conf.getStripedWriteMaxSecondsGetStripedBlock());
+      endBlocks = new MultipleBlockingQueue<>(numDataBlocks, 1,
+          conf.getStripedWriteMaxSecondsGetEndedBlock());
+      updateBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1,
+          conf.getStripedWriteMaxSecondsGetStripedBlock());
+    }
+
+    void putEndBlock(int i, ExtendedBlock block) {
+      endBlocks.offer(i, block);
     }
 
     ExtendedBlock getEndBlock(int i) throws InterruptedIOException {
-      try {
-        return endBlocks.get(i).poll(
-            conf.getStripedWriteMaxSecondsGetEndedBlock(),
-            TimeUnit.SECONDS);
-      } catch (InterruptedException e) {
-        throw DFSUtil.toInterruptedIOException(
-            "getEndBlock interrupted, i=" + i, e);
-      }
+      return endBlocks.poll(i);
+    }
+
+    void putUpdateBlock(int i, ExtendedBlock block) {
+      updateBlocks.offer(i, block);
+    }
+
+    ExtendedBlock getUpdateBlock(int i) throws InterruptedIOException {
+      return updateBlocks.poll(i);
     }
 
     void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) {
-      ExtendedBlock b = endBlocks.get(i).peek();
+      ExtendedBlock b = endBlocks.peek(i);
       if (b == null) {
         // streamer just has failed, put end block and continue
         b = block;
@@ -119,22 +138,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       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);
-      }
+      stripedBlocks.offer(i, block);
     }
 
     LocatedBlock getStripedBlock(int i) throws IOException {
-      final LocatedBlock lb;
-      try {
-        lb = stripedBlocks.get(i).poll(
-            conf.getStripedWriteMaxSecondsGetStripedBlock(),
-            TimeUnit.SECONDS);
-      } catch (InterruptedException e) {
-        throw DFSUtil.toInterruptedIOException("getStripedBlock interrupted", e);
-      }
-
+      final LocatedBlock lb = stripedBlocks.poll(i);
       if (lb == null) {
         throw new IOException("Failed: i=" + i);
       }
@@ -218,6 +226,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     return streamers.get(0);
   }
 
+  @Override
+  ExtendedBlock getBlock() {
+    return getLeadingStreamer().getBlock();
+  }
+
   /** Construct a new output stream for creating a file. */
   DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
                          EnumSet<CreateFlag> flag, Progressable progress,
@@ -292,6 +305,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     int count = 0;
     for(StripedDataStreamer s : streamers) {
       if (!s.isFailed()) {
+        s.getErrorState().initExtenalError();
         count++;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/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 8f07341..1344d54 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
@@ -44,7 +44,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -210,6 +209,7 @@ class DataStreamer extends Daemon {
 
   static class ErrorState {
     private boolean error = false;
+    private boolean extenalError = false;
     private int badNodeIndex = -1;
     private int restartingNodeIndex = -1;
     private long restartingNodeDeadline = 0;
@@ -221,6 +221,7 @@ class DataStreamer extends Daemon {
 
     synchronized void reset() {
       error = false;
+      extenalError = false;
       badNodeIndex = -1;
       restartingNodeIndex = -1;
       restartingNodeDeadline = 0;
@@ -231,13 +232,19 @@ class DataStreamer extends Daemon {
     }
 
     synchronized boolean hasDatanodeError() {
-      return error && isNodeMarked();
+      return error && (isNodeMarked() || extenalError);
     }
 
     synchronized void setError(boolean err) {
       this.error = err;
     }
 
+    synchronized void initExtenalError() {
+      setError(true);
+      this.extenalError = true;
+    }
+
+
     synchronized void setBadNodeIndex(int index) {
       this.badNodeIndex = index;
     }
@@ -1736,6 +1743,10 @@ class DataStreamer extends Daemon {
     return accessToken;
   }
 
+  ErrorState getErrorState() {
+    return errorState;
+  }
+
   /**
    * Put a packet to the data queue
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/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 258fc65..7b7db75 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
@@ -94,36 +94,69 @@ public class StripedDataStreamer extends DataStreamer {
   protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
       throws IOException {
     if (isLeadingStreamer()) {
-      if (coordinator.shouldLocateFollowingBlock()) {
+      if (block != null) {
         // 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();
+          if (b != null) {
+            StripedBlockUtil.checkBlocks(block, i, b);
+            bytes += b.getNumBytes();
+          }
         }
         block.setNumBytes(bytes);
       }
 
-      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;
-      for (int i = 0; i < blocks.length; i++) {
-        coordinator.putStripedBlock(i, blocks[i]);
-      }
+      putLoactedBlocks(super.locateFollowingBlock(excludedNodes));
     }
 
     return coordinator.getStripedBlock(index);
   }
 
+  void putLoactedBlocks(LocatedBlock lb) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Obtained block group " + lb);
+    }
+    LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
+        (LocatedStripedBlock)lb,
+        BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+
+    // TODO allow write to continue if blocks.length >= NUM_DATA_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;
+    for (int i = 0; i < blocks.length; i++) {
+      coordinator.putStripedBlock(i, blocks[i]);
+    }
+  }
+
+  @Override
+  LocatedBlock updateBlockForPipeline() throws IOException {
+    if (isLeadingStreamer()) {
+      final LocatedBlock updated = super.updateBlockForPipeline();
+      final ExtendedBlock block = updated.getBlock();
+      for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) {
+        final LocatedBlock lb = new LocatedBlock(block, null, null, null,
+                -1, updated.isCorrupt(), null);
+        lb.setBlockToken(updated.getBlockToken());
+        coordinator.putStripedBlock(i, lb);
+      }
+    }
+    return coordinator.getStripedBlock(index);
+  }
+
+  @Override
+  ExtendedBlock updatePipeline(long newGS) throws IOException {
+    if (isLeadingStreamer()) {
+      final ExtendedBlock newBlock = super.updatePipeline(newGS);
+      for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) {
+        coordinator.putUpdateBlock(i, new ExtendedBlock(newBlock));
+      }
+    }
+    return coordinator.getUpdateBlock(index);
+  }
+
   @Override
   public String toString() {
     return "#" + index + ": isFailed? " + Boolean.toString(isFailed).charAt(0)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index 9b5a923..2275d91 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@ -21,18 +21,15 @@ import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 
-import com.google.common.annotations.VisibleForTesting;
-
 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.DatanodeInfoWithStorage;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * A Datanode has one or more storages. A storage in the Datanode is represented
  * by this class.
@@ -41,7 +38,7 @@ public class DatanodeStorageInfo {
   public static final DatanodeStorageInfo[] EMPTY_ARRAY = {};
 
   public static DatanodeInfo[] toDatanodeInfos(DatanodeStorageInfo[] storages) {
-    return toDatanodeInfos(Arrays.asList(storages));
+    return storages == null? null: toDatanodeInfos(Arrays.asList(storages));
   }
   static DatanodeInfo[] toDatanodeInfos(List<DatanodeStorageInfo> storages) {
     final DatanodeInfo[] datanodes = new DatanodeInfo[storages.size()];
@@ -61,6 +58,9 @@ public class DatanodeStorageInfo {
   }
 
   public static String[] toStorageIDs(DatanodeStorageInfo[] storages) {
+    if (storages == null) {
+      return null;
+    }
     String[] storageIDs = new String[storages.length];
     for(int i = 0; i < storageIDs.length; i++) {
       storageIDs[i] = storages[i].getStorageID();
@@ -69,6 +69,9 @@ public class DatanodeStorageInfo {
   }
 
   public static StorageType[] toStorageTypes(DatanodeStorageInfo[] storages) {
+    if (storages == null) {
+      return null;
+    }
     StorageType[] storageTypes = new StorageType[storages.length];
     for(int i = 0; i < storageTypes.length; i++) {
       storageTypes[i] = storages[i].getStorageType();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/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 60f86d6..7e45e90 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
@@ -5787,29 +5787,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Get a new generation stamp together with an access token for 
    * a block under construction
    * 
-   * This method is called for recovering a failed pipeline or setting up
-   * a pipeline to append to a block.
+   * This method is called for recovering a failed write or setting up
+   * a block for appended.
    * 
    * @param block a block
    * @param clientName the name of a client
    * @return a located block with a new generation stamp and an access token
    * @throws IOException if any error occurs
    */
-  LocatedBlock updateBlockForPipeline(ExtendedBlock block, 
+  LocatedBlock bumpBlockGenerationStamp(ExtendedBlock block,
       String clientName) throws IOException {
-    LocatedBlock locatedBlock;
+    final LocatedBlock locatedBlock;
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
 
       // check vadility of parameters
-      checkUCBlock(block, clientName);
+      final INodeFile file = checkUCBlock(block, clientName);
   
       // get a new generation stamp and an access token
       block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock())));
-      locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]);
-      blockManager.setBlockToken(locatedBlock, BlockTokenIdentifier.AccessMode.WRITE);
+
+      locatedBlock = BlockManager.newLocatedBlock(
+          block, file.getLastBlock(), null, -1);
     } finally {
       writeUnlock();
     }
@@ -5864,23 +5865,26 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // check the vadility of the block and lease holder name
     final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
     final BlockInfo lastBlock = pendingFile.getLastBlock();
-    // when updating pipeline, the last block must be contiguous block
-    assert lastBlock instanceof BlockInfoContiguousUnderConstruction;
-    BlockInfoContiguousUnderConstruction blockinfo =
-        (BlockInfoContiguousUnderConstruction) lastBlock;
+    final BlockInfoUnderConstruction blockinfo = (BlockInfoUnderConstruction)lastBlock;
 
     // check new GS & length: this is not expected
-    if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() ||
-        newBlock.getNumBytes() < blockinfo.getNumBytes()) {
-      String msg = "Update " + oldBlock + " (len = " + 
-        blockinfo.getNumBytes() + ") to an older state: " + newBlock + 
-        " (len = " + newBlock.getNumBytes() +")";
+    if (newBlock.getGenerationStamp() <= lastBlock.getGenerationStamp()) {
+      final String msg = "Update " + oldBlock + " but the new block " + newBlock
+          + " does not have a larger generation stamp than the last block "
+          + lastBlock;
+      LOG.warn(msg);
+      throw new IOException(msg);
+    }
+    if (newBlock.getNumBytes() < lastBlock.getNumBytes()) {
+      final String msg = "Update " + oldBlock + " (size="
+          + oldBlock.getNumBytes() + ") to a smaller size block " + newBlock
+          + " (size=" + newBlock.getNumBytes() + ")";
       LOG.warn(msg);
       throw new IOException(msg);
     }
 
     // Update old block with the new generation stamp and new length
-    blockinfo.setNumBytes(newBlock.getNumBytes());
+    lastBlock.setNumBytes(newBlock.getNumBytes());
     blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
 
     // find the DatanodeDescriptor objects

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/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 747f528..9e94b90 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
@@ -788,7 +788,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName)
       throws IOException {
     checkNNStartup();
-    return namesystem.updateBlockForPipeline(block, clientName);
+    return namesystem.bumpBlockGenerationStamp(block, clientName);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/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 6f7dcb1..0b09f37 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
@@ -34,6 +34,7 @@ import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 
 import java.util.*;
+import java.io.IOException;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutionException;
@@ -104,12 +105,17 @@ public class StripedBlockUtil {
     final ExtendedBlock blk = constructInternalBlock(
         bg.getBlock(), 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);
+    final long offset = bg.getStartOffset() + idxInBlockGroup * cellSize;
+    if (idxInReturnedLocs < bg.getLocations().length) {
+      return new LocatedBlock(blk,
+          new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
+          new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
+          new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
+          offset, bg.isCorrupt(), null);
+    } else {
+      return new LocatedBlock(blk, null, null, null,
+          offset, bg.isCorrupt(), null);
+    }
   }
 
   /**
@@ -823,4 +829,26 @@ public class StripedBlockUtil {
       return "(index=" + index + ", state =" + state + ")";
     }
   }
+
+  /**
+   * Check if the information such as IDs and generation stamps in block-i
+   * match block-0.
+   */
+  public static void checkBlocks(ExtendedBlock block0, int i,
+      ExtendedBlock blocki) throws IOException {
+
+    if (!blocki.getBlockPoolId().equals(block0.getBlockPoolId())) {
+      throw new IOException("Block pool IDs mismatched: block0="
+          + block0 + ", block" + i + "=" + blocki);
+    }
+    if (blocki.getBlockId() - i != block0.getBlockId()) {
+      throw new IOException("Block IDs mismatched: block0="
+          + block0 + ", block" + i + "=" + blocki);
+    }
+    if (blocki.getGenerationStamp() != block0.getGenerationStamp()) {
+      throw new IOException("Generation stamps mismatched: block0="
+          + block0 + ", block" + i + "=" + blocki);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/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 558c45d..82c0781 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
@@ -1953,11 +1953,9 @@ public class DFSTestUtil {
    * 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);
+  public static ExtendedBlock flushInternal(DFSStripedOutputStream out)
+      throws IOException {
     out.flushInternal();
+    return out.getBlock();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index 4ad3b2e..c232e13 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -21,7 +21,6 @@ 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;
@@ -44,6 +43,8 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.base.Preconditions;
+
 public class TestDFSStripedOutputStreamWithFailure {
   public static final Log LOG = LogFactory.getLog(
       TestDFSStripedOutputStreamWithFailure.class);
@@ -59,6 +60,9 @@ public class TestDFSStripedOutputStreamWithFailure {
   private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK;
   private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS;
 
+  private static final int FLUSH_POS
+      = 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
+
   private final HdfsConfiguration conf = new HdfsConfiguration();
   private MiniDFSCluster cluster;
   private DistributedFileSystem dfs;
@@ -149,50 +153,53 @@ public class TestDFSStripedOutputStreamWithFailure {
       cluster.startDataNodes(conf, 1, true, null, null);
       cluster.waitActive();
 
-      runTest(new Path(dir, src), length, dnIndex);
+      runTest(new Path(dir, src), length, length/2, dnIndex);
     } catch(Exception e) {
       LOG.info("FAILED", e);
       Assert.fail(StringUtils.stringifyException(e));
     }
   }
 
-  private void runTest(final Path p, final int length,
+  private void runTest(final Path p, final int length, final int killPos,
       final int dnIndex) throws Exception {
-    LOG.info("p=" + p + ", length=" + length + ", dnIndex=" + dnIndex);
+    LOG.info("p=" + p + ", length=" + length + ", killPos=" + killPos
+        + ", dnIndex=" + dnIndex);
+    Preconditions.checkArgument(killPos < length);
+    Preconditions.checkArgument(killPos > FLUSH_POS);
     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 DFSStripedOutputStream stripedOut
+        = (DFSStripedOutputStream)out.getWrappedStream();
 
-    final int mask = (1 << 16) - 1;
+    long oldGS = -1;
+    boolean killed = false;
     for(; pos.get() < length; ) {
       final int i = pos.getAndIncrement();
+      if (i == killPos) {
+        final long gs = getGenerationStamp(stripedOut);
+        Assert.assertTrue(oldGS != -1);
+        Assert.assertEquals(oldGS, gs);
+
+        killDatanode(cluster, stripedOut, dnIndex, pos);
+        killed = true;
+      }
+
       write(out, i);
-      if ((i & mask) == 0) {
-        final long ms = 100;
-        LOG.info("i=" + i + " sleep " + ms);
-        Thread.sleep(ms);
+
+      if (i == FLUSH_POS) {
+        oldGS = getGenerationStamp(stripedOut);
       }
     }
-    killer.join(10000);
-    Assert.assertTrue(killed.get());
     out.close();
+    Assert.assertTrue(killed);
 
     // check file length
     final FileStatus status = dfs.getFileStatus(p);
     Assert.assertEquals(length, status.getLen());
 
-    checkData(dfs, fullPath, length, dnIndex);
+    checkData(dfs, fullPath, length, dnIndex, oldGS);
   }
 
   static void write(FSDataOutputStream out, int i) throws IOException {
@@ -203,6 +210,14 @@ public class TestDFSStripedOutputStreamWithFailure {
     }
   }
 
+  static long getGenerationStamp(DFSStripedOutputStream out)
+      throws IOException {
+    final long gs = DFSTestUtil.flushInternal(out).getGenerationStamp();
+    LOG.info("getGenerationStamp returns " + gs);
+    return gs;
+
+  }
+
   static DatanodeInfo getDatanodes(StripedDataStreamer streamer) {
     for(;;) {
       final DatanodeInfo[] datanodes = streamer.getNodes();
@@ -228,7 +243,7 @@ public class TestDFSStripedOutputStreamWithFailure {
   }
 
   static void checkData(DistributedFileSystem dfs, String src, int length,
-      int killedDnIndex) throws IOException {
+      int killedDnIndex, long oldGS) throws IOException {
     List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
     LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(src, 0L);
     final int expectedNumGroup = (length - 1)/BLOCK_GROUP_SIZE + 1;
@@ -236,6 +251,12 @@ public class TestDFSStripedOutputStreamWithFailure {
 
     for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
       Assert.assertTrue(firstBlock instanceof LocatedStripedBlock);
+
+      final long gs = firstBlock.getBlock().getGenerationStamp();
+      final String s = "gs=" + gs + ", oldGS=" + oldGS;
+      LOG.info(s);
+      Assert.assertTrue(s, gs > oldGS);
+
       LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
           (LocatedStripedBlock) firstBlock,
           CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
@@ -247,7 +268,7 @@ public class TestDFSStripedOutputStreamWithFailure {
       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 numCellInGroup = (groupSize - 1)/CELL_SIZE + 1;
       final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS;
       final int lastCellSize = groupSize - (numCellInGroup - 1)*CELL_SIZE;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/45db1a0b/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 a35cbf4..7876d1a 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
@@ -105,6 +105,14 @@ public class TestAddStripedBlocks {
     Assert.assertEquals(firstId + HdfsServerConstants.MAX_BLOCKS_IN_GROUP, secondId);
   }
 
+  private static void writeAndFlushStripedOutputStream(
+      DFSStripedOutputStream out, int chunkSize) throws IOException {
+    // FSOutputSummer.BUFFER_NUM_CHUNKS == 9
+    byte[] toWrite = new byte[chunkSize * 9 + 1];
+    out.write(toWrite);
+    DFSTestUtil.flushInternal(out);
+  }
+
   @Test (timeout=60000)
   public void testAddStripedBlock() throws Exception {
     final Path file = new Path("/file1");
@@ -112,7 +120,7 @@ public class TestAddStripedBlocks {
     FSDataOutputStream out = null;
     try {
       out = dfs.create(file, (short) 1);
-      DFSTestUtil.writeAndFlushStripedOutputStream(
+      writeAndFlushStripedOutputStream(
           (DFSStripedOutputStream) out.getWrappedStream(),
           DFS_BYTES_PER_CHECKSUM_DEFAULT);
 
@@ -190,7 +198,7 @@ public class TestAddStripedBlocks {
     FSDataOutputStream out = null;
     try {
       out = dfs.create(file, (short) 1);
-      DFSTestUtil.writeAndFlushStripedOutputStream(
+      writeAndFlushStripedOutputStream(
           (DFSStripedOutputStream) out.getWrappedStream(),
           DFS_BYTES_PER_CHECKSUM_DEFAULT);
 


[06/50] [abbrv] 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/e849be2d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e849be2d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e849be2d

Branch: refs/heads/HDFS-7285
Commit: e849be2d31b6b5cda14f3c19b5469ca4d2742d7d
Parents: 4392325
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Tue May 5 19:25:21 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 11:59:58 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/e849be2d/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/e849be2d/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 722bb39..84e7871 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
@@ -3358,11 +3358,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();
     }
   }
 }


[12/50] [abbrv] 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/cea46f79
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cea46f79
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cea46f79

Branch: refs/heads/HDFS-7285
Commit: cea46f79b0fabdf42bd3ac6e25fa263b75791f57
Parents: 6616de2
Author: Zhe Zhang <zh...@apache.org>
Authored: Wed May 6 15:34:37 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:48 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/cea46f79/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/cea46f79/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/cea46f79/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/cea46f79/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());
+    }
+  }
+}


[45/50] [abbrv] hadoop git commit: HDFS-7768. Change fsck to support EC files. Contributed by Takanobu Asanuma

Posted by zh...@apache.org.
HDFS-7768. Change fsck to support EC files.  Contributed by Takanobu Asanuma


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

Branch: refs/heads/HDFS-7285
Commit: 0ed92e5b13f6bbc0ea7475bc67488359413a980e
Parents: 4ae32ab
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Sun May 24 23:07:34 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:07:15 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   2 +
 .../hdfs/server/namenode/NamenodeFsck.java      | 206 ++++++++++++++++---
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  75 ++++++-
 3 files changed, 238 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ed92e5b/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 d71b9c3..efbadaf 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -253,3 +253,5 @@
 
     HDFS-8441. Erasure Coding: make condition check earlier for setReplication.
     (waltersu4549)
+
+    HDFS-7768. Change fsck to support EC files.  (Takanobu Asanuma via szetszwo)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ed92e5b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 1067394..1c60a88 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NodeBase;
@@ -123,6 +124,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   private final int totalDatanodes;
   private final InetAddress remoteAddress;
 
+  private long totalDirs = 0L;
+  private long totalSymlinks = 0L;
+
   private String lostFound = null;
   private boolean lfInited = false;
   private boolean lfInitedOk = false;
@@ -356,13 +360,21 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
               namenode.getNamesystem().getBlockManager().getStoragePolicies());
         }
 
-        Result res = new Result(conf);
+        Result replRes = new ReplicationResult(conf);
+        Result ecRes = new ErasureCodingResult(conf);
 
-        check(path, file, res);
+        check(path, file, replRes, ecRes);
 
-        out.println(res);
-        out.println(" Number of data-nodes:\t\t" + totalDatanodes);
+        out.print("\nStatus: ");
+        out.println(replRes.isHealthy() && ecRes.isHealthy() ? "HEALTHY" : "CORRUPT");
+        out.println(" Number of data-nodes:\t" + totalDatanodes);
         out.println(" Number of racks:\t\t" + networktopology.getNumOfRacks());
+        out.println(" Total dirs:\t\t\t" + totalDirs);
+        out.println(" Total symlinks:\t\t" + totalSymlinks);
+        out.println("\nReplicated Blocks:");
+        out.println(replRes);
+        out.println("\nErasure Coded Block Groups:");
+        out.println(ecRes);
 
         if (this.showStoragePolcies) {
           out.print(storageTypeSummary.toString());
@@ -382,7 +394,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         // of file system and return appropriate code. Changing the output
         // string might break testcases. Also note this must be the last line
         // of the report.
-        if (res.isHealthy()) {
+        if (replRes.isHealthy() && ecRes.isHealthy()) {
           out.print("\n\nThe filesystem under path '" + path + "' " + HEALTHY_STATUS);
         } else {
           out.print("\n\nThe filesystem under path '" + path + "' " + CORRUPT_STATUS);
@@ -425,42 +437,49 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   }
 
   @VisibleForTesting
-  void check(String parent, HdfsFileStatus file, Result res) throws IOException {
+  void check(String parent, HdfsFileStatus file, Result replRes, Result ecRes)
+      throws IOException {
     String path = file.getFullName(parent);
     if (file.isDir()) {
-      checkDir(path, res);
+      checkDir(path, replRes, ecRes);
       return;
     }
     if (file.isSymlink()) {
       if (showFiles) {
         out.println(path + " <symlink>");
       }
-      res.totalSymlinks++;
+      totalSymlinks++;
       return;
     }
     LocatedBlocks blocks = getBlockLocations(path, file);
     if (blocks == null) { // the file is deleted
       return;
     }
-    collectFileSummary(path, file, res, blocks);
-    collectBlocksSummary(parent, file, res, blocks);
+
+    final Result r = file.getReplication() == 0? ecRes: replRes;
+    collectFileSummary(path, file, r, blocks);
+    if (showprogress && (replRes.totalFiles + ecRes.totalFiles) % 100 == 0) {
+      out.println();
+      out.flush();
+    }
+    collectBlocksSummary(parent, file, r, blocks);
   }
 
-  private void checkDir(String path, Result res) throws IOException {
+  private void checkDir(String path, Result replRes, Result ecRes) throws IOException {
     if (snapshottableDirs != null && snapshottableDirs.contains(path)) {
       String snapshotPath = (path.endsWith(Path.SEPARATOR) ? path : path
           + Path.SEPARATOR)
           + HdfsConstants.DOT_SNAPSHOT_DIR;
       HdfsFileStatus snapshotFileInfo = namenode.getRpcServer().getFileInfo(
           snapshotPath);
-      check(snapshotPath, snapshotFileInfo, res);
+      check(snapshotPath, snapshotFileInfo, replRes, ecRes);
     }
     byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;
     DirectoryListing thisListing;
     if (showFiles) {
       out.println(path + " <dir>");
     }
-    res.totalDirs++;
+    totalDirs++;
     do {
       assert lastReturnedName != null;
       thisListing = namenode.getRpcServer().getListing(
@@ -470,7 +489,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       }
       HdfsFileStatus[] files = thisListing.getPartialListing();
       for (int i = 0; i < files.length; i++) {
-        check(path, files[i], res);
+        check(path, files[i], replRes, ecRes);
       }
       lastReturnedName = thisListing.getLastName();
     } while (thisListing.hasMore());
@@ -517,10 +536,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     } else if (showprogress) {
       out.print('.');
     }
-    if ((showprogress) && res.totalFiles % 100 == 0) {
-      out.println();
-      out.flush();
-    }
   }
 
   private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res,
@@ -543,7 +558,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
           block.getLocalBlock());
       // count decommissionedReplicas / decommissioningReplicas
       NumberReplicas numberReplicas = bm.countNodes(storedBlock);
-      int decommissionedReplicas = numberReplicas.decommissioned();;
+      int decommissionedReplicas = numberReplicas.decommissioned();
       int decommissioningReplicas = numberReplicas.decommissioning();
       res.decommissionedReplicas +=  decommissionedReplicas;
       res.decommissioningReplicas += decommissioningReplicas;
@@ -555,7 +570,15 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       res.totalReplicas += totalReplicasPerBlock;
 
       // count expected replicas
-      short targetFileReplication = file.getReplication();
+      short targetFileReplication;
+      if(file.getReplication() == 0) {
+        INode inode = namenode.getNamesystem().getFSDirectory().getINode(path);
+        INodesInPath iip = INodesInPath.fromINode(inode);
+        ECSchema ecSchema = namenode.getNamesystem().getFSDirectory().getECSchema(iip);
+        targetFileReplication = (short) (ecSchema.getNumDataUnits() + ecSchema.getNumParityUnits());
+      } else {
+        targetFileReplication = file.getReplication();
+      }
       res.numExpectedReplicas += targetFileReplication;
 
       // count under min repl'd blocks
@@ -981,7 +1004,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     long missingReplicas = 0L;
     long decommissionedReplicas = 0L;
     long decommissioningReplicas = 0L;
-    long numUnderMinReplicatedBlocks=0L;
+    long numUnderMinReplicatedBlocks = 0L;
     long numOverReplicatedBlocks = 0L;
     long numUnderReplicatedBlocks = 0L;
     long numMisReplicatedBlocks = 0L;  // blocks that do not satisfy block placement policy
@@ -991,20 +1014,14 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     long totalOpenFilesBlocks = 0L;
     long totalFiles = 0L;
     long totalOpenFiles = 0L;
-    long totalDirs = 0L;
-    long totalSymlinks = 0L;
     long totalSize = 0L;
     long totalOpenFilesSize = 0L;
     long totalReplicas = 0L;
 
-    final short replication;
     final int minReplication;
 
-    Result(Configuration conf) {
-      this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
-                                            DFSConfigKeys.DFS_REPLICATION_DEFAULT);
-      this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
-                                            DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
+    Result(int minReplication) {
+      this.minReplication = minReplication;
     }
 
     /**
@@ -1032,19 +1049,28 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         return 0.0f;
       return (float) (totalReplicas) / (float) totalBlocks;
     }
+  }
+
+  @VisibleForTesting
+  static class ReplicationResult extends Result {
+    final short replication;
+
+    ReplicationResult(Configuration conf) {
+      super(conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
+                        DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT));
+      this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
+                                            DFSConfigKeys.DFS_REPLICATION_DEFAULT);
+    }
 
     @Override
     public String toString() {
       StringBuilder res = new StringBuilder();
-      res.append("Status: ").append((isHealthy() ? "HEALTHY" : "CORRUPT"))
-          .append("\n Total size:\t").append(totalSize).append(" B");
+      res.append(" Total size:\t").append(totalSize).append(" B");
       if (totalOpenFilesSize != 0) {
         res.append(" (Total open files size: ").append(totalOpenFilesSize)
             .append(" B)");
       }
-      res.append("\n Total dirs:\t").append(totalDirs).append(
-          "\n Total files:\t").append(totalFiles);
-      res.append("\n Total symlinks:\t\t").append(totalSymlinks);
+      res.append("\n Total files:\t").append(totalFiles);
       if (totalOpenFiles != 0) {
         res.append(" (Files currently being written: ").append(totalOpenFiles)
             .append(")");
@@ -1134,4 +1160,116 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       return res.toString();
     }
   }
+
+  @VisibleForTesting
+  static class ErasureCodingResult extends Result {
+    final String ecSchema;
+
+    ErasureCodingResult(Configuration conf) {
+      this(ErasureCodingSchemaManager.getSystemDefaultSchema());
+    }
+
+    ErasureCodingResult(ECSchema ecSchema) {
+      super(ecSchema.getNumDataUnits());
+      this.ecSchema = ecSchema.getSchemaName();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder res = new StringBuilder();
+      res.append(" Total size:\t").append(totalSize).append(" B");
+      if (totalOpenFilesSize != 0) {
+        res.append(" (Total open files size: ").append(totalOpenFilesSize)
+            .append(" B)");
+      }
+      res.append("\n Total files:\t").append(totalFiles);
+      if (totalOpenFiles != 0) {
+        res.append(" (Files currently being written: ").append(totalOpenFiles)
+            .append(")");
+      }
+      res.append("\n Total block groups (validated):\t").append(totalBlocks);
+      if (totalBlocks > 0) {
+        res.append(" (avg. block group size ").append((totalSize / totalBlocks))
+            .append(" B)");
+      }
+      if (totalOpenFilesBlocks != 0) {
+        res.append(" (Total open file block groups (not validated): ").append(
+            totalOpenFilesBlocks).append(")");
+      }
+      if (corruptFiles > 0 || numUnderMinReplicatedBlocks > 0) {
+        res.append("\n  ********************************");
+        if(numUnderMinReplicatedBlocks>0){
+          res.append("\n  UNRECOVERABLE BLOCK GROUPS:\t").append(numUnderMinReplicatedBlocks);
+          if(totalBlocks>0){
+            res.append(" (").append(
+                ((float) (numUnderMinReplicatedBlocks * 100) / (float) totalBlocks))
+                .append(" %)");
+          }
+          res.append("\n  ").append("MIN REQUIRED EC BLOCK:\t")
+              .append(minReplication);
+        }
+        if(corruptFiles>0) {
+          res.append(
+              "\n  CORRUPT FILES:\t").append(corruptFiles);
+          if (missingSize > 0) {
+            res.append("\n  MISSING BLOCK GROUPS:\t").append(missingIds.size()).append(
+                "\n  MISSING SIZE:\t\t").append(missingSize).append(" B");
+          }
+          if (corruptBlocks > 0) {
+            res.append("\n  CORRUPT BLOCK GROUPS: \t").append(corruptBlocks).append(
+                "\n  CORRUPT SIZE:\t\t").append(corruptSize).append(" B");
+          }
+        }
+        res.append("\n  ********************************");
+      }
+      res.append("\n Minimally erasure-coded block groups:\t").append(
+          numMinReplicatedBlocks);
+      if (totalBlocks > 0) {
+        res.append(" (").append(
+            ((float) (numMinReplicatedBlocks * 100) / (float) totalBlocks))
+            .append(" %)");
+      }
+      res.append("\n Over-erasure-coded block groups:\t")
+          .append(numOverReplicatedBlocks);
+      if (totalBlocks > 0) {
+        res.append(" (").append(
+            ((float) (numOverReplicatedBlocks * 100) / (float) totalBlocks))
+            .append(" %)");
+      }
+      res.append("\n Under-erasure-coded block groups:\t").append(
+          numUnderReplicatedBlocks);
+      if (totalBlocks > 0) {
+        res.append(" (").append(
+            ((float) (numUnderReplicatedBlocks * 100) / (float) totalBlocks))
+            .append(" %)");
+      }
+      res.append("\n Unsatisfactory placement block groups:\t\t")
+          .append(numMisReplicatedBlocks);
+      if (totalBlocks > 0) {
+        res.append(" (").append(
+            ((float) (numMisReplicatedBlocks * 100) / (float) totalBlocks))
+            .append(" %)");
+      }
+      res.append("\n Default schema:\t").append(ecSchema)
+          .append("\n Average block group size:\t").append(
+          getReplicationFactor()).append("\n Missing block groups:\t\t").append(
+          missingIds.size()).append("\n Corrupt block groups:\t\t").append(
+          corruptBlocks).append("\n Missing ec-blocks:\t\t").append(
+          missingReplicas);
+      if (totalReplicas > 0) {
+        res.append(" (").append(
+            ((float) (missingReplicas * 100) / (float) numExpectedReplicas)).append(
+            " %)");
+      }
+      if (decommissionedReplicas > 0) {
+        res.append("\n Decommissioned ec-blocks:\t").append(
+            decommissionedReplicas);
+      }
+      if (decommissioningReplicas > 0) {
+        res.append("\n Decommissioning ec-blocks:\t").append(
+            decommissioningReplicas);
+      }
+      return res.toString();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ed92e5b/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 b36beb2..e5482d4 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
@@ -84,6 +84,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result;
+import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ReplicationResult;
+import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ErasureCodingResult;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSck;
 import org.apache.hadoop.io.IOUtils;
@@ -1058,13 +1060,14 @@ public class TestFsck {
       final HdfsFileStatus file = 
           namenode.getRpcServer().getFileInfo(pathString);
       assertNotNull(file);
-      Result res = new Result(conf);
-      fsck.check(pathString, file, res);
+      Result replRes = new ReplicationResult(conf);
+      Result ecRes = new ErasureCodingResult(conf);
+      fsck.check(pathString, file, replRes, ecRes);
       // Also print the output from the fsck, for ex post facto sanity checks
       System.out.println(result.toString());
-      assertEquals(res.missingReplicas, 
+      assertEquals(replRes.missingReplicas,
           (NUM_BLOCKS*REPL_FACTOR) - (NUM_BLOCKS*NUM_REPLICAS));
-      assertEquals(res.numExpectedReplicas, NUM_BLOCKS*REPL_FACTOR);
+      assertEquals(replRes.numExpectedReplicas, NUM_BLOCKS*REPL_FACTOR);
     } finally {
       if(dfs != null) {
         dfs.close();
@@ -1135,10 +1138,11 @@ public class TestFsck {
       final HdfsFileStatus file = 
           namenode.getRpcServer().getFileInfo(pathString);
       assertNotNull(file);
-      Result res = new Result(conf);
-      fsck.check(pathString, file, res);
+      Result replRes = new ReplicationResult(conf);
+      Result ecRes = new ErasureCodingResult(conf);
+      fsck.check(pathString, file, replRes, ecRes);
       // check misReplicatedBlock number.
-      assertEquals(res.numMisReplicatedBlocks, NUM_BLOCKS);
+      assertEquals(replRes.numMisReplicatedBlocks, NUM_BLOCKS);
     } finally {
       if(dfs != null) {
         dfs.close();
@@ -1199,14 +1203,15 @@ public class TestFsck {
     HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication,
         blockSize, modTime, accessTime, perms, owner, group, symlink,
         path, fileId, numChildren, null, storagePolicy, null, 0);
-    Result res = new Result(conf);
+    Result replRes = new ReplicationResult(conf);
+    Result ecRes = new ErasureCodingResult(conf);
 
     try {
-      fsck.check(pathString, file, res);
+      fsck.check(pathString, file, replRes, ecRes);
     } catch (Exception e) {
       fail("Unexpected exception "+ e.getMessage());
     }
-    assertTrue(res.toString().contains("HEALTHY"));
+    assertTrue(replRes.isHealthy());
   }
 
   /** Test fsck with symlinks in the filesystem */
@@ -1629,4 +1634,52 @@ public class TestFsck {
       }
     }
   }
-}
+
+  @Test
+  public void testECFsck() throws Exception {
+    MiniDFSCluster cluster = null;
+    FileSystem fs = null;
+    try {
+      Configuration conf = new HdfsConfiguration();
+      final long precision = 1L;
+      conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
+      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
+      int totalSize = ErasureCodingSchemaManager.getSystemDefaultSchema().getNumDataUnits()
+                      + ErasureCodingSchemaManager.getSystemDefaultSchema().getNumParityUnits();
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(totalSize).build();
+      fs = cluster.getFileSystem();
+      Path replDirPath = new Path("/replicated");
+      Path replFilePath = new Path(replDirPath, "replfile");
+      final short factor = 3;
+      DFSTestUtil.createFile(fs, replFilePath, 1024, factor, 0);
+      DFSTestUtil.waitReplication(fs, replFilePath, factor);
+      Path ecDirPath = new Path("/striped");
+      Path ecFilePath = new Path(ecDirPath, "ecfile");
+      final int numBlocks = 4;
+      DFSTestUtil.createStripedFile(cluster, ecFilePath, ecDirPath, numBlocks, 2, true);
+      long replTime = fs.getFileStatus(replFilePath).getAccessTime();
+      long ecTime = fs.getFileStatus(ecFilePath).getAccessTime();
+      Thread.sleep(precision);
+      setupAuditLogs();
+      String outStr = runFsck(conf, 0, true, "/");
+      verifyAuditLogs();
+      assertEquals(replTime, fs.getFileStatus(replFilePath).getAccessTime());
+      assertEquals(ecTime, fs.getFileStatus(ecFilePath).getAccessTime());
+      System.out.println(outStr);
+      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
+      if (fs != null) {try{fs.close();} catch(Exception e){}}
+      cluster.shutdown();
+
+      // restart the cluster; bring up namenode but not the data nodes
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0).format(false).build();
+      outStr = runFsck(conf, 1, true, "/");
+      // expect the result is corrupt
+      assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
+      System.out.println(outStr);
+    } finally {
+      if (fs != null) {try{fs.close();} catch(Exception e){}}
+      if (cluster != null) { cluster.shutdown(); }
+    }
+  }
+}
\ No newline at end of file


[05/50] [abbrv] 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/ea6c66ed
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ea6c66ed
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ea6c66ed

Branch: refs/heads/HDFS-7285
Commit: ea6c66ed5704dc8a9448b4c1b56e9664086c70fc
Parents: 77d94dd
Author: Jing Zhao <ji...@apache.org>
Authored: Mon May 4 11:24:35 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 11:59:57 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/ea6c66ed/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/ea6c66ed/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/ea6c66ed/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/ea6c66ed/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/ea6c66ed/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/ea6c66ed/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 2ebc927..4043928 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
@@ -396,6 +396,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/ea6c66ed/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/ea6c66ed/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


[25/50] [abbrv] hadoop git commit: HDFS-8367 BlockInfoStriped uses EC schema. Contributed by Kai Sasaki

Posted by zh...@apache.org.
HDFS-8367 BlockInfoStriped uses EC schema. 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/c9103e9c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c9103e9c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c9103e9c

Branch: refs/heads/HDFS-7285
Commit: c9103e9cacc67a614940e32fa87c5dbc3daa60de
Parents: d8ea443
Author: Kai Zheng <ka...@intel.com>
Authored: Tue May 19 00:10:30 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:30 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  2 +
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  6 +--
 .../blockmanagement/BlockInfoStriped.java       | 24 ++++++++----
 .../BlockInfoStripedUnderConstruction.java      | 12 +++---
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  4 +-
 .../hdfs/server/namenode/FSDirectory.java       |  3 ++
 .../hdfs/server/namenode/FSEditLogLoader.java   | 34 +++++++++--------
 .../hdfs/server/namenode/FSImageFormat.java     | 10 +++--
 .../server/namenode/FSImageFormatPBINode.java   |  7 +++-
 .../server/namenode/FSImageSerialization.java   | 14 ++++---
 .../hdfs/server/namenode/FSNamesystem.java      |  2 +-
 .../blockmanagement/TestBlockInfoStriped.java   |  8 +++-
 .../server/namenode/TestFSEditLogLoader.java    |  8 +++-
 .../hdfs/server/namenode/TestFSImage.java       |  6 ++-
 .../server/namenode/TestStripedINodeFile.java   | 39 ++++++--------------
 15 files changed, 99 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 1456434..333d85f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -215,3 +215,5 @@
 
     HDFS-8391. NN should consider current EC tasks handling count from DN while 
     assigning new tasks. (umamahesh)
+
+    HDFS-8367. BlockInfoStriped uses EC schema. (Kai Sasaki via Kai Zheng)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 94b2ff9..a6a356c 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
@@ -203,6 +203,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
@@ -445,9 +446,8 @@ public class PBHelper {
     return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
   }
 
-  public static BlockInfoStriped convert(StripedBlockProto p) {
-    return new BlockInfoStriped(convert(p.getBlock()),
-        (short) p.getDataBlockNum(), (short) p.getParityBlockNum());
+  public static BlockInfoStriped convert(StripedBlockProto p, ECSchema schema) {
+    return new BlockInfoStriped(convert(p.getBlock()), schema);
   }
 
   public static StripedBlockProto convert(BlockInfoStriped blk) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 f0e52e3..d7a48a0 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,7 +19,9 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 
@@ -39,6 +41,7 @@ import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_S
 public class BlockInfoStriped extends BlockInfo {
   private final short dataBlockNum;
   private final short parityBlockNum;
+  private final ECSchema schema;
   /**
    * Always the same size with triplets. Record the block index for each triplet
    * TODO: actually this is only necessary for over-replicated block. Thus can
@@ -46,16 +49,17 @@ public class BlockInfoStriped extends BlockInfo {
    */
   private byte[] indices;
 
-  public BlockInfoStriped(Block blk, short dataBlockNum, short parityBlockNum) {
-    super(blk, (short) (dataBlockNum + parityBlockNum));
-    indices = new byte[dataBlockNum + parityBlockNum];
+  public BlockInfoStriped(Block blk, ECSchema schema) {
+    super(blk, (short) (schema.getNumDataUnits() + schema.getNumParityUnits()));
+    indices = new byte[schema.getNumDataUnits() + schema.getNumParityUnits()];
     initIndices();
-    this.dataBlockNum = dataBlockNum;
-    this.parityBlockNum = parityBlockNum;
+    this.schema = schema;
+    this.dataBlockNum = (short)schema.getNumDataUnits();
+    this.parityBlockNum = (short)schema.getNumParityUnits();
   }
 
   BlockInfoStriped(BlockInfoStriped b) {
-    this(b, b.dataBlockNum, b.parityBlockNum);
+    this(b, b.getSchema());
     this.setBlockCollection(b.getBlockCollection());
   }
 
@@ -71,6 +75,10 @@ public class BlockInfoStriped extends BlockInfo {
     return parityBlockNum;
   }
 
+  public ECSchema getSchema() {
+    return schema;
+  }
+
   private void initIndices() {
     for (int i = 0; i < indices.length; i++) {
       indices[i] = -1;
@@ -231,8 +239,8 @@ public class BlockInfoStriped extends BlockInfo {
       BlockUCState s, DatanodeStorageInfo[] targets) {
     final BlockInfoStripedUnderConstruction ucBlock;
     if(isComplete()) {
-      ucBlock = new BlockInfoStripedUnderConstruction(this, getDataBlockNum(),
-              getParityBlockNum(), s, targets);
+      ucBlock = new BlockInfoStripedUnderConstruction(this, schema,
+          s, targets);
       ucBlock.setBlockCollection(getBlockCollection());
     } else {
       // the block is already under construction

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 b991615..40f880f 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
@@ -21,6 +21,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import java.io.IOException;
 
@@ -56,17 +57,16 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
   /**
    * Constructor with null storage targets.
    */
-  public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum,
-      short parityBlockNum) {
-    this(blk, dataBlockNum, parityBlockNum, UNDER_CONSTRUCTION, null);
+  public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema) {
+    this(blk, schema, UNDER_CONSTRUCTION, null);
   }
 
   /**
    * Create a striped block that is currently being constructed.
    */
-  public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum,
-      short parityBlockNum, BlockUCState state, DatanodeStorageInfo[] targets) {
-    super(blk, dataBlockNum, parityBlockNum);
+  public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema,
+      BlockUCState state, DatanodeStorageInfo[] targets) {
+    super(blk, schema);
     assert getBlockUCState() != COMPLETE :
       "BlockInfoStripedUnderConstruction cannot be in COMPLETE state";
     this.blockUCState = state;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 81a2fa7..6137fd5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -532,8 +532,8 @@ class FSDirWriteFileOp {
         // check quota limits and updated space consumed
         fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
             numLocations, true);
-        blockInfo = new BlockInfoStripedUnderConstruction(block, numDataUnits,
-            numParityUnits, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+        blockInfo = new BlockInfoStripedUnderConstruction(block, ecSchema,
+            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
             targets);
       } else {
         // check quota limits and updated space consumed

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 7989ccb..bb8282d 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
@@ -54,6 +54,9 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 592c421..66b2f82 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
@@ -104,6 +104,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.util.Holder;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.ChunkedArrayList;
 
 import com.google.common.base.Joiner;
@@ -416,8 +417,8 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       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.isInECZone(iip));
+      updateBlocks(fsDir, addCloseOp, iip, newFile,
+          fsDir.getECSchema(iip), fsDir.isInECZone(iip));
       break;
     }
     case OP_CLOSE: {
@@ -437,8 +438,8 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       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.isInECZone(iip));
+      updateBlocks(fsDir, addCloseOp, iip, file,
+          fsDir.getECSchema(iip), fsDir.isInECZone(iip));
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -496,9 +497,9 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       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.isInECZone(iip));
-      
+      updateBlocks(fsDir, updateOp, iip, oldFile,
+          fsDir.getECSchema(iip), fsDir.isInECZone(iip));
+
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
       }
@@ -514,8 +515,8 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       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.isInECZone(iip));
+      addNewBlock(addBlockOp, oldFile,
+          fsDir.getECSchema(iip), fsDir.isInECZone(iip));
       break;
     }
     case OP_SET_REPLICATION: {
@@ -956,7 +957,8 @@ public class FSEditLogLoader {
   /**
    * Add a new block into the given INodeFile
    */
-  private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped)
+  private void addNewBlock(AddBlockOp op, INodeFile file,
+                           ECSchema schema, boolean isStriped)
       throws IOException {
     BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
@@ -985,8 +987,7 @@ public class FSEditLogLoader {
     // add the new block
     final BlockInfo newBlockInfo;
     if (isStriped) {
-      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
-          HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, schema);
     } else {
       newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
           file.getPreferredBlockReplication());
@@ -1001,7 +1002,8 @@ public class FSEditLogLoader {
    * @throws IOException
    */
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
-      INodesInPath iip, INodeFile file, boolean isStriped) throws IOException {
+      INodesInPath iip, INodeFile file, ECSchema schema,
+      boolean isStriped) throws IOException {
     // Update its block list
     BlockInfo[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
@@ -1069,8 +1071,7 @@ public class FSEditLogLoader {
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
           if (isStriped) {
-            newBI = new BlockInfoStripedUnderConstruction(newBlock,
-                HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+            newBI = new BlockInfoStripedUnderConstruction(newBlock, schema);
           } else {
             newBI = new BlockInfoContiguousUnderConstruction(newBlock,
                 file.getPreferredBlockReplication());
@@ -1080,8 +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.
+          // TODO: ECSchema can be restored from persisted file (HDFS-7859).
           newBI = isStriped ? new BlockInfoStriped(newBlock,
-              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS) :
+              ErasureCodingSchemaManager.getSystemDefaultSchema()) :
               new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication());
         }
         fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 6f485f5..3083952 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
@@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -762,6 +763,9 @@ public class FSImageFormat {
             NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
             && (in.readBoolean());
     final int numBlocks = in.readInt();
+    // TODO: ECSchema can be restored from persisted file (HDFS-7859).
+    final ECSchema schema = isStriped ?
+        ErasureCodingSchemaManager.getSystemDefaultSchema() : null;
 
     if (numBlocks >= 0) {
       // file
@@ -771,8 +775,7 @@ public class FSImageFormat {
       if (isStriped) {
         blocks = new Block[numBlocks];
         for (int j = 0; j < numBlocks; j++) {
-          blocks[j] = new BlockInfoStriped(new Block(),
-              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+          blocks[j] = new BlockInfoStriped(new Block(), schema);
           blocks[j].readFields(in);
         }
       } else {
@@ -804,8 +807,7 @@ public class FSImageFormat {
                 BlockInfoStriped lastStripedBlk = (BlockInfoStriped) lastBlk;
                 blocks[blocks.length - 1]
                         = new BlockInfoStripedUnderConstruction(lastBlk,
-                                lastStripedBlk.getDataBlockNum(),
-                                lastStripedBlk.getParityBlockNum());
+                                lastStripedBlk.getSchema());
               } else {
                 blocks[blocks.length - 1]
                         = new BlockInfoContiguousUnderConstruction(lastBlk,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 583f4b0..e157b95 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
@@ -69,6 +69,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.protobuf.ByteString;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 @InterfaceAudience.Private
 public final class FSImageFormatPBINode {
@@ -354,10 +355,12 @@ public final class FSImageFormatPBINode {
 
       FileWithStripedBlocksFeature stripeFeature = null;
       if (f.hasStripedBlocks()) {
+        // TODO: HDFS-7859
+        ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
         StripedBlocksFeature sb = f.getStripedBlocks();
         stripeFeature = file.addStripedBlocksFeature();
         for (StripedBlockProto sp : sb.getBlocksList()) {
-          stripeFeature.addBlock(PBHelper.convert(sp));
+          stripeFeature.addBlock(PBHelper.convert(sp, schema));
         }
       }
 
@@ -372,7 +375,7 @@ public final class FSImageFormatPBINode {
           if (stripeFeature != null) {
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
             ucBlk = new BlockInfoStripedUnderConstruction(striped,
-                striped.getDataBlockNum(), striped.getParityBlockNum());
+                striped.getSchema());
           } else {
             ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
                 replication);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 25febd4..4744716 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
@@ -50,6 +50,7 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.xml.sax.ContentHandler;
 import org.xml.sax.SAXException;
 
@@ -130,7 +131,11 @@ public class FSImageSerialization {
     final boolean isStriped = NameNodeLayoutVersion.supports(
         NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
         && (in.readBoolean());
-  
+
+    // TODO: ECSchema can be restored from persisted file (HDFS-7859).
+    final ECSchema schema = isStriped ?
+        ErasureCodingSchemaManager.getSystemDefaultSchema() : null;
+
     int numBlocks = in.readInt();
 
     final BlockInfoContiguous[] blocksContiguous;
@@ -140,15 +145,12 @@ public class FSImageSerialization {
       blocksStriped = new BlockInfoStriped[numBlocks];
       int i = 0;
       for (; i < numBlocks - 1; i++) {
-        blocksStriped[i] = new BlockInfoStriped(new Block(),
-            HdfsConstants.NUM_DATA_BLOCKS,
-            HdfsConstants.NUM_PARITY_BLOCKS);
+        blocksStriped[i] = new BlockInfoStriped(new Block(), schema);
         blocksStriped[i].readFields(in);
       }
       if (numBlocks > 0) {
         blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(),
-            HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS,
-            BlockUCState.UNDER_CONSTRUCTION, null);
+            schema, BlockUCState.UNDER_CONSTRUCTION, null);
         blocksStriped[i].readFields(in);
       }
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 71a3dcf..6643dfe 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
@@ -3672,7 +3672,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           fileINode.getPreferredBlockSize() * numBlocks;
 
       final BlockInfoStriped striped = new BlockInfoStriped(commitBlock,
-          numDataUnits, numParityUnits);
+          ecSchema);
       final long actualBlockGroupSize = striped.spaceConsumed();
 
       diff = fullBlockGroupSize - actualBlockGroupSize;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 3b689eb..2d6b509 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
@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
@@ -42,8 +44,10 @@ public class TestBlockInfoStriped {
   private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
   private static final long BASE_ID = -1600;
   private static final Block baseBlock = new Block(BASE_ID);
+  private static final ECSchema testSchema
+      = ErasureCodingSchemaManager.getSystemDefaultSchema();
   private final BlockInfoStriped info = new BlockInfoStriped(baseBlock,
-      NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+      testSchema);
 
   private Block[] createReportedBlocks(int num) {
     Block[] blocks = new Block[num];
@@ -231,7 +235,7 @@ public class TestBlockInfoStriped {
     ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
     DataOutput out = new DataOutputStream(byteStream);
     BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes,
-        generationStamp), NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+        generationStamp), testSchema);
 
     try {
       blk.write(out);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 57f62c9..2909423 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.junit.Test;
@@ -72,6 +73,9 @@ public class TestFSEditLogLoader {
   private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class);
 
   private static final int NUM_DATA_NODES = 0;
+
+  private static final ECSchema testSchema
+      = ErasureCodingSchemaManager.getSystemDefaultSchema();
   
   @Test
   public void testDisplayRecentEditLogOpCodes() throws IOException {
@@ -459,7 +463,7 @@ public class TestFSEditLogLoader {
 
       // Add a striped block to the file
       BlockInfoStriped stripedBlk = new BlockInfoStriped(
-          new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum);
+          new Block(blkId, blkNumBytes, timestamp), testSchema);
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
       file.getStripedBlocksFeature().addBlock(stripedBlk);
@@ -525,7 +529,7 @@ public class TestFSEditLogLoader {
       Path p = new Path(testFilePath);
       DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
       BlockInfoStriped stripedBlk = new BlockInfoStriped(
-          new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum);
+          new Block(blkId, blkNumBytes, timestamp), testSchema);
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
       file.getStripedBlocksFeature().addBlock(stripedBlk);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 bb37534..5e015bf 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
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.Assert;
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -67,6 +68,9 @@ public class TestFSImage {
 
   private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ =
       "image-with-zero-block-size.tar.gz";
+  private static final ECSchema testSchema
+      = ErasureCodingSchemaManager.getSystemDefaultSchema();
+
   @Test
   public void testPersist() throws IOException {
     Configuration conf = new Configuration();
@@ -158,7 +162,7 @@ public class TestFSImage {
     for (int i = 0; i < stripedBlks.length; i++) {
       stripedBlks[i] = new BlockInfoStriped(
               new Block(stripedBlkId + i, preferredBlockSize, timestamp),
-              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+              testSchema);
       file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9103e9c/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 e24bc54..9e4da84 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
@@ -51,6 +51,9 @@ public class TestStripedINodeFile {
   private final BlockStoragePolicy defaultPolicy =
       defaultSuite.getDefaultPolicy();
 
+  private static final ECSchema testSchema
+      = ErasureCodingSchemaManager.getSystemDefaultSchema();
+
   private static INodeFile createStripedINodeFile() {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID);
@@ -69,9 +72,7 @@ public class TestStripedINodeFile {
     ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk, testSchema);
     assertEquals(9, blockInfoStriped.getTotalBlockNum());
   }
 
@@ -83,9 +84,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk, testSchema);
     inf.addBlock(blockInfoStriped);
     assertEquals(1, inf.getBlocks().length);
   }
@@ -98,9 +97,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk, testSchema);
     blockInfoStriped.setNumBytes(1);
     inf.addBlock(blockInfoStriped);
     //   0. Calculate the total bytes per stripes <Num Bytes per Stripes>
@@ -127,15 +124,11 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     Block blk1 = new Block(1);
     BlockInfoStriped blockInfoStriped1
-        = new BlockInfoStriped(blk1,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk1, testSchema);
     blockInfoStriped1.setNumBytes(1);
     Block blk2 = new Block(2);
     BlockInfoStriped blockInfoStriped2
-        = new BlockInfoStriped(blk2,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk2, testSchema);
     blockInfoStriped2.setNumBytes(1);
     inf.addBlock(blockInfoStriped1);
     inf.addBlock(blockInfoStriped2);
@@ -152,9 +145,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk, testSchema);
     blockInfoStriped.setNumBytes(100);
     inf.addBlock(blockInfoStriped);
     // Compute file size should return actual data
@@ -171,9 +162,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStripedUnderConstruction bInfoStripedUC
-        = new BlockInfoStripedUnderConstruction(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStripedUnderConstruction(blk, testSchema);
     bInfoStripedUC.setNumBytes(100);
     inf.addBlock(bInfoStripedUC);
     assertEquals(100, inf.computeFileSize());
@@ -188,9 +177,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk, testSchema);
     blockInfoStriped.setNumBytes(100);
     inf.addBlock(blockInfoStriped);
 
@@ -213,9 +200,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStripedUnderConstruction bInfoStripedUC
-        = new BlockInfoStripedUnderConstruction(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStripedUnderConstruction(blk, testSchema);
     bInfoStripedUC.setNumBytes(100);
     inf.addBlock(bInfoStripedUC);
 


[43/50] [abbrv] hadoop git commit: HADOOP-12013 Generate fixed data to perform erasure coder test. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-12013 Generate fixed data to perform erasure coder test. 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/4ae32abd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4ae32abd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4ae32abd

Branch: refs/heads/HDFS-7285
Commit: 4ae32abdf474f419d65464580f30c678ffff8182
Parents: 3d734df
Author: Kai Zheng <ka...@intel.com>
Authored: Fri May 22 23:47:52 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:07:14 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  2 +
 .../hadoop/io/erasurecode/TestCoderBase.java    | 87 +++++++++++++++++++-
 .../io/erasurecode/rawcoder/TestRSRawCoder.java |  8 +-
 .../rawcoder/TestRSRawCoderBase.java            | 21 +++--
 4 files changed, 110 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ae32abd/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 c799b4f..531b8d5 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -54,3 +54,5 @@
 
     HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure 
     coder. (Kai Zheng via Zhe Zhang)
+
+    HADOOP-12013. Generate fixed data to perform erasure coder test. (Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ae32abd/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 cc3617c..3686695 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
@@ -52,6 +52,12 @@ public abstract class TestCoderBase {
   // may go to different coding implementations.
   protected boolean usingDirectBuffer = true;
 
+  protected boolean usingFixedData = true;
+  // Using this the generated data can be repeatable across multiple calls to
+  // encode(), in order for troubleshooting.
+  private static int FIXED_DATA_GENERATOR = 0;
+  protected byte[][] fixedData;
+
   protected int getChunkSize() {
     return chunkSize;
   }
@@ -63,13 +69,17 @@ public abstract class TestCoderBase {
 
   /**
    * Prepare before running the case.
+   * @param conf
    * @param numDataUnits
    * @param numParityUnits
    * @param erasedDataIndexes
+   * @param erasedParityIndexes
+   * @param usingFixedData Using fixed or pre-generated data to test instead of
+   *                       generating data
    */
   protected void prepare(Configuration conf, int numDataUnits,
                          int numParityUnits, int[] erasedDataIndexes,
-                         int[] erasedParityIndexes) {
+                         int[] erasedParityIndexes, boolean usingFixedData) {
     this.conf = conf;
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
@@ -77,6 +87,38 @@ public abstract class TestCoderBase {
         erasedDataIndexes : new int[] {0};
     this.erasedParityIndexes = erasedParityIndexes != null ?
         erasedParityIndexes : new int[] {0};
+    this.usingFixedData = usingFixedData;
+    if (usingFixedData) {
+      prepareFixedData();
+    }
+  }
+
+  /**
+   * Prepare before running the case.
+   * @param conf
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedDataIndexes
+   * @param erasedParityIndexes
+   */
+  protected void prepare(Configuration conf, int numDataUnits,
+                         int numParityUnits, int[] erasedDataIndexes,
+                         int[] erasedParityIndexes) {
+    prepare(conf, numDataUnits, numParityUnits, erasedDataIndexes,
+        erasedParityIndexes, false);
+  }
+
+  /**
+   * Prepare before running the case.
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedDataIndexes
+   * @param erasedParityIndexes
+   */
+  protected void prepare(int numDataUnits, int numParityUnits,
+                         int[] erasedDataIndexes, int[] erasedParityIndexes) {
+    prepare(null, numDataUnits, numParityUnits, erasedDataIndexes,
+        erasedParityIndexes, false);
   }
 
   /**
@@ -278,6 +320,29 @@ public abstract class TestCoderBase {
    * @return
    */
   protected ECChunk[] prepareDataChunksForEncoding() {
+    if (usingFixedData) {
+      ECChunk[] chunks = new ECChunk[numDataUnits];
+      for (int i = 0; i < chunks.length; i++) {
+        chunks[i] = makeChunkUsingData(fixedData[i]);
+      }
+      return chunks;
+    }
+
+    return generateDataChunks();
+  }
+
+  private ECChunk makeChunkUsingData(byte[] data) {
+    ECChunk chunk = allocateOutputChunk();
+    ByteBuffer buffer = chunk.getBuffer();
+    int pos = buffer.position();
+    buffer.put(data, 0, chunkSize);
+    buffer.flip();
+    buffer.position(pos);
+
+    return chunk;
+  }
+
+  private ECChunk[] generateDataChunks() {
     ECChunk[] chunks = new ECChunk[numDataUnits];
     for (int i = 0; i < chunks.length; i++) {
       chunks[i] = generateDataChunk();
@@ -286,6 +351,15 @@ public abstract class TestCoderBase {
     return chunks;
   }
 
+  private void prepareFixedData() {
+    // We may load test data from a resource, or just generate randomly.
+    // The generated data will be used across subsequent encode/decode calls.
+    this.fixedData = new byte[numDataUnits][];
+    for (int i = 0; i < numDataUnits; i++) {
+      fixedData[i] = generateFixedData(baseChunkSize * 2);
+    }
+  }
+
   /**
    * Generate data chunk by making random data.
    * @return
@@ -319,6 +393,17 @@ public abstract class TestCoderBase {
     return buffer;
   }
 
+  protected byte[] generateFixedData(int len) {
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) FIXED_DATA_GENERATOR++;
+      if (FIXED_DATA_GENERATOR == 256) {
+        FIXED_DATA_GENERATOR = 0;
+      }
+    }
+    return buffer;
+  }
+
   /**
    * Prepare parity chunks for encoding, each chunk for each parity unit.
    * @return

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ae32abd/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 02b9eea..80ec04d 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
@@ -80,7 +80,13 @@ public class TestRSRawCoder extends TestRSRawCoderBase {
 
   @Test
   public void testCodingDirectBuffer_10x4_erasure_of_d2_d4_p0() {
-    prepare(null, 10, 4, new int[] {2, 4}, new int[] {0});
+    prepare(null, 10, 4, new int[]{2, 4}, new int[]{0});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_usingFixedData_10x4_erasure_of_d2_d4_p0() {
+    prepare(null, 10, 4, new int[] {2, 4}, new int[] {0}, true);
     testCoding(true);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ae32abd/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
index c06aded..efde332 100644
--- 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
@@ -17,12 +17,7 @@
  */
 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.
@@ -32,6 +27,8 @@ public abstract class TestRSRawCoderBase extends TestRawCoderBase {
   private static int symbolSize = 0;
   private static int symbolMax = 0;
 
+  private static int RS_FIXED_DATA_GENERATOR = 0;
+
   static {
     symbolSize = (int) Math.round(Math.log(
         RSUtil.GF.getFieldSize()) / Math.log(2));
@@ -41,9 +38,21 @@ public abstract class TestRSRawCoderBase extends TestRawCoderBase {
   @Override
   protected byte[] generateData(int len) {
     byte[] buffer = new byte[len];
-    for (int i = 0; i < len; i++) {
+    for (int i = 0; i < buffer.length; i++) {
       buffer[i] = (byte) RAND.nextInt(symbolMax);
     }
     return buffer;
   }
+
+  @Override
+  protected byte[] generateFixedData(int len) {
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) RS_FIXED_DATA_GENERATOR++;
+      if (RS_FIXED_DATA_GENERATOR == symbolMax) {
+        RS_FIXED_DATA_GENERATOR = 0;
+      }
+    }
+    return buffer;
+  }
 }


[20/50] [abbrv] hadoop git commit: HDFS-8372. Erasure coding: compute storage type quotas for striped files, to be consistent with HDFS-8327. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-8372. Erasure coding: compute storage type quotas for striped files, to be consistent with HDFS-8327. 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/97a2396a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/97a2396a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/97a2396a

Branch: refs/heads/HDFS-7285
Commit: 97a2396af685838c9fcb31e48573e758c124d8d7
Parents: 8d3030f
Author: Jing Zhao <ji...@apache.org>
Authored: Tue May 12 11:43:04 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:50 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 ++
 .../namenode/FileWithStripedBlocksFeature.java  | 12 +++--
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 53 +++++++++++++++-----
 .../server/namenode/TestStripedINodeFile.java   | 22 ++++----
 4 files changed, 64 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/97a2396a/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 0acf746..f026a5c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -198,3 +198,6 @@
 
     HDFS-7678. Erasure coding: DFSInputStream with decode functionality (pread).
     (Zhe Zhang)
+
+    HDFS-8372. Erasure coding: compute storage type quotas for striped files,
+    to be consistent with HDFS-8327. (Zhe Zhang via jing9)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97a2396a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java
index 47445be..94ab527 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java
@@ -21,6 +21,7 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 
 /**
  * Feature for file with striped blocks
@@ -78,20 +79,23 @@ class FileWithStripedBlocksFeature implements INode.Feature {
     }
   }
 
-  boolean removeLastBlock(Block oldblock) {
+  BlockInfoStripedUnderConstruction removeLastBlock(
+      Block oldblock) {
     if (blocks == null || blocks.length == 0) {
-      return false;
+      return null;
     }
     int newSize = blocks.length - 1;
     if (!blocks[newSize].equals(oldblock)) {
-      return false;
+      return null;
     }
 
+    BlockInfoStripedUnderConstruction uc =
+        (BlockInfoStripedUnderConstruction) blocks[newSize];
     //copy to a new list
     BlockInfoStriped[] newlist = new BlockInfoStriped[newSize];
     System.arraycopy(blocks, 0, newlist, 0, newSize);
     setBlocks(newlist);
-    return true;
+    return uc;
   }
 
   void truncateStripedBlocks(int n) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97a2396a/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 cc18770..154198c 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
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
@@ -295,7 +296,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * Remove a block from the block list. This block should be
    * the last one on the list.
    */
-  BlockInfoContiguousUnderConstruction removeLastBlock(Block oldblock) {
+  BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
     FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
@@ -317,7 +318,7 @@ public class INodeFile extends INodeWithAdditionalFields
       return uc;
     } else {
       assert hasNoContiguousBlock();
-      return null;
+      return sb.removeLastBlock(oldblock);
     }
   }
 
@@ -676,7 +677,7 @@ public class INodeFile extends INodeWithAdditionalFields
     final long ssDeltaNoReplication;
     short replication;
     if (isStriped()) {
-      return computeQuotaUsageWithStriped(bsps, counts);
+      return computeQuotaUsageWithStriped(bsp, counts);
     }
     
     if (last < lastSnapshotId) {
@@ -702,11 +703,15 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /**
-   * Compute quota of striped file
+   * Compute quota of striped file. Note that currently EC files do not support
+   * append/hflush/hsync, thus the file length recorded in snapshots should be
+   * the same with the current file length.
    */
   public final QuotaCounts computeQuotaUsageWithStriped(
-      BlockStoragePolicySuite bsps, QuotaCounts counts) {
-    return null;
+      BlockStoragePolicy bsp, QuotaCounts counts) {
+    counts.addNameSpace(1);
+    counts.add(storagespaceConsumed(bsp));
+    return counts;
   }
 
   @Override
@@ -828,21 +833,44 @@ public class INodeFile extends INodeWithAdditionalFields
    * Use preferred block size for the last block if it is under construction.
    */
   public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
-    QuotaCounts counts = new QuotaCounts.Builder().build();
     if (isStriped()) {
-      return storagespaceConsumedWithStriped(bsp);
+      return storagespaceConsumedWithStriped();
     } else {
       return storagespaceConsumedWithReplication(bsp);
     }
   }
 
-  public final QuotaCounts storagespaceConsumedWithStriped(
-      BlockStoragePolicy bsp) {
-    return  null;
+  // TODO: support EC with heterogeneous storage
+  public final QuotaCounts storagespaceConsumedWithStriped() {
+    QuotaCounts counts = new QuotaCounts.Builder().build();
+    BlockInfo[] blockInfos = getBlocks();
+    if (blockInfos == null || blockInfos.length == 0) {
+      return counts;
+    }
+
+    long size;
+    final int last = blockInfos.length - 1;
+    if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
+      BlockInfoStripedUnderConstruction blockInfoStripedUC
+          =(BlockInfoStripedUnderConstruction)blockInfos[last];
+      size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum();
+    } else {
+      // In case of last block is complete
+      BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[last];
+      size = blockInfoStriped.spaceConsumed();
+    }
+    for (int i = 0; i < last; i++) {
+      BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[i];
+      size += blockInfoStriped.spaceConsumed();
+    }
+
+    counts.addStorageSpace(size);
+    return  counts;
   }
 
   public final QuotaCounts storagespaceConsumedWithReplication(
-      BlockStoragePolicy bsp) {    QuotaCounts counts = new QuotaCounts.Builder().build();
+      BlockStoragePolicy bsp) {
+    QuotaCounts counts = new QuotaCounts.Builder().build();
     final Iterable<BlockInfo> blocks;
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf == null) {
@@ -965,6 +993,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * compute the quota usage change for a truncate op
    * @param newLength the length for truncation
+   * TODO: properly handle striped blocks (HDFS-7622)
    **/
   void computeQuotaDeltaForTruncate(
       long newLength, BlockStoragePolicy bsps,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97a2396a/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 a725e6b..e24bc54 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
@@ -27,6 +27,7 @@ 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.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
@@ -45,6 +46,11 @@ public class TestStripedINodeFile {
   private static final PermissionStatus perm = new PermissionStatus(
       "userName", null, FsPermission.getDefault());
 
+  private final BlockStoragePolicySuite defaultSuite =
+      BlockStoragePolicySuite.createDefaultSuite();
+  private final BlockStoragePolicy defaultPolicy =
+      defaultSuite.getDefaultPolicy();
+
   private static INodeFile createStripedINodeFile() {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID);
@@ -109,8 +115,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(null));
-    assertEquals(4, inf.storagespaceConsumed(null));
+    assertEquals(4, inf.storagespaceConsumedWithStriped().getStorageSpace());
+    assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
   }
 
   @Test
@@ -134,8 +140,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(null));
-    assertEquals(4 * 2, inf.storagespaceConsumed(null));
+    assertEquals(4 * 2, inf.storagespaceConsumedWithStriped().getStorageSpace());
+    assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
   }
 
   @Test
@@ -188,10 +194,8 @@ public class TestStripedINodeFile {
     blockInfoStriped.setNumBytes(100);
     inf.addBlock(blockInfoStriped);
 
-    BlockStoragePolicySuite suite =
-        BlockStoragePolicySuite.createDefaultSuite();
     QuotaCounts counts =
-        inf.computeQuotaUsageWithStriped(suite,
+        inf.computeQuotaUsageWithStriped(defaultPolicy,
             new QuotaCounts.Builder().build());
     assertEquals(1, counts.getNameSpace());
     // The total consumed space is the sum of
@@ -215,10 +219,8 @@ public class TestStripedINodeFile {
     bInfoStripedUC.setNumBytes(100);
     inf.addBlock(bInfoStripedUC);
 
-    BlockStoragePolicySuite suite
-        = BlockStoragePolicySuite.createDefaultSuite();
     QuotaCounts counts
-        = inf.computeQuotaUsageWithStriped(suite,
+        = inf.computeQuotaUsageWithStriped(defaultPolicy,
               new QuotaCounts.Builder().build());
     assertEquals(1024, inf.getPreferredBlockSize());
     assertEquals(1, counts.getNameSpace());


[13/50] [abbrv] hadoop git commit: HDFS-7678. Erasure coding: DFSInputStream with decode functionality (pread). Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-7678. Erasure coding: DFSInputStream with decode functionality (pread). 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/8d3030f0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8d3030f0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8d3030f0

Branch: refs/heads/HDFS-7285
Commit: 8d3030f064116a657c2cbb7c7560af6bed1d5586
Parents: 6bacaa9
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon May 11 21:10:23 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:49 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   3 +
 .../hadoop/hdfs/DFSStripedInputStream.java      | 164 ++++--
 .../erasurecode/ErasureCodingWorker.java        |  10 +-
 .../hadoop/hdfs/util/StripedBlockUtil.java      | 517 +++++++++++++++++--
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |  97 +++-
 .../hadoop/hdfs/TestWriteReadStripedFile.java   |  49 ++
 6 files changed, 768 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3030f0/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 c7d01c7..0acf746 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -195,3 +195,6 @@
 
     HDFS-8355. Erasure Coding: Refactor BlockInfo and BlockInfoUnderConstruction.
     (Tsz Wo Nicholas Sze via jing9)
+
+    HDFS-7678. Erasure coding: DFSInputStream with decode functionality (pread).
+    (Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3030f0/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 7425e75..7678fae 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,15 +21,27 @@ 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.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.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;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.divideByteRangeIntoStripes;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.initDecodeInputs;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.decodeAndFillBuffer;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getNextCompletedStripedRead;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.ReadPortion;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
 
 import org.apache.hadoop.io.erasurecode.ECSchema;
+
 import org.apache.hadoop.net.NetUtils;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
@@ -37,10 +49,12 @@ import org.apache.htrace.TraceScope;
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.EnumSet;
 import java.util.Set;
+import java.util.Collection;
 import java.util.Map;
 import java.util.HashMap;
 import java.util.concurrent.CompletionService;
@@ -51,7 +65,6 @@ import java.util.concurrent.CancellationException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Future;
 
-
 /******************************************************************************
  * DFSStripedInputStream reads from striped block groups, illustrated below:
  *
@@ -125,6 +138,7 @@ public class DFSStripedInputStream extends DFSInputStream {
   private final short parityBlkNum;
   /** the buffer for a complete stripe */
   private ByteBuffer curStripeBuf;
+  private final ECSchema schema;
   /**
    * indicate the start/end offset of the current buffered stripe in the
    * block group
@@ -137,6 +151,7 @@ public class DFSStripedInputStream extends DFSInputStream {
     super(dfsClient, src, verifyChecksum);
 
     assert schema != null;
+    this.schema = schema;
     cellSize = schema.getChunkSize();
     dataBlkNum = (short) schema.getNumDataUnits();
     parityBlkNum = (short) schema.getNumParityUnits();
@@ -472,12 +487,10 @@ public class DFSStripedInputStream extends DFSInputStream {
    */
   @Override
   protected LocatedBlock getBlockAt(long blkStartOffset) throws IOException {
-    LocatedBlock lb = super.getBlockAt(blkStartOffset);
-    assert lb instanceof LocatedStripedBlock : "NameNode should return a " +
-        "LocatedStripedBlock for a striped file";
+    LocatedBlock lb = getBlockGroupAt(blkStartOffset);
 
-    int idx = (int) (((blkStartOffset - lb.getStartOffset()) / cellSize)
-        % dataBlkNum);
+    int idx = (int) ((blkStartOffset - lb.getStartOffset())
+        % (dataBlkNum + parityBlkNum));
     // If indexing information is returned, iterate through the index array
     // to find the entry for position idx in the group
     LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
@@ -509,48 +522,121 @@ public class DFSStripedInputStream extends DFSInputStream {
       long end, byte[] buf, int offset,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
-    Map<Future<Void>, Integer> futures = new HashMap<>();
-    CompletionService<Void> stripedReadsService =
-        new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool());
-    int len = (int) (end - start + 1);
-
     // Refresh the striped block group
     LocatedStripedBlock blockGroup = getBlockGroupAt(blockStartOffset);
 
+    AlignedStripe[] stripes = divideByteRangeIntoStripes(schema, blockGroup,
+        start, end, buf, offset);
+    for (AlignedStripe stripe : stripes) {
+      fetchOneStripe(blockGroup, buf, stripe, corruptedBlockMap);
+    }
+  }
 
-    // Planning the portion of I/O for each shard
-    ReadPortion[] readPortions = planReadPortions(dataBlkNum, cellSize, start,
-        len, offset);
-
+  private void fetchOneStripe(LocatedStripedBlock blockGroup,
+      byte[] buf, AlignedStripe alignedStripe, Map<ExtendedBlock,
+      Set<DatanodeInfo>> corruptedBlockMap) throws IOException {
+    Map<Future<Void>, Integer> futures = new HashMap<>();
+    CompletionService<Void> service =
+        new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool());
+    if (alignedStripe.getSpanInBlock() == 0) {
+      DFSClient.LOG.warn("Trying to read an empty stripe from" + blockGroup);
+      return;
+    }
     // Parse group to get chosen DN location
     LocatedBlock[] blks = StripedBlockUtil.
         parseStripedBlockGroup(blockGroup, cellSize, dataBlkNum, parityBlkNum);
-
     for (short i = 0; i < dataBlkNum; i++) {
-      ReadPortion rp = readPortions[i];
-      if (rp.getReadLength() <= 0) {
-        continue;
+      if (alignedStripe.chunks[i] != null
+          && alignedStripe.chunks[i].state != StripingChunk.ALLZERO) {
+        fetchOneStripingChunk(futures, service, blks[i], alignedStripe, i,
+            corruptedBlockMap);
       }
-      DatanodeInfo loc = blks[i].getLocations()[0];
-      StorageType type = blks[i].getStorageTypes()[0];
-      DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr(
-          loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())),
-          type);
-      Callable<Void> readCallable = getFromOneDataNode(dnAddr,
-          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]);
-      futures.put(getFromDNRequest, (int) i);
     }
+    // Input buffers for potential decode operation, which remains null until
+    // first read failure
+    byte[][] decodeInputs = null;
     while (!futures.isEmpty()) {
       try {
-        waitNextCompletion(stripedReadsService, futures);
+        StripingChunkReadResult r = getNextCompletedStripedRead(
+            service, futures, 0);
+        if (DFSClient.LOG.isDebugEnabled()) {
+          DFSClient.LOG.debug("Read task returned: " + r + ", for stripe " + alignedStripe);
+        }
+        StripingChunk returnedChunk = alignedStripe.chunks[r.index];
+        Preconditions.checkNotNull(returnedChunk);
+        Preconditions.checkState(returnedChunk.state == StripingChunk.PENDING);
+        if (r.state == StripingChunkReadResult.SUCCESSFUL) {
+          returnedChunk.state = StripingChunk.FETCHED;
+          alignedStripe.fetchedChunksNum++;
+          if (alignedStripe.fetchedChunksNum == dataBlkNum) {
+            clearFutures(futures.keySet());
+            break;
+          }
+        } else {
+          returnedChunk.state = StripingChunk.MISSING;
+          alignedStripe.missingChunksNum++;
+          if (alignedStripe.missingChunksNum > parityBlkNum) {
+            clearFutures(futures.keySet());
+            throw new IOException("Too many blocks are missing: " + alignedStripe);
+          }
+          // When seeing first missing block, initialize decode input buffers
+          if (decodeInputs == null) {
+            decodeInputs = initDecodeInputs(alignedStripe, dataBlkNum, parityBlkNum);
+          }
+          for (int i = 0; i < alignedStripe.chunks.length; i++) {
+            StripingChunk chunk = alignedStripe.chunks[i];
+            Preconditions.checkNotNull(chunk);
+            if (chunk.state == StripingChunk.REQUESTED && i <= dataBlkNum) {
+              fetchOneStripingChunk(futures, service, blks[i], alignedStripe, i,
+                  corruptedBlockMap);
+            }
+          }
+        }
       } catch (InterruptedException ie) {
-        // Ignore and retry
+        String err = "Read request interrupted";
+        DFSClient.LOG.error(err);
+        clearFutures(futures.keySet());
+        // Don't decode if read interrupted
+        throw new InterruptedIOException(err);
       }
     }
+
+    if (alignedStripe.missingChunksNum > 0) {
+      decodeAndFillBuffer(decodeInputs, buf, alignedStripe,
+          dataBlkNum, parityBlkNum);
+    }
+  }
+
+  /**
+   * Schedule a single read request to an internal block
+   * @param block The internal block
+   * @param index Index of the internal block in the group
+   * @param corruptedBlockMap Map of corrupted blocks
+   */
+  private void fetchOneStripingChunk(Map<Future<Void>, Integer> futures,
+      final CompletionService<Void> service, final LocatedBlock block,
+      final AlignedStripe alignedStripe, final int index,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
+    DatanodeInfo loc = block.getLocations()[0];
+    StorageType type = block.getStorageTypes()[0];
+    DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr(
+        loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())),
+        type);
+    StripingChunk chunk = alignedStripe.chunks[index];
+    chunk.state = StripingChunk.PENDING;
+    Callable<Void> readCallable = getFromOneDataNode(dnAddr,
+        block.getStartOffset(), alignedStripe.getOffsetInBlock(),
+        alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1, chunk.buf,
+        chunk.getOffsets(), chunk.getLengths(),
+        corruptedBlockMap, index);
+    Future<Void> getFromDNRequest = service.submit(readCallable);
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("Submitting striped read request for " + index +
+          ". Info of the block: " + block + ", offset in block is " +
+          alignedStripe.getOffsetInBlock() + ", end is " +
+          (alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1));
+    }
+    futures.put(getFromDNRequest, index);
   }
 
   private Callable<Void> getFromOneDataNode(final DNAddrPair datanode,
@@ -609,4 +695,12 @@ public class DFSStripedInputStream extends DFSInputStream {
     throw new UnsupportedOperationException(
         "Not support enhanced byte buffer access.");
   }
+
+  /** A variation to {@link DFSInputStream#cancelAll} */
+  private void clearFutures(Collection<Future<Void>> futures) {
+    for (Future<Void> future : futures) {
+      future.cancel(false);
+    }
+    futures.clear();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3030f0/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 5ede508..eedb191 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
@@ -67,7 +67,7 @@ import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
-import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripedReadResult;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
@@ -462,10 +462,10 @@ public final class ErasureCodingWorker {
       int nsuccess = 0;
       while (!futures.isEmpty()) {
         try {
-          StripedReadResult result = 
+          StripingChunkReadResult result =
               StripedBlockUtil.getNextCompletedStripedRead(
                   readService, futures, STRIPED_READ_THRESHOLD_MILLIS);
-          if (result.state == StripedReadResult.SUCCESSFUL) {
+          if (result.state == StripingChunkReadResult.SUCCESSFUL) {
             success[nsuccess++] = result.index;
             if (nsuccess >= dataBlkNum) {
               // cancel remaining reads if we read successfully from minimum
@@ -474,14 +474,14 @@ public final class ErasureCodingWorker {
               futures.clear();
               break;
             }
-          } else if (result.state == StripedReadResult.FAILED) {
+          } else if (result.state == StripingChunkReadResult.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) {
+          } else if (result.state == StripingChunkReadResult.TIMEOUT) {
             // If timeout, we also schedule a new read.
             scheduleNewRead(used);
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3030f0/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 45bbf6b..f7ae88a 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
@@ -22,16 +22,18 @@ import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSStripedOutputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutionException;
@@ -85,7 +87,7 @@ public class StripedBlockUtil {
         new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
         new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
         new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
-        bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(),
+        bg.getStartOffset() + idxInBlockGroup, bg.isCorrupt(),
         null);
   }
 
@@ -238,33 +240,37 @@ public class StripedBlockUtil {
   /**
    * Get the next completed striped read task
    *
-   * @return {@link StripedReadResult} indicating the status of the read task
+   * @return {@link StripingChunkReadResult} indicating the status of the read task
    *          succeeded, and the block index of the task. If the method times
    *          out without getting any completed read tasks, -1 is returned as
    *          block index.
    * @throws InterruptedException
    */
-  public static StripedReadResult getNextCompletedStripedRead(
+  public static StripingChunkReadResult 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 (threshold > 0) {
+        future = readService.poll(threshold, TimeUnit.MILLISECONDS);
+      } else {
+        future = readService.take();
+      }
       if (future != null) {
         future.get();
-        return new StripedReadResult(futures.remove(future),
-            StripedReadResult.SUCCESSFUL);
+        return new StripingChunkReadResult(futures.remove(future),
+            StripingChunkReadResult.SUCCESSFUL);
       } else {
-        return new StripedReadResult(StripedReadResult.TIMEOUT);
+        return new StripingChunkReadResult(StripingChunkReadResult.TIMEOUT);
       }
     } catch (ExecutionException e) {
-      return new StripedReadResult(futures.remove(future),
-          StripedReadResult.FAILED);
+      DFSClient.LOG.error("ExecutionException " + e);
+      return new StripingChunkReadResult(futures.remove(future),
+          StripingChunkReadResult.FAILED);
     } catch (CancellationException e) {
-      return new StripedReadResult(futures.remove(future),
-          StripedReadResult.CANCELLED);
+      return new StripingChunkReadResult(futures.remove(future),
+          StripingChunkReadResult.CANCELLED);
     }
   }
 
@@ -291,26 +297,247 @@ public class StripedBlockUtil {
   }
 
   /**
-   * This class represents the portion of I/O associated with each block in the
-   * striped block group.
+   * Initialize the decoding input buffers based on the chunk states in an
+   * AlignedStripe
    */
-  public static class ReadPortion {
+  public static byte[][] initDecodeInputs(AlignedStripe alignedStripe,
+      int dataBlkNum, int parityBlkNum) {
+    byte[][] decodeInputs =
+        new byte[dataBlkNum + parityBlkNum][(int) alignedStripe.getSpanInBlock()];
+    for (int i = 0; i < alignedStripe.chunks.length; i++) {
+      StripingChunk chunk = alignedStripe.chunks[i];
+      if (chunk == null) {
+        alignedStripe.chunks[i] = new StripingChunk(decodeInputs[i]);
+        alignedStripe.chunks[i].offsetsInBuf.add(0);
+        alignedStripe.chunks[i].lengthsInBuf.add((int) alignedStripe.getSpanInBlock());
+      } else if (chunk.state == StripingChunk.FETCHED) {
+        int posInBuf = 0;
+        for (int j = 0; j < chunk.offsetsInBuf.size(); j++) {
+          System.arraycopy(chunk.buf, chunk.offsetsInBuf.get(j),
+              decodeInputs[i], posInBuf, chunk.lengthsInBuf.get(j));
+          posInBuf += chunk.lengthsInBuf.get(j);
+        }
+      } else if (chunk.state == StripingChunk.ALLZERO) {
+        Arrays.fill(decodeInputs[i], (byte)0);
+      }
+    }
+    return decodeInputs;
+  }
+
+  /**
+   * Decode based on the given input buffers and schema
+   */
+  public static void decodeAndFillBuffer(final byte[][] decodeInputs, byte[] buf,
+      AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum) {
+    int[] decodeIndices = new int[parityBlkNum];
+    int pos = 0;
+    for (int i = 0; i < alignedStripe.chunks.length; i++) {
+      if (alignedStripe.chunks[i].state != StripingChunk.FETCHED &&
+          alignedStripe.chunks[i].state != StripingChunk.ALLZERO) {
+        decodeIndices[pos++] = i;
+      }
+    }
+
+    byte[][] outputs = new byte[parityBlkNum][(int) alignedStripe.getSpanInBlock()];
+    RSRawDecoder rsRawDecoder = new RSRawDecoder();
+    rsRawDecoder.initialize(dataBlkNum, parityBlkNum, (int) alignedStripe.getSpanInBlock());
+    rsRawDecoder.decode(decodeInputs, decodeIndices, outputs);
+
+    for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {
+      StripingChunk chunk = alignedStripe.chunks[i];
+      if (chunk.state == StripingChunk.MISSING) {
+        int srcPos = 0;
+        for (int j = 0; j < chunk.offsetsInBuf.size(); j++) {
+          //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938
+//          System.arraycopy(outputs[i], srcPos, buf, chunk.offsetsInBuf.get(j),
+//              chunk.lengthsInBuf.get(j));
+          Arrays.fill(buf, chunk.offsetsInBuf.get(j),
+              chunk.offsetsInBuf.get(j) + chunk.lengthsInBuf.get(j), (byte)7);
+          srcPos += chunk.lengthsInBuf.get(j);
+        }
+      }
+    }
+  }
+
+  /**
+   * This method divides a requested byte range into an array of
+   * {@link AlignedStripe}
+   *
+   *
+   * At most 5 stripes will be generated from each logical range
+   * TODO: cleanup and get rid of planReadPortions
+   */
+  public static AlignedStripe[] divideByteRangeIntoStripes (
+      ECSchema ecSchema, LocatedStripedBlock blockGroup, long start, long end,
+      byte[] buf, int offsetInBuf) {
+    // TODO: change ECSchema naming to use cell size instead of chunk size
+
+    // Step 0: analyze range and calculate basic parameters
+    int cellSize = ecSchema.getChunkSize();
+    int dataBlkNum = ecSchema.getNumDataUnits();
+    int len = (int) (end - start + 1);
+    int firstCellIdxInBG = (int) (start / cellSize);
+    int lastCellIdxInBG = (int) (end / cellSize);
+    int firstCellSize = Math.min(cellSize - (int) (start % cellSize), len);
+    long firstCellOffsetInBlk = start % cellSize;
+    int lastCellSize = lastCellIdxInBG == firstCellIdxInBG ?
+        firstCellSize : (int) (end % cellSize) + 1;
+
+    // Step 1: get the unmerged ranges on each internal block
+    // TODO: StripingCell should carry info on size and start offset (HDFS-8320)
+    VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema,
+        firstCellIdxInBG, lastCellIdxInBG, firstCellSize, firstCellOffsetInBlk,
+        lastCellSize);
+
+    // Step 2: merge into at most 5 stripes
+    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges);
+
+    // Step 3: calculate each chunk's position in destination buffer
+    calcualteChunkPositionsInBuf(ecSchema, blockGroup, buf, offsetInBuf,
+        firstCellIdxInBG, lastCellIdxInBG, firstCellSize, firstCellOffsetInBlk,
+        lastCellSize, stripes);
+
+    // Step 4: prepare ALLZERO blocks
+    prepareAllZeroChunks(blockGroup, buf, stripes, cellSize, dataBlkNum);
+
+    return stripes;
+  }
+
+  private static VerticalRange[] getRangesForInternalBlocks (ECSchema ecSchema,
+      int firstCellIdxInBG, int lastCellIdxInBG, int firstCellSize,
+      long firstCellOffsetInBlk, int lastCellSize) {
+    int cellSize = ecSchema.getChunkSize();
+    int dataBlkNum = ecSchema.getNumDataUnits();
+
+    StripingCell firstCell = new StripingCell(ecSchema, firstCellIdxInBG);
+    StripingCell lastCell = new StripingCell(ecSchema, lastCellIdxInBG);
+
+    VerticalRange ranges[] = new VerticalRange[dataBlkNum];
+    ranges[firstCell.idxInStripe] =
+        new VerticalRange(firstCellOffsetInBlk, firstCellSize);
+    for (int i = firstCellIdxInBG + 1; i < lastCellIdxInBG; i++) {
+      // iterate through all cells and update the list of StripeRanges
+      StripingCell cell = new StripingCell(ecSchema, i);
+      if (ranges[cell.idxInStripe] == null) {
+        ranges[cell.idxInStripe] = new VerticalRange(
+            cell.idxInInternalBlk * cellSize, cellSize);
+      } else {
+        ranges[cell.idxInStripe].spanInBlock += cellSize;
+      }
+    }
+    if (ranges[lastCell.idxInStripe] == null) {
+      ranges[lastCell.idxInStripe] = new VerticalRange(
+          lastCell.idxInInternalBlk * cellSize, lastCellSize);
+    } else if (lastCell.idxInBlkGroup != firstCell.idxInBlkGroup) {
+      ranges[lastCell.idxInStripe].spanInBlock += lastCellSize;
+    }
+
+    return ranges;
+  }
+
+  private static AlignedStripe[] mergeRangesForInternalBlocks(ECSchema ecSchema,
+      VerticalRange[] ranges) {
+    int dataBlkNum = ecSchema.getNumDataUnits();
+    int parityBlkNum = ecSchema.getNumParityUnits();
+    List<AlignedStripe> stripes = new ArrayList<>();
+    SortedSet<Long> stripePoints = new TreeSet<>();
+    for (VerticalRange r : ranges) {
+      if (r != null) {
+        stripePoints.add(r.offsetInBlock);
+        stripePoints.add(r.offsetInBlock + r.spanInBlock);
+      }
+    }
+
+    long prev = -1;
+    for (long point : stripePoints) {
+      if (prev >= 0) {
+        stripes.add(new AlignedStripe(prev, point - prev,
+            dataBlkNum + parityBlkNum));
+      }
+      prev = point;
+    }
+    return stripes.toArray(new AlignedStripe[stripes.size()]);
+  }
+
+  private static void calcualteChunkPositionsInBuf(ECSchema ecSchema,
+      LocatedStripedBlock blockGroup, byte[] buf, int offsetInBuf,
+      int firstCellIdxInBG, int lastCellIdxInBG, int firstCellSize,
+      long firstCellOffsetInBlk, int lastCellSize, AlignedStripe[] stripes) {
+    int cellSize = ecSchema.getChunkSize();
+    int dataBlkNum = ecSchema.getNumDataUnits();
+    // Step 3: calculate each chunk's position in destination buffer
     /**
-     * startOffsetInBlock
-     *     |
-     *     v
-     *     |<-lengths[0]->|<-  lengths[1]  ->|<-lengths[2]->|
+     *     | <--------------- AlignedStripe --------------->|
+     *
+     *     |<- length_0 ->|<--  length_1  -->|<- length_2 ->|
      * +------------------+------------------+----------------+
-     * |      cell_0      |      cell_3      |     cell_6     |  <- blk_0
+     * |    cell_0_0_0    |    cell_3_1_0    |   cell_6_2_0   |  <- 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)   |          |
-     * +------------------------------------------------------+
+     *  v offset_0                                 v offset_1
+     * +----------------------------------------------------------+
+     * |  cell_0_0_0 |  cell_1_0_1 and cell_2_0_2  |cell_3_1_0 ...|   <- buf
+     * |  (partial)  |    (from blk_1 and blk_2)   |              |
+     * +----------------------------------------------------------+
+     *
+     * Cell indexing convention defined in {@link StripingCell}
      */
+    int done = 0;
+    for (int i = firstCellIdxInBG; i <= lastCellIdxInBG; i++) {
+      StripingCell cell  = new StripingCell(ecSchema, i);
+      long cellStart = i == firstCellIdxInBG ?
+          firstCellOffsetInBlk : cell.idxInInternalBlk * cellSize;
+      int cellLen;
+      if (i == firstCellIdxInBG) {
+        cellLen = firstCellSize;
+      } else if (i == lastCellIdxInBG) {
+        cellLen = lastCellSize;
+      } else {
+        cellLen = cellSize;
+      }
+      long cellEnd = cellStart + cellLen - 1;
+      for (AlignedStripe s : stripes) {
+        long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1;
+        long overlapStart = Math.max(cellStart, s.getOffsetInBlock());
+        long overlapEnd = Math.min(cellEnd, stripeEnd);
+        int overLapLen = (int) (overlapEnd - overlapStart + 1);
+        if (overLapLen <= 0) {
+          continue;
+        }
+        if (s.chunks[cell.idxInStripe] == null) {
+          s.chunks[cell.idxInStripe] = new StripingChunk(buf);
+        }
+
+        s.chunks[cell.idxInStripe].offsetsInBuf.
+            add((int)(offsetInBuf + done + overlapStart - cellStart));
+        s.chunks[cell.idxInStripe].lengthsInBuf.add(overLapLen);
+      }
+      done += cellLen;
+    }
+  }
+
+  private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup,
+      byte[] buf, AlignedStripe[] stripes, int cellSize, int dataBlkNum) {
+    for (AlignedStripe s : stripes) {
+      for (int i = 0; i < dataBlkNum; i++) {
+        long internalBlkLen = getInternalBlockLength(blockGroup.getBlockSize(),
+            cellSize, dataBlkNum, i);
+        if (internalBlkLen <= s.getOffsetInBlock()) {
+          Preconditions.checkState(s.chunks[i] == null);
+          s.chunks[i] = new StripingChunk(buf);
+          s.chunks[i].state = StripingChunk.ALLZERO;
+        }
+      }
+    }
+  }
+
+  /**
+   * This class represents the portion of I/O associated with each block in the
+   * striped block group.
+   * TODO: consolidate ReadPortion with AlignedStripe
+   */
+  public static class ReadPortion {
     private long startOffsetInBlock = 0;
     private int readLength = 0;
     public final List<Integer> offsetsInBuf = new ArrayList<>();
@@ -350,11 +577,234 @@ public class StripedBlockUtil {
   }
 
   /**
+   * The unit of encoding used in {@link DFSStripedOutputStream}
+   *  | <------- Striped Block Group -------> |
+   *    blk_0          blk_1          blk_2
+   *      |              |              |
+   *      v              v              v
+   * +----------+   +----------+   +----------+
+   * |cell_0_0_0|   |cell_1_0_1|   |cell_2_0_2|
+   * +----------+   +----------+   +----------+
+   * |cell_3_1_0|   |cell_4_1_1|   |cell_5_1_2| <- {@link idxInBlkGroup} = 5
+   * +----------+   +----------+   +----------+    {@link idxInInternalBlk} = 1
+   *                                               {@link idxInStripe} = 2
+   * A StripingCell is a special instance of {@link StripingChunk} whose offset
+   * and size align with the cell used when writing data.
+   * TODO: consider parity cells
+   */
+  public static class StripingCell {
+    public final ECSchema schema;
+    /** Logical order in a block group, used when doing I/O to a block group */
+    public final int idxInBlkGroup;
+    public final int idxInInternalBlk;
+    public final int idxInStripe;
+
+    public StripingCell(ECSchema ecSchema, int idxInBlkGroup) {
+      this.schema = ecSchema;
+      this.idxInBlkGroup = idxInBlkGroup;
+      this.idxInInternalBlk = idxInBlkGroup / ecSchema.getNumDataUnits();
+      this.idxInStripe = idxInBlkGroup -
+          this.idxInInternalBlk * ecSchema.getNumDataUnits();
+    }
+
+    public StripingCell(ECSchema ecSchema, int idxInInternalBlk,
+        int idxInStripe) {
+      this.schema = ecSchema;
+      this.idxInInternalBlk = idxInInternalBlk;
+      this.idxInStripe = idxInStripe;
+      this.idxInBlkGroup =
+          idxInInternalBlk * ecSchema.getNumDataUnits() + idxInStripe;
+    }
+  }
+
+  /**
+   * Given a requested byte range on a striped block group, an AlignedStripe
+   * represents a {@link VerticalRange} that is aligned with both the byte range
+   * and boundaries of all internal blocks. As illustrated in the diagram, any
+   * given byte range on a block group leads to 1~5 AlignedStripe's.
+   *
+   * |<-------- Striped Block Group -------->|
+   * blk_0   blk_1   blk_2      blk_3   blk_4
+   *                 +----+  |  +----+  +----+
+   *                 |full|  |  |    |  |    | <- AlignedStripe0:
+   *         +----+  |~~~~|  |  |~~~~|  |~~~~|      1st cell is partial
+   *         |part|  |    |  |  |    |  |    | <- AlignedStripe1: byte range
+   * +----+  +----+  +----+  |  |~~~~|  |~~~~|      doesn't start at 1st block
+   * |full|  |full|  |full|  |  |    |  |    |
+   * |cell|  |cell|  |cell|  |  |    |  |    | <- AlignedStripe2 (full stripe)
+   * |    |  |    |  |    |  |  |    |  |    |
+   * +----+  +----+  +----+  |  |~~~~|  |~~~~|
+   * |full|  |part|          |  |    |  |    | <- AlignedStripe3: byte range
+   * |~~~~|  +----+          |  |~~~~|  |~~~~|      doesn't end at last block
+   * |    |                  |  |    |  |    | <- AlignedStripe4:
+   * +----+                  |  +----+  +----+      last cell is partial
+   *                         |
+   * <---- data blocks ----> | <--- parity --->
+   *
+   * An AlignedStripe is the basic unit of reading from a striped block group,
+   * because within the AlignedStripe, all internal blocks can be processed in
+   * a uniform manner.
+   *
+   * The coverage of an AlignedStripe on an internal block is represented as a
+   * {@link StripingChunk}.
+   * To simplify the logic of reading a logical byte range from a block group,
+   * a StripingChunk is either completely in the requested byte range or
+   * completely outside the requested byte range.
+   */
+  public static class AlignedStripe {
+    public VerticalRange range;
+    /** status of each chunk in the stripe */
+    public final StripingChunk[] chunks;
+    public int fetchedChunksNum = 0;
+    public int missingChunksNum = 0;
+
+    public AlignedStripe(long offsetInBlock, long length, int width) {
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      this.range = new VerticalRange(offsetInBlock, length);
+      this.chunks = new StripingChunk[width];
+    }
+
+    public AlignedStripe(VerticalRange range, int width) {
+      this.range = range;
+      this.chunks = new StripingChunk[width];
+    }
+
+    public boolean include(long pos) {
+      return range.include(pos);
+    }
+
+    public long getOffsetInBlock() {
+      return range.offsetInBlock;
+    }
+
+    public long getSpanInBlock() {
+      return range.spanInBlock;
+    }
+
+    @Override
+    public String toString() {
+      return "Offset=" + range.offsetInBlock + ", length=" + range.spanInBlock +
+          ", fetchedChunksNum=" + fetchedChunksNum +
+          ", missingChunksNum=" + missingChunksNum;
+    }
+  }
+
+  /**
+   * A simple utility class representing an arbitrary vertical inclusive range
+   * starting at {@link offsetInBlock} and lasting for {@link length} bytes in
+   * an internal block. Note that VerticalRange doesn't necessarily align with
+   * {@link StripingCell}.
+   *
+   * |<- Striped Block Group ->|
+   *  blk_0
+   *    |
+   *    v
+   * +-----+
+   * |~~~~~| <-- {@link offsetInBlock}
+   * |     |  ^
+   * |     |  |
+   * |     |  | {@link spanInBlock}
+   * |     |  v
+   * |~~~~~| ---
+   * |     |
+   * +-----+
+   */
+  public static class VerticalRange {
+    /** start offset in the block group (inclusive) */
+    public long offsetInBlock;
+    /** length of the stripe range */
+    public long spanInBlock;
+
+    public VerticalRange(long offsetInBlock, long length) {
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      this.offsetInBlock = offsetInBlock;
+      this.spanInBlock = length;
+    }
+
+    /** whether a position is in the range */
+    public boolean include(long pos) {
+      return pos >= offsetInBlock && pos < offsetInBlock + spanInBlock;
+    }
+  }
+
+  /**
+   * Indicates the coverage of an {@link AlignedStripe} on an internal block,
+   * and the state of the chunk in the context of the read request.
+   *
+   * |<---------------- Striped Block Group --------------->|
+   *   blk_0        blk_1        blk_2          blk_3   blk_4
+   *                           +---------+  |  +----+  +----+
+   *     null         null     |REQUESTED|  |  |null|  |null| <- AlignedStripe0
+   *              +---------+  |---------|  |  |----|  |----|
+   *     null     |REQUESTED|  |REQUESTED|  |  |null|  |null| <- AlignedStripe1
+   * +---------+  +---------+  +---------+  |  +----+  +----+
+   * |REQUESTED|  |REQUESTED|    ALLZERO    |  |null|  |null| <- AlignedStripe2
+   * +---------+  +---------+               |  +----+  +----+
+   * <----------- data blocks ------------> | <--- parity --->
+   *
+   * The class also carries {@link buf}, {@link offsetsInBuf}, and
+   * {@link lengthsInBuf} to define how read task for this chunk should deliver
+   * the returned data.
+   */
+  public static class StripingChunk {
+    /** Chunk has been successfully fetched */
+    public static final int FETCHED = 0x01;
+    /** Chunk has encountered failed when being fetched */
+    public static final int MISSING = 0x02;
+    /** Chunk being fetched (fetching task is in-flight) */
+    public static final int PENDING = 0x04;
+    /**
+     * Chunk is requested either by application or for decoding, need to
+     * schedule read task
+     */
+    public static final int REQUESTED = 0X08;
+    /**
+     * Internal block is short and has no overlap with chunk. Chunk considered
+     * all-zero bytes in codec calculations.
+     */
+    public static final int ALLZERO = 0X0f;
+
+    /**
+     * If a chunk is completely in requested range, the state transition is:
+     * REQUESTED (when AlignedStripe created) -> PENDING -> {FETCHED | MISSING}
+     * If a chunk is completely outside requested range (including parity
+     * chunks), state transition is:
+     * null (AlignedStripe created) -> REQUESTED (upon failure) -> PENDING ...
+     */
+    public int state = REQUESTED;
+    public byte[] buf;
+    public List<Integer> offsetsInBuf;
+    public List<Integer> lengthsInBuf;
+
+    public StripingChunk(byte[] buf) {
+      this.buf = buf;
+      this.offsetsInBuf = new ArrayList<>();
+      this.lengthsInBuf = new ArrayList<>();
+    }
+
+    public int[] getOffsets() {
+      int[] offsets = new int[offsetsInBuf.size()];
+      for (int i = 0; i < offsets.length; i++) {
+        offsets[i] = offsetsInBuf.get(i);
+      }
+      return offsets;
+    }
+
+    public int[] getLengths() {
+      int[] lens = new int[this.lengthsInBuf.size()];
+      for (int i = 0; i < lens.length; i++) {
+        lens[i] = this.lengthsInBuf.get(i);
+      }
+      return lens;
+    }
+  }
+
+  /**
    * 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 class StripingChunkReadResult {
     public static final int SUCCESSFUL = 0x01;
     public static final int FAILED = 0x02;
     public static final int TIMEOUT = 0x04;
@@ -363,18 +813,23 @@ public class StripedBlockUtil {
     public final int index;
     public final int state;
 
-    public StripedReadResult(int state) {
+    public StripingChunkReadResult(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) {
+    public StripingChunkReadResult(int index, int state) {
       Preconditions.checkArgument(state != TIMEOUT,
           "Timeout result should return negative index.");
       this.index = index;
       this.state = state;
     }
+
+    @Override
+    public String toString() {
+      return "(index=" + index + ", state =" + state + ")";
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3030f0/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 3f79933..452cc2b 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
@@ -36,6 +36,7 @@ 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.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -133,8 +134,102 @@ public class TestDFSStripedInputStream {
     byte[] readBuffer = new byte[readSize];
     int ret = in.read(0, readBuffer, 0, readSize);
 
+    byte[] expected = new byte[readSize];
+    /** 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 = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k;
+          expected[posInFile] = SimulatedFSDataset.simulatedByte(
+              new Block(bg.getBlock().getBlockId() + j), posInBlk);
+        }
+      }
+    }
+
     assertEquals(readSize, ret);
-    // TODO: verify read results with patterned data from HDFS-8117
+    assertArrayEquals(expected, readBuffer);
+  }
+
+  @Test
+  public void testPreadWithDNFailure() throws Exception {
+    final int numBlocks = 4;
+    final int failedDNIdx = 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 + PARITY_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,
+            ErasureCodingSchemaManager.getSystemDefaultSchema());
+    int readSize = BLOCK_GROUP_SIZE;
+    byte[] readBuffer = new byte[readSize];
+    byte[] expected = new byte[readSize];
+    cluster.stopDataNode(failedDNIdx);
+    /** 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 = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k;
+          expected[posInFile] = SimulatedFSDataset.simulatedByte(
+              new Block(bg.getBlock().getBlockId() + j), posInBlk);
+        }
+      }
+    }
+
+    // Update the expected content for decoded data
+    for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) {
+      byte[][] decodeInputs = new byte[DATA_BLK_NUM + PARITY_BLK_NUM][CELLSIZE];
+      int[] missingBlkIdx = new int[]{failedDNIdx, DATA_BLK_NUM+1, DATA_BLK_NUM+2};
+      byte[][] decodeOutputs = new byte[PARITY_BLK_NUM][CELLSIZE];
+      for (int j = 0; j < DATA_BLK_NUM; j++) {
+        int posInBuf = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE;
+        if (j != failedDNIdx) {
+          System.arraycopy(expected, posInBuf, decodeInputs[j], 0, CELLSIZE);
+        }
+      }
+      for (int k = 0; k < CELLSIZE; k++) {
+        int posInBlk = i * CELLSIZE + k;
+        decodeInputs[DATA_BLK_NUM][k] = SimulatedFSDataset.simulatedByte(
+            new Block(bg.getBlock().getBlockId() + DATA_BLK_NUM), posInBlk);
+      }
+//      RSRawDecoder rsRawDecoder = new RSRawDecoder();
+//      rsRawDecoder.initialize(DATA_BLK_NUM, PARITY_BLK_NUM, CELLSIZE);
+//      rsRawDecoder.decode(decodeInputs, missingBlkIdx, decodeOutputs);
+      int posInBuf = i * CELLSIZE * DATA_BLK_NUM + failedDNIdx * CELLSIZE;
+//      System.arraycopy(decodeOutputs[0], 0, expected, posInBuf, CELLSIZE);
+      //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938
+      Arrays.fill(expected, posInBuf, posInBuf + CELLSIZE, (byte)7);
+    }
+    int delta = 10;
+    int done = 0;
+    // read a small delta, shouldn't trigger decode
+    // |cell_0 |
+    // |10     |
+    done += in.read(0, readBuffer, 0, delta);
+    assertEquals(delta, done);
+    // both head and trail cells are partial
+    // |c_0      |c_1    |c_2 |c_3 |c_4      |c_5         |
+    // |256K - 10|missing|256K|256K|256K - 10|not in range|
+    done += in.read(delta, readBuffer, delta,
+        CELLSIZE * (DATA_BLK_NUM - 1) - 2 * delta);
+    assertEquals(CELLSIZE * (DATA_BLK_NUM - 1) - delta, done);
+    // read the rest
+    done += in.read(done, readBuffer, done, readSize - done);
+    assertEquals(readSize, done);
+    assertArrayEquals(expected, readBuffer);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3030f0/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 5c6f449..57d6eb9 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
@@ -18,10 +18,13 @@
 package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
 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.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -321,4 +324,50 @@ public class TestWriteReadStripedFile {
       Assert.assertArrayEquals(bytes, result.array());
     }
   }
+
+  @Test
+  public void testWritePreadWithDNFailure() throws IOException {
+    final int failedDNIdx = 2;
+    final int length = cellSize * (dataBlocks + 2);
+    Path testPath = new Path("/foo");
+    final byte[] bytes = generateBytes(length);
+    DFSTestUtil.writeFile(fs, testPath, new String(bytes));
+
+    // shut down the DN that holds the last internal data block
+    BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5,
+        cellSize);
+    String name = (locs[0].getNames())[failedDNIdx];
+    for (DataNode dn : cluster.getDataNodes()) {
+      int port = dn.getXferPort();
+      if (name.contains(Integer.toString(port))) {
+        dn.shutdown();
+        break;
+      }
+    }
+
+    // pread
+    int startOffsetInFile = cellSize * 5;
+    try (FSDataInputStream fsdis = fs.open(testPath)) {
+      byte[] buf = new byte[length];
+      int readLen = fsdis.read(startOffsetInFile, buf, 0, buf.length);
+      Assert.assertEquals("The length of file should be the same to write size",
+          length - startOffsetInFile, readLen);
+
+      RSRawDecoder rsRawDecoder = new RSRawDecoder();
+      rsRawDecoder.initialize(dataBlocks, parityBlocks, 1);
+      byte[] expected = new byte[readLen];
+      for (int i = startOffsetInFile; i < length; i++) {
+        //TODO: workaround (filling fixed bytes), to remove after HADOOP-11938
+        if ((i / cellSize) % dataBlocks == failedDNIdx) {
+          expected[i - startOffsetInFile] = (byte)7;
+        } else {
+          expected[i - startOffsetInFile] = getByte(i);
+        }
+      }
+      for (int i = startOffsetInFile; i < length; i++) {
+        Assert.assertEquals("Byte at " + i + " should be the same",
+            expected[i - startOffsetInFile], buf[i - startOffsetInFile]);
+      }
+    }
+  }
 }


[22/50] [abbrv] hadoop git commit: HDFS-8364. Erasure coding: fix some minor bugs in EC CLI (Contributed by Walter Su)

Posted by zh...@apache.org.
HDFS-8364. Erasure coding: fix some minor bugs in EC CLI (Contributed by Walter Su)


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

Branch: refs/heads/HDFS-7285
Commit: bba15e06d29d79ab1c379f258e5efb13a1ea39ae
Parents: 9798065
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed May 13 12:43:39 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:51 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  3 ++
 .../hadoop-hdfs/src/main/bin/hdfs               |  1 +
 .../hdfs/tools/erasurecode/ECCommand.java       | 12 ++++---
 .../test/resources/testErasureCodingConf.xml    | 35 ++++++++++++++++++++
 4 files changed, 47 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bba15e06/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 0945d72..190ddd6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -209,3 +209,6 @@
 
     HDFS-8195. Erasure coding: Fix file quota change when we complete/commit 
     the striped blocks. (Takuya Fukudome via zhz)
+
+    HDFS-8364. Erasure coding: fix some minor bugs in EC CLI
+    (Walter Su via vinayakumarb)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bba15e06/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 84c79b8..5ee7f4d 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -28,6 +28,7 @@ function hadoop_usage
   echo "  datanode             run a DFS datanode"
   echo "  dfs                  run a filesystem command on the file system"
   echo "  dfsadmin             run a DFS admin client"
+  echo "  erasurecode          configure HDFS erasure coding zones"
   echo "  fetchdt              fetch a delegation token from the NameNode"
   echo "  fsck                 run a DFS filesystem checking utility"
   echo "  getconf              get config values from configuration"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bba15e06/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 d53844d..2b6a6a5 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
@@ -135,7 +135,7 @@ public abstract class ECCommand extends Command {
         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);
+            + item.path + ". " + e.getMessage());
       }
     }
   }
@@ -165,10 +165,14 @@ public abstract class ECCommand extends Command {
       DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
       try {
         ErasureCodingZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path);
-        out.println(ecZoneInfo.toString());
+        if (ecZoneInfo != null) {
+          out.println(ecZoneInfo.toString());
+        } else {
+          out.println("Path " + item.path + " is not in EC zone");
+        }
       } catch (IOException e) {
-        throw new IOException("Unable to create EC zone for the path "
-            + item.path, e);
+        throw new IOException("Unable to get EC zone for the path "
+            + item.path + ". " + e.getMessage());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bba15e06/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index b7b29d3..66892f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -123,6 +123,24 @@
     </test>
 
     <test>
+      <description>createZone : create a zone twice</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /eczone</command>
+        <ec-admin-command>-fs NAMENODE -createZone /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -createZone /eczone</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /eczone</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Directory /eczone is already in an erasure coding zone</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>createZone : default schema</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /eczone</command>
@@ -141,6 +159,23 @@
     </test>
 
     <test>
+      <description>getZoneInfo : get information about the EC zone at specified path not in zone</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /noec</command>
+        <ec-admin-command>-fs NAMENODE -getZoneInfo /noec</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /noec</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Path NAMENODE/noec is not in EC zone</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>


[19/50] [abbrv] hadoop git commit: HDFS-8368. Erasure Coding: DFS opening a non-existent file need to be handled properly. Contributed by Rakesh R.

Posted by zh...@apache.org.
HDFS-8368. Erasure Coding: DFS opening a non-existent file need to be handled properly. 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/54d28275
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/54d28275
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/54d28275

Branch: refs/heads/HDFS-7285
Commit: 54d28275226e0bc3b0d46142ce20f74a52e66395
Parents: 97a2396
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue May 12 14:31:28 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:50 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d28275/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 f026a5c..79ad208 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -201,3 +201,6 @@
 
     HDFS-8372. Erasure coding: compute storage type quotas for striped files,
     to be consistent with HDFS-8327. (Zhe Zhang via jing9)
+
+    HDFS-8368. Erasure Coding: DFS opening a non-existent file need to be 
+    handled properly (Rakesh R via zhz)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d28275/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 9155b4d..ffeb568 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
@@ -1193,12 +1193,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     //    Get block info from namenode
     TraceScope scope = getPathTraceScope("newDFSInputStream", src);
     try {
-      ECSchema schema = getFileInfo(src).getECSchema();
-      if (schema != null) {
-        return new DFSStripedInputStream(this, src, verifyChecksum, schema);
-      } else {
-        return new DFSInputStream(this, src, verifyChecksum);
+      HdfsFileStatus fileInfo = getFileInfo(src);
+      if (fileInfo != null) {
+        ECSchema schema = fileInfo.getECSchema();
+        if (schema != null) {
+          return new DFSStripedInputStream(this, src, verifyChecksum, schema);
+        }
       }
+      return new DFSInputStream(this, src, verifyChecksum);
     } finally {
       scope.close();
     }


[47/50] [abbrv] hadoop git commit: HDFS-8408. Revisit and refactor ErasureCodingInfo (Contributed by Vinayakumar B)

Posted by zh...@apache.org.
HDFS-8408. Revisit and refactor ErasureCodingInfo (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/9a18598e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9a18598e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9a18598e

Branch: refs/heads/HDFS-7285
Commit: 9a18598e2da8e699ed852ffa30fd7f503902190c
Parents: b30e96b
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon May 25 11:57:17 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:07:15 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  2 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 22 ++-----
 .../hadoop/hdfs/DistributedFileSystem.java      | 16 ++---
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 12 +---
 .../hadoop/hdfs/protocol/ErasureCodingInfo.java | 41 ------------
 .../hdfs/protocol/ErasureCodingZoneInfo.java    | 66 --------------------
 ...tNamenodeProtocolServerSideTranslatorPB.java | 37 +++--------
 .../ClientNamenodeProtocolTranslatorPB.java     | 34 +++-------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 34 ++++------
 .../server/blockmanagement/BlockManager.java    | 12 ++--
 .../namenode/ErasureCodingZoneManager.java      | 10 +--
 .../server/namenode/FSDirStatAndListingOp.java  | 14 ++---
 .../hdfs/server/namenode/FSDirectory.java       |  6 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 23 ++-----
 .../hdfs/server/namenode/NameNodeRpcServer.java | 13 +---
 .../hadoop/hdfs/server/namenode/Namesystem.java | 10 ++-
 .../hdfs/tools/erasurecode/ECCommand.java       | 16 ++---
 .../src/main/proto/ClientNamenodeProtocol.proto |  6 +-
 .../src/main/proto/erasurecoding.proto          | 26 ++------
 .../hadoop/hdfs/TestErasureCodingZones.java     | 16 +++--
 .../test/resources/testErasureCodingConf.xml    | 38 +++++------
 21 files changed, 117 insertions(+), 337 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 4609fb6..d045ee5 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -257,3 +257,5 @@
     HDFS-7768. Change fsck to support EC files.  (Takanobu Asanuma via szetszwo)
 
     HDFS-8382. Remove chunkSize and initialize from erasure coder. (Kai Zheng)
+
+    HDFS-8408. Revisit and refactor ErasureCodingInfo (vinayakumarb)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 42f48f0..a2a6221 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,8 +119,7 @@ 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.ErasureCodingInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -3137,19 +3136,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
-    checkOpen();
-    TraceScope scope = getPathTraceScope("getErasureCodingInfo", src);
-    try {
-      return namenode.getErasureCodingInfo(src);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-          FileNotFoundException.class, UnresolvedPathException.class);
-    } finally {
-      scope.close();
-    }
-  }
-
   public ECSchema[] getECSchemas() throws IOException {
     checkOpen();
     TraceScope scope = Trace.startSpan("getECSchemas", traceSampler);
@@ -3359,11 +3345,11 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @return Returns the zone information if path is in EC Zone, null otherwise
    * @throws IOException
    */
-  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+  public ErasureCodingZone getErasureCodingZone(String src) throws IOException {
     checkOpen();
-    TraceScope scope = getPathTraceScope("getErasureCodingZoneInfo", src);
+    TraceScope scope = getPathTraceScope("getErasureCodingZone", src);
     try {
-      return namenode.getErasureCodingZoneInfo(src);
+      return namenode.getErasureCodingZone(src);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(FileNotFoundException.class,
           AccessControlException.class, UnresolvedPathException.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 2e21372..6d55922 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
@@ -75,7 +75,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.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -2316,25 +2316,25 @@ public class DistributedFileSystem extends FileSystem {
    * @return Returns the zone information if path is in EC zone, null otherwise
    * @throws IOException
    */
-  public ErasureCodingZoneInfo getErasureCodingZoneInfo(final Path path)
+  public ErasureCodingZone getErasureCodingZone(final Path path)
       throws IOException {
     Path absF = fixRelativePart(path);
-    return new FileSystemLinkResolver<ErasureCodingZoneInfo>() {
+    return new FileSystemLinkResolver<ErasureCodingZone>() {
       @Override
-      public ErasureCodingZoneInfo doCall(final Path p) throws IOException,
+      public ErasureCodingZone doCall(final Path p) throws IOException,
           UnresolvedLinkException {
-        return dfs.getErasureCodingZoneInfo(getPathName(p));
+        return dfs.getErasureCodingZone(getPathName(p));
       }
 
       @Override
-      public ErasureCodingZoneInfo next(final FileSystem fs, final Path p)
+      public ErasureCodingZone next(final FileSystem fs, final Path p)
           throws IOException {
         if (fs instanceof DistributedFileSystem) {
           DistributedFileSystem myDfs = (DistributedFileSystem) fs;
-          return myDfs.getErasureCodingZoneInfo(p);
+          return myDfs.getErasureCodingZone(p);
         }
         throw new UnsupportedOperationException(
-            "Cannot getErasureCodingZoneInfo through a symlink to a "
+            "Cannot getErasureCodingZone through a symlink to a "
                 + "non-DistributedFileSystem: " + path + " -> " + p);
       }
     }.resolve(this, absF);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 4f985ba..d2b8a51 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
@@ -1467,16 +1467,6 @@ public interface ClientProtocol {
       throws IOException;
 
   /**
-   * Gets the ECInfo for the specified file/directory
-   * 
-   * @param src
-   * @return Returns the ECInfo if the file/directory is erasure coded, null otherwise
-   * @throws IOException
-   */
-  @Idempotent
-  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException;
-
-  /**
    * Gets list of ECSchemas loaded in Namenode
    *
    * @return Returns the list of ECSchemas loaded at Namenode
@@ -1492,5 +1482,5 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException;
+  public ErasureCodingZone getErasureCodingZone(String src) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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
deleted file mode 100644
index bad09b3..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingInfo.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 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/9a18598e/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
deleted file mode 100644
index 282eeaf..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingZoneInfo.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.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;
-  private int cellSize;
-
-  public ErasureCodingZoneInfo(String dir, ECSchema schema, int cellSize) {
-    this.dir = dir;
-    this.schema = schema;
-    this.cellSize = cellSize;
-  }
-
-  /**
-   * 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;
-  }
-
-  /**
-   * Get cellSize for the EC Zone
-   */
-  public int getCellSize() {
-    return cellSize;
-  }
-
-  @Override
-  public String toString() {
-    return "Dir: " + getDir() + ", Schema: " + schema + ", cellSize: "
-        + cellSize;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 863b217..4228a65 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,7 @@ 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.ErasureCodingInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -202,10 +201,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.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.GetErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
@@ -1525,22 +1522,6 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public GetErasureCodingInfoResponseProto getErasureCodingInfo(RpcController controller,
-      GetErasureCodingInfoRequestProto request) throws ServiceException {
-    try {
-      ErasureCodingInfo ecInfo = server.getErasureCodingInfo(request.getSrc());
-      GetErasureCodingInfoResponseProto.Builder resBuilder = GetErasureCodingInfoResponseProto
-          .newBuilder();
-      if (ecInfo != null) {
-        resBuilder.setECInfo(PBHelper.convertECInfo(ecInfo));
-      }
-      return resBuilder.build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
-  @Override
   public GetECSchemasResponseProto getECSchemas(RpcController controller,
       GetECSchemasRequestProto request) throws ServiceException {
     try {
@@ -1557,13 +1538,13 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public GetErasureCodingZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller,
-      GetErasureCodingZoneInfoRequestProto request) throws ServiceException {
+  public GetErasureCodingZoneResponseProto getErasureCodingZone(RpcController controller,
+      GetErasureCodingZoneRequestProto request) throws ServiceException {
     try {
-      ErasureCodingZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc());
-      GetErasureCodingZoneInfoResponseProto.Builder builder = GetErasureCodingZoneInfoResponseProto.newBuilder();
-      if (ecZoneInfo != null) {
-        builder.setECZoneInfo(PBHelper.convertECZoneInfo(ecZoneInfo));
+      ErasureCodingZone ecZone = server.getErasureCodingZone(request.getSrc());
+      GetErasureCodingZoneResponseProto.Builder builder = GetErasureCodingZoneResponseProto.newBuilder();
+      if (ecZone != null) {
+        builder.setECZone(PBHelper.convertErasureCodingZone(ecZone));
       }
       return builder.build();
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 336e3a2..2c88a93 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,7 @@ 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.ErasureCodingInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -168,10 +167,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.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.GetErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
@@ -1553,21 +1550,6 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
-    GetErasureCodingInfoRequestProto req = GetErasureCodingInfoRequestProto.newBuilder()
-        .setSrc(src).build();
-    try {
-      GetErasureCodingInfoResponseProto res = rpcProxy.getErasureCodingInfo(null, req);
-      if (res.hasECInfo()) {
-        return PBHelper.convertECInfo(res.getECInfo());
-      }
-      return null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
   public ECSchema[] getECSchemas() throws IOException {
     try {
       GetECSchemasResponseProto response = rpcProxy.getECSchemas(null,
@@ -1584,14 +1566,14 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
-    GetErasureCodingZoneInfoRequestProto req = GetErasureCodingZoneInfoRequestProto.newBuilder()
+  public ErasureCodingZone getErasureCodingZone(String src) throws IOException {
+    GetErasureCodingZoneRequestProto req = GetErasureCodingZoneRequestProto.newBuilder()
         .setSrc(src).build();
     try {
-      GetErasureCodingZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo(
+      GetErasureCodingZoneResponseProto response = rpcProxy.getErasureCodingZone(
           null, req);
-      if (response.hasECZoneInfo()) {
-        return PBHelper.convertECZoneInfo(response.getECZoneInfo());
+      if (response.hasECZone()) {
+        return PBHelper.convertErasureCodingZone(response.getECZone());
       }
       return null;
     } catch (ServiceException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 4d0f871..b2415fa 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,12 @@ 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.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-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,8 +134,7 @@ 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.ErasureCodingInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ErasureCodingZoneProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
@@ -203,7 +201,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
@@ -3123,16 +3120,6 @@ public class PBHelper {
         build();
   }
 
-  public static ErasureCodingInfo convertECInfo(ErasureCodingInfoProto ecInfoProto) {
-    return new ErasureCodingInfo(ecInfoProto.getSrc(),
-        convertECSchema(ecInfoProto.getSchema()));
-  }
-
-  public static ErasureCodingInfoProto convertECInfo(ErasureCodingInfo ecInfo) {
-    return ErasureCodingInfoProto.newBuilder().setSrc(ecInfo.getSrc())
-        .setSchema(convertECSchema(ecInfo.getSchema())).build();
-  }
-
   public static ECSchema convertECSchema(ECSchemaProto schema) {
     List<ECSchemaOptionEntryProto> optionsList = schema.getOptionsList();
     Map<String, String> options = new HashMap<>(optionsList.size());
@@ -3157,16 +3144,17 @@ public class PBHelper {
     return builder.build();
   }
 
-  public static ErasureCodingZoneInfoProto convertECZoneInfo(ErasureCodingZoneInfo ecZoneInfo) {
-    return ErasureCodingZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir())
-        .setSchema(convertECSchema(ecZoneInfo.getSchema()))
-        .setCellSize(ecZoneInfo.getCellSize()).build();
+  public static ErasureCodingZoneProto convertErasureCodingZone(
+      ErasureCodingZone ecZone) {
+    return ErasureCodingZoneProto.newBuilder().setDir(ecZone.getDir())
+        .setSchema(convertECSchema(ecZone.getSchema()))
+        .setCellSize(ecZone.getCellSize()).build();
   }
 
-  public static ErasureCodingZoneInfo convertECZoneInfo(ErasureCodingZoneInfoProto ecZoneInfoProto) {
-    return new ErasureCodingZoneInfo(ecZoneInfoProto.getDir(),
-        convertECSchema(ecZoneInfoProto.getSchema()),
-        ecZoneInfoProto.getCellSize());
+  public static ErasureCodingZone convertErasureCodingZone(
+      ErasureCodingZoneProto ecZoneProto) {
+    return new ErasureCodingZone(ecZoneProto.getDir(),
+        convertECSchema(ecZoneProto.getSchema()), ecZoneProto.getCellSize());
   }
   
   public static BlockECRecoveryInfo convertBlockECRecoveryInfo(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 79cbcc6..8f1f6b7 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
@@ -51,7 +51,7 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -1555,14 +1555,14 @@ public class BlockManager {
             assert rw instanceof ErasureCodingWork;
             assert rw.targets.length > 0;
             String src = block.getBlockCollection().getName();
-            ErasureCodingZoneInfo ecZoneInfo = null;
+            ErasureCodingZone ecZone = null;
             try {
-              ecZoneInfo = namesystem.getErasureCodingZoneInfoForPath(src);
+              ecZone = namesystem.getErasureCodingZoneForPath(src);
             } catch (IOException e) {
               blockLog
-                  .warn("Failed to get the EC zone info for the file {} ", src);
+                  .warn("Failed to get the EC zone for the file {} ", src);
             }
-            if (ecZoneInfo == null) {
+            if (ecZone == 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
@@ -1573,7 +1573,7 @@ public class BlockManager {
                 new ExtendedBlock(namesystem.getBlockPoolId(), block),
                 rw.srcNodes, rw.targets,
                 ((ErasureCodingWork) rw).liveBlockIndicies,
-                ecZoneInfo.getSchema(), ecZoneInfo.getCellSize());
+                ecZone.getSchema(), ecZone.getCellSize());
           } else {
             rw.srcNodes[0].addBlockToBeReplicated(block, targets);
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 89fecc6..4b3e747 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,7 +23,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.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -61,11 +61,11 @@ public class ErasureCodingZoneManager {
   }
 
   ECSchema getECSchema(INodesInPath iip) throws IOException {
-    ErasureCodingZoneInfo ecZoneInfo = getECZoneInfo(iip);
-    return ecZoneInfo == null ? null : ecZoneInfo.getSchema();
+    ErasureCodingZone ecZone = getECZone(iip);
+    return ecZone == null ? null : ecZone.getSchema();
   }
 
-  ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
+  ErasureCodingZone getECZone(INodesInPath iip) throws IOException {
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     List<INode> inodes = iip.getReadOnlyINodes();
@@ -92,7 +92,7 @@ public class ErasureCodingZoneManager {
           String schemaName = WritableUtils.readString(dIn);
           ECSchema schema = dir.getFSNamesystem().getECSchemaManager()
               .getSchema(schemaName);
-          return new ErasureCodingZoneInfo(dir.getInode(inode.getId())
+          return new ErasureCodingZone(dir.getInode(inode.getId())
               .getFullPathName(), schema, cellSize);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 eba5013..1b1d79e 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
@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -386,9 +386,9 @@ class FSDirStatAndListingOp {
     final FileEncryptionInfo feInfo = isRawPath ? null :
         fsd.getFileEncryptionInfo(node, snapshot, iip);
     
-    final ErasureCodingZoneInfo ecZoneInfo = fsd.getECZoneInfo(iip);
-    final ECSchema schema = ecZoneInfo != null ? ecZoneInfo.getSchema() : null;
-    final int cellSize = ecZoneInfo != null ? ecZoneInfo.getCellSize() : 0;
+    final ErasureCodingZone ecZone = fsd.getECZone(iip);
+    final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
+    final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;
 
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
@@ -468,9 +468,9 @@ class FSDirStatAndListingOp {
     }
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
-    final ErasureCodingZoneInfo ecZoneInfo = fsd.getECZoneInfo(iip);
-    final ECSchema schema = ecZoneInfo != null ? ecZoneInfo.getSchema() : null;
-    final int cellSize = ecZoneInfo != null ? ecZoneInfo.getCellSize() : 0;
+    final ErasureCodingZone ecZone = fsd.getECZone(iip);
+    final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
+    final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;
 
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 735ea5a..0e3694f 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
@@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
@@ -1253,10 +1253,10 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  ErasureCodingZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
+  ErasureCodingZone getECZone(INodesInPath iip) throws IOException {
     readLock();
     try {
-      return ecZoneManager.getECZoneInfo(iip);
+      return ecZoneManager.getECZone(iip);
     } finally {
       readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 a53cd61..59c9a03 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,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.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -7624,27 +7623,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
-   * Get the erasure coding information for specified src
-   */
-  ErasureCodingInfo getErasureCodingInfo(String src) throws AccessControlException,
-      UnresolvedLinkException, IOException {
-    ErasureCodingZoneInfo zoneInfo = getErasureCodingZoneInfo(src);
-    if (zoneInfo != null) {
-      return new ErasureCodingInfo(src, zoneInfo.getSchema());
-    }
-    return null;
-  }
-
-  /**
    * Get the erasure coding zone information for specified path
    */
-  ErasureCodingZoneInfo getErasureCodingZoneInfo(String src)
+  ErasureCodingZone getErasureCodingZone(String src)
       throws AccessControlException, UnresolvedLinkException, IOException {
     checkOperation(OperationCategory.READ);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return getErasureCodingZoneInfoForPath(src);
+      return getErasureCodingZoneForPath(src);
     } finally {
       readUnlock();
     }
@@ -7865,7 +7852,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override
-  public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src)
+  public ErasureCodingZone getErasureCodingZoneForPath(String src)
       throws IOException {
     final byte[][] pathComponents = FSDirectory
         .getPathComponentsForReservedPath(src);
@@ -7875,7 +7862,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if (isPermissionEnabled) {
       dir.checkPathAccess(pc, iip, FsAction.READ);
     }
-    return dir.getECZoneInfo(iip);
+    return dir.getECZone(iip);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 9e94b90..1377bbe 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,7 @@ 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.ErasureCodingInfo;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
@@ -2036,20 +2035,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public ErasureCodingInfo getErasureCodingInfo(String src) throws IOException {
-    checkNNStartup();
-    return namesystem.getErasureCodingInfo(src);
-  }
-
-  @Override // ClientProtocol
   public ECSchema[] getECSchemas() throws IOException {
     checkNNStartup();
     return namesystem.getECSchemas();
   }
 
   @Override // ClientProtocol
-  public ErasureCodingZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+  public ErasureCodingZone getErasureCodingZone(String src) throws IOException {
     checkNNStartup();
-    return namesystem.getErasureCodingZoneInfo(src);
+    return namesystem.getErasureCodingZone(src);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 a32e800..2a9ab39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -21,11 +21,10 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 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;
 
@@ -52,13 +51,12 @@ public interface Namesystem extends RwLock, SafeMode {
   public boolean isInSnapshot(BlockCollection bc);
 
   /**
-   * Gets the ECZone info for path
-   * 
+   * Gets the ECZone for path
    * @param src
    *          - path
-   * @return {@link ErasureCodingZoneInfo}
+   * @return {@link ErasureCodingZone}
    * @throws IOException
    */
-  public ErasureCodingZoneInfo getErasureCodingZoneInfoForPath(String src)
+  public ErasureCodingZone getErasureCodingZoneForPath(String src)
       throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 2d82208..34965d2 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.ErasureCodingZoneInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.io.erasurecode.ECSchema;
@@ -47,8 +47,8 @@ public abstract class ECCommand extends Command {
     // 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(GetECZoneCommand.class, "-"
+        + GetECZoneCommand.NAME);
     factory.addClass(ListECSchemas.class, "-" + ListECSchemas.NAME);
   }
 
@@ -153,8 +153,8 @@ public abstract class ECCommand extends Command {
   /**
    * Get the information about the zone
    */
-  static class GetECZoneInfoCommand extends ECCommand {
-    public static final String NAME = "getZoneInfo";
+  static class GetECZoneCommand extends ECCommand {
+    public static final String NAME = "getZone";
     public static final String USAGE = "<path>";
     public static final String DESCRIPTION =
         "Get information about the EC zone at specified path\n";
@@ -174,9 +174,9 @@ public abstract class ECCommand extends Command {
       super.processPath(item);
       DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
       try {
-        ErasureCodingZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path);
-        if (ecZoneInfo != null) {
-          out.println(ecZoneInfo.toString());
+        ErasureCodingZone ecZone = dfs.getErasureCodingZone(item.path);
+        if (ecZone != null) {
+          out.println(ecZone.toString());
         } else {
           out.println("Path " + item.path + " is not in EC zone");
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 0a0a4c4..2b64ca0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -863,10 +863,8 @@ service ClientNamenodeProtocol {
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
       returns(GetEditsFromTxidResponseProto);
-  rpc getErasureCodingInfo(GetErasureCodingInfoRequestProto)
-      returns(GetErasureCodingInfoResponseProto);
   rpc getECSchemas(GetECSchemasRequestProto)
       returns(GetECSchemasResponseProto);
-  rpc getErasureCodingZoneInfo(GetErasureCodingZoneInfoRequestProto) 
-      returns(GetErasureCodingZoneInfoResponseProto);
+  rpc getErasureCodingZone(GetErasureCodingZoneRequestProto)
+      returns(GetErasureCodingZoneResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 058ed96..56bb7a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -24,17 +24,9 @@ package hadoop.hdfs;
 import "hdfs.proto";
 
 /**
- * ErasureCodingInfo
+ * ErasureCodingZone
  */
-message ErasureCodingInfoProto {
- required string src = 1;
- required ECSchemaProto schema = 2;
-}
-
-/**
- * ErasureCodingZoneInfo
- */
-message ErasureCodingZoneInfoProto {
+message ErasureCodingZoneProto {
   required string dir = 1;
   required ECSchemaProto schema = 2;
   required uint32 cellSize = 3;
@@ -49,14 +41,6 @@ message CreateErasureCodingZoneRequestProto {
 message CreateErasureCodingZoneResponseProto {
 }
 
-message GetErasureCodingInfoRequestProto {
-  required string src = 1;
-}
-
-message GetErasureCodingInfoResponseProto {
-  optional ErasureCodingInfoProto ECInfo = 1;
-}
-
 message GetECSchemasRequestProto { // void request
 }
 
@@ -64,12 +48,12 @@ message GetECSchemasResponseProto {
   repeated ECSchemaProto schemas = 1;
 }
 
-message GetErasureCodingZoneInfoRequestProto {
+message GetErasureCodingZoneRequestProto {
   required string src = 1; // path to get the zone info
 }
 
-message GetErasureCodingZoneInfoResponseProto {
-  optional ErasureCodingZoneInfoProto ECZoneInfo = 1;
+message GetErasureCodingZoneResponseProto {
+  optional ErasureCodingZoneProto ECZone = 1;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/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 480791e..d724b53 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,7 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -177,12 +177,12 @@ public class TestErasureCodingZones {
     final Path ecDir = new Path(src);
     fs.mkdir(ecDir, FsPermission.getDirDefault());
     // dir ECInfo before creating ec zone
-    assertNull(fs.getClient().getErasureCodingInfo(src));
+    assertNull(fs.getClient().getFileInfo(src).getECSchema());
     // dir ECInfo after creating ec zone
     fs.getClient().createErasureCodingZone(src, null, 0); //Default one will be used.
     ECSchema sysDefaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     verifyErasureCodingInfo(src, sysDefaultSchema);
-    fs.create(new Path(ecDir, "/child1")).close();
+    fs.create(new Path(ecDir, "child1")).close();
     // verify for the files in ec zone
     verifyErasureCodingInfo(src + "/child1", sysDefaultSchema);
   }
@@ -198,21 +198,19 @@ public class TestErasureCodingZones {
     final Path ecDir = new Path(src);
     fs.mkdir(ecDir, FsPermission.getDirDefault());
     // dir ECInfo before creating ec zone
-    assertNull(fs.getClient().getErasureCodingInfo(src));
+    assertNull(fs.getClient().getFileInfo(src).getECSchema());
     // dir ECInfo after creating ec zone
     fs.getClient().createErasureCodingZone(src, usingSchema, 0);
     verifyErasureCodingInfo(src, usingSchema);
-    fs.create(new Path(ecDir, "/child1")).close();
+    fs.create(new Path(ecDir, "child1")).close();
     // verify for the files in ec zone
     verifyErasureCodingInfo(src + "/child1", usingSchema);
   }
 
   private void verifyErasureCodingInfo(
       String src, ECSchema usingSchema) throws IOException {
-    ErasureCodingInfo ecInfo = fs.getClient().getErasureCodingInfo(src);
-    assertNotNull("ECInfo should have been non-null", ecInfo);
-    assertEquals(src, ecInfo.getSrc());
-    ECSchema schema = ecInfo.getSchema();
+    HdfsFileStatus hdfsFileStatus = fs.getClient().getFileInfo(src);
+    ECSchema schema = hdfsFileStatus.getECSchema();
     assertNotNull(schema);
     assertEquals("Actually used schema should be equal with target schema",
         usingSchema, schema);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a18598e/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index 66892f0..ee1a19a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -67,9 +67,9 @@
     </test>
 
     <test>
-      <description>help: getZoneInfo command</description>
+      <description>help: getZone command</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -help getZoneInfo</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -help getZone</ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
@@ -80,7 +80,7 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-getZoneInfo &lt;path&gt;(.)*</expected-output>
+          <expected-output>^-getZone &lt;path&gt;(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -145,7 +145,7 @@
       <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>
+        <ec-admin-command>-fs NAMENODE -getZone /eczone</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /eczone</command>
@@ -159,10 +159,10 @@
     </test>
 
     <test>
-      <description>getZoneInfo : get information about the EC zone at specified path not in zone</description>
+      <description>getZone : get information about the EC zone at specified path not in zone</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /noec</command>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo /noec</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone /noec</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /noec</command>
@@ -176,11 +176,11 @@
     </test>
 
     <test>
-      <description>getZoneInfo : get information about the EC zone at specified path</description>
+      <description>getZone : 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>
+        <ec-admin-command>-fs NAMENODE -getZone /eczone</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /eczone</command>
@@ -194,12 +194,12 @@
     </test>
 
     <test>
-      <description>getZoneInfo : get EC zone at specified file path</description>
+      <description>getZone : 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>
+        <ec-admin-command>-fs NAMENODE -getZone /eczone/ecfile</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rm /eczone/ecfile</command>
@@ -313,24 +313,24 @@
     </test>
 
     <test>
-      <description>getZoneInfo : illegal parameters - path is missing</description>
+      <description>getZone : illegal parameters - path is missing</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo </ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone </ec-admin-command>
       </test-commands>
       <cleanup-commands>
       </cleanup-commands>
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-getZoneInfo: &lt;path&gt; is missing(.)*</expected-output>
+          <expected-output>^-getZone: &lt;path&gt; is missing(.)*</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>getZoneInfo : illegal parameters - too many arguments</description>
+      <description>getZone : illegal parameters - too many arguments</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone /eczone /eczone</ec-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rm /eczone</command>
@@ -338,22 +338,22 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>-getZoneInfo: Too many arguments</expected-output>
+          <expected-output>-getZone: Too many arguments</expected-output>
         </comparator>
       </comparators>
     </test>
 
     <test>
-      <description>getZoneInfo : illegal parameters - no such file</description>
+      <description>getZone : illegal parameters - no such file</description>
       <test-commands>
-        <ec-admin-command>-fs NAMENODE -getZoneInfo /eczone</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getZone /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>
+          <expected-output>^getZone: `/eczone': No such file or directory(.)*</expected-output>
         </comparator>
       </comparators>
     </test>


[03/50] [abbrv] 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/436c1485
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/436c1485
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/436c1485

Branch: refs/heads/HDFS-7285
Commit: 436c14855aef58a551d5cab91eba6e88775c6797
Parents: 6dea01f
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Tue May 5 11:22:52 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 11:59:57 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/436c1485/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/436c1485/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/436c1485/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 6657e5b..a3d75b5 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
@@ -66,7 +66,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;
@@ -84,7 +83,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;
@@ -94,6 +96,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;
 
@@ -1554,10 +1557,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/436c1485/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/436c1485/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 3fa8818..79dd3d7 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
@@ -7581,25 +7581,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;
   }
@@ -7841,5 +7825,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/436c1485/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/436c1485/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/436c1485/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/436c1485/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,


[21/50] [abbrv] hadoop git commit: HDFS-8391. NN should consider current EC tasks handling count from DN while assigning new tasks. Contributed by Uma Maheswara Rao G.

Posted by zh...@apache.org.
HDFS-8391. NN should consider current EC tasks handling count from DN while assigning new tasks. 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/c99c3379
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c99c3379
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c99c3379

Branch: refs/heads/HDFS-7285
Commit: c99c3379282779e11ebda88d845bb89407f2f350
Parents: bba15e0
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Thu May 14 11:27:48 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:51 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt         |  3 +++
 .../hadoop/hdfs/server/datanode/DataNode.java    | 19 +++++++++++++++++--
 .../erasurecode/ErasureCodingWorker.java         |  4 +++-
 3 files changed, 23 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c99c3379/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 190ddd6..1456434 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -212,3 +212,6 @@
 
     HDFS-8364. Erasure coding: fix some minor bugs in EC CLI
     (Walter Su via vinayakumarb)
+
+    HDFS-8391. NN should consider current EC tasks handling count from DN while 
+    assigning new tasks. (umamahesh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c99c3379/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 5eca2c7..a1a80ee 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
@@ -1909,6 +1909,21 @@ public class DataNode extends ReconfigurableBase
   int getXmitsInProgress() {
     return xmitsInProgress.get();
   }
+  
+  /**
+   * Increments the xmitsInProgress count. xmitsInProgress count represents the
+   * number of data replication/reconstruction tasks running currently.
+   */
+  public void incrementXmitsInProgress() {
+    xmitsInProgress.getAndIncrement();
+  }
+
+  /**
+   * Decrements the xmitsInProgress count
+   */
+  public void decrementXmitsInProgress() {
+    xmitsInProgress.getAndDecrement();
+  }
 
   private void reportBadBlock(final BPOfferService bpos,
       final ExtendedBlock block, final String msg) {
@@ -2128,7 +2143,7 @@ public class DataNode extends ReconfigurableBase
      */
     @Override
     public void run() {
-      xmitsInProgress.getAndIncrement();
+      incrementXmitsInProgress();
       Socket sock = null;
       DataOutputStream out = null;
       DataInputStream in = null;
@@ -2207,7 +2222,7 @@ public class DataNode extends ReconfigurableBase
         // check if there are any disk problem
         checkDiskErrorAsync();
       } finally {
-        xmitsInProgress.getAndDecrement();
+        decrementXmitsInProgress();
         IOUtils.closeStream(blockSender);
         IOUtils.closeStream(out);
         IOUtils.closeStream(in);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c99c3379/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 eedb191..7b3c24d 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
@@ -312,6 +312,7 @@ public final class ErasureCodingWorker {
 
     @Override
     public void run() {
+      datanode.incrementXmitsInProgress();
       try {
         // Store the indices of successfully read source
         // This will be updated after doing real read.
@@ -397,8 +398,9 @@ public final class ErasureCodingWorker {
         // 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);
+        LOG.warn("Failed to recover striped block: " + blockGroup, e);
       } finally {
+        datanode.decrementXmitsInProgress();
         // close block readers
         for (StripedReader stripedReader : stripedReaders) {
           closeBlockReader(stripedReader.blockReader);


[17/50] [abbrv] hadoop git commit: HDFS-8363. Erasure Coding: DFSStripedInputStream#seekToNewSource. (yliu)

Posted by zh...@apache.org.
HDFS-8363. Erasure Coding: DFSStripedInputStream#seekToNewSource. (yliu)


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

Branch: refs/heads/HDFS-7285
Commit: 8da9e18dadf536d340d5f291a105e48380d0282e
Parents: 54d2827
Author: yliu <yl...@apache.org>
Authored: Wed May 13 08:48:56 2015 +0800
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:01:50 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8da9e18d/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 79ad208..0a2bb9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -204,3 +204,5 @@
 
     HDFS-8368. Erasure Coding: DFS opening a non-existent file need to be 
     handled properly (Rakesh R via zhz)
+
+    HDFS-8363. Erasure Coding: DFSStripedInputStream#seekToNewSource. (yliu)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8da9e18d/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 7678fae..8f15eda 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
@@ -130,12 +130,12 @@ public class DFSStripedInputStream extends DFSInputStream {
     }
   }
 
-  private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS;
-  private final BlockReader[] blockReaders = new BlockReader[groupSize];
-  private final DatanodeInfo[] currentNodes = new DatanodeInfo[groupSize];
+  private final BlockReader[] blockReaders;
+  private final DatanodeInfo[] currentNodes;
   private final int cellSize;
   private final short dataBlkNum;
   private final short parityBlkNum;
+  private final short groupSize;
   /** the buffer for a complete stripe */
   private ByteBuffer curStripeBuf;
   private final ECSchema schema;
@@ -155,6 +155,9 @@ public class DFSStripedInputStream extends DFSInputStream {
     cellSize = schema.getChunkSize();
     dataBlkNum = (short) schema.getNumDataUnits();
     parityBlkNum = (short) schema.getNumParityUnits();
+    groupSize = dataBlkNum;
+    blockReaders = new BlockReader[groupSize];
+    currentNodes = new DatanodeInfo[groupSize];
     curStripeRange = new StripeRange(0, 0);
     readingService =
         new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool());
@@ -392,6 +395,12 @@ public class DFSStripedInputStream extends DFSInputStream {
   }
 
   @Override
+  public synchronized boolean seekToNewSource(long targetPos)
+      throws IOException {
+    return false;
+  }
+
+  @Override
   protected synchronized int readWithStrategy(ReaderStrategy strategy,
       int off, int len) throws IOException {
     dfsClient.checkOpen();


[26/50] [abbrv] hadoop git commit: HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure coder. Contributed by Kai Zheng.

Posted by zh...@apache.org.
HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure coder. 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/343c0e76
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/343c0e76
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/343c0e76

Branch: refs/heads/HDFS-7285
Commit: 343c0e76fcd95ac739ca7cd6742c9d617e19fc37
Parents: b64f674
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon May 18 10:14:54 2015 -0700
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:02:31 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |   3 +
 .../apache/hadoop/io/erasurecode/ECChunk.java   |  35 ++---
 .../rawcoder/AbstractRawErasureCoder.java       |  77 +++++------
 .../rawcoder/AbstractRawErasureDecoder.java     |  69 ++++++++--
 .../rawcoder/AbstractRawErasureEncoder.java     |  66 ++++++++--
 .../io/erasurecode/rawcoder/RSRawDecoder.java   |  22 ++--
 .../io/erasurecode/rawcoder/RSRawEncoder.java   |  41 +++---
 .../io/erasurecode/rawcoder/XORRawDecoder.java  |  30 +++--
 .../io/erasurecode/rawcoder/XORRawEncoder.java  |  40 +++---
 .../erasurecode/rawcoder/util/GaloisField.java  | 112 ++++++++++++----
 .../hadoop/io/erasurecode/TestCoderBase.java    | 131 +++++++++++++++----
 .../erasurecode/coder/TestErasureCoderBase.java |  21 ++-
 .../io/erasurecode/rawcoder/TestRSRawCoder.java |  12 +-
 .../rawcoder/TestRSRawCoderBase.java            |  12 +-
 .../erasurecode/rawcoder/TestRawCoderBase.java  |  57 +++++++-
 .../erasurecode/rawcoder/TestXORRawCoder.java   |  19 +++
 16 files changed, 535 insertions(+), 212 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 34dfc9e..c799b4f 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -51,3 +51,6 @@
 
     HADOOP-11566. Add tests and fix for erasure coders to recover erased parity 
     units. (Kai Zheng via Zhe Zhang)
+
+    HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure 
+    coder. (Kai Zheng via Zhe Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 69a8343..310c738 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
@@ -72,34 +72,15 @@ public class ECChunk {
   }
 
   /**
-   * Convert an array of this chunks to an array of byte array.
-   * Note the chunk buffers are not affected.
-   * @param chunks
-   * @return an array of byte array
+   * Convert to a bytes array, just for test usage.
+   * @return bytes array
    */
-  public static byte[][] toArrays(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();
-      if (buffer.hasArray()) {
-        bytesArr[i] = buffer.array();
-      } else {
-        bytesArr[i] = new byte[buffer.remaining()];
-        // Avoid affecting the original one
-        buffer.mark();
-        buffer.get(bytesArr[i]);
-        buffer.reset();
-      }
-    }
+  public byte[] toBytesArray() {
+    byte[] bytesArr = new byte[chunkBuffer.remaining()];
+    // Avoid affecting the original one
+    chunkBuffer.mark();
+    chunkBuffer.get(bytesArr);
+    chunkBuffer.reset();
 
     return bytesArr;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 2400313..5268962 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configured;
 
 import java.nio.ByteBuffer;
@@ -30,9 +31,6 @@ import java.util.Arrays;
 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;
@@ -43,8 +41,6 @@ public abstract class AbstractRawErasureCoder
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
     this.chunkSize = chunkSize;
-
-    zeroChunkBytes = new byte[chunkSize]; // With ZERO by default
   }
 
   @Override
@@ -73,55 +69,60 @@ public abstract class AbstractRawErasureCoder
   }
 
   /**
-   * Convert an array of heap ByteBuffers to an array of byte array.
-   * @param buffers
-   * @return an array of byte array
+   * Ensure output buffer filled with ZERO bytes fully in chunkSize.
+   * @param buffer a buffer ready to write chunk size bytes
+   * @return the buffer itself, with ZERO bytes written, the position and limit
+   *         are not changed after the call
    */
-  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");
-      }
+  protected ByteBuffer resetOutputBuffer(ByteBuffer buffer) {
+    int pos = buffer.position();
+    for (int i = pos; i < buffer.limit(); ++i) {
+      buffer.put((byte) 0);
     }
+    buffer.position(pos);
 
-    return bytesArr;
+    return buffer;
   }
 
   /**
    * Ensure the buffer (either input or output) ready to read or write with ZERO
    * bytes fully in chunkSize.
-   * @param buffer
+   * @param buffer bytes array buffer
    * @return the buffer itself
    */
-  protected ByteBuffer resetBuffer(ByteBuffer buffer) {
-    buffer.clear();
-    buffer.put(zeroChunkBytes);
-    buffer.position(0);
+  protected byte[] resetBuffer(byte[] buffer, int offset, int len) {
+    for (int i = offset; i < len; ++i) {
+      buffer[i] = (byte) 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
+   * Check and ensure the buffers are of the length specified by dataLen.
+   * @param buffers
+   * @param dataLen
    */
-  protected byte[] resetBuffer(byte[] buffer) {
-    System.arraycopy(zeroChunkBytes, 0, buffer, 0, buffer.length);
+  protected void ensureLength(ByteBuffer[] buffers, int dataLen) {
+    for (int i = 0; i < buffers.length; ++i) {
+      if (buffers[i].remaining() != dataLen) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer, not of length " + dataLen);
+      }
+    }
+  }
 
-    return buffer;
+  /**
+   * Check and ensure the buffers are of the length specified by dataLen.
+   * @param buffers
+   * @param dataLen
+   */
+  protected void ensureLength(byte[][] buffers, int dataLen) {
+    for (int i = 0; i < buffers.length; ++i) {
+      if (buffers[i].length != dataLen) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer not of length " + dataLen);
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 b247543..31f4fb8 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.io.erasurecode.ECChunk;
 
 import java.nio.ByteBuffer;
@@ -33,14 +34,43 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs) {
     checkParameters(inputs, erasedIndexes, outputs);
+    int dataLen = inputs[0].remaining();
+    if (dataLen == 0) {
+      return;
+    }
+    ensureLength(inputs, dataLen);
+    ensureLength(outputs, dataLen);
 
-    boolean hasArray = inputs[0].hasArray();
-    if (hasArray) {
-      byte[][] newInputs = toArrays(inputs);
-      byte[][] newOutputs = toArrays(outputs);
-      doDecode(newInputs, erasedIndexes, newOutputs);
-    } else {
+    boolean usingDirectBuffer = inputs[0].isDirect();
+    if (usingDirectBuffer) {
       doDecode(inputs, erasedIndexes, outputs);
+      return;
+    }
+
+    int[] inputOffsets = new int[inputs.length];
+    int[] outputOffsets = new int[outputs.length];
+    byte[][] newInputs = new byte[inputs.length][];
+    byte[][] newOutputs = new byte[outputs.length][];
+
+    ByteBuffer buffer;
+    for (int i = 0; i < inputs.length; ++i) {
+      buffer = inputs[i];
+      inputOffsets[i] = buffer.position();
+      newInputs[i] = buffer.array();
+    }
+
+    for (int i = 0; i < outputs.length; ++i) {
+      buffer = outputs[i];
+      outputOffsets[i] = buffer.position();
+      newOutputs[i] = buffer.array();
+    }
+
+    doDecode(newInputs, inputOffsets, dataLen,
+        erasedIndexes, newOutputs, outputOffsets);
+
+    for (int i = 0; i < inputs.length; ++i) {
+      buffer = inputs[i];
+      buffer.position(inputOffsets[i] + dataLen); // dataLen bytes consumed
     }
   }
 
@@ -56,18 +86,33 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   @Override
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) {
     checkParameters(inputs, erasedIndexes, outputs);
+    int dataLen = inputs[0].length;
+    if (dataLen == 0) {
+      return;
+    }
+    ensureLength(inputs, dataLen);
+    ensureLength(outputs, dataLen);
+
+    int[] inputOffsets = new int[inputs.length]; // ALL ZERO
+    int[] outputOffsets = new int[outputs.length]; // ALL ZERO
 
-    doDecode(inputs, erasedIndexes, outputs);
+    doDecode(inputs, inputOffsets, dataLen, erasedIndexes, outputs,
+        outputOffsets);
   }
 
   /**
-   * Perform the real decoding using bytes array
+   * Perform the real decoding using bytes array, supporting offsets and
+   * lengths.
    * @param inputs
+   * @param inputOffsets
+   * @param dataLen
    * @param erasedIndexes
    * @param outputs
+   * @param outputOffsets
    */
-  protected abstract void doDecode(byte[][] inputs, int[] erasedIndexes,
-                                   byte[][] outputs);
+  protected abstract void doDecode(byte[][] inputs, int[] inputOffsets,
+                                   int dataLen, int[] erasedIndexes,
+                                   byte[][] outputs, int[] outputOffsets);
 
   @Override
   public void decode(ECChunk[] inputs, int[] erasedIndexes,
@@ -91,12 +136,12 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
     }
 
     if (erasedIndexes.length != outputs.length) {
-      throw new IllegalArgumentException(
+      throw new HadoopIllegalArgumentException(
           "erasedIndexes and outputs mismatch in length");
     }
 
     if (erasedIndexes.length > getNumParityUnits()) {
-      throw new IllegalArgumentException(
+      throw new HadoopIllegalArgumentException(
           "Too many erased, not recoverable");
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 06e88bf..0ae54c5 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.io.erasurecode.ECChunk;
 
 import java.nio.ByteBuffer;
@@ -32,14 +33,42 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
   @Override
   public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
     checkParameters(inputs, outputs);
+    int dataLen = inputs[0].remaining();
+    if (dataLen == 0) {
+      return;
+    }
+    ensureLength(inputs, dataLen);
+    ensureLength(outputs, dataLen);
 
-    boolean hasArray = inputs[0].hasArray();
-    if (hasArray) {
-      byte[][] newInputs = toArrays(inputs);
-      byte[][] newOutputs = toArrays(outputs);
-      doEncode(newInputs, newOutputs);
-    } else {
+    boolean usingDirectBuffer = inputs[0].isDirect();
+    if (usingDirectBuffer) {
       doEncode(inputs, outputs);
+      return;
+    }
+
+    int[] inputOffsets = new int[inputs.length];
+    int[] outputOffsets = new int[outputs.length];
+    byte[][] newInputs = new byte[inputs.length][];
+    byte[][] newOutputs = new byte[outputs.length][];
+
+    ByteBuffer buffer;
+    for (int i = 0; i < inputs.length; ++i) {
+      buffer = inputs[i];
+      inputOffsets[i] = buffer.position();
+      newInputs[i] = buffer.array();
+    }
+
+    for (int i = 0; i < outputs.length; ++i) {
+      buffer = outputs[i];
+      outputOffsets[i] = buffer.position();
+      newOutputs[i] = buffer.array();
+    }
+
+    doEncode(newInputs, inputOffsets, dataLen, newOutputs, outputOffsets);
+
+    for (int i = 0; i < inputs.length; ++i) {
+      buffer = inputs[i];
+      buffer.position(buffer.position() + dataLen); // dataLen bytes consumed
     }
   }
 
@@ -53,16 +82,31 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
   @Override
   public void encode(byte[][] inputs, byte[][] outputs) {
     checkParameters(inputs, outputs);
+    int dataLen = inputs[0].length;
+    if (dataLen == 0) {
+      return;
+    }
+    ensureLength(inputs, dataLen);
+    ensureLength(outputs, dataLen);
+
+    int[] inputOffsets = new int[inputs.length]; // ALL ZERO
+    int[] outputOffsets = new int[outputs.length]; // ALL ZERO
 
-    doEncode(inputs, outputs);
+    doEncode(inputs, inputOffsets, dataLen, outputs, outputOffsets);
   }
 
   /**
-   * Perform the real encoding work using bytes array
+   * Perform the real encoding work using bytes array, supporting offsets
+   * and lengths.
    * @param inputs
+   * @param inputOffsets
+   * @param dataLen
    * @param outputs
+   * @param outputOffsets
    */
-  protected abstract void doEncode(byte[][] inputs, byte[][] outputs);
+  protected abstract void doEncode(byte[][] inputs, int[] inputOffsets,
+                                   int dataLen, byte[][] outputs,
+                                   int[] outputOffsets);
 
   @Override
   public void encode(ECChunk[] inputs, ECChunk[] outputs) {
@@ -78,10 +122,10 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
    */
   protected void checkParameters(Object[] inputs, Object[] outputs) {
     if (inputs.length != getNumDataUnits()) {
-      throw new IllegalArgumentException("Invalid inputs length");
+      throw new HadoopIllegalArgumentException("Invalid inputs length");
     }
     if (outputs.length != getNumParityUnits()) {
-      throw new IllegalArgumentException("Invalid outputs length");
+      throw new HadoopIllegalArgumentException("Invalid outputs length");
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
index 24fa637..ff1162f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
@@ -36,9 +36,9 @@ public class RSRawDecoder extends AbstractRawErasureDecoder {
     super.initialize(numDataUnits, numParityUnits, chunkSize);
     assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
 
-    this.errSignature = new int[getNumParityUnits()];
-    this.primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
-        getNumParityUnits());
+    this.errSignature = new int[numParityUnits];
+    this.primitivePower = RSUtil.getPrimitivePower(numDataUnits,
+        numParityUnits);
   }
 
   @Override
@@ -49,21 +49,21 @@ public class RSRawDecoder extends AbstractRawErasureDecoder {
       RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
     }
 
-    int dataLen = inputs[0].remaining();
-    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
-        erasedIndexes.length, dataLen);
+    RSUtil.GF.solveVandermondeSystem(errSignature,
+        outputs, erasedIndexes.length);
   }
 
   @Override
-  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
-                          byte[][] outputs) {
+  protected void doDecode(byte[][] inputs, int[] inputOffsets,
+                          int dataLen, int[] erasedIndexes,
+                          byte[][] outputs, int[] outputOffsets) {
     for (int i = 0; i < erasedIndexes.length; i++) {
       errSignature[i] = primitivePower[erasedIndexes[i]];
-      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
+      RSUtil.GF.substitute(inputs, inputOffsets, dataLen, outputs[i],
+          outputOffsets[i], primitivePower[i]);
     }
 
-    int dataLen = inputs[0].length;
-    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
+    RSUtil.GF.solveVandermondeSystem(errSignature, outputs, outputOffsets,
         erasedIndexes.length, dataLen);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
index 7b501ce..9136331 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
@@ -34,12 +34,12 @@ public class RSRawEncoder extends AbstractRawErasureEncoder {
     super.initialize(numDataUnits, numParityUnits, chunkSize);
     assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
 
-    int[] primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
-        getNumParityUnits());
+    int[] primitivePower = RSUtil.getPrimitivePower(numDataUnits,
+        numParityUnits);
     // compute generating polynomial
     int[] gen = {1};
     int[] poly = new int[2];
-    for (int i = 0; i < getNumParityUnits(); i++) {
+    for (int i = 0; i < numParityUnits; i++) {
       poly[0] = primitivePower[i];
       poly[1] = 1;
       gen = RSUtil.GF.multiply(gen, poly);
@@ -50,29 +50,30 @@ public class RSRawEncoder extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    ByteBuffer[] data = new ByteBuffer[getNumDataUnits() + getNumParityUnits()];
-    for (int i = 0; i < getNumParityUnits(); i++) {
-      data[i] = outputs[i];
-    }
-    for (int i = 0; i < getNumDataUnits(); i++) {
-      data[i + getNumParityUnits()] = inputs[i];
-    }
+    // parity units + data units
+    ByteBuffer[] all = new ByteBuffer[outputs.length + inputs.length];
+    System.arraycopy(outputs, 0, all, 0, outputs.length);
+    System.arraycopy(inputs, 0, all, outputs.length, inputs.length);
 
     // Compute the remainder
-    RSUtil.GF.remainder(data, generatingPolynomial);
+    RSUtil.GF.remainder(all, generatingPolynomial);
   }
 
   @Override
-  protected void doEncode(byte[][] inputs, byte[][] outputs) {
-    byte[][] data = new byte[getNumDataUnits() + getNumParityUnits()][];
-    for (int i = 0; i < getNumParityUnits(); i++) {
-      data[i] = outputs[i];
-    }
-    for (int i = 0; i < getNumDataUnits(); i++) {
-      data[i + getNumParityUnits()] = inputs[i];
-    }
+  protected void doEncode(byte[][] inputs, int[] inputOffsets,
+                          int dataLen, byte[][] outputs,
+                          int[] outputOffsets) {
+    // parity units + data units
+    byte[][] all = new byte[outputs.length + inputs.length][];
+    System.arraycopy(outputs, 0, all, 0, outputs.length);
+    System.arraycopy(inputs, 0, all, outputs.length, inputs.length);
+
+    int[] offsets = new int[inputOffsets.length + outputOffsets.length];
+    System.arraycopy(outputOffsets, 0, offsets, 0, outputOffsets.length);
+    System.arraycopy(inputOffsets, 0, offsets,
+        outputOffsets.length, inputOffsets.length);
 
     // Compute the remainder
-    RSUtil.GF.remainder(data, generatingPolynomial);
+    RSUtil.GF.remainder(all, offsets, dataLen, generatingPolynomial);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 2ea1b3d..bf6e894 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
@@ -21,47 +21,57 @@ import java.nio.ByteBuffer;
 
 /**
  * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ *
+ * XOR code is an important primitive code scheme in erasure coding and often
+ * used in advanced codes, like HitchHiker and LRC, though itself is rarely
+ * deployed independently.
  */
 public class XORRawDecoder extends AbstractRawErasureDecoder {
 
   @Override
   protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
                           ByteBuffer[] outputs) {
-    resetBuffer(outputs[0]);
+    ByteBuffer output = outputs[0];
+    resetOutputBuffer(output);
 
-    int bufSize = getChunkSize();
     int erasedIdx = erasedIndexes[0];
 
     // Process the inputs.
+    int iIdx, oIdx;
     for (int i = 0; i < inputs.length; i++) {
       // Skip the erased location.
       if (i == erasedIdx) {
         continue;
       }
 
-      for (int j = 0; j < bufSize; j++) {
-        outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j)));
+      for (iIdx = inputs[i].position(), oIdx = output.position();
+           iIdx < inputs[i].limit();
+           iIdx++, oIdx++) {
+        output.put(oIdx, (byte) (output.get(oIdx) ^ inputs[i].get(iIdx)));
       }
     }
   }
 
   @Override
-  protected void doDecode(byte[][] inputs,
-                          int[] erasedIndexes, byte[][] outputs) {
-    resetBuffer(outputs[0]);
+  protected void doDecode(byte[][] inputs, int[] inputOffsets, int dataLen,
+                          int[] erasedIndexes, byte[][] outputs,
+                          int[] outputOffsets) {
+    byte[] output = outputs[0];
+    resetBuffer(output, outputOffsets[0], dataLen);
 
-    int bufSize = getChunkSize();
     int erasedIdx = erasedIndexes[0];
 
     // Process the inputs.
+    int iIdx, oIdx;
     for (int i = 0; i < inputs.length; i++) {
       // Skip the erased location.
       if (i == erasedIdx) {
         continue;
       }
 
-      for (int j = 0; j < bufSize; j++) {
-        outputs[0][j] ^= inputs[i][j];
+      for (iIdx = inputOffsets[i], oIdx = outputOffsets[0];
+           iIdx < inputOffsets[i] + dataLen; iIdx++, oIdx++) {
+        output[oIdx] ^= inputs[i][iIdx];
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 116cb91..feffbbf 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
@@ -21,43 +21,53 @@ import java.nio.ByteBuffer;
 
 /**
  * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ *
+ * XOR code is an important primitive code scheme in erasure coding and often
+ * used in advanced codes, like HitchHiker and LRC, though itself is rarely
+ * deployed independently.
  */
 public class XORRawEncoder extends AbstractRawErasureEncoder {
 
-  @Override
   protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    resetBuffer(outputs[0]);
+    ByteBuffer output = outputs[0];
+    resetOutputBuffer(output);
 
-    int bufSize = getChunkSize();
     // Get the first buffer's data.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0].put(j, inputs[0].get(j));
+    int iIdx, oIdx;
+    for (iIdx = inputs[0].position(), oIdx = output.position();
+         iIdx < inputs[0].limit(); iIdx++, oIdx++) {
+      output.put(oIdx, inputs[0].get(iIdx));
     }
 
     // XOR with everything else.
     for (int i = 1; i < inputs.length; i++) {
-      for (int j = 0; j < bufSize; j++) {
-        outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j)));
+      for (iIdx = inputs[i].position(), oIdx = output.position();
+           iIdx < inputs[i].limit();
+           iIdx++, oIdx++) {
+        output.put(oIdx, (byte) (output.get(oIdx) ^ inputs[i].get(iIdx)));
       }
     }
   }
 
   @Override
-  protected void doEncode(byte[][] inputs, byte[][] outputs) {
-    resetBuffer(outputs[0]);
+  protected void doEncode(byte[][] inputs, int[] inputOffsets, int dataLen,
+                          byte[][] outputs, int[] outputOffsets) {
+    byte[] output = outputs[0];
+    resetBuffer(output, outputOffsets[0], dataLen);
 
-    int bufSize = getChunkSize();
     // Get the first buffer's data.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0][j] = inputs[0][j];
+    int iIdx, oIdx;
+    for (iIdx = inputOffsets[0], oIdx = outputOffsets[0];
+         iIdx < inputOffsets[0] + dataLen; iIdx++, oIdx++) {
+      output[oIdx] = inputs[0][iIdx];
     }
 
     // XOR with everything else.
     for (int i = 1; i < inputs.length; i++) {
-      for (int j = 0; j < bufSize; j++) {
-        outputs[0][j] ^= inputs[i][j];
+      for (iIdx = inputOffsets[i], oIdx = outputOffsets[0];
+           iIdx < inputOffsets[i] + dataLen; iIdx++, oIdx++) {
+        output[oIdx] ^= inputs[i][iIdx];
       }
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
index 77544c6..62b22c9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
@@ -235,26 +235,30 @@ public class GaloisField {
   /**
    * A "bulk" version to the solving of Vandermonde System
    */
-  public void solveVandermondeSystem(int[] x, byte[][] y,
+  public void solveVandermondeSystem(int[] x, byte[][] y, int[] outputOffsets,
                                      int len, int dataLen) {
+    int idx1, idx2;
     for (int i = 0; i < len - 1; i++) {
       for (int j = len - 1; j > i; j--) {
-        for (int k = 0; k < dataLen; k++) {
-          y[j][k] = (byte) (y[j][k] ^ mulTable[x[i]][y[j - 1][k] &
+        for (idx2 = outputOffsets[j-1], idx1 = outputOffsets[j];
+             idx1 < outputOffsets[j] + dataLen; idx1++, idx2++) {
+          y[j][idx1] = (byte) (y[j][idx1] ^ mulTable[x[i]][y[j - 1][idx2] &
               0x000000FF]);
         }
       }
     }
     for (int i = len - 1; i >= 0; i--) {
       for (int j = i + 1; j < len; j++) {
-        for (int k = 0; k < dataLen; k++) {
-          y[j][k] = (byte) (divTable[y[j][k] & 0x000000FF][x[j] ^
+        for (idx1 = outputOffsets[j];
+             idx1 < outputOffsets[j] + dataLen; idx1++) {
+          y[j][idx1] = (byte) (divTable[y[j][idx1] & 0x000000FF][x[j] ^
               x[j - i - 1]]);
         }
       }
       for (int j = i; j < len - 1; j++) {
-        for (int k = 0; k < dataLen; k++) {
-          y[j][k] = (byte) (y[j][k] ^ y[j + 1][k]);
+        for (idx2 = outputOffsets[j+1], idx1 = outputOffsets[j];
+             idx1 < outputOffsets[j] + dataLen; idx1++, idx2++) {
+          y[j][idx1] = (byte) (y[j][idx1] ^ y[j + 1][idx2]);
         }
       }
     }
@@ -263,26 +267,34 @@ public class GaloisField {
   /**
    * A "bulk" version of the solveVandermondeSystem, using ByteBuffer.
    */
-  public void solveVandermondeSystem(int[] x, ByteBuffer[] y,
-                                     int len, int dataLen) {
+  public void solveVandermondeSystem(int[] x, ByteBuffer[] y, int len) {
+    ByteBuffer p;
+    int idx1, idx2;
     for (int i = 0; i < len - 1; i++) {
       for (int j = len - 1; j > i; j--) {
-        for (int k = 0; k < dataLen; k++) {
-          y[j].put(k, (byte) (y[j].get(k) ^ mulTable[x[i]][y[j - 1].get(k) &
+        p = y[j];
+        for (idx1 = p.position(), idx2 = y[j-1].position();
+             idx1 < p.limit(); idx1++, idx2++) {
+          p.put(idx1, (byte) (p.get(idx1) ^ mulTable[x[i]][y[j-1].get(idx2) &
               0x000000FF]));
         }
       }
     }
+
     for (int i = len - 1; i >= 0; i--) {
       for (int j = i + 1; j < len; j++) {
-        for (int k = 0; k < dataLen; k++) {
-          y[j].put(k, (byte) (divTable[y[j].get(k) & 0x000000FF][x[j] ^
-              x[j - i - 1]]));
+        p = y[j];
+        for (idx1 = p.position(); idx1 < p.limit(); idx1++) {
+          p.put(idx1, (byte) (divTable[p.get(idx1) &
+              0x000000FF][x[j] ^ x[j - i - 1]]));
         }
       }
+
       for (int j = i; j < len - 1; j++) {
-        for (int k = 0; k < dataLen; k++) {
-          y[j].put(k, (byte) (y[j].get(k) ^ y[j + 1].get(k)));
+        p = y[j];
+        for (idx1 = p.position(), idx2 = y[j+1].position();
+             idx1 < p.limit(); idx1++, idx2++) {
+          p.put(idx1, (byte) (p.get(idx1) ^ y[j+1].get(idx2)));
         }
       }
     }
@@ -394,6 +406,31 @@ public class GaloisField {
   }
 
   /**
+   * A "bulk" version of the substitute.
+   * Tends to be 2X faster than the "int" substitute in a loop.
+   *
+   * @param p input polynomial
+   * @param offsets
+   * @param len
+   * @param q store the return result
+   * @param offset
+   * @param x input field
+   */
+  public void substitute(byte[][] p, int[] offsets,
+                         int len, byte[] q, int offset, int x) {
+    int y = 1, iIdx, oIdx;
+    for (int i = 0; i < p.length; i++) {
+      byte[] pi = p[i];
+      for (iIdx = offsets[i], oIdx = offset;
+           iIdx < offsets[i] + len; iIdx++, oIdx++) {
+        int pij = pi[iIdx] & 0x000000FF;
+        q[oIdx] = (byte) (q[oIdx] ^ mulTable[pij][y]);
+      }
+      y = mulTable[x][y];
+    }
+  }
+
+  /**
    * A "bulk" version of the substitute, using ByteBuffer.
    * Tends to be 2X faster than the "int" substitute in a loop.
    *
@@ -402,13 +439,13 @@ public class GaloisField {
    * @param x input field
    */
   public void substitute(ByteBuffer[] p, ByteBuffer q, int x) {
-    int y = 1;
+    int y = 1, iIdx, oIdx;
     for (int i = 0; i < p.length; i++) {
       ByteBuffer pi = p[i];
-      int len = pi.remaining();
-      for (int j = 0; j < len; j++) {
-        int pij = pi.get(j) & 0x000000FF;
-        q.put(j, (byte) (q.get(j) ^ mulTable[pij][y]));
+      for (iIdx = pi.position(), oIdx = q.position();
+           iIdx < pi.limit(); iIdx++, oIdx++) {
+        int pij = pi.get(iIdx) & 0x000000FF;
+        q.put(oIdx, (byte) (q.get(oIdx) ^ mulTable[pij][y]));
       }
       y = mulTable[x][y];
     }
@@ -432,17 +469,42 @@ public class GaloisField {
   }
 
   /**
+   * The "bulk" version of the remainder.
+   * Warning: This function will modify the "dividend" inputs.
+   */
+  public void remainder(byte[][] dividend, int[] offsets,
+                        int len, int[] divisor) {
+    int idx1, idx2;
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      for (int j = 0; j < divisor.length; j++) {
+        for (idx2 = offsets[j + i], idx1 = offsets[i + divisor.length - 1];
+             idx1 < offsets[i + divisor.length - 1] + len;
+             idx1++, idx2++) {
+          int ratio = divTable[dividend[i + divisor.length - 1][idx1] &
+              0x00FF][divisor[divisor.length - 1]];
+          dividend[j + i][idx2] = (byte) ((dividend[j + i][idx2] & 0x00FF) ^
+              mulTable[ratio][divisor[j]]);
+        }
+      }
+    }
+  }
+
+  /**
    * The "bulk" version of the remainder, using ByteBuffer.
    * Warning: This function will modify the "dividend" inputs.
    */
   public void remainder(ByteBuffer[] dividend, int[] divisor) {
+    int idx1, idx2;
+    ByteBuffer b1, b2;
     for (int i = dividend.length - divisor.length; i >= 0; i--) {
-      int width = dividend[i].remaining();
       for (int j = 0; j < divisor.length; j++) {
-        for (int k = 0; k < width; k++) {
-          int ratio = divTable[dividend[i + divisor.length - 1].get(k) &
+        b1 = dividend[i + divisor.length - 1];
+        b2 = dividend[j + i];
+        for (idx1 = b1.position(), idx2 = b2.position();
+             idx1 < b1.limit(); idx1++, idx2++) {
+          int ratio = divTable[b1.get(idx1) &
               0x00FF][divisor[divisor.length - 1]];
-          dividend[j + i].put(k, (byte) ((dividend[j + i].get(k) & 0x00FF) ^
+          b2.put(idx2, (byte) ((b2.get(idx2) & 0x00FF) ^
               mulTable[ratio][divisor[j]]));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 769427d..cc3617c 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
@@ -35,7 +35,12 @@ public abstract class TestCoderBase {
   private Configuration conf;
   protected int numDataUnits;
   protected int numParityUnits;
-  protected int chunkSize = 16 * 1024;
+  protected int baseChunkSize = 16 * 1024;
+  private int chunkSize = baseChunkSize;
+
+  private byte[] zeroChunkBytes;
+
+  private boolean startBufferWithZero = true;
 
   // Indexes of erased data units.
   protected int[] erasedDataIndexes = new int[] {0};
@@ -47,6 +52,15 @@ public abstract class TestCoderBase {
   // may go to different coding implementations.
   protected boolean usingDirectBuffer = true;
 
+  protected int getChunkSize() {
+    return chunkSize;
+  }
+
+  protected void setChunkSize(int chunkSize) {
+    this.chunkSize = chunkSize;
+    this.zeroChunkBytes = new byte[chunkSize]; // With ZERO by default
+  }
+
   /**
    * Prepare before running the case.
    * @param numDataUnits
@@ -80,8 +94,8 @@ public abstract class TestCoderBase {
    */
   protected void compareAndVerify(ECChunk[] erasedChunks,
                                   ECChunk[] recoveredChunks) {
-    byte[][] erased = ECChunk.toArrays(erasedChunks);
-    byte[][] recovered = ECChunk.toArrays(recoveredChunks);
+    byte[][] erased = toArrays(erasedChunks);
+    byte[][] recovered = toArrays(recoveredChunks);
     boolean result = Arrays.deepEquals(erased, recovered);
     assertTrue("Decoding and comparing failed.", result);
   }
@@ -171,16 +185,19 @@ public abstract class TestCoderBase {
 
   /**
    * Erase data from the specified chunk, putting ZERO bytes to the buffer.
-   * @param chunk
+   * @param chunk with a buffer ready to read at the current position
    */
   protected void eraseDataFromChunk(ECChunk chunk) {
     ByteBuffer chunkBuffer = chunk.getBuffer();
-    // erase the data
-    chunkBuffer.position(0);
-    for (int i = 0; i < chunkSize; i++) {
-      chunkBuffer.put((byte) 0);
-    }
+    // Erase the data at the position, and restore the buffer ready for reading
+    // same many bytes but all ZERO.
+    int pos = chunkBuffer.position();
+    int len = chunkBuffer.remaining();
+    chunkBuffer.put(zeroChunkBytes, 0, len);
+    // Back to readable again after data erased
     chunkBuffer.flip();
+    chunkBuffer.position(pos);
+    chunkBuffer.limit(pos + len);
   }
 
   /**
@@ -190,7 +207,7 @@ public abstract class TestCoderBase {
    * @param chunks
    * @return
    */
-  protected static ECChunk[] cloneChunksWithData(ECChunk[] chunks) {
+  protected ECChunk[] cloneChunksWithData(ECChunk[] chunks) {
     ECChunk[] results = new ECChunk[chunks.length];
     for (int i = 0; i < chunks.length; i++) {
       results[i] = cloneChunkWithData(chunks[i]);
@@ -206,22 +223,19 @@ public abstract class TestCoderBase {
    * @param chunk
    * @return a new chunk
    */
-  protected static ECChunk cloneChunkWithData(ECChunk chunk) {
+  protected ECChunk cloneChunkWithData(ECChunk chunk) {
     ByteBuffer srcBuffer = chunk.getBuffer();
-    ByteBuffer destBuffer;
 
     byte[] bytesArr = new byte[srcBuffer.remaining()];
     srcBuffer.mark();
-    srcBuffer.get(bytesArr);
+    srcBuffer.get(bytesArr, 0, bytesArr.length);
     srcBuffer.reset();
 
-    if (srcBuffer.hasArray()) {
-      destBuffer = ByteBuffer.wrap(bytesArr);
-    } else {
-      destBuffer = ByteBuffer.allocateDirect(srcBuffer.remaining());
-      destBuffer.put(bytesArr);
-      destBuffer.flip();
-    }
+    ByteBuffer destBuffer = allocateOutputBuffer(bytesArr.length);
+    int pos = destBuffer.position();
+    destBuffer.put(bytesArr);
+    destBuffer.flip();
+    destBuffer.position(pos);
 
     return new ECChunk(destBuffer);
   }
@@ -231,18 +245,30 @@ public abstract class TestCoderBase {
    * @return
    */
   protected ECChunk allocateOutputChunk() {
-    ByteBuffer buffer = allocateOutputBuffer();
+    ByteBuffer buffer = allocateOutputBuffer(chunkSize);
 
     return new ECChunk(buffer);
   }
 
   /**
-   * Allocate a buffer for output or writing.
-   * @return
+   * Allocate a buffer for output or writing. It can prepare for two kinds of
+   * data buffers: one with position as 0, the other with position > 0
+   * @return a buffer ready to write chunkSize bytes from current position
    */
-  protected ByteBuffer allocateOutputBuffer() {
+  protected ByteBuffer allocateOutputBuffer(int bufferLen) {
+    /**
+     * When startBufferWithZero, will prepare a buffer as:---------------
+     * otherwise, the buffer will be like:             ___TO--BE--WRITTEN___,
+     * and in the beginning, dummy data are prefixed, to simulate a buffer of
+     * position > 0.
+     */
+    int startOffset = startBufferWithZero ? 0 : 11; // 11 is arbitrary
+    int allocLen = startOffset + bufferLen + startOffset;
     ByteBuffer buffer = usingDirectBuffer ?
-        ByteBuffer.allocateDirect(chunkSize) : ByteBuffer.allocate(chunkSize);
+        ByteBuffer.allocateDirect(allocLen) : ByteBuffer.allocate(allocLen);
+    buffer.limit(startOffset + bufferLen);
+    fillDummyData(buffer, startOffset);
+    startBufferWithZero = ! startBufferWithZero;
 
     return buffer;
   }
@@ -265,16 +291,35 @@ public abstract class TestCoderBase {
    * @return
    */
   protected ECChunk generateDataChunk() {
-    ByteBuffer buffer = allocateOutputBuffer();
-    for (int i = 0; i < chunkSize; i++) {
-      buffer.put((byte) RAND.nextInt(256));
-    }
+    ByteBuffer buffer = allocateOutputBuffer(chunkSize);
+    int pos = buffer.position();
+    buffer.put(generateData(chunkSize));
     buffer.flip();
+    buffer.position(pos);
 
     return new ECChunk(buffer);
   }
 
   /**
+   * Fill len of dummy data in the buffer at the current position.
+   * @param buffer
+   * @param len
+   */
+  protected void fillDummyData(ByteBuffer buffer, int len) {
+    byte[] dummy = new byte[len];
+    RAND.nextBytes(dummy);
+    buffer.put(dummy);
+  }
+
+  protected byte[] generateData(int len) {
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) RAND.nextInt(256);
+    }
+    return buffer;
+  }
+
+  /**
    * Prepare parity chunks for encoding, each chunk for each parity unit.
    * @return
    */
@@ -303,4 +348,32 @@ public abstract class TestCoderBase {
     return chunks;
   }
 
+  /**
+   * Convert an array of this chunks to an array of byte array.
+   * Note the chunk buffers are not affected.
+   * @param chunks
+   * @return an array of byte array
+   */
+  protected byte[][] toArrays(ECChunk[] chunks) {
+    byte[][] bytesArr = new byte[chunks.length][];
+
+    for (int i = 0; i < chunks.length; i++) {
+      bytesArr[i] = chunks[i].toBytesArray();
+    }
+
+    return bytesArr;
+  }
+
+
+  /**
+   * Make some chunk messy or not correct any more
+   * @param chunks
+   */
+  protected void corruptSomeChunk(ECChunk[] chunks) {
+    int idx = new Random().nextInt(chunks.length);
+    ByteBuffer buffer = chunks[idx].getBuffer();
+    if (buffer.hasRemaining()) {
+      buffer.position(buffer.position() + 1);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 f30323b..154ec18 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
@@ -59,6 +59,19 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     this.usingDirectBuffer = usingDirectBuffer;
     prepareCoders();
 
+    /**
+     * The following runs will use 3 different chunkSize for inputs and outputs,
+     * to verify the same encoder/decoder can process variable width of data.
+     */
+    performTestCoding(baseChunkSize);
+    performTestCoding(baseChunkSize - 17);
+    performTestCoding(baseChunkSize + 16);
+  }
+
+  private void performTestCoding(int chunkSize) {
+    setChunkSize(chunkSize);
+
+
     // Generate data and encode
     ECBlockGroup blockGroup = prepareBlockGroupForEncoding();
     // Backup all the source chunks for later recovering because some coders
@@ -138,7 +151,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
       throw new RuntimeException("Failed to create encoder", e);
     }
 
-    encoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    encoder.initialize(numDataUnits, numParityUnits, getChunkSize());
     encoder.setConf(getConf());
     return encoder;
   }
@@ -165,7 +178,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
       throw new RuntimeException("Failed to create decoder", e);
     }
 
-    decoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    decoder.initialize(numDataUnits, numParityUnits, getChunkSize());
     decoder.setConf(getConf());
     return decoder;
   }
@@ -249,7 +262,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
    * @param blocks
    * @return
    */
-  protected static TestBlock[] cloneBlocksWithData(TestBlock[] blocks) {
+  protected TestBlock[] cloneBlocksWithData(TestBlock[] blocks) {
     TestBlock[] results = new TestBlock[blocks.length];
     for (int i = 0; i < blocks.length; ++i) {
       results[i] = cloneBlockWithData(blocks[i]);
@@ -263,7 +276,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
    * @param block
    * @return a new block
    */
-  protected static TestBlock cloneBlockWithData(TestBlock block) {
+  protected TestBlock cloneBlockWithData(TestBlock block) {
     ECChunk[] newChunks = cloneChunksWithData(block.chunks);
 
     return new TestBlock(newChunks);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 84bad92..02b9eea 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
@@ -44,7 +44,7 @@ public class TestRSRawCoder extends TestRSRawCoderBase {
 
   @Test
   public void testCodingDirectBuffer_10x4_erasing_p1() {
-    prepare(null, 10, 4, new int[] {}, new int[] {1});
+    prepare(null, 10, 4, new int[0], new int[] {1});
     testCoding(true);
     testCoding(true);
   }
@@ -101,4 +101,14 @@ public class TestRSRawCoder extends TestRSRawCoderBase {
     prepare(null, 3, 3, new int[] {0}, new int[] {0});
     testCoding(true);
   }
+
+  @Test
+  public void testCodingNegative_10x4_erasing_d2_d4() {
+    prepare(null, 10, 4, new int[]{2, 4}, new int[0]);
+
+    testCodingWithBadInput(true);
+    testCodingWithBadOutput(false);
+    testCodingWithBadInput(true);
+    testCodingWithBadOutput(false);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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
index f9e8a6b..c06aded 100644
--- 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
@@ -39,13 +39,11 @@ public abstract class TestRSRawCoderBase extends TestRawCoderBase {
   }
 
   @Override
-  protected ECChunk generateDataChunk() {
-    ByteBuffer buffer = allocateOutputBuffer();
-    for (int i = 0; i < chunkSize; i++) {
-      buffer.put((byte) RAND.nextInt(symbolMax));
+  protected byte[] generateData(int len) {
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < len; i++) {
+      buffer[i] = (byte) RAND.nextInt(symbolMax);
     }
-    buffer.flip();
-
-    return new ECChunk(buffer);
+    return buffer;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 8543c4d..4582317 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 
 import org.apache.hadoop.io.erasurecode.ECChunk;
 import org.apache.hadoop.io.erasurecode.TestCoderBase;
+import org.junit.Assert;
 
 /**
  * Raw coder test base with utilities.
@@ -41,8 +42,57 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     this.usingDirectBuffer = usingDirectBuffer;
     prepareCoders();
 
+    /**
+     * The following runs will use 3 different chunkSize for inputs and outputs,
+     * to verify the same encoder/decoder can process variable width of data.
+     */
+    performTestCoding(baseChunkSize, false, false);
+    performTestCoding(baseChunkSize - 17, false, false);
+    performTestCoding(baseChunkSize + 16, false, false);
+  }
+
+  /**
+   * Similar to above, but perform negative cases using bad input for encoding.
+   * @param usingDirectBuffer
+   */
+  protected void testCodingWithBadInput(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders();
+
+    try {
+      performTestCoding(baseChunkSize, true, false);
+      Assert.fail("Encoding test with bad input should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  /**
+   * Similar to above, but perform negative cases using bad output for decoding.
+   * @param usingDirectBuffer
+   */
+  protected void testCodingWithBadOutput(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders();
+
+    try {
+      performTestCoding(baseChunkSize, false, true);
+      Assert.fail("Decoding test with bad output should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  private void performTestCoding(int chunkSize,
+                                 boolean useBadInput, boolean useBadOutput) {
+    setChunkSize(chunkSize);
+
     // Generate data and encode
     ECChunk[] dataChunks = prepareDataChunksForEncoding();
+    if (useBadInput) {
+      corruptSomeChunk(dataChunks);
+    }
+
     ECChunk[] parityChunks = prepareParityChunksForEncoding();
 
     // Backup all the source chunks for later recovering because some coders
@@ -59,6 +109,9 @@ public abstract class TestRawCoderBase extends TestCoderBase {
         clonedDataChunks, parityChunks);
 
     ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
+    if (useBadOutput) {
+      corruptSomeChunk(recoveredChunks);
+    }
 
     decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks);
 
@@ -88,7 +141,7 @@ public abstract class TestRawCoderBase extends TestCoderBase {
       throw new RuntimeException("Failed to create encoder", e);
     }
 
-    encoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    encoder.initialize(numDataUnits, numParityUnits, getChunkSize());
     encoder.setConf(getConf());
     return encoder;
   }
@@ -105,7 +158,7 @@ public abstract class TestRawCoderBase extends TestCoderBase {
       throw new RuntimeException("Failed to create decoder", e);
     }
 
-    decoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    decoder.initialize(numDataUnits, numParityUnits, getChunkSize());
     decoder.setConf(getConf());
     return decoder;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/343c0e76/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 b8912a9..327174e 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
@@ -49,6 +49,15 @@ public class TestXORRawCoder extends TestRawCoderBase {
   @Test
   public void testCodingDirectBuffer_erasing_p0() {
     prepare(null, 10, 1, new int[0], new int[] {0});
+
+    testCoding(true);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_erasing_d0() {
+    prepare(null, 10, 1, new int[] {0}, new int[0]);
+
     testCoding(true);
     testCoding(true);
   }
@@ -67,4 +76,14 @@ public class TestXORRawCoder extends TestRawCoderBase {
     testCoding(true);
     testCoding(false);
   }
+
+  @Test
+  public void testCodingNegative_erasing_d5() {
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
+
+    testCodingWithBadInput(true);
+    testCodingWithBadOutput(false);
+    testCodingWithBadInput(true);
+    testCodingWithBadOutput(false);
+  }
 }


[48/50] [abbrv] hadoop git commit: HADOOP-12029. Remove chunkSize from ECSchema as its not required for coders (Contributed by Vinayakumar B)

Posted by zh...@apache.org.
HADOOP-12029. Remove chunkSize from ECSchema as its not required for coders (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/5a391e1d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5a391e1d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5a391e1d

Branch: refs/heads/HDFS-7285
Commit: 5a391e1d2584dc5d77fb1325ea91c8e5854934d1
Parents: 80c56c2
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon May 25 16:02:37 2015 +0530
Committer: Zhe Zhang <zh...@cloudera.com>
Committed: Tue May 26 12:07:16 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  5 ++-
 .../apache/hadoop/io/erasurecode/ECSchema.java  | 40 +-------------------
 .../hadoop/io/erasurecode/TestECSchema.java     |  3 --
 3 files changed, 6 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a391e1d/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 531b8d5..c9b80d3 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -55,4 +55,7 @@
     HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure 
     coder. (Kai Zheng via Zhe Zhang)
 
-    HADOOP-12013. Generate fixed data to perform erasure coder test. (Kai Zheng)
\ No newline at end of file
+    HADOOP-12013. Generate fixed data to perform erasure coder test. (Kai Zheng)
+
+    HADOOP-12029. Remove chunkSize from ECSchema as its not required for coders
+    (vinayakumarb)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a391e1d/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 f058ea7..fdc569e 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
@@ -28,8 +28,6 @@ public final class ECSchema {
   public static final String NUM_DATA_UNITS_KEY = "k";
   public static final String NUM_PARITY_UNITS_KEY = "m";
   public static final String CODEC_NAME_KEY = "codec";
-  public static final String CHUNK_SIZE_KEY = "chunkSize";
-  public static final int DEFAULT_CHUNK_SIZE = 256 * 1024; // 256K
 
   /**
    * A friendly and understandable name that can mean what's it, also serves as
@@ -52,11 +50,6 @@ public final class ECSchema {
    */
   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.
@@ -92,17 +85,9 @@ public final class ECSchema {
     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;
-    }
-
     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);
   }
@@ -144,14 +129,6 @@ public final class ECSchema {
       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);
   }
@@ -217,14 +194,6 @@ public final class ECSchema {
   }
 
   /**
-   * Get chunk buffer size for the erasure encoding/decoding.
-   * @return chunk buffer size
-   */
-  public int getChunkSize() {
-    return chunkSize;
-  }
-
-  /**
    * Make a meaningful string representation for log output.
    * @return string representation
    */
@@ -235,9 +204,8 @@ public final class ECSchema {
     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() ? "" : ", "));
+    sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits);
+    sb.append((extraOptions.isEmpty() ? "" : ", "));
 
     int i = 0;
     for (String opt : extraOptions.keySet()) {
@@ -267,9 +235,6 @@ public final class ECSchema {
     if (numParityUnits != ecSchema.numParityUnits) {
       return false;
     }
-    if (chunkSize != ecSchema.chunkSize) {
-      return false;
-    }
     if (!schemaName.equals(ecSchema.schemaName)) {
       return false;
     }
@@ -286,7 +251,6 @@ public final class ECSchema {
     result = 31 * result + extraOptions.hashCode();
     result = 31 * result + numDataUnits;
     result = 31 * result + numParityUnits;
-    result = 31 * result + chunkSize;
 
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a391e1d/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 15e672f..c362b96 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
@@ -29,7 +29,6 @@ public class TestECSchema {
     String schemaName = "goodSchema";
     int numDataUnits = 6;
     int numParityUnits = 3;
-    int chunkSize = 64 * 1024 * 1024;
     String codec = "rs";
     String extraOption = "extraOption";
     String extraOptionValue = "extraOptionValue";
@@ -38,7 +37,6 @@ public class TestECSchema {
     options.put(ECSchema.NUM_DATA_UNITS_KEY, String.valueOf(numDataUnits));
     options.put(ECSchema.NUM_PARITY_UNITS_KEY, String.valueOf(numParityUnits));
     options.put(ECSchema.CODEC_NAME_KEY, codec);
-    options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(chunkSize));
     options.put(extraOption, extraOptionValue);
 
     ECSchema schema = new ECSchema(schemaName, options);
@@ -47,7 +45,6 @@ public class TestECSchema {
     assertEquals(schemaName, schema.getSchemaName());
     assertEquals(numDataUnits, schema.getNumDataUnits());
     assertEquals(numParityUnits, schema.getNumParityUnits());
-    assertEquals(chunkSize, schema.getChunkSize());
     assertEquals(codec, schema.getCodecName());
     assertEquals(extraOptionValue, schema.getExtraOptions().get(extraOption));
   }