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 ji...@apache.org on 2015/04/29 21:43:29 UTC

[01/50] hadoop git commit: HDFS-7969. Erasure coding: NameNode support for lease recovery of striped block groups. Contributed by Zhe Zhang.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 bd5cb5981 -> f3d0e5588 (forced update)


HDFS-7969. Erasure coding: NameNode support for lease recovery of striped block groups. 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/80e6018c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/80e6018c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/80e6018c

Branch: refs/heads/HDFS-7285
Commit: 80e6018c430ac2a4c0e2b6fb88bf9c771f48e49a
Parents: 979f453
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 6 12:52:44 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:06:49 2015 -0700

----------------------------------------------------------------------
 .../BlockInfoContiguousUnderConstruction.java   | 33 ++++----
 .../BlockInfoStripedUnderConstruction.java      | 80 ++++++++++++++++----
 .../BlockInfoUnderConstruction.java             | 57 ++++++++++++++
 .../blockmanagement/DatanodeDescriptor.java     | 12 +--
 .../server/blockmanagement/DatanodeManager.java | 10 +--
 .../hdfs/server/namenode/FSNamesystem.java      | 24 +++---
 .../TestBlockInfoUnderConstruction.java         |  2 +-
 7 files changed, 163 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/80e6018c/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 7a052fd..9ba2978 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
@@ -31,7 +31,8 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  * Represents a block that is currently being constructed.<br>
  * This is usually the last block of a file opened for write or append.
  */
-public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
+public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous
+    implements BlockInfoUnderConstruction{
   /** Block state. See {@link BlockUCState} */
   private BlockUCState blockUCState;
 
@@ -94,7 +95,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     return new BlockInfoContiguous(this);
   }
 
-  /** Set expected locations */
+  @Override
   public void setExpectedLocations(DatanodeStorageInfo[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ArrayList<>(numLocations);
@@ -104,10 +105,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     }
   }
 
-  /**
-   * Create array of expected replica locations
-   * (as has been assigned by chooseTargets()).
-   */
+  @Override
   public DatanodeStorageInfo[] getExpectedStorageLocations() {
     int numLocations = replicas == null ? 0 : replicas.size();
     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
@@ -117,7 +115,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     return storages;
   }
 
-  /** Get the number of expected locations */
+  @Override
   public int getNumExpectedLocations() {
     return replicas == null ? 0 : replicas.size();
   }
@@ -135,25 +133,26 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     blockUCState = s;
   }
 
-  /** Get block recovery ID */
+  @Override
   public long getBlockRecoveryId() {
     return blockRecoveryId;
   }
 
-  /** Get recover block */
+  @Override
   public Block getTruncateBlock() {
     return truncateBlock;
   }
 
+  @Override
+  public Block toBlock(){
+    return this;
+  }
+
   public void setTruncateBlock(Block recoveryBlock) {
     this.truncateBlock = recoveryBlock;
   }
 
-  /**
-   * Process the recorded replicas. When about to commit or finish the
-   * pipeline recovery sort out bad replicas.
-   * @param genStamp  The final generation stamp for the block.
-   */
+  @Override
   public void setGenerationStampAndVerifyReplicas(long genStamp) {
     // Set the generation stamp for the block.
     setGenerationStamp(genStamp);
@@ -187,11 +186,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
   }
 
-  /**
-   * Initialize lease recovery for this block.
-   * Find the first alive data-node starting from the previous primary and
-   * make it primary.
-   */
+  @Override
   public void initializeBlockRecovery(long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80e6018c/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 b1857bb..cfaf3a0 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
@@ -31,7 +31,8 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCSt
  * Represents a striped block that is currently being constructed.
  * This is usually the last block of a file opened for write or append.
  */
-public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
+public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
+    implements BlockInfoUnderConstruction{
   private BlockUCState blockUCState;
 
   /**
@@ -40,6 +41,12 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   private ReplicaUnderConstruction[] replicas;
 
   /**
+   * Index of the primary data node doing the recovery. Useful for log
+   * messages.
+   */
+  private int primaryNodeIndex = -1;
+
+  /**
    * The new generation stamp, which this block will have
    * after the recovery succeeds. Also used as a recovery id to identify
    * the right recovery if any of the abandoned recoveries re-appear.
@@ -82,6 +89,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   }
 
   /** Set expected locations */
+  @Override
   public void setExpectedLocations(DatanodeStorageInfo[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ReplicaUnderConstruction[numLocations];
@@ -98,6 +106,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
    * Create array of expected replica locations
    * (as has been assigned by chooseTargets()).
    */
+  @Override
   public DatanodeStorageInfo[] getExpectedStorageLocations() {
     int numLocations = getNumExpectedLocations();
     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
@@ -117,7 +126,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
     return indices;
   }
 
-  /** Get the number of expected locations */
+  @Override
   public int getNumExpectedLocations() {
     return replicas == null ? 0 : replicas.length;
   }
@@ -135,16 +144,22 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
     blockUCState = s;
   }
 
-  /** Get block recovery ID */
+  @Override
   public long getBlockRecoveryId() {
     return blockRecoveryId;
   }
 
-  /**
-   * Process the recorded replicas. When about to commit or finish the
-   * pipeline recovery sort out bad replicas.
-   * @param genStamp  The final generation stamp for the block.
-   */
+  @Override
+  public Block getTruncateBlock() {
+    return null;
+  }
+
+  @Override
+  public Block toBlock(){
+    return this;
+  }
+
+  @Override
   public void setGenerationStampAndVerifyReplicas(long genStamp) {
     // Set the generation stamp for the block.
     setGenerationStamp(genStamp);
@@ -178,18 +193,53 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
     setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
   }
 
-  /**
-   * Initialize lease recovery for this striped block.
-   */
+  @Override
   public void initializeBlockRecovery(long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
     if (replicas == null || replicas.length == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK*" +
-          " BlockInfoUnderConstruction.initLeaseRecovery:" +
+          " BlockInfoStripedUnderConstruction.initLeaseRecovery:" +
           " No blocks found, lease removed.");
     }
-    // TODO we need to implement different recovery logic here
+    boolean allLiveReplicasTriedAsPrimary = true;
+    for (ReplicaUnderConstruction replica : replicas) {
+      // Check if all replicas have been tried or not.
+      if (replica.isAlive()) {
+        allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary &&
+            replica.getChosenAsPrimary());
+      }
+    }
+    if (allLiveReplicasTriedAsPrimary) {
+      // Just set all the replicas to be chosen whether they are alive or not.
+      for (ReplicaUnderConstruction replica : replicas) {
+        replica.setChosenAsPrimary(false);
+      }
+    }
+    long mostRecentLastUpdate = 0;
+    ReplicaUnderConstruction primary = null;
+    primaryNodeIndex = -1;
+    for(int i = 0; i < replicas.length; i++) {
+      // Skip alive replicas which have been chosen for recovery.
+      if (!(replicas[i].isAlive() && !replicas[i].getChosenAsPrimary())) {
+        continue;
+      }
+      final ReplicaUnderConstruction ruc = replicas[i];
+      final long lastUpdate = ruc.getExpectedStorageLocation()
+          .getDatanodeDescriptor().getLastUpdateMonotonic();
+      if (lastUpdate > mostRecentLastUpdate) {
+        primaryNodeIndex = i;
+        primary = ruc;
+        mostRecentLastUpdate = lastUpdate;
+      }
+    }
+    if (primary != null) {
+      primary.getExpectedStorageLocation().getDatanodeDescriptor()
+          .addBlockToBeRecovered(this);
+      primary.setChosenAsPrimary(true);
+      NameNode.blockStateChangeLog.info(
+          "BLOCK* {} recovery started, primary={}", this, primary);
+    }
   }
 
   void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block reportedBlock,
@@ -238,7 +288,9 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   }
 
   private void appendUCParts(StringBuilder sb) {
-    sb.append("{UCState=").append(blockUCState).append(", replicas=[");
+    sb.append("{UCState=").append(blockUCState).
+        append(", primaryNodeIndex=").append(primaryNodeIndex).
+        append(", replicas=[");
     if (replicas != null) {
       int i = 0;
       for (ReplicaUnderConstruction r : replicas) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80e6018c/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
new file mode 100644
index 0000000..bfdd386
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+
+public interface BlockInfoUnderConstruction {
+  /**
+   * Create array of expected replica locations
+   * (as has been assigned by chooseTargets()).
+   */
+  public DatanodeStorageInfo[] getExpectedStorageLocations();
+
+  /** Get recover block */
+  public Block getTruncateBlock();
+
+  /** Convert to a Block object */
+  public Block toBlock();
+
+  /** Get block recovery ID */
+  public long getBlockRecoveryId();
+
+  /** Get the number of expected locations */
+  public int getNumExpectedLocations();
+
+  /** Set expected locations */
+  public void setExpectedLocations(DatanodeStorageInfo[] targets);
+
+  /**
+   * Process the recorded replicas. When about to commit or finish the
+   * pipeline recovery sort out bad replicas.
+   * @param genStamp  The final generation stamp for the block.
+   */
+  public void setGenerationStampAndVerifyReplicas(long genStamp);
+
+  /**
+   * Initialize lease recovery for this block.
+   * Find the first alive data-node starting from the previous primary and
+   * make it primary.
+   */
+  public void initializeBlockRecovery(long recoveryId);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80e6018c/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 15427f7..7ec71a2 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
@@ -253,8 +253,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final BlockQueue<BlockECRecoveryInfo> erasurecodeBlocks =
       new BlockQueue<>();
   /** A queue of blocks to be recovered by this datanode */
-  private final BlockQueue<BlockInfoContiguousUnderConstruction>
-      recoverBlocks = new BlockQueue<>();
+  private final BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
+      new BlockQueue<>();
   /** A set of blocks to be invalidated by this datanode */
   private final LightWeightHashSet<Block> invalidateBlocks =
       new LightWeightHashSet<>();
@@ -649,7 +649,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Store block recovery work.
    */
-  void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction block) {
+  void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
     if(recoverBlocks.contains(block)) {
       // this prevents adding the same block twice to the recovery queue
       BlockManager.LOG.info(block + " is already in the recovery queue");
@@ -703,11 +703,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return erasurecodeBlocks.poll(maxTransfers);
   }
 
-  public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
-    List<BlockInfoContiguousUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
+  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
+    List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
       return null;
-    return blocks.toArray(new BlockInfoContiguousUnderConstruction[blocks.size()]);
+    return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80e6018c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index f787339..3fcbfb8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1382,12 +1382,12 @@ public class DatanodeManager {
         }
 
         //check lease recovery
-        BlockInfoContiguousUnderConstruction[] blocks = nodeinfo
+        BlockInfoUnderConstruction[] blocks = nodeinfo
             .getLeaseRecoveryCommand(Integer.MAX_VALUE);
         if (blocks != null) {
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
               blocks.length);
-          for (BlockInfoContiguousUnderConstruction b : blocks) {
+          for (BlockInfoUnderConstruction b : blocks) {
             final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =
@@ -1401,10 +1401,10 @@ public class DatanodeManager {
             // to old block.
             boolean truncateRecovery = b.getTruncateBlock() != null;
             boolean copyOnTruncateRecovery = truncateRecovery &&
-                b.getTruncateBlock().getBlockId() != b.getBlockId();
+                b.getTruncateBlock().getBlockId() != b.toBlock().getBlockId();
             ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ?
                 new ExtendedBlock(blockPoolId, b.getTruncateBlock()) :
-                new ExtendedBlock(blockPoolId, b);
+                new ExtendedBlock(blockPoolId, b.toBlock());
             // If we only get 1 replica after eliminating stale nodes, then choose all
             // replicas for recovery and let the primary data node handle failures.
             DatanodeInfo[] recoveryInfos;
@@ -1421,7 +1421,7 @@ public class DatanodeManager {
               recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages);
             }
             if(truncateRecovery) {
-              Block recoveryBlock = (copyOnTruncateRecovery) ? b :
+              Block recoveryBlock = (copyOnTruncateRecovery) ? b.toBlock() :
                   b.getTruncateBlock();
               brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos,
                                                 recoveryBlock));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80e6018c/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 9c71574..c359baa 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
@@ -210,6 +210,7 @@ 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.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -4106,18 +4107,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new AlreadyBeingCreatedException(message);
     case UNDER_CONSTRUCTION:
     case UNDER_RECOVERY:
-      // TODO support Striped block's recovery
-      final BlockInfoContiguousUnderConstruction uc =
-          (BlockInfoContiguousUnderConstruction)lastBlock;
+      // TODO support truncate of striped blocks
+      final BlockInfoUnderConstruction uc =
+          (BlockInfoUnderConstruction)lastBlock;
       // determine if last block was intended to be truncated
       Block recoveryBlock = uc.getTruncateBlock();
       boolean truncateRecovery = recoveryBlock != null;
       boolean copyOnTruncate = truncateRecovery &&
-          recoveryBlock.getBlockId() != uc.getBlockId();
+          recoveryBlock.getBlockId() != uc.toBlock().getBlockId();
       assert !copyOnTruncate ||
-          recoveryBlock.getBlockId() < uc.getBlockId() &&
-          recoveryBlock.getGenerationStamp() < uc.getGenerationStamp() &&
-          recoveryBlock.getNumBytes() > uc.getNumBytes() :
+          recoveryBlock.getBlockId() < uc.toBlock().getBlockId() &&
+          recoveryBlock.getGenerationStamp() < uc.toBlock().
+              getGenerationStamp() &&
+          recoveryBlock.getNumBytes() > uc.toBlock().getNumBytes() :
             "wrong recoveryBlock";
 
       // setup the last block locations from the blockManager if not known
@@ -4125,7 +4127,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         uc.setExpectedLocations(blockManager.getStorages(lastBlock));
       }
 
-      if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) {
+      if (uc.getNumExpectedLocations() == 0 &&
+          uc.toBlock().getNumBytes() == 0) {
         // There is no datanode reported to this block.
         // may be client have crashed before writing data to pipeline.
         // This blocks doesn't need any recovery.
@@ -4138,10 +4141,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         return true;
       }
       // start recovery of the last block for this file
-      long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc));
+      long blockRecoveryId =
+          nextGenerationStamp(blockIdManager.isLegacyBlock(uc.toBlock()));
       lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
       if(copyOnTruncate) {
-        uc.setGenerationStamp(blockRecoveryId);
+        uc.toBlock().setGenerationStamp(blockRecoveryId);
       } else if(truncateRecovery) {
         recoveryBlock.setGenerationStamp(blockRecoveryId);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80e6018c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
index a7ba293..f5a9cc4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
@@ -51,7 +51,7 @@ public class TestBlockInfoUnderConstruction {
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000);
     blockInfo.initializeBlockRecovery(1);
-    BlockInfoContiguousUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
+    BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
     // Recovery attempt #2.


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

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


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

Branch: refs/heads/HDFS-7285
Commit: d5683a3d4502d666c326f0a30c0c6a11985e0c4f
Parents: d27f5dc
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Apr 16 13:16:37 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:54 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/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 8006405..dd00e6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -273,6 +273,9 @@ public class BlockManager {
   /** Check whether name system is running before terminating */
   private boolean checkNSRunning = true;
 
+  /** Check whether there are any non-EC blocks using StripedID */
+  private boolean hasNonEcBlockUsingStripedID = false;
+
   public BlockManager(final Namesystem namesystem, final Configuration conf)
     throws IOException {
     this.namesystem = namesystem;
@@ -2912,6 +2915,24 @@ public class BlockManager {
   }
 
   /**
+   * Get the value of whether there are any non-EC blocks using StripedID.
+   *
+   * @return Returns the value of whether there are any non-EC blocks using StripedID.
+   */
+  public boolean hasNonEcBlockUsingStripedID(){
+    return hasNonEcBlockUsingStripedID;
+  }
+
+  /**
+   * Set the value of whether there are any non-EC blocks using StripedID.
+   *
+   * @param has - the value of whether there are any non-EC blocks using StripedID.
+   */
+  public void hasNonEcBlockUsingStripedID(boolean has){
+    hasNonEcBlockUsingStripedID = has;
+  }
+
+  /**
    * Process a single possibly misreplicated block. This adds it to the
    * appropriate queues if necessary, and returns a result code indicating
    * what happened with it.
@@ -3507,8 +3528,10 @@ public class BlockManager {
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
       info = blocksMap.getStoredBlock(
           new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
-    }
-    if (info == null) {
+      if ((info == null) && hasNonEcBlockUsingStripedID()){
+        info = blocksMap.getStoredBlock(block);
+      }
+    } else {
       info = blocksMap.getStoredBlock(block);
     }
     return info;
@@ -3682,6 +3705,21 @@ public class BlockManager {
     return blocksMap.addBlockCollection(block, bc);
   }
 
+  /**
+   * Do some check when adding a block to blocksmap.
+   * For HDFS-7994 to check whether then block is a NonEcBlockUsingStripedID.
+   *
+   */
+  public BlockInfo addBlockCollectionWithCheck(
+      BlockInfo block, BlockCollection bc) {
+    if (!hasNonEcBlockUsingStripedID()){
+      if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
+        hasNonEcBlockUsingStripedID(true);
+      }
+    }
+    return addBlockCollection(block, bc);
+  }
+
   public BlockCollection getBlockCollection(Block b) {
     return blocksMap.getBlockCollection(b);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/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 de930e6..2c5566c 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
@@ -991,7 +991,7 @@ public class FSEditLogLoader {
       newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
           file.getBlockReplication());
     }
-    fsNamesys.getBlockManager().addBlockCollection(newBlockInfo, file);
+    fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file);
     file.addBlock(newBlockInfo);
     fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
   }
@@ -1081,7 +1081,7 @@ public class FSEditLogLoader {
           // OP_ADD operations as each block is allocated.
           newBI = new BlockInfoContiguous(newBlock, file.getBlockReplication());
         }
-        fsNamesys.getBlockManager().addBlockCollection(newBI, file);
+        fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);
         file.addBlock(newBI);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/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 88b0a97d..704f31e 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
@@ -700,7 +700,7 @@ public class FSImageFormat {
       if (blocks != null) {
         final BlockManager bm = namesystem.getBlockManager();
         for (int i = 0; i < blocks.length; i++) {
-          file.setBlock(i, bm.addBlockCollection(blocks[i], file));
+          file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file));
         } 
       }
     }
@@ -1005,8 +1005,8 @@ public class FSImageFormat {
         if (oldnode.numBlocks() > 0) {
           BlockInfo ucBlock = cons.getLastBlock();
           // we do not replace the inode, just replace the last block of oldnode
-          BlockInfo info = namesystem.getBlockManager().addBlockCollection(
-              ucBlock, oldnode);
+          BlockInfo info = namesystem.getBlockManager()
+              .addBlockCollectionWithCheck(ucBlock, oldnode);
           oldnode.setBlock(oldnode.numBlocks() - 1, info);
         }
 

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

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

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

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


[04/50] hadoop git commit: HDFS-7839. Erasure coding: implement facilities in NameNode to create and manage EC zones. Contributed by Zhe Zhang

Posted by ji...@apache.org.
HDFS-7839. Erasure coding: implement facilities in NameNode to create and manage EC zones. 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/d717dc3e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d717dc3e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d717dc3e

Branch: refs/heads/HDFS-7285
Commit: d717dc3e9ee5d81c6e4365e4ed1015b392bdd653
Parents: 45db005
Author: Zhe Zhang <zh...@apache.org>
Authored: Thu Apr 2 22:38:29 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:06:49 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  15 ++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   8 +
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   2 -
 ...tNamenodeProtocolServerSideTranslatorPB.java |  14 ++
 .../ClientNamenodeProtocolTranslatorPB.java     |  16 ++
 .../BlockStoragePolicySuite.java                |   5 -
 .../hdfs/server/common/HdfsServerConstants.java |   2 +
 .../namenode/ErasureCodingZoneManager.java      | 112 ++++++++++++++
 .../hdfs/server/namenode/FSDirRenameOp.java     |   2 +
 .../hdfs/server/namenode/FSDirectory.java       |  26 +++-
 .../hdfs/server/namenode/FSNamesystem.java      |  40 +++++
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  10 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  16 ++
 .../src/main/proto/ClientNamenodeProtocol.proto |   9 ++
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |  12 +-
 .../hadoop/hdfs/TestErasureCodingZones.java     | 151 +++++++++++++++++++
 .../TestBlockInitialEncoding.java               |  75 ---------
 .../server/namenode/TestAddStripedBlocks.java   |   2 +-
 .../server/namenode/TestFSEditLogLoader.java    |   6 +-
 .../hdfs/server/namenode/TestFSImage.java       |  23 ++-
 .../namenode/TestRecoverStripedBlocks.java      |   7 +-
 21 files changed, 431 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 8fc9e77..da3b0e5 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
@@ -2954,6 +2954,21 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return new EncryptionZoneIterator(namenode, traceSampler);
   }
 
+  public void createErasureCodingZone(String src)
+      throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("createErasureCodingZone", src);
+    try {
+      namenode.createErasureCodingZone(src);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class,
+          UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
   public void setXAttr(String src, String name, byte[] value, 
       EnumSet<XAttrSetFlag> flag) throws IOException {
     checkOpen();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 bafb02b..8efe344 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
@@ -1363,6 +1363,14 @@ public interface ClientProtocol {
       long prevId) throws IOException;
 
   /**
+   * Create an erasure coding zone (currently with hardcoded schema)
+   * TODO: Configurable and pluggable schemas (HDFS-7337)
+   */
+  @Idempotent
+  public void createErasureCodingZone(String src)
+      throws IOException;
+
+  /**
    * Set xattr of a file or directory.
    * The name must be prefixed with the namespace followed by ".". For example,
    * "user.attr".

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index ed5f27d..892c5f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -164,7 +164,6 @@ public class HdfsConstants {
   public static final String ONESSD_STORAGE_POLICY_NAME = "ONE_SSD";
   public static final String HOT_STORAGE_POLICY_NAME = "HOT";
   public static final String WARM_STORAGE_POLICY_NAME = "WARM";
-  public static final String EC_STORAGE_POLICY_NAME = "EC";
   public static final String COLD_STORAGE_POLICY_NAME = "COLD";
 
   public static final byte MEMORY_STORAGE_POLICY_ID = 15;
@@ -172,7 +171,6 @@ public class HdfsConstants {
   public static final byte ONESSD_STORAGE_POLICY_ID = 10;
   public static final byte HOT_STORAGE_POLICY_ID = 7;
   public static final byte WARM_STORAGE_POLICY_ID = 5;
-  public static final byte EC_STORAGE_POLICY_ID = 4;
   public static final byte COLD_STORAGE_POLICY_ID = 2;
 
   public static final byte NUM_DATA_BLOCKS = 3;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 a292d8f..2b8215e 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
@@ -192,6 +192,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto;
@@ -1391,6 +1393,18 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
+  public CreateErasureCodingZoneResponseProto createErasureCodingZone(
+      RpcController controller, CreateErasureCodingZoneRequestProto req)
+      throws ServiceException {
+    try {
+      server.createErasureCodingZone(req.getSrc());
+      return CreateErasureCodingZoneResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public SetXAttrResponseProto setXAttr(RpcController controller,
       SetXAttrRequestProto req) throws ServiceException {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 22da083..43a0322 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
@@ -160,6 +160,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Trunca
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
@@ -1407,6 +1409,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
+  public void createErasureCodingZone(String src)
+      throws IOException {
+    final CreateErasureCodingZoneRequestProto.Builder builder =
+        CreateErasureCodingZoneRequestProto.newBuilder();
+    builder.setSrc(src);
+    CreateErasureCodingZoneRequestProto req = builder.build();
+    try {
+      rpcProxy.createErasureCodingZone(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
     SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
index a1c96b9..6c352f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
@@ -77,11 +77,6 @@ public class BlockStoragePolicySuite {
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE});
-    final byte ecId = HdfsConstants.EC_STORAGE_POLICY_ID;
-    policies[ecId] = new BlockStoragePolicy(ecId,
-        HdfsConstants.EC_STORAGE_POLICY_NAME,
-        new StorageType[]{StorageType.DISK}, StorageType.EMPTY_ARRAY,
-        new StorageType[]{StorageType.ARCHIVE});
     final byte coldId = HdfsConstants.COLD_STORAGE_POLICY_ID;
     policies[coldId] = new BlockStoragePolicy(coldId,
         HdfsConstants.COLD_STORAGE_POLICY_NAME,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 2d267ce..8fd0197 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
@@ -327,4 +327,6 @@ public final class HdfsServerConstants {
       "raw.hdfs.crypto.file.encryption.info";
   public static final String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER =
       "security.hdfs.unreadable.by.superuser";
+  public static final String XATTR_ERASURECODING_ZONE =
+      "raw.hdfs.erasurecoding.zone";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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
new file mode 100644
index 0000000..d4ff7c5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.hdfs.XAttrHelper;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_ZONE;
+
+/**
+ * Manages the list of erasure coding zones in the filesystem.
+ * <p/>
+ * The ErasureCodingZoneManager has its own lock, but relies on the FSDirectory
+ * lock being held for many operations. The FSDirectory lock should not be
+ * taken if the manager lock is already held.
+ * TODO: consolidate zone logic w/ encrypt. zones {@link EncryptionZoneManager}
+ */
+public class ErasureCodingZoneManager {
+  private final FSDirectory dir;
+
+  /**
+   * Construct a new ErasureCodingZoneManager.
+   *
+   * @param dir Enclosing FSDirectory
+   */
+  public ErasureCodingZoneManager(FSDirectory dir) {
+    this.dir = dir;
+  }
+
+  boolean getECPolicy(INodesInPath iip) {
+    assert dir.hasReadLock();
+    Preconditions.checkNotNull(iip);
+    List<INode> inodes = iip.getReadOnlyINodes();
+    for (int i = inodes.size() - 1; i >= 0; i--) {
+      final INode inode = inodes.get(i);
+      if (inode == null) {
+        continue;
+      }
+      final List<XAttr> xAttrs = inode.getXAttrFeature() == null ?
+          new ArrayList<XAttr>(0)
+          : inode.getXAttrFeature().getXAttrs();
+      for (XAttr xAttr : xAttrs) {
+        if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  XAttr createErasureCodingZone(String src)
+      throws IOException {
+    assert dir.hasWriteLock();
+    final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false);
+    if (dir.isNonEmptyDirectory(srcIIP)) {
+      throw new IOException(
+          "Attempt to create an erasure coding zone for a " +
+              "non-empty directory.");
+    }
+    if (srcIIP != null &&
+        srcIIP.getLastINode() != null &&
+        !srcIIP.getLastINode().isDirectory()) {
+      throw new IOException("Attempt to create an erasure coding zone " +
+          "for a file.");
+    }
+    if (getECPolicy(srcIIP)) {
+      throw new IOException("Directory " + src + " is already in an " +
+          "erasure coding zone.");
+    }
+    final XAttr ecXAttr = XAttrHelper
+        .buildXAttr(XATTR_ERASURECODING_ZONE, null);
+    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
+    xattrs.add(ecXAttr);
+    FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs,
+        EnumSet.of(XAttrSetFlag.CREATE));
+    return ecXAttr;
+  }
+
+  void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src)
+      throws IOException {
+    assert dir.hasReadLock();
+    if (getECPolicy(srcIIP)
+        != getECPolicy(dstIIP)) {
+      throw new IOException(
+          src + " can't be moved because the source and destination have " +
+              "different erasure coding policies.");
+    }
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 56a20a3..a07ff23 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
@@ -212,6 +212,9 @@ public class FSDirectory implements Closeable {
   @VisibleForTesting
   public final EncryptionZoneManager ezManager;
 
+  @VisibleForTesting
+  public final ErasureCodingZoneManager ecZoneManager;
+
   /**
    * Caches frequently used file names used in {@link INode} to reuse 
    * byte[] objects and reduce heap usage.
@@ -303,6 +306,7 @@ public class FSDirectory implements Closeable {
     namesystem = ns;
     this.editLog = ns.getEditLog();
     ezManager = new EncryptionZoneManager(this, conf);
+    ecZoneManager = new ErasureCodingZoneManager(this);
   }
     
   FSNamesystem getFSNamesystem() {
@@ -426,8 +430,7 @@ public class FSDirectory implements Closeable {
     writeLock();
     try {
       newiip = addINode(existing, newNode);
-      // TODO: we will no longer use storage policy for "Erasure Coding Zone"
-      if (newiip != null && newNode.isStriped()) {
+      if (newiip != null && getECPolicy(newiip)) {
         newNode.addStripedBlocksFeature();
       }
     } finally {
@@ -1397,6 +1400,25 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  XAttr createErasureCodingZone(String src)
+      throws IOException {
+    writeLock();
+    try {
+      return ecZoneManager.createErasureCodingZone(src);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public boolean getECPolicy(INodesInPath iip) {
+    readLock();
+    try {
+      return ecZoneManager.getECPolicy(iip);
+    } finally {
+      readUnlock();
+    }
+  }
+
   static INode resolveLastINode(INodesInPath iip) throws FileNotFoundException {
     INode inode = iip.getLastINode();
     if (inode == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 f1b0eb8..1740365 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
@@ -8111,6 +8111,46 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
+  /**
+   * Create an erasure coding zone on directory src.
+   *
+   * @param src     the path of a directory which will be the root of the
+   *                erasure coding zone. The directory must be empty.
+   * @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 boolean logRetryCache)
+      throws IOException, UnresolvedLinkException,
+      SafeModeException, AccessControlException {
+    String src = srcArg;
+    HdfsFileStatus resultingStat = null;
+    checkSuperuserPrivilege();
+    checkOperation(OperationCategory.WRITE);
+    final byte[][] pathComponents =
+        FSDirectory.getPathComponentsForReservedPath(src);
+    FSPermissionChecker pc = getPermissionChecker();
+    writeLock();
+    try {
+      checkSuperuserPrivilege();
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot create erasure coding zone on " + src);
+      src = dir.resolvePath(pc, src, pathComponents);
+
+      final XAttr ecXAttr = dir.createErasureCodingZone(src);
+      List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
+      xAttrs.add(ecXAttr);
+      getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
+      final INodesInPath iip = dir.getINodesInPath4Write(src, false);
+      resultingStat = dir.getAuditFileInfo(iip);
+    } finally {
+      writeUnlock();
+    }
+    getEditLog().logSync();
+    logAuditEvent(true, "createErasureCodingZone", srcArg, null, resultingStat);
+  }
+
   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
                 boolean logRetryCache)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 807f9b3..deb89d7 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
@@ -419,7 +419,7 @@ public class INodeFile extends INodeWithAdditionalFields
       }
       max = maxInSnapshot > max ? maxInSnapshot : max;
     }
-    return isStriped()?
+    return isStriped() ?
         HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max;
   }
 
@@ -841,7 +841,10 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   public final long storagespaceConsumedWithStriped() {
     BlockInfo[] blockInfos = getBlocks();
-    long size = 0;
+    if (blockInfos == null || blockInfos.length == 0) {
+      return 0;
+    }
+    long size;
     final int last = blockInfos.length - 1;
     if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
       BlockInfoStripedUnderConstruction blockInfoStripedUC
@@ -1100,8 +1103,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   @VisibleForTesting
   @Override
-  // TODO: move erasure coding policy to file XAttr
   public boolean isStriped() {
-    return getStoragePolicyID() == HdfsConstants.EC_STORAGE_POLICY_ID;
+    return getStripedBlocksFeature() != null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 83686e0..09357cb 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
@@ -1843,6 +1843,22 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
+  public void createErasureCodingZone(String src)
+    throws IOException {
+    checkNNStartup();
+    final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return;
+    }
+    boolean success = false;
+    try {
+      namesystem.createErasureCodingZone(src, cacheEntry != null);
+    } finally {
+      RetryCache.setState(cacheEntry, success);
+    }
+  }
+
+  @Override // ClientProtocol
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
     checkNNStartup();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 b44c556..183aff8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -714,6 +714,13 @@ message GetEditsFromTxidResponseProto {
   required EventsListProto eventsList = 1;
 }
 
+message CreateErasureCodingZoneRequestProto {
+  required string src = 1;
+}
+
+message CreateErasureCodingZoneResponseProto {
+}
+
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -856,6 +863,8 @@ service ClientNamenodeProtocol {
       returns(ListEncryptionZonesResponseProto);
   rpc getEZForPath(GetEZForPathRequestProto)
       returns(GetEZForPathResponseProto);
+  rpc createErasureCodingZone(CreateErasureCodingZoneRequestProto)
+      returns(CreateErasureCodingZoneResponseProto);
   rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto)
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index f3cdd7d..89c8e11 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -68,7 +68,6 @@ public class TestBlockStoragePolicy {
   static final short REPLICATION = 3;
 
   static final byte COLD = HdfsConstants.COLD_STORAGE_POLICY_ID;
-  static final byte EC = HdfsConstants.EC_STORAGE_POLICY_ID;
   static final byte WARM = HdfsConstants.WARM_STORAGE_POLICY_ID;
   static final byte HOT  = HdfsConstants.HOT_STORAGE_POLICY_ID;
   static final byte ONESSD  = HdfsConstants.ONESSD_STORAGE_POLICY_ID;
@@ -116,9 +115,6 @@ public class TestBlockStoragePolicy {
     expectedPolicyStrings.put(COLD,
         "BlockStoragePolicy{COLD:" + COLD + ", storageTypes=[ARCHIVE], " +
             "creationFallbacks=[], replicationFallbacks=[]}");
-    expectedPolicyStrings.put(EC,
-        "BlockStoragePolicy{EC:" + EC + ", storageTypes=[DISK], " +
-            "creationFallbacks=[], replicationFallbacks=[ARCHIVE]}");
     expectedPolicyStrings.put(WARM,
         "BlockStoragePolicy{WARM:" + WARM + ", storageTypes=[DISK, ARCHIVE], " +
             "creationFallbacks=[DISK, ARCHIVE], " +
@@ -1163,15 +1159,13 @@ public class TestBlockStoragePolicy {
     final DistributedFileSystem fs = cluster.getFileSystem();
     try {
       BlockStoragePolicy[] policies = fs.getStoragePolicies();
-      Assert.assertEquals(7, policies.length);
+      Assert.assertEquals(6, policies.length);
       Assert.assertEquals(POLICY_SUITE.getPolicy(COLD).toString(),
           policies[0].toString());
-      Assert.assertEquals(POLICY_SUITE.getPolicy(EC).toString(),
-          policies[1].toString());
       Assert.assertEquals(POLICY_SUITE.getPolicy(WARM).toString(),
-          policies[2].toString());
+          policies[1].toString());
       Assert.assertEquals(POLICY_SUITE.getPolicy(HOT).toString(),
-          policies[3].toString());
+          policies[2].toString());
     } finally {
       IOUtils.cleanup(null, fs);
       cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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
new file mode 100644
index 0000000..49f08eef
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+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;
+  private FSNamesystem namesystem;
+
+  @Before
+  public void setupCluster() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    cluster = new MiniDFSCluster.Builder(conf).
+        numDataNodes(NUM_OF_DATANODES).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    namesystem = cluster.getNamesystem();
+  }
+
+  @After
+  public void shutdownCluster() throws IOException {
+    cluster.shutdown();
+  }
+
+  @Test
+  public void testCreateECZone()
+      throws IOException, InterruptedException {
+    final Path testDir = new Path("/ec");
+    fs.mkdir(testDir, FsPermission.getDirDefault());
+
+    /* Normal creation of an erasure coding zone */
+    fs.getClient().createErasureCodingZone(testDir.toString());
+
+    /* Verify files under the zone are striped */
+    final Path ECFilePath = new Path(testDir, "foo");
+    fs.create(ECFilePath);
+    INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString());
+    assertTrue(inode.asFile().isStriped());
+
+    /* Verify that EC zone cannot be created on non-empty dir */
+    final Path notEmpty = new Path("/nonEmpty");
+    fs.mkdir(notEmpty, FsPermission.getDirDefault());
+    fs.create(new Path(notEmpty, "foo"));
+    try {
+      fs.getClient().createErasureCodingZone(notEmpty.toString());
+      fail("Erasure coding zone on non-empty dir");
+    } catch (IOException e) {
+      assertExceptionContains("erasure coding zone for a non-empty directory", e);
+    }
+
+    /* Verify that nested EC zones cannot be created */
+    final Path zone1 = new Path("/zone1");
+    final Path zone2 = new Path(zone1, "zone2");
+    fs.mkdir(zone1, FsPermission.getDirDefault());
+    fs.getClient().createErasureCodingZone(zone1.toString());
+    fs.mkdir(zone2, FsPermission.getDirDefault());
+    try {
+      fs.getClient().createErasureCodingZone(zone2.toString());
+      fail("Nested erasure coding zones");
+    } catch (IOException e) {
+      assertExceptionContains("already in an erasure coding zone", e);
+    }
+
+    /* Verify that EC zone cannot be created on a file */
+    final Path fPath = new Path("/file");
+    fs.create(fPath);
+    try {
+      fs.getClient().createErasureCodingZone(fPath.toString());
+      fail("Erasure coding zone on file");
+    } catch (IOException e) {
+      assertExceptionContains("erasure coding zone for a file", e);
+    }
+  }
+
+  @Test
+  public void testMoveValidity() throws IOException, InterruptedException {
+    final Path srcECDir = new Path("/srcEC");
+    final Path dstECDir = new Path("/dstEC");
+    fs.mkdir(srcECDir, FsPermission.getDirDefault());
+    fs.mkdir(dstECDir, FsPermission.getDirDefault());
+    fs.getClient().createErasureCodingZone(srcECDir.toString());
+    fs.getClient().createErasureCodingZone(dstECDir.toString());
+    final Path srcFile = new Path(srcECDir, "foo");
+    fs.create(srcFile);
+
+    /* Verify that a file can be moved between 2 EC zones */
+    try {
+      fs.rename(srcFile, dstECDir);
+    } catch (IOException e) {
+      fail("A file should be able to move between 2 EC zones " + e);
+    }
+
+    // Move the file back
+    fs.rename(new Path(dstECDir, "foo"), srcECDir);
+
+    /* Verify that a file cannot be moved from a non-EC dir to an EC zone */
+    final Path nonECDir = new Path("/nonEC");
+    fs.mkdir(nonECDir, FsPermission.getDirDefault());
+    try {
+      fs.rename(srcFile, nonECDir);
+      fail("A file shouldn't be able to move from a non-EC dir to an EC zone");
+    } catch (IOException e) {
+      assertExceptionContains("can't be moved because the source and " +
+          "destination have different erasure coding policies", e);
+    }
+
+    /* Verify that a file cannot be moved from an EC zone to a non-EC dir */
+    final Path nonECFile = new Path(nonECDir, "nonECFile");
+    fs.create(nonECFile);
+    try {
+      fs.rename(nonECFile, dstECDir);
+    } catch (IOException e) {
+      assertExceptionContains("can't be moved because the source and " +
+          "destination have different erasure coding policies", e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
deleted file mode 100644
index a84f67b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.*;
-import org.apache.hadoop.hdfs.client.HdfsAdmin;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_ID;
-import static org.junit.Assert.assertEquals;
-
-public class TestBlockInitialEncoding {
-  private final int NUM_OF_DATANODES = 3;
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private DistributedFileSystem fs;
-  private static final int BLOCK_SIZE = 1024;
-  private HdfsAdmin dfsAdmin;
-  private FSNamesystem namesystem;
-
-  @Before
-  public void setupCluster() throws IOException {
-    conf = new HdfsConfiguration();
-    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    cluster = new MiniDFSCluster.Builder(conf).
-        numDataNodes(NUM_OF_DATANODES).build();
-    cluster.waitActive();
-    fs = cluster.getFileSystem();
-    dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
-    namesystem = cluster.getNamesystem();
-  }
-
-  @After
-  public void shutdownCluster() throws IOException {
-    cluster.shutdown();
-  }
-
-  @Test
-  public void testBlockInitialEncoding()
-      throws IOException, InterruptedException {
-    final Path testDir = new Path("/test");
-    fs.mkdir(testDir, FsPermission.getDirDefault());
-    dfsAdmin.setStoragePolicy(testDir, EC_STORAGE_POLICY_NAME);
-    final Path ECFilePath = new Path("/test/foo.ec");
-    DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, (short) 3, 0);
-    INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString());
-    assertEquals(EC_STORAGE_POLICY_ID, inode.getStoragePolicyID());
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 215a4e4..c3c8239 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
@@ -68,7 +68,7 @@ public class TestAddStripedBlocks {
         .numDataNodes(GROUP_SIZE).build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
-    dfs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+    dfs.getClient().createErasureCodingZone("/");
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 d3cb749..407d07e 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
@@ -444,8 +444,7 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.setStoragePolicy(new Path(testDir),
-          HdfsConstants.EC_STORAGE_POLICY_NAME);
+      fs.getClient().getNamenode().createErasureCodingZone(testDir);
 
       // Create a file with striped block
       Path p = new Path(testFilePath);
@@ -517,8 +516,7 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.setStoragePolicy(new Path(testDir),
-          HdfsConstants.EC_STORAGE_POLICY_NAME);
+      fs.getClient().getNamenode().createErasureCodingZone(testDir);
 
       //create a file with striped blocks
       Path p = new Path(testFilePath);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d717dc3e/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 440f5cd..83f01c6 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
@@ -33,18 +33,14 @@ import java.io.IOException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 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;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -59,7 +55,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
@@ -137,9 +132,10 @@ public class TestFSImage {
     }
   }
 
-  private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf,
-      boolean isUC) throws IOException{
+  private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf,
+                                               boolean isUC) throws IOException{
     // contruct a INode with StripedBlock for saving and loading
+    fsn.createErasureCodingZone("/", false);
     long id = 123456789;
     byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
     PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
@@ -149,9 +145,8 @@ public class TestFSImage {
     BlockInfoContiguous[] blks = new BlockInfoContiguous[0];
     short replication = 3;
     long preferredBlockSize = 128*1024*1024;
-    byte storagePolicyID = HdfsConstants.EC_STORAGE_POLICY_ID;
     INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
-        blks, replication, preferredBlockSize, storagePolicyID);
+        blks, replication, preferredBlockSize);
     ByteArrayOutputStream bs = new ByteArrayOutputStream();
     file.addStripedBlocksFeature();
 
@@ -237,13 +232,13 @@ public class TestFSImage {
    * FSImageSerialization and loaded by FSImageFormat#Loader.
    */
   @Test
-  public void testSaveAndLoadInodeFile() throws IOException{
+  public void testSaveAndLoadStripedINodeFile() throws IOException{
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
-      testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, false);
+      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, false);
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -256,14 +251,14 @@ public class TestFSImage {
    * saved and loaded by FSImageSerialization
    */
   @Test
-  public void testSaveAndLoadInodeFileUC() throws IOException{
+  public void testSaveAndLoadStripedINodeFileUC() throws IOException{
     // construct a INode with StripedBlock for saving and loading
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
-      testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, true);
+      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, true);
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -402,7 +397,7 @@ public class TestFSImage {
           .build();
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
-      fs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+      fs.getClient().getNamenode().createErasureCodingZone("/");
       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/d717dc3e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
index b9fd4fe..d965ae7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
@@ -37,23 +36,19 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.List;
 import java.util.UUID;
 
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -93,7 +88,7 @@ public class TestRecoverStripedBlocks {
       int numBlocks) throws Exception {
     DistributedFileSystem dfs = cluster.getFileSystem();
     dfs.mkdirs(dir);
-    dfs.setStoragePolicy(dir, EC_STORAGE_POLICY_NAME);
+    dfs.getClient().getNamenode().createErasureCodingZone(dir.toString());
 
     FSDataOutputStream out = null;
     try {


[13/50] hadoop git commit: HDFS-7782. Erasure coding: pread from files in striped layout. Contributed by Zhe Zhang and Jing Zhao

Posted by ji...@apache.org.
HDFS-7782. Erasure coding: pread from files in striped layout. 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/631916c3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/631916c3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/631916c3

Branch: refs/heads/HDFS-7285
Commit: 631916c34a39f57924a70174180683fcd0440df1
Parents: 593bbfd
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Apr 7 11:20:13 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:14:10 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSClient.java            | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/631916c3/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 ff8bad0..f4eea49 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
@@ -3165,7 +3165,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       LOG.debug("Using hedged reads; pool threads=" + num);
     }
   }
-  
+
   /**
    * Create thread pool for parallel reading in striped layout,
    * STRIPED_READ_THREAD_POOL, if it does not already exist.


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 5e8cea7a540035b73ffbc55e456ed644eb0a199f
Parents: b65f508
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 28 14:14:33 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:17:52 2015 -0700

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


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

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


[14/50] hadoop git commit: HDFS-8104 Make hard-coded values consistent with the system default schema first before remove them. Contributed by Kai Zheng

Posted by ji...@apache.org.
HDFS-8104 Make hard-coded values consistent with the system default schema first before remove them. 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/ae857573
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ae857573
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ae857573

Branch: refs/heads/HDFS-7285
Commit: ae857573dd38f0c689f8bb394c6f9d9d220d3967
Parents: 44aa26a
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Apr 10 00:16:28 2015 +0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:14:10 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   4 +-
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  12 +-
 .../hadoop/hdfs/TestPlanReadPortions.java       | 142 +++++++++++++++++++
 .../apache/hadoop/hdfs/TestReadStripedFile.java | 112 ---------------
 4 files changed, 154 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae857573/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 5078a15..1e695c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -54,4 +54,6 @@
     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)
\ No newline at end of file
+    HDFS-8074. Define a system-wide default EC schema. (Kai Zheng)
+
+    HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae857573/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index 72b8f38..4c67371 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -173,11 +173,17 @@ public class HdfsConstants {
   public static final byte WARM_STORAGE_POLICY_ID = 5;
   public static final byte COLD_STORAGE_POLICY_ID = 2;
 
-  public static final byte NUM_DATA_BLOCKS = 3;
-  public static final byte NUM_PARITY_BLOCKS = 2;
+
   public static final long BLOCK_GROUP_INDEX_MASK = 15;
   public static final byte MAX_BLOCKS_IN_GROUP = 16;
 
+  /*
+   * These values correspond to the values used by the system default 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 = 128 * 1024;
+  public static final int BLOCK_STRIPED_CELL_SIZE = 256 * 1024;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae857573/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
new file mode 100644
index 0000000..cf84b30
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 static org.apache.hadoop.hdfs.DFSStripedInputStream.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 = DFSStripedInputStream.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 DFSStripedInputStream#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/ae857573/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
index 0032bdd..849e12e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
@@ -29,7 +29,6 @@ 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.apache.hadoop.hdfs.DFSStripedInputStream.ReadPortion;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -78,117 +77,6 @@ public class TestReadStripedFile {
     }
   }
 
-  private void testPlanReadPortions(int startInBlk, int length,
-      int bufferOffset, int[] readLengths, int[] offsetsInBlock,
-      int[][] bufferOffsets, int[][] bufferLengths) {
-    ReadPortion[] results = DFSStripedInputStream.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 DFSStripedInputStream#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}});
-  }
-
   private LocatedStripedBlock createDummyLocatedBlock() {
     final long blockGroupID = -1048576;
     DatanodeInfo[] locs = new DatanodeInfo[TOTAL_SIZE];


[18/50] hadoop git commit: HDFS-7889 Subclass DFSOutputStream to support writing striping layout files. Contributed by Li Bo

Posted by ji...@apache.org.
HDFS-7889 Subclass DFSOutputStream to support writing striping layout files. 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/1ce17bc3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1ce17bc3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1ce17bc3

Branch: refs/heads/HDFS-7285
Commit: 1ce17bc366f0f8553f3bc0df606a5f4fe5e87083
Parents: 19f5d1f
Author: Kai Zheng <ka...@intel.com>
Authored: Sat Apr 11 01:03:37 2015 +0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:15:35 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   4 +-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  13 +-
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  26 +-
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 439 +++++++++++++++++++
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  11 +-
 .../apache/hadoop/hdfs/StripedDataStreamer.java | 241 ++++++++++
 .../hadoop/hdfs/TestDFSStripedOutputStream.java | 311 +++++++++++++
 7 files changed, 1031 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ce17bc3/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 1e695c4..753795a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -56,4 +56,6 @@
     
     HDFS-8074. Define a system-wide default EC schema. (Kai Zheng)
 
-    HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng)
\ No newline at end of file
+    HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng)
+
+    HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ce17bc3/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 4646b60..99d48b2 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
@@ -266,8 +266,14 @@ public class DFSOutputStream extends FSOutputSummer
         }
       }
       Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!");
-      final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
-          flag, progress, checksum, favoredNodes);
+      final DFSOutputStream out;
+      if(stat.getReplication() == 0) {
+        out = new DFSStripedOutputStream(dfsClient, src, stat,
+            flag, progress, checksum, favoredNodes);
+      } else {
+        out = new DFSOutputStream(dfsClient, src, stat,
+            flag, progress, checksum, favoredNodes);
+      }
       out.start();
       return out;
     } finally {
@@ -347,6 +353,9 @@ public class DFSOutputStream extends FSOutputSummer
       String[] favoredNodes) throws IOException {
     TraceScope scope =
         dfsClient.getPathTraceScope("newStreamForAppend", src);
+	if(stat.getReplication() == 0) {
+      throw new IOException("Not support appending to a striping layout file yet.");
+    }
     try {
       final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
           progress, lastBlock, stat, checksum, favoredNodes);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ce17bc3/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 22055c3..9cd1ec1 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
 import java.nio.channels.ClosedChannelException;
 import java.util.Arrays;
 
@@ -113,6 +114,19 @@ class DFSPacket {
     dataPos += len;
   }
 
+  synchronized void writeData(ByteBuffer inBuffer, int len)
+      throws ClosedChannelException {
+    checkBuffer();
+    len =  len > inBuffer.remaining() ? inBuffer.remaining() : len;
+    if (dataPos + len > buf.length) {
+      throw new BufferOverflowException();
+    }
+    for (int i = 0; i < len; i++) {
+      buf[dataPos + i] = inBuffer.get();
+    }
+    dataPos += len;
+  }
+
   /**
    * Write checksums to this packet
    *
@@ -222,7 +236,7 @@ class DFSPacket {
    *
    * @return true if the packet is the last packet
    */
-  boolean isLastPacketInBlock(){
+  boolean isLastPacketInBlock() {
     return lastPacketInBlock;
   }
 
@@ -231,7 +245,7 @@ class DFSPacket {
    *
    * @return the sequence number of this packet
    */
-  long getSeqno(){
+  long getSeqno() {
     return seqno;
   }
 
@@ -240,14 +254,14 @@ class DFSPacket {
    *
    * @return the number of chunks in this packet
    */
-  synchronized int getNumChunks(){
+  synchronized int getNumChunks() {
     return numChunks;
   }
 
   /**
    * increase the number of chunks by one
    */
-  synchronized void incNumChunks(){
+  synchronized void incNumChunks() {
     numChunks++;
   }
 
@@ -256,7 +270,7 @@ class DFSPacket {
    *
    * @return the maximum number of packets
    */
-  int getMaxChunks(){
+  int getMaxChunks() {
     return maxChunks;
   }
 
@@ -265,7 +279,7 @@ class DFSPacket {
    *
    * @param syncBlock if to sync block
    */
-  synchronized void setSyncBlock(boolean syncBlock){
+  synchronized void setSyncBlock(boolean syncBlock) {
     this.syncBlock = syncBlock;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ce17bc3/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
new file mode 100644
index 0000000..aded4fe
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -0,0 +1,439 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Progressable;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+
+/****************************************************************
+ * The DFSStripedOutputStream class supports writing files in striped
+ * layout. Each stripe contains a sequence of cells and multiple
+ * {@link StripedDataStreamer}s in DFSStripedOutputStream are responsible
+ * for writing the cells to different datanodes.
+ *
+ ****************************************************************/
+
+@InterfaceAudience.Private
+public class DFSStripedOutputStream extends DFSOutputStream {
+
+  private final List<StripedDataStreamer> streamers;
+  /**
+   * Size of each striping cell, must be a multiple of bytesPerChecksum
+   */
+  private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private ByteBuffer[] cellBuffers;
+  private final short blockGroupBlocks = HdfsConstants.NUM_DATA_BLOCKS
+      + HdfsConstants.NUM_PARITY_BLOCKS;
+  private final short blockGroupDataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
+  private int curIdx = 0;
+  /* bytes written in current block group */
+  private long currentBlockGroupBytes = 0;
+
+  //TODO: Use ErasureCoder interface (HDFS-7781)
+  private RawErasureEncoder encoder;
+
+  private StripedDataStreamer getLeadingStreamer() {
+    return streamers.get(0);
+  }
+
+  private long getBlockGroupSize() {
+    return blockSize * HdfsConstants.NUM_DATA_BLOCKS;
+  }
+
+  /** Construct a new output stream for creating a file. */
+  DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
+                         EnumSet<CreateFlag> flag, Progressable progress,
+                         DataChecksum checksum, String[] favoredNodes)
+                         throws IOException {
+    super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
+    DFSClient.LOG.info("Creating striped output stream");
+    if (blockGroupBlocks <= 1) {
+      throw new IOException("The block group must contain more than one block.");
+    }
+
+    cellBuffers = new ByteBuffer[blockGroupBlocks];
+    List<BlockingQueue<LocatedBlock>> stripeBlocks = new ArrayList<>();
+
+    for (int i = 0; i < blockGroupBlocks; i++) {
+      stripeBlocks.add(new LinkedBlockingQueue<LocatedBlock>(blockGroupBlocks));
+      try {
+        cellBuffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
+      } catch (InterruptedException ie) {
+        final InterruptedIOException iioe = new InterruptedIOException(
+            "create cell buffers");
+        iioe.initCause(ie);
+        throw iioe;
+      }
+    }
+    encoder = new RSRawEncoder();
+    encoder.initialize(blockGroupDataBlocks,
+        blockGroupBlocks - blockGroupDataBlocks, cellSize);
+
+    streamers = new ArrayList<>(blockGroupBlocks);
+    for (short i = 0; i < blockGroupBlocks; i++) {
+      StripedDataStreamer streamer = new StripedDataStreamer(stat, null,
+          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
+          i, stripeBlocks);
+      if (favoredNodes != null && favoredNodes.length != 0) {
+        streamer.setFavoredNodes(favoredNodes);
+      }
+      streamers.add(streamer);
+    }
+
+    refreshStreamer();
+  }
+
+  private void refreshStreamer() {
+    streamer = streamers.get(curIdx);
+  }
+
+  private void moveToNextStreamer() {
+    curIdx = (curIdx + 1) % blockGroupBlocks;
+    refreshStreamer();
+  }
+
+  /**
+   * encode the buffers.
+   * After encoding, flip each buffer.
+   *
+   * @param buffers data buffers + parity buffers
+   */
+  private void encode(ByteBuffer[] buffers) {
+    ByteBuffer[] dataBuffers = new ByteBuffer[blockGroupDataBlocks];
+    ByteBuffer[] parityBuffers = new ByteBuffer[blockGroupBlocks - blockGroupDataBlocks];
+    for (int i = 0; i < blockGroupBlocks; i++) {
+      if (i < blockGroupDataBlocks) {
+        dataBuffers[i] = buffers[i];
+      } else {
+        parityBuffers[i - blockGroupDataBlocks] = buffers[i];
+      }
+    }
+    encoder.encode(dataBuffers, parityBuffers);
+  }
+
+  /**
+   * Generate packets from a given buffer
+   *
+   * @param byteBuffer the given buffer to generate packets
+   * @return packets generated
+   * @throws IOException
+   */
+  private List<DFSPacket> generatePackets(ByteBuffer byteBuffer)
+      throws IOException{
+    List<DFSPacket> packets = new ArrayList<>();
+    while (byteBuffer.remaining() > 0) {
+      DFSPacket p = createPacket(packetSize, chunksPerPacket,
+          streamer.getBytesCurBlock(),
+          streamer.getAndIncCurrentSeqno(), false);
+      int maxBytesToPacket = p.getMaxChunks() * bytesPerChecksum;
+      int toWrite = byteBuffer.remaining() > maxBytesToPacket ?
+          maxBytesToPacket: byteBuffer.remaining();
+      p.writeData(byteBuffer, toWrite);
+      streamer.incBytesCurBlock(toWrite);
+      packets.add(p);
+    }
+    return packets;
+  }
+
+  @Override
+  protected synchronized void writeChunk(byte[] b, 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());
+      }
+      streamer.waitAndQueuePacket(currentPacket);
+      currentPacket = null;
+      adjustChunkBoundary();
+      endBlock();
+    }
+
+    // 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();
+      //When all data cells in a stripe are ready, we need to encode
+      //them and generate some parity cells. These cells will be
+      //converted to packets and put to their DataStreamer's queue.
+      if (curIdx == blockGroupDataBlocks) {
+        //encode the data cells
+        for (int k = 0; k < blockGroupDataBlocks; k++) {
+          cellBuffers[k].flip();
+        }
+        encode(cellBuffers);
+        for (int i = blockGroupDataBlocks; i < blockGroupBlocks; 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< blockGroupBlocks; i++) {
+      cellBuffers[i].clear();
+    }
+  }
+
+  private int stripeDataSize() {
+    return blockGroupDataBlocks * 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");
+  }
+
+  @Override
+  public void hsync() throws IOException {
+    notSupported("hsync");
+  }
+
+
+  @Override
+  protected synchronized void start() {
+    for (StripedDataStreamer streamer : streamers) {
+      streamer.start();
+    }
+  }
+
+  @Override
+  synchronized void abort() throws IOException {
+    if (isClosed()) {
+      return;
+    }
+    for (StripedDataStreamer streamer : streamers) {
+      streamer.setLastException(new IOException("Lease timeout of "
+          + (dfsClient.getHdfsTimeout()/1000) + " seconds expired."));
+    }
+    closeThreads(true);
+    dfsClient.endFileLease(fileId);
+  }
+
+  //TODO: Handle slow writers (HDFS-7786)
+  //Cuurently only check if the leading streamer is terminated
+  boolean isClosed() {
+    return closed || getLeadingStreamer().streamerClosed();
+  }
+
+  // shutdown datastreamer and responseprocessor threads.
+  // interrupt datastreamer if force is true
+  @Override
+  protected void closeThreads(boolean force) throws IOException {
+    StripedDataStreamer leadingStreamer = null;
+    for (StripedDataStreamer streamer : streamers) {
+      try {
+        streamer.close(force);
+        streamer.join();
+        streamer.closeSocket();
+        if (streamer.isLeadingStreamer()) {
+          leadingStreamer = streamer;
+        } else {
+          streamer.countTailingBlockGroupBytes();
+        }
+
+      } catch (InterruptedException e) {
+        throw new IOException("Failed to shutdown streamer");
+      } finally {
+        streamer.setSocketToNull();
+        setClosed();
+      }
+    }
+    leadingStreamer.countTailingBlockGroupBytes();
+  }
+
+  @Override
+  public synchronized void write(int b) throws IOException {
+    super.write(b);
+    currentBlockGroupBytes = (currentBlockGroupBytes + 1) % getBlockGroupSize();
+  }
+
+  @Override
+  public synchronized void write(byte b[], int off, int len)
+      throws IOException {
+    super.write(b, off, len);
+    currentBlockGroupBytes = (currentBlockGroupBytes + len) % getBlockGroupSize();
+  }
+
+  private void writeParityCellsForLastStripe() throws IOException{
+    if(currentBlockGroupBytes == 0 ||
+        currentBlockGroupBytes % stripeDataSize() == 0)
+      return;
+    int lastStripeLen =(int)(currentBlockGroupBytes % stripeDataSize());
+    // Size of parity cells should equal the size of the first cell, if it
+    // is not full.
+    int parityCellSize = cellSize;
+    int index = lastStripeLen / cellSize;
+    if (lastStripeLen < cellSize) {
+      parityCellSize = lastStripeLen;
+      index++;
+    }
+    for (int i = 0; i < blockGroupBlocks; i++) {
+      if (i >= index) {
+        int position = cellBuffers[i].position();
+        for (int j = 0; j < parityCellSize - position; j++) {
+          cellBuffers[i].put((byte)0);
+        }
+      }
+      cellBuffers[i].flip();
+    }
+    encode(cellBuffers);
+
+    //write parity cells
+    curIdx = blockGroupDataBlocks;
+    refreshStreamer();
+    for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) {
+      ByteBuffer parityBuffer = cellBuffers[i];
+      List<DFSPacket> packets = generatePackets(parityBuffer);
+      for (DFSPacket p : packets) {
+        currentPacket = p;
+        streamer.waitAndQueuePacket(currentPacket);
+        currentPacket = null;
+      }
+      endBlock();
+      moveToNextStreamer();
+    }
+
+    clearCellBuffers();
+  }
+
+  @Override
+  void setClosed() {
+    super.setClosed();
+    for (int i = 0; i < blockGroupBlocks; i++) {
+      byteArrayManager.release(cellBuffers[i].array());
+      streamers.get(i).release();
+    }
+  }
+
+  @Override
+  protected synchronized void closeImpl() throws IOException {
+    if (isClosed()) {
+      IOException e = getLeadingStreamer().getLastException().getAndSet(null);
+      if (e == null)
+        return;
+      else
+        throw e;
+    }
+
+    try {
+      // flush from all upper layers
+      flushBuffer();
+      if (currentPacket != null) {
+        streamer.waitAndQueuePacket(currentPacket);
+        currentPacket = null;
+      }
+      //if the last stripe is incomplete, generate and write parity cells
+      writeParityCellsForLastStripe();
+
+      for (int i = 0; i < blockGroupBlocks; i++) {
+        curIdx = i;
+        refreshStreamer();
+        if (streamer.getBytesCurBlock()!= 0 ||
+            currentBlockGroupBytes < getBlockGroupSize()) {
+          // send an empty packet to mark the end of the block
+          currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
+              streamer.getAndIncCurrentSeqno(), true);
+          currentPacket.setSyncBlock(shouldSyncBlock);
+        }
+        // flush all data to Datanode
+        flushInternal();
+      }
+
+      // get last block before destroying the streamer
+      ExtendedBlock lastBlock = streamers.get(0).getBlock();
+      closeThreads(false);
+      TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
+      try {
+        completeFile(lastBlock);
+      } finally {
+        scope.close();
+      }
+      dfsClient.endFileLease(fileId);
+    } catch (ClosedChannelException e) {
+    } finally {
+      setClosed();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ce17bc3/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 3727d20..5ce0f98 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
@@ -203,7 +203,7 @@ class DataStreamer extends Daemon {
   }
 
   private volatile boolean streamerClosed = false;
-  private ExtendedBlock block; // its length is number of bytes acked
+  protected ExtendedBlock block; // its length is number of bytes acked
   private Token<BlockTokenIdentifier> accessToken;
   private DataOutputStream blockStream;
   private DataInputStream blockReplyStream;
@@ -211,6 +211,7 @@ class DataStreamer extends Daemon {
   private volatile DatanodeInfo[] nodes = null; // list of targets for current block
   private volatile StorageType[] storageTypes = null;
   private volatile String[] storageIDs = null;
+  protected String[] favoredNodes;
   volatile boolean hasError = false;
   volatile int errorIndex = -1;
   // Restarting node index
@@ -237,12 +238,12 @@ class DataStreamer extends Daemon {
   private final LastException lastException = new LastException();
   private Socket s;
 
-  private final DFSClient dfsClient;
-  private final String src;
+  protected final DFSClient dfsClient;
+  protected final String src;
   /** Only for DataTransferProtocol.writeBlock(..) */
   private final DataChecksum checksum4WriteBlock;
   private final Progressable progress;
-  private final HdfsFileStatus stat;
+  protected final HdfsFileStatus stat;
   // appending to existing partial block
   private volatile boolean appendChunk = false;
   // both dataQueue and ackQueue are protected by dataQueue lock
@@ -358,7 +359,7 @@ class DataStreamer extends Daemon {
     stage = BlockConstructionStage.DATA_STREAMING;
   }
 
-  private void endBlock() {
+  protected void endBlock() {
     if(LOG.isDebugEnabled()) {
       LOG.debug("Closing old block " + block);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ce17bc3/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
new file mode 100644
index 0000000..710d92d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -0,0 +1,241 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.List;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.hadoop.security.token.Token;
+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;
+
+/****************************************************************************
+ * The StripedDataStreamer class is used by {@link DFSStripedOutputStream}.
+ * There are two kinds of StripedDataStreamer, leading streamer and ordinary
+ * stream. Leading streamer requests a block group from NameNode, unwraps
+ * it to located blocks and transfers each located block to its corresponding
+ * ordinary streamer via a blocking queue.
+ *
+ ****************************************************************************/
+public class StripedDataStreamer extends DataStreamer {
+  private final short index;
+  private final  List<BlockingQueue<LocatedBlock>> stripedBlocks;
+  private static short blockGroupSize = HdfsConstants.NUM_DATA_BLOCKS
+      + HdfsConstants.NUM_PARITY_BLOCKS;
+  private boolean hasCommittedBlock = false;
+
+  StripedDataStreamer(HdfsFileStatus stat, ExtendedBlock block,
+                      DFSClient dfsClient, String src,
+                      Progressable progress, DataChecksum checksum,
+                      AtomicReference<CachingStrategy> cachingStrategy,
+                      ByteArrayManager byteArrayManage, short index,
+                      List<BlockingQueue<LocatedBlock>> stripedBlocks) {
+    super(stat,block, dfsClient, src, progress, checksum, cachingStrategy,
+        byteArrayManage);
+    this.index = index;
+    this.stripedBlocks = stripedBlocks;
+  }
+
+  /**
+   * 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;
+  }
+
+  public boolean isLeadingStreamer () {
+    return index == 0;
+  }
+
+  private boolean isParityStreamer() {
+    return index >= HdfsConstants.NUM_DATA_BLOCKS;
+  }
+
+  @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)
+      }
+    }
+    super.endBlock();
+  }
+
+  /**
+   * This function is called after the streamer is closed.
+   */
+  void countTailingBlockGroupBytes () throws IOException {
+    if (isLeadingStreamer()) {
+      //when committing a block group, leading streamer has to adjust
+      // {@link block} including the size of block group
+      for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) {
+        try {
+          LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30,
+              TimeUnit.SECONDS);
+          if (finishedLocatedBlock == null) {
+            throw new IOException("Fail to get finished LocatedBlock " +
+                "from streamer, i=" + i);
+          }
+          ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock();
+          long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes();
+          if (block != null) {
+            block.setNumBytes(block.getNumBytes() + bytes);
+          }
+        } catch (InterruptedException ie) {
+          DFSClient.LOG.info("InterruptedException received when " +
+              "putting a block to stripeBlocks, ie = " + ie);
+        }
+      }
+    } else if (!isParityStreamer()) {
+      if (block == null || block.getNumBytes() == 0) {
+        LocatedBlock finishedBlock = new LocatedBlock(null, null);
+        try {
+          boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30,
+              TimeUnit.SECONDS);
+        } catch (InterruptedException ie) {
+          //TODO: Handle InterruptedException (HDFS-7786)
+          ie.printStackTrace();
+        }
+      }
+    }
+
+  }
+
+  @Override
+  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} including the size of block group
+        for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) {
+          try {
+            LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30,
+                TimeUnit.SECONDS);
+            if (finishedLocatedBlock == null) {
+              throw new IOException("Fail to get finished LocatedBlock " +
+                  "from streamer, i=" + i);
+            }
+            ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock();
+            long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes();
+            if(block != null) {
+              block.setNumBytes(block.getNumBytes() + bytes);
+            }
+          } catch (InterruptedException ie) {
+            DFSClient.LOG.info("InterruptedException received when putting" +
+                " a block to stripeBlocks, ie = " + ie);
+          }
+        }
+      }
+
+      lb = super.locateFollowingBlock(excludedNodes);
+      hasCommittedBlock = true;
+      LocatedBlock[] blocks = unwrapBlockGroup(lb);
+      assert blocks.length == blockGroupSize :
+          "Fail to get block group from namenode: blockGroupSize: " +
+              blockGroupSize + ", 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);
+      }
+    }
+    return lb;
+  }
+
+  /**
+   * Generate other blocks in a block group according to the first one.
+   *
+   * @param firstBlockInGroup the first block in a block group
+   * @return  other blocks in this group
+   */
+  public static LocatedBlock[] unwrapBlockGroup(
+      final LocatedBlock firstBlockInGroup) {
+    ExtendedBlock eb = firstBlockInGroup.getBlock();
+    DatanodeInfo[] locs = firstBlockInGroup.getLocations();
+    String[] storageIDs = firstBlockInGroup.getStorageIDs();
+    StorageType[] storageTypes = firstBlockInGroup.getStorageTypes();
+    Token<BlockTokenIdentifier> blockToken = firstBlockInGroup.getBlockToken();
+    LocatedBlock[] blocksInGroup = new LocatedBlock[locs.length];
+    for (int i = 0; i < blocksInGroup.length; i++) {
+      //each block in a group has the same number of bytes and timestamp
+      ExtendedBlock extendedBlock = new ExtendedBlock(eb.getBlockPoolId(),
+          eb.getBlockId() + i, eb.getNumBytes(), eb.getGenerationStamp());
+      blocksInGroup[i] = new LocatedBlock(extendedBlock,
+          new DatanodeInfo[] {locs[i]}, new String[]{storageIDs[i]},
+          new StorageType[] {storageTypes[i]});
+      blocksInGroup[i].setBlockToken(blockToken);
+    }
+    return blocksInGroup;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ce17bc3/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
new file mode 100644
index 0000000..f5a37f3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -0,0 +1,311 @@
+package org.apache.hadoop.hdfs;
+
+import java.util.Arrays;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.net.TcpPeerServer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TestDFSStripedOutputStream {
+  private int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
+  private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
+
+  private MiniDFSCluster cluster;
+  private Configuration conf = new Configuration();
+  private DistributedFileSystem fs;
+  int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  int blockSize = 8 * 1024 * 1024;
+  int cellsInBlock = blockSize / cellSize;
+  private int mod = 29;
+
+  @Before
+  public void setup() throws IOException {
+    int numDNs = dataBlocks + parityBlocks + 2;
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, cellsInBlock * cellSize);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.getFileSystem().getClient().createErasureCodingZone("/");
+    fs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void TestFileEmpty() throws IOException {
+    testOneFile("/EmptyFile", 0);
+  }
+
+  @Test
+  public void TestFileSmallerThanOneCell1() throws IOException {
+    testOneFile("/SmallerThanOneCell", 1);
+  }
+
+  @Test
+  public void TestFileSmallerThanOneCell2() throws IOException {
+    testOneFile("/SmallerThanOneCell", cellSize - 1);
+  }
+
+  @Test
+  public void TestFileEqualsWithOneCell() throws IOException {
+    testOneFile("/EqualsWithOneCell", cellSize);
+  }
+
+  @Test
+  public void TestFileSmallerThanOneStripe1() throws IOException {
+    testOneFile("/SmallerThanOneStripe", cellSize * dataBlocks - 1);
+  }
+
+  @Test
+  public void TestFileSmallerThanOneStripe2() throws IOException {
+    testOneFile("/SmallerThanOneStripe", cellSize + 123);
+  }
+
+  @Test
+  public void TestFileEqualsWithOneStripe() throws IOException {
+    testOneFile("/EqualsWithOneStripe", cellSize * dataBlocks);
+  }
+
+  @Test
+  public void TestFileMoreThanOneStripe1() throws IOException {
+    testOneFile("/MoreThanOneStripe1", cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void TestFileMoreThanOneStripe2() throws IOException {
+    testOneFile("/MoreThanOneStripe2",
+        cellSize * dataBlocks * (cellsInBlock >= 2 ? cellsInBlock / 2 : 1)
+            + cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void TestFileFullBlockGroup() throws IOException {
+    testOneFile("/FullBlockGroup", blockSize * dataBlocks);
+  }
+
+  //TODO: The following tests will pass after HDFS-8121 fixed
+//  @Test
+  public void TestFileMoreThanABlockGroup1() throws IOException {
+    testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123);
+  }
+
+  //  @Test
+  public void TestFileMoreThanABlockGroup2() throws IOException {
+    testOneFile("/MoreThanABlockGroup2",
+        blockSize * dataBlocks * 3
+            + (cellsInBlock >= 2 ? cellsInBlock / 2 : 1) * cellSize * dataBlocks
+            + 123);
+  }
+
+  private int stripeDataSize() {
+    return cellSize * dataBlocks;
+  }
+
+  private byte[] generateBytes(int cnt) {
+    byte[] bytes = new byte[cnt];
+    for (int i = 0; i < cnt; i++) {
+      bytes[i] = getByte(i);
+    }
+    return bytes;
+  }
+
+  private byte getByte(long pos) {
+    return (byte) (pos % mod + 1);
+  }
+
+  private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes)
+      throws IOException {
+    Path TestPath = new Path(src);
+    byte[] bytes = generateBytes(writeBytes);
+    DFSTestUtil.writeFile(fs, TestPath, new String(bytes));
+
+    //check file length
+    FileStatus status = fs.getFileStatus(TestPath);
+    long fileLength = status.getLen();
+    if (fileLength != writeBytes) {
+      Assert.fail("File Length error: expect=" + writeBytes
+          + ", actual=" + fileLength);
+    }
+
+    DFSStripedInputStream dis = new DFSStripedInputStream(
+        fs.getClient(), src, true);
+    byte[] buf = new byte[writeBytes + 100];
+    int readLen = dis.read(0, buf, 0, buf.length);
+    readLen = readLen >= 0 ? readLen : 0;
+    if (readLen != writeBytes) {
+      Assert.fail("The length of file is not correct.");
+    }
+
+    for (int i = 0; i < writeBytes; i++) {
+      if (getByte(i) != buf[i]) {
+        Assert.fail("Byte at i = " + i + " is wrongly written.");
+      }
+    }
+  }
+
+  private void testOneFile(String src, int writeBytes)
+      throws IOException {
+    Path TestPath = new Path(src);
+
+    int allBlocks = dataBlocks + parityBlocks;
+    byte[] bytes = generateBytes(writeBytes);
+    DFSTestUtil.writeFile(fs, TestPath, new String(bytes));
+
+    //check file length
+    FileStatus status = fs.getFileStatus(TestPath);
+    long fileLength = status.getLen();
+    if (fileLength != writeBytes) {
+      Assert.fail("File Length error: expect=" + writeBytes
+          + ", actual=" + fileLength);
+    }
+
+    List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
+    LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L);
+
+    for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
+      LocatedBlock[] blocks = StripedDataStreamer.unwrapBlockGroup(firstBlock);
+      List<LocatedBlock> oneGroup = Arrays.asList(blocks);
+      blockGroupList.add(oneGroup);
+    }
+
+    //test each block group
+    for (int group = 0; group < blockGroupList.size(); group++) {
+      //get the data of this block
+      List<LocatedBlock> blockList = blockGroupList.get(group);
+      byte[][] dataBlockBytes = new byte[dataBlocks][];
+      byte[][] parityBlockBytes = new byte[allBlocks - dataBlocks][];
+
+      //calculate the size of this block group
+      int lenOfBlockGroup = group < blockGroupList.size() - 1 ?
+          blockSize * dataBlocks :
+          writeBytes - blockSize * (blockGroupList.size() - 1) * dataBlocks;
+      int intactStripes = lenOfBlockGroup / stripeDataSize();
+      int lastStripeLen = lenOfBlockGroup % stripeDataSize();
+
+      //for each block, use BlockReader to read data
+      for (int i = 0; i < blockList.size(); i++) {
+        LocatedBlock lblock = blockList.get(i);
+        if (lblock == null) {
+          continue;
+        }
+        DatanodeInfo[] nodes = lblock.getLocations();
+        ExtendedBlock block = lblock.getBlock();
+        InetSocketAddress targetAddr = NetUtils.createSocketAddr(
+            nodes[0].getXferAddr());
+
+        int lenOfCell = cellSize;
+        if (i == lastStripeLen / cellSize) {
+          lenOfCell = lastStripeLen % cellSize;
+        } else if (i > lastStripeLen / cellSize) {
+          lenOfCell = 0;
+        }
+        int lenOfBlock = cellSize * intactStripes + lenOfCell;
+        byte[] blockBytes = new byte[lenOfBlock];
+        if (i < dataBlocks) {
+          dataBlockBytes[i] = blockBytes;
+        } else {
+          parityBlockBytes[i - dataBlocks] = blockBytes;
+        }
+
+        if (lenOfBlock == 0) {
+          continue;
+        }
+
+        block.setNumBytes(lenOfBlock);
+        BlockReader blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)).
+            setFileName(src).
+            setBlock(block).
+            setBlockToken(lblock.getBlockToken()).
+            setInetSocketAddress(targetAddr).
+            setStartOffset(0).
+            setLength(block.getNumBytes()).
+            setVerifyChecksum(true).
+            setClientName("TestStripeLayoutWrite").
+            setDatanodeInfo(nodes[0]).
+            setCachingStrategy(CachingStrategy.newDefaultStrategy()).
+            setClientCacheContext(ClientContext.getFromConf(conf)).
+            setConfiguration(conf).
+            setRemotePeerFactory(new RemotePeerFactory() {
+              @Override
+              public Peer newConnectedPeer(InetSocketAddress addr,
+                                           Token<BlockTokenIdentifier> blockToken,
+                                           DatanodeID datanodeId)
+                  throws IOException {
+                Peer peer = null;
+                Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
+                try {
+                  sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
+                  sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
+                  peer = TcpPeerServer.peerFromSocket(sock);
+                } finally {
+                  if (peer == null) {
+                    IOUtils.closeSocket(sock);
+                  }
+                }
+                return peer;
+              }
+            }).build();
+
+        blockReader.readAll(blockBytes, 0, lenOfBlock);
+        blockReader.close();
+      }
+
+      //check if we write the data correctly
+      for (int i = 0; i < dataBlockBytes.length; i++) {
+        byte[] cells = dataBlockBytes[i];
+        if (cells == null) {
+          continue;
+        }
+        for (int j = 0; j < cells.length; j++) {
+          byte expected;
+          //calculate the postion of this byte in the file
+          long pos = group * dataBlocks * blockSize
+              + (i * cellSize + j / cellSize * cellSize * dataBlocks)
+              + j % cellSize;
+          if (pos >= writeBytes) {
+            expected = 0;
+          } else {
+            expected = getByte(pos);
+          }
+
+          if (expected != cells[j]) {
+            Assert.fail("Unexpected byte " + cells[j] + ", expect " + expected
+                + ". Block group index is " + group +
+                ", stripe index is " + j / cellSize +
+                ", cell index is " + i + ", byte index is " + j % cellSize);
+          }
+        }
+      }
+    }
+  }
+
+}


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

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


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

Branch: refs/heads/HDFS-7285
Commit: d27f5dce8f7cfc7eff009817de65c1acdcf254ca
Parents: 186f139
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Apr 15 12:59:27 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:53 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d27f5dce/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 9104f84..16250dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1148,9 +1148,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         for (int i = 0; i < offsets.length; i++) {
           int nread = reader.readAll(buf, offsets[i], lengths[i]);
           updateReadStatistics(readStatistics, nread, reader);
-          if (nread != len) {
+          if (nread != lengths[i]) {
             throw new IOException("truncated return from reader.read(): " +
-                "excpected " + len + ", got " + nread);
+                "excpected " + lengths[i] + ", got " + nread);
           }
         }
         DFSClientFaultInjector.get().readFromDatanodeDelay();

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d27f5dce/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 1d0e1be..f11a657 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.util.DataChecksum;
@@ -309,10 +310,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
         streamer.closeSocket();
         if (streamer.isLeadingStreamer()) {
           leadingStreamer = streamer;
-        } else {
-          streamer.countTailingBlockGroupBytes();
         }
-
       } catch (InterruptedException e) {
         throw new IOException("Failed to shutdown streamer");
       } finally {
@@ -320,6 +318,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
         setClosed();
       }
     }
+    assert leadingStreamer != null : "One streamer should be leader";
     leadingStreamer.countTailingBlockGroupBytes();
   }
 
@@ -337,23 +336,28 @@ public class DFSStripedOutputStream extends DFSOutputStream {
   }
 
   private void writeParityCellsForLastStripe() throws IOException{
-    if(currentBlockGroupBytes == 0 ||
-        currentBlockGroupBytes % stripeDataSize() == 0)
+    long parityBlkSize = StripedBlockUtil.getInternalBlockLength(
+        currentBlockGroupBytes, cellSize, blockGroupDataBlocks,
+        blockGroupDataBlocks + 1);
+    if (parityBlkSize == 0 || currentBlockGroupBytes % stripeDataSize() == 0) {
       return;
-    int lastStripeLen =(int)(currentBlockGroupBytes % stripeDataSize());
-    // Size of parity cells should equal the size of the first cell, if it
-    // is not full.
-    int parityCellSize = cellSize;
-    int index = lastStripeLen / cellSize;
-    if (lastStripeLen < cellSize) {
-      parityCellSize = lastStripeLen;
-      index++;
     }
+    int parityCellSize = parityBlkSize % cellSize == 0 ? cellSize :
+                        (int) (parityBlkSize % cellSize);
+
     for (int i = 0; i < blockGroupBlocks; i++) {
-      if (i >= index) {
+      long internalBlkLen = StripedBlockUtil.getInternalBlockLength(
+          currentBlockGroupBytes, cellSize, blockGroupDataBlocks, i);
+      // Pad zero bytes to make all cells exactly the size of parityCellSize
+      // If internal block is smaller than parity block, pad zero bytes.
+      // Also pad zero bytes to all parity cells
+      if (internalBlkLen < parityBlkSize || i >= blockGroupDataBlocks) {
         int position = cellBuffers[i].position();
+        assert position <= parityCellSize : "If an internal block is smaller" +
+            " than parity block, then its last cell should be small than last" +
+            " parity cell";
         for (int j = 0; j < parityCellSize - position; j++) {
-          cellBuffers[i].put((byte)0);
+          cellBuffers[i].put((byte) 0);
         }
       }
       cellBuffers[i].flip();

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d27f5dce/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
new file mode 100644
index 0000000..2368021
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+
+/**
+ * Utility class for analyzing striped block groups
+ */
+@InterfaceAudience.Private
+public class StripedBlockUtil {
+
+  /**
+   * This method parses a striped block group into individual blocks.
+   *
+   * @param bg The striped block group
+   * @param cellSize The size of a striping cell
+   * @param dataBlkNum The number of data blocks
+   * @return An array containing the blocks in the group
+   */
+  public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg,
+      int cellSize, int dataBlkNum, int parityBlkNum) {
+    int locatedBGSize = bg.getBlockIndices().length;
+    // TODO not considering missing blocks for now, only identify data blocks
+    LocatedBlock[] lbs = new LocatedBlock[dataBlkNum + parityBlkNum];
+    for (short i = 0; i < locatedBGSize; i++) {
+      final int idx = bg.getBlockIndices()[i];
+      if (idx < (dataBlkNum + parityBlkNum) && lbs[idx] == null) {
+        lbs[idx] = constructInternalBlock(bg, i, cellSize,
+            dataBlkNum, idx);
+      }
+    }
+    return lbs;
+  }
+
+  /**
+   * This method creates an internal block at the given index of a block group
+   *
+   * @param idxInReturnedLocs The index in the stored locations in the
+   *                          {@link LocatedStripedBlock} object
+   * @param idxInBlockGroup The logical index in the striped block group
+   * @return The constructed internal block
+   */
+  public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg,
+      int idxInReturnedLocs, int cellSize, int dataBlkNum,
+      int idxInBlockGroup) {
+    final ExtendedBlock blk = new ExtendedBlock(bg.getBlock());
+    blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup);
+    blk.setNumBytes(getInternalBlockLength(bg.getBlockSize(),
+        cellSize, dataBlkNum, idxInBlockGroup));
+
+    return new LocatedBlock(blk,
+        new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
+        new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
+        new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
+        bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(),
+        null);
+  }
+
+  /**
+   * Get the size of an internal block at the given index of a block group
+   *
+   * @param numBytesInGroup Size of the block group only counting data blocks
+   * @param cellSize The size of a striping cell
+   * @param dataBlkNum The number of data blocks
+   * @param idxInGroup The logical index in the striped block group
+   * @return The size of the internal block at the specified index
+   */
+  public static long getInternalBlockLength(long numBytesInGroup,
+      int cellSize, int dataBlkNum, int idxInGroup) {
+    // Size of each stripe (only counting data blocks)
+    final long numBytesPerStripe = cellSize * dataBlkNum;
+    assert numBytesPerStripe  > 0:
+        "getInternalBlockLength should only be called on valid striped blocks";
+    // If block group ends at stripe boundary, each internal block has an equal
+    // share of the group
+    if (numBytesInGroup % numBytesPerStripe == 0) {
+      return numBytesInGroup / dataBlkNum;
+    }
+
+    int numStripes = (int) ((numBytesInGroup - 1) / numBytesPerStripe + 1);
+    assert numStripes >= 1 : "There should be at least 1 stripe";
+
+    // All stripes but the last one are full stripes. The block should at least
+    // contain (numStripes - 1) full cells.
+    long blkSize = (numStripes - 1) * cellSize;
+
+    long lastStripeLen = numBytesInGroup % numBytesPerStripe;
+    // Size of parity cells should equal the size of the first cell, if it
+    // is not full.
+    long lastParityCellLen = Math.min(cellSize, lastStripeLen);
+
+    if (idxInGroup >= dataBlkNum) {
+      // for parity blocks
+      blkSize += lastParityCellLen;
+    } else {
+      // for data blocks
+      blkSize +=  Math.min(cellSize,
+          Math.max(0, lastStripeLen - cellSize * idxInGroup));
+    }
+
+    return blkSize;
+  }
+
+  /**
+   * Given a byte's offset in an internal block, calculate the offset in
+   * the block group
+   */
+  public static long offsetInBlkToOffsetInBG(int cellSize, int dataBlkNum,
+      long offsetInBlk, int idxInBlockGroup) {
+    int cellIdxInBlk = (int) (offsetInBlk / cellSize);
+    return cellIdxInBlk * cellSize * dataBlkNum // n full stripes before offset
+        + idxInBlockGroup * cellSize // m full cells before offset
+        + offsetInBlk % cellSize; // partial cell
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d27f5dce/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 aaacf23..db14292 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
@@ -106,7 +106,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@@ -1811,11 +1810,30 @@ public class DFSTestUtil {
     return reports;
   }
 
-  public static void createECFile(MiniDFSCluster cluster, Path file, Path dir,
-      int numBlocks, int numStripesPerBlk) throws Exception {
+  /**
+   * Creates the metadata of a file in striped layout. This method only
+   * manipulates the NameNode state without injecting data to DataNode.
+   *  @param file Path of the file to create
+   * @param dir Parent path of the file
+   * @param numBlocks Number of striped block groups to add to the file
+   * @param numStripesPerBlk Number of striped cells in each block
+   * @param toMkdir
+   */
+  public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir,
+      int numBlocks, int numStripesPerBlk, boolean toMkdir) throws Exception {
     DistributedFileSystem dfs = cluster.getFileSystem();
-    dfs.mkdirs(dir);
-    dfs.getClient().createErasureCodingZone(dir.toString(), null);
+    // If outer test already created EC zone, dir should be left as null
+    if (toMkdir) {
+      assert dir != null;
+      dfs.mkdirs(dir);
+      try {
+        dfs.getClient().createErasureCodingZone(dir.toString(), null);
+      } catch (IOException e) {
+        if (!e.getMessage().contains("non-empty directory")) {
+          throw e;
+        }
+      }
+    }
 
     FSDataOutputStream out = null;
     try {
@@ -1827,7 +1845,7 @@ public class DFSTestUtil {
 
       ExtendedBlock previous = null;
       for (int i = 0; i < numBlocks; i++) {
-        Block newBlock = createBlock(cluster.getDataNodes(), dfs, ns,
+        Block newBlock = addStripedBlockToFile(cluster.getDataNodes(), dfs, ns,
             file.toString(), fileNode, dfs.getClient().getClientName(),
             previous, numStripesPerBlk);
         previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
@@ -1840,43 +1858,50 @@ public class DFSTestUtil {
     }
   }
 
-  static Block createBlock(List<DataNode> dataNodes, DistributedFileSystem fs,
-      FSNamesystem ns, String file, INodeFile fileNode, String clientName,
-      ExtendedBlock previous, int numStripes) throws Exception {
+  /**
+   * Adds a striped block group to a file. This method only manipulates NameNode
+   * states of the file and the block without injecting data to DataNode.
+   * It does mimic block reports.
+   * @param dataNodes List DataNodes to host the striped block group
+   * @param previous Previous block in the file
+   * @param numStripes Number of stripes in each block group
+   * @return The added block group
+   */
+  public static Block addStripedBlockToFile(List<DataNode> dataNodes,
+      DistributedFileSystem fs, FSNamesystem ns, String file, INodeFile fileNode,
+      String clientName, ExtendedBlock previous, int numStripes)
+      throws Exception {
     fs.getClient().namenode.addBlock(file, clientName, previous, null,
         fileNode.getId(), null);
 
     final BlockInfo lastBlock = fileNode.getLastBlock();
     final int groupSize = fileNode.getBlockReplication();
+    assert dataNodes.size() >= groupSize;
     // 1. RECEIVING_BLOCK IBR
-    int i = 0;
-    for (DataNode dn : dataNodes) {
-      if (i < groupSize) {
-        final Block block = new Block(lastBlock.getBlockId() + i++, 0,
-            lastBlock.getGenerationStamp());
-        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
-        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
-            .makeReportForReceivedBlock(block,
-                ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
-        for (StorageReceivedDeletedBlocks report : reports) {
-          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
-        }
+    for (int i = 0; i < groupSize; i++) {
+      DataNode dn = dataNodes.get(i);
+      final Block block = new Block(lastBlock.getBlockId() + i, 0,
+          lastBlock.getGenerationStamp());
+      DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+      StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+          .makeReportForReceivedBlock(block,
+              ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
+      for (StorageReceivedDeletedBlocks report : reports) {
+        ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
       }
     }
 
     // 2. RECEIVED_BLOCK IBR
-    i = 0;
-    for (DataNode dn : dataNodes) {
-      if (i < groupSize) {
-        final Block block = new Block(lastBlock.getBlockId() + i++,
-            numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp());
-        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
-        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
-            .makeReportForReceivedBlock(block,
-                ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
-        for (StorageReceivedDeletedBlocks report : reports) {
-          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
-        }
+    for (int i = 0; i < groupSize; i++) {
+      DataNode dn = dataNodes.get(i);
+      final Block block = new Block(lastBlock.getBlockId() + i,
+          numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp());
+      DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+      StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+          .makeReportForReceivedBlock(block,
+              ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+      for (StorageReceivedDeletedBlocks report : reports) {
+        ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
       }
     }
 

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d27f5dce/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index 27df1cd..6bb1162 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -34,11 +34,13 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
@@ -53,6 +55,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
 
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 import static org.junit.Assert.assertEquals;
 
 public class TestAddStripedBlocks {
@@ -284,4 +288,107 @@ public class TestAddStripedBlocks {
       Assert.assertEquals(GROUP_SIZE - i - 1, indices[i]);
     }
   }
+
+  @Test
+  public void testCheckStripedReplicaCorrupt() throws Exception {
+    final int numBlocks = 4;
+    final int numStripes = 4;
+    final Path filePath = new Path("/corrupt");
+    final FSNamesystem ns = cluster.getNameNode().getNamesystem();
+    DFSTestUtil.createStripedFile(cluster, filePath, null,
+        numBlocks, numStripes, false);
+
+    INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()).
+        asFile();
+    Assert.assertTrue(fileNode.isStriped());
+    BlockInfoStriped stored = fileNode.getStripedBlocksFeature().getBlocks()[0];
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(0, ns.getCorruptReplicaBlocks());
+
+    // Now send a block report with correct size
+    DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+    final Block reported = new Block(stored);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
+    StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+        .makeReportForReceivedBlock(reported,
+            ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(0).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(0, ns.getCorruptReplicaBlocks());
+
+    // Now send a block report with wrong size
+    reported.setBlockId(stored.getBlockId() + 1);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE - 1);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+            ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(1).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
+
+    // Now send a parity block report with correct size
+    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(2).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
+
+    // Now send a parity block report with wrong size
+    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 1);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(3).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(2, ns.getCorruptReplicaBlocks());
+
+    // Now change the size of stored block, and test verifying the last
+    // block size
+    stored.setNumBytes(stored.getNumBytes() + 10);
+    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS + 2);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(3).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(3, ns.getCorruptReplicaBlocks());
+
+    // Now send a parity block report with correct size based on adjusted
+    // size of stored block
+    /** Now stored block has {@link numStripes} full stripes + a cell + 10 */
+    stored.setNumBytes(stored.getNumBytes() + BLOCK_STRIPED_CELL_SIZE);
+    reported.setBlockId(stored.getBlockId());
+    reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(0).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(3, ns.getCorruptReplicaBlocks());
+
+    reported.setBlockId(stored.getBlockId() + 1);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 10);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(1).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(3, ns.getCorruptReplicaBlocks());
+
+    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
+    reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(2).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(3, ns.getCorruptReplicaBlocks());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d27f5dce/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
index 4292f9a..ea18c3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -78,7 +78,8 @@ public class TestRecoverStripedBlocks {
   @Test
   public void testMissingStripedBlock() throws Exception {
     final int numBlocks = 4;
-    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, 1);
+    DFSTestUtil.createStripedFile(cluster, filePath,
+        dirPath, numBlocks, 1, true);
 
     // make sure the file is complete in NN
     final INodeFile fileNode = cluster.getNamesystem().getFSDirectory()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d27f5dce/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
new file mode 100644
index 0000000..ec0b1bb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.util;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.parseStripedBlockGroup;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class TestStripedBlockUtil {
+  private final short DATA_BLK_NUM = HdfsConstants.NUM_DATA_BLOCKS;
+  private final short PARITY_BLK_NUM = HdfsConstants.NUM_PARITY_BLOCKS;
+  private final short BLK_GROUP_SIZE = DATA_BLK_NUM + PARITY_BLK_NUM;
+  private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+
+  private LocatedStripedBlock createDummyLocatedBlock() {
+    final long blockGroupID = -1048576;
+    DatanodeInfo[] locs = new DatanodeInfo[BLK_GROUP_SIZE];
+    String[] storageIDs = new String[BLK_GROUP_SIZE];
+    StorageType[] storageTypes = new StorageType[BLK_GROUP_SIZE];
+    int[] indices = new int[BLK_GROUP_SIZE];
+    for (int i = 0; i < BLK_GROUP_SIZE; i++) {
+      indices[i] = (i + 2) % DATA_BLK_NUM;
+      // Location port always equal to logical index of a block,
+      // for easier verification
+      locs[i] = DFSTestUtil.getLocalDatanodeInfo(indices[i]);
+      storageIDs[i] = locs[i].getDatanodeUuid();
+      storageTypes[i] = StorageType.DISK;
+    }
+    return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID),
+        locs, storageIDs, storageTypes, indices, 0, false, null);
+  }
+
+  @Test
+  public void testParseDummyStripedBlock() {
+    LocatedStripedBlock lsb = createDummyLocatedBlock();
+    LocatedBlock[] blocks = parseStripedBlockGroup(
+        lsb, CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM);
+    assertEquals(DATA_BLK_NUM + PARITY_BLK_NUM, blocks.length);
+    for (int i = 0; i < DATA_BLK_NUM; i++) {
+      assertFalse(blocks[i].isStriped());
+      assertEquals(i,
+          BlockIdManager.getBlockIndex(blocks[i].getBlock().getLocalBlock()));
+      assertEquals(i * CELLSIZE, blocks[i].getStartOffset());
+      assertEquals(1, blocks[i].getLocations().length);
+      assertEquals(i, blocks[i].getLocations()[0].getIpcPort());
+      assertEquals(i, blocks[i].getLocations()[0].getXferPort());
+    }
+  }
+
+  private void verifyInternalBlocks (long numBytesInGroup, long[] expected) {
+    for (int i = 1; i < BLK_GROUP_SIZE; i++) {
+      assertEquals(expected[i],
+          getInternalBlockLength(numBytesInGroup, CELLSIZE, DATA_BLK_NUM, i));
+    }
+  }
+
+  @Test
+  public void testGetInternalBlockLength () {
+    // A small delta that is smaller than a cell
+    final int delta = 10;
+    assert delta < CELLSIZE;
+
+    // Block group is smaller than a cell
+    verifyInternalBlocks(CELLSIZE - delta,
+        new long[] {CELLSIZE - delta, 0, 0, 0, 0, 0,
+            CELLSIZE - delta, CELLSIZE - delta, CELLSIZE - delta});
+
+    // Block group is exactly as large as a cell
+    verifyInternalBlocks(CELLSIZE,
+        new long[] {CELLSIZE, 0, 0, 0, 0, 0,
+            CELLSIZE, CELLSIZE, CELLSIZE});
+
+    // Block group is a little larger than a cell
+    verifyInternalBlocks(CELLSIZE + delta,
+        new long[] {CELLSIZE, delta, 0, 0, 0, 0,
+            CELLSIZE, CELLSIZE, CELLSIZE});
+
+    // Block group contains multiple stripes and ends at stripe boundary
+    verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE,
+        new long[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE});
+
+    // Block group contains multiple stripes and ends at cell boundary
+    // (not ending at stripe boundary)
+    verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE + CELLSIZE,
+        new long[] {3 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            3 * CELLSIZE, 3 * CELLSIZE, 3 * CELLSIZE});
+
+    // Block group contains multiple stripes and doesn't end at cell boundary
+    verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE - delta,
+        new long[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE - delta,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE});
+  }
+
+}


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

Posted by ji...@apache.org.
HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema in FileSystemLinkResolver. Contributed by Tsz Wo Nicholas Sze.


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

Branch: refs/heads/HDFS-7285
Commit: 7dfd6d42b01013bc7ebf08a4f0e55da18c8019c6
Parents: dbeb2c9
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Apr 21 21:03:07 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:55 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7dfd6d42/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 4c8fff3..ede4f48 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2281,7 +2281,7 @@ public class DistributedFileSystem extends FileSystem {
       @Override
       public Void doCall(final Path p) throws IOException,
           UnresolvedLinkException {
-        dfs.createErasureCodingZone(getPathName(p), null);
+        dfs.createErasureCodingZone(getPathName(p), schema);
         return null;
       }
 


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 6452510ac57a3ecadb3353d5abd8af2a3e8fc27f
Parents: 5e8cea7
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 28 14:24:17 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:17:52 2015 -0700

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


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

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


[17/50] hadoop git commit: HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (Contributed by Vinayakumar B)

Posted by ji...@apache.org.
HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (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/96173cea
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/96173cea
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/96173cea

Branch: refs/heads/HDFS-7285
Commit: 96173ceac4ac8fd32c5f4c5414df1bb69d8466c9
Parents: 631916c
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Apr 8 12:48:59 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:14:10 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  5 ++-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 14 ++++++
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 10 +++++
 ...tNamenodeProtocolServerSideTranslatorPB.java | 19 ++++++++
 .../ClientNamenodeProtocolTranslatorPB.java     | 18 ++++++++
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 46 ++++++++++++++++++++
 .../hdfs/server/namenode/FSNamesystem.java      | 31 +++++++++++++
 .../hdfs/server/namenode/NameNodeRpcServer.java |  7 +++
 .../src/main/proto/ClientNamenodeProtocol.proto | 10 +++++
 .../hadoop-hdfs/src/main/proto/hdfs.proto       | 28 ++++++++++++
 .../hadoop/hdfs/TestErasureCodingZones.java     | 38 +++++++++++++++-
 11 files changed, 223 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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 9927ccf..7423033 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -49,4 +49,7 @@
     (Hui Zheng via Zhe Zhang)
 
     HDFS-7839. Erasure coding: implement facilities in NameNode to create and
-    manage EC zones (Zhe Zhang)
\ No newline at end of file
+    manage EC zones (Zhe Zhang)
+
+    HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from
+    NameNode (vinayakumarb)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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 f4eea49..16f876c 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
@@ -117,6 +117,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.ECInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -3095,6 +3096,19 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  public ECInfo 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 DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
     return new DFSInotifyEventInputStream(traceSampler, namenode);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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 8efe344..45d92f3 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
@@ -1464,4 +1464,14 @@ public interface ClientProtocol {
    */
   @Idempotent
   public EventBatchList getEditsFromTxid(long txid) 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 ECInfo getErasureCodingInfo(String src) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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 2b8215e..7569819 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,6 +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.ECInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -108,6 +109,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
@@ -1511,4 +1514,20 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetErasureCodingInfoResponseProto getErasureCodingInfo(RpcController controller,
+      GetErasureCodingInfoRequestProto request) throws ServiceException {
+    try {
+      ECInfo 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);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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 43a0322..568da68 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,6 +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.ECInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -107,6 +108,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
@@ -1532,4 +1535,19 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public ECInfo 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);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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 450e0bb..9aff943 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
@@ -29,7 +29,11 @@ import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
@@ -77,6 +81,7 @@ 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.HdfsConstantsClient;
+import org.apache.hadoop.hdfs.protocol.ECInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -146,6 +151,9 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeLocalInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
@@ -226,6 +234,7 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
@@ -3095,4 +3104,41 @@ public class PBHelper {
         setId(context.getReportId()).
         build();
   }
+
+  public static ECInfo convertECInfo(ECInfoProto ecInfoProto) {
+    return new ECInfo(ecInfoProto.getSrc(),
+        convertECSchema(ecInfoProto.getSchema()));
+  }
+
+  public static ECInfoProto convertECInfo(ECInfo ecInfo) {
+    return ECInfoProto.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());
+    for (ECSchemaOptionEntryProto option : optionsList) {
+      options.put(option.getKey(), option.getValue());
+    }
+    // include chunksize in options.
+    options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(schema.getChunkSize()));
+    return new ECSchema(schema.getSchemaName(), schema.getCodecName(),
+        schema.getDataUnits(), schema.getParityUnits(), options);
+  }
+
+  public static ECSchemaProto convertECSchema(ECSchema schema) {
+    ECSchemaProto.Builder builder = ECSchemaProto.newBuilder()
+        .setSchemaName(schema.getSchemaName())
+        .setCodecName(schema.getCodecName())
+        .setDataUnits(schema.getNumDataUnits())
+        .setParityUnits(schema.getNumParityUnits())
+        .setChunkSize(schema.getChunkSize());
+    Set<Entry<String, String>> entrySet = schema.getOptions().entrySet();
+    for (Entry<String, String> entry : entrySet) {
+      builder.addOptions(ECSchemaOptionEntryProto.newBuilder()
+          .setKey(entry.getKey()).setValue(entry.getValue()).build());
+    }
+    return builder.build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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 c359baa..5eb84c7 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,6 +181,7 @@ 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.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -263,6 +264,7 @@ import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.ipc.RetryCache;
 import org.apache.hadoop.ipc.Server;
@@ -8155,6 +8157,35 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     logAuditEvent(true, "createErasureCodingZone", srcArg, null, resultingStat);
   }
 
+  /**
+   * Get the erasure coding information for specified src
+   */
+  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);
+      }
+      if (dir.getECPolicy(iip)) {
+        // TODO HDFS-8074 and HDFS-7859 : To get from loaded schemas
+        Map<String, String> options = new HashMap<String, String>();
+        ECSchema defaultSchema = new ECSchema("RS-6-3", "rs", 6, 3, options);
+        return new ECInfo(src, defaultSchema);
+      }
+    } finally {
+      readUnlock();
+    }
+    return null;
+  }
+
   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
                 boolean logRetryCache)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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 09357cb..d2b20a8 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,6 +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.ECInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
@@ -2051,4 +2052,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.checkSuperuserPrivilege();
     nn.spanReceiverHost.removeSpanReceiver(id);
   }
+
+  @Override // ClientNameNodeProtocol
+  public ECInfo getErasureCodingInfo(String src) throws IOException {
+    checkNNStartup();
+    return namesystem.getErasureCodingInfo(src);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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 183aff8..9488aed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -721,6 +721,14 @@ message CreateErasureCodingZoneRequestProto {
 message CreateErasureCodingZoneResponseProto {
 }
 
+message GetErasureCodingInfoRequestProto {
+  required string src = 1;
+}
+
+message GetErasureCodingInfoResponseProto {
+  optional ECInfoProto ECInfo = 1;
+}
+
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -869,4 +877,6 @@ service ClientNamenodeProtocol {
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
       returns(GetEditsFromTxidResponseProto);
+  rpc getErasureCodingInfo(GetErasureCodingInfoRequestProto)
+      returns(GetErasureCodingInfoResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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..1314ea0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -620,3 +620,31 @@ message RollingUpgradeStatusProto {
   required string blockPoolId = 1;
   optional bool finalized = 2 [default = false];
 }
+
+/**
+ * ECSchema options entry
+ */
+message ECSchemaOptionEntryProto {
+  required string key = 1;
+  required string value = 2;
+}
+
+/**
+ * ECShema for erasurecoding
+ */
+message ECSchemaProto {
+  required string schemaName = 1;
+  required string codecName = 2;
+  required uint32 dataUnits = 3;
+  required uint32 parityUnits = 4;
+  required uint32 chunkSize = 5;
+  repeated ECSchemaOptionEntryProto options = 6;
+}
+
+/**
+ * ECInfo
+ */
+message ECInfoProto {
+ required string src = 1;
+ required ECSchemaProto schema = 2;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96173cea/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 49f08eef..bdca915 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,10 @@ 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.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -29,8 +31,7 @@ import org.junit.Test;
 import java.io.IOException;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
 public class TestErasureCodingZones {
   private final int NUM_OF_DATANODES = 3;
@@ -148,4 +149,37 @@ public class TestErasureCodingZones {
           "destination have different erasure coding policies", e);
     }
   }
+
+  @Test
+  public void testGetErasureCodingInfo() throws Exception {
+    String src = "/ec";
+    final Path ecDir = new Path(src);
+    fs.mkdir(ecDir, FsPermission.getDirDefault());
+    // dir ECInfo before creating ec zone
+    assertNull(fs.getClient().getErasureCodingInfo(src));
+    // dir ECInfo after creating ec zone
+    fs.getClient().createErasureCodingZone(src);
+    verifyErasureCodingInfo(src);
+    fs.create(new Path(ecDir, "/child1")).close();
+    // verify for the files in ec zone
+    verifyErasureCodingInfo(src + "/child1");
+  }
+
+  private void verifyErasureCodingInfo(String src) throws IOException {
+    ECInfo ecInfo = fs.getClient().getErasureCodingInfo(src);
+    assertNotNull("ECInfo should have been non-null", ecInfo);
+    assertEquals(src, ecInfo.getSrc());
+    ECSchema schema = ecInfo.getSchema();
+    assertNotNull(schema);
+    assertEquals("Default schema should be returned", "RS-6-3",
+        schema.getSchemaName());
+    assertEquals("Default codec(rs) should be returned", "rs",
+        schema.getCodecName());
+    assertEquals("Default numDataUnits should be used", 6,
+        schema.getNumDataUnits());
+    assertEquals("Default numParityUnits should be used", 3,
+        schema.getNumParityUnits());
+    assertEquals("Default chunkSize should be used",
+        ECSchema.DEFAULT_CHUNK_SIZE, schema.getChunkSize());
+  }
 }
\ No newline at end of file


[09/50] hadoop git commit: HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng

Posted by ji...@apache.org.
HADOOP-11805 Better to rename some raw 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/f24e2442
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f24e2442
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f24e2442

Branch: refs/heads/HDFS-7285
Commit: f24e2442b74942370de99dfeb8566a8d56f8861d
Parents: 80e6018
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Apr 8 01:26:40 2015 +0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:06:50 2015 -0700

----------------------------------------------------------------------
 .../io/erasurecode/coder/RSErasureDecoder.java  |  8 +-
 .../io/erasurecode/coder/RSErasureEncoder.java  |  4 +-
 .../io/erasurecode/coder/XORErasureDecoder.java | 78 ++++++++++++++++
 .../io/erasurecode/coder/XORErasureEncoder.java | 45 ++++++++++
 .../io/erasurecode/coder/XorErasureDecoder.java | 78 ----------------
 .../io/erasurecode/coder/XorErasureEncoder.java | 45 ----------
 .../io/erasurecode/rawcoder/JRSRawDecoder.java  | 69 ---------------
 .../io/erasurecode/rawcoder/JRSRawEncoder.java  | 78 ----------------
 .../rawcoder/JRSRawErasureCoderFactory.java     | 34 -------
 .../io/erasurecode/rawcoder/RSRawDecoder.java   | 69 +++++++++++++++
 .../io/erasurecode/rawcoder/RSRawEncoder.java   | 78 ++++++++++++++++
 .../rawcoder/RSRawErasureCoderFactory.java      | 34 +++++++
 .../io/erasurecode/rawcoder/XORRawDecoder.java  | 81 +++++++++++++++++
 .../io/erasurecode/rawcoder/XORRawEncoder.java  | 61 +++++++++++++
 .../rawcoder/XORRawErasureCoderFactory.java     | 34 +++++++
 .../io/erasurecode/rawcoder/XorRawDecoder.java  | 81 -----------------
 .../io/erasurecode/rawcoder/XorRawEncoder.java  | 61 -------------
 .../rawcoder/XorRawErasureCoderFactory.java     | 34 -------
 .../erasurecode/coder/TestRSErasureCoder.java   |  4 +-
 .../io/erasurecode/coder/TestXORCoder.java      | 50 +++++++++++
 .../io/erasurecode/coder/TestXorCoder.java      | 50 -----------
 .../erasurecode/rawcoder/TestJRSRawCoder.java   | 93 --------------------
 .../io/erasurecode/rawcoder/TestRSRawCoder.java | 93 ++++++++++++++++++++
 .../erasurecode/rawcoder/TestXORRawCoder.java   | 49 +++++++++++
 .../erasurecode/rawcoder/TestXorRawCoder.java   | 51 -----------
 25 files changed, 680 insertions(+), 682 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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 ba32f04..e2c5051 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
@@ -4,9 +4,9 @@ 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.JRSRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
-import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 
 /**
  * Reed-Solomon erasure decoder that decodes a block group.
@@ -56,7 +56,7 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
       rsRawDecoder = createRawDecoder(
           CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
       if (rsRawDecoder == null) {
-        rsRawDecoder = new JRSRawDecoder();
+        rsRawDecoder = new RSRawDecoder();
       }
       rsRawDecoder.initialize(getNumDataUnits(),
           getNumParityUnits(), getChunkSize());
@@ -66,7 +66,7 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
 
   private RawErasureDecoder checkCreateXorRawDecoder() {
     if (xorRawDecoder == null) {
-      xorRawDecoder = new XorRawDecoder();
+      xorRawDecoder = new XORRawDecoder();
       xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize());
     }
     return xorRawDecoder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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 430749d..a7d02b5 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
@@ -3,7 +3,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.rawcoder.JRSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 
 /**
@@ -30,7 +30,7 @@ public class RSErasureEncoder extends AbstractErasureEncoder {
       rawEncoder = createRawEncoder(
           CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
       if (rawEncoder == null) {
-        rawEncoder = new JRSRawEncoder();
+        rawEncoder = new RSRawEncoder();
       }
       rawEncoder.initialize(getNumDataUnits(),
           getNumParityUnits(), getChunkSize());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
new file mode 100644
index 0000000..6f4b423
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
+
+/**
+ * Xor erasure decoder that decodes a block group.
+ *
+ * It implements {@link ErasureDecoder}.
+ */
+public class XORErasureDecoder extends AbstractErasureDecoder {
+
+  @Override
+  protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
+    // May be configured
+    RawErasureDecoder rawDecoder = new XORRawDecoder();
+    rawDecoder.initialize(getNumDataUnits(),
+        getNumParityUnits(), getChunkSize());
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureDecodingStep(inputBlocks,
+        getErasedIndexes(inputBlocks),
+        getOutputBlocks(blockGroup), rawDecoder);
+  }
+
+  /**
+   * Which blocks were erased ? For XOR it's simple we only allow and return one
+   * erased block, either data or parity.
+   * @param blockGroup
+   * @return output blocks to recover
+   */
+  @Override
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    /**
+     * If more than one blocks (either data or parity) erased, then it's not
+     * edible to recover. We don't have the check here since it will be done
+     * by upper level: ErasreCoder call can be avoid if not possible to recover
+     * at all.
+     */
+    int erasedNum = getNumErasedBlocks(blockGroup);
+    ECBlock[] outputBlocks = new ECBlock[erasedNum];
+
+    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];
+      }
+    }
+
+    return outputBlocks;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
new file mode 100644
index 0000000..9011857
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+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}.
+ */
+public class XORErasureEncoder extends AbstractErasureEncoder {
+
+  @Override
+  protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
+    // May be configured
+    RawErasureEncoder rawEncoder = new XORRawEncoder();
+    rawEncoder.initialize(getNumDataUnits(),
+        getNumParityUnits(), getChunkSize());
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureEncodingStep(inputBlocks,
+        getOutputBlocks(blockGroup), rawEncoder);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
deleted file mode 100644
index 33f5386..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
+++ /dev/null
@@ -1,78 +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.io.erasurecode.coder;
-
-import org.apache.hadoop.io.erasurecode.ECBlock;
-import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
-import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
-
-/**
- * Xor erasure decoder that decodes a block group.
- *
- * It implements {@link ErasureDecoder}.
- */
-public class XorErasureDecoder extends AbstractErasureDecoder {
-
-  @Override
-  protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
-    // May be configured
-    RawErasureDecoder rawDecoder = new XorRawDecoder();
-    rawDecoder.initialize(getNumDataUnits(),
-        getNumParityUnits(), getChunkSize());
-
-    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
-
-    return new ErasureDecodingStep(inputBlocks,
-        getErasedIndexes(inputBlocks),
-        getOutputBlocks(blockGroup), rawDecoder);
-  }
-
-  /**
-   * Which blocks were erased ? For XOR it's simple we only allow and return one
-   * erased block, either data or parity.
-   * @param blockGroup
-   * @return output blocks to recover
-   */
-  @Override
-  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
-    /**
-     * If more than one blocks (either data or parity) erased, then it's not
-     * edible to recover. We don't have the check here since it will be done
-     * by upper level: ErasreCoder call can be avoid if not possible to recover
-     * at all.
-     */
-    int erasedNum = getNumErasedBlocks(blockGroup);
-    ECBlock[] outputBlocks = new ECBlock[erasedNum];
-
-    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];
-      }
-    }
-
-    return outputBlocks;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
deleted file mode 100644
index f8d67c3..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
+++ /dev/null
@@ -1,45 +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.io.erasurecode.coder;
-
-import org.apache.hadoop.io.erasurecode.ECBlock;
-import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-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}.
- */
-public class XorErasureEncoder extends AbstractErasureEncoder {
-
-  @Override
-  protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
-    // May be configured
-    RawErasureEncoder rawEncoder = new XorRawEncoder();
-    rawEncoder.initialize(getNumDataUnits(),
-        getNumParityUnits(), getChunkSize());
-
-    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
-
-    return new ErasureEncodingStep(inputBlocks,
-        getOutputBlocks(blockGroup), rawEncoder);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java
deleted file mode 100644
index dbb689e..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.io.erasurecode.rawcoder;
-
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-
-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.
- */
-public class JRSRawDecoder extends AbstractRawErasureDecoder {
-  // To describe and calculate the needed Vandermonde matrix
-  private int[] errSignature;
-  private int[] primitivePower;
-
-  @Override
-  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
-    super.initialize(numDataUnits, numParityUnits, chunkSize);
-    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
-
-    this.errSignature = new int[getNumParityUnits()];
-    this.primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
-        getNumParityUnits());
-  }
-
-  @Override
-  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
-                          ByteBuffer[] outputs) {
-    for (int i = 0; i < erasedIndexes.length; i++) {
-      errSignature[i] = primitivePower[erasedIndexes[i]];
-      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
-    }
-
-    int dataLen = inputs[0].remaining();
-    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
-        erasedIndexes.length, dataLen);
-  }
-
-  @Override
-  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
-                          byte[][] outputs) {
-    for (int i = 0; i < erasedIndexes.length; i++) {
-      errSignature[i] = primitivePower[erasedIndexes[i]];
-      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
-    }
-
-    int dataLen = inputs[0].length;
-    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
-        erasedIndexes.length, dataLen);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java
deleted file mode 100644
index 6ea7551..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java
+++ /dev/null
@@ -1,78 +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.io.erasurecode.rawcoder;
-
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-
-import java.nio.ByteBuffer;
-
-/**
- * A raw erasure encoder in RS code scheme in pure Java in case native one
- * isn't available in some environment. Please always use native implementations
- * when possible.
- */
-public class JRSRawEncoder extends AbstractRawErasureEncoder {
-  private int[] generatingPolynomial;
-
-  @Override
-  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
-    super.initialize(numDataUnits, numParityUnits, chunkSize);
-    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
-
-    int[] primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
-        getNumParityUnits());
-    // compute generating polynomial
-    int[] gen = {1};
-    int[] poly = new int[2];
-    for (int i = 0; i < getNumParityUnits(); i++) {
-      poly[0] = primitivePower[i];
-      poly[1] = 1;
-      gen = RSUtil.GF.multiply(gen, poly);
-    }
-    // generating polynomial has all generating roots
-    generatingPolynomial = gen;
-  }
-
-  @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];
-    }
-
-    // Compute the remainder
-    RSUtil.GF.remainder(data, 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];
-    }
-
-    // Compute the remainder
-    RSUtil.GF.remainder(data, generatingPolynomial);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java
deleted file mode 100644
index d6b40aa..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.io.erasurecode.rawcoder;
-
-/**
- * A raw coder factory for raw Reed-Solomon coder in Java.
- */
-public class JRSRawErasureCoderFactory implements RawErasureCoderFactory {
-
-  @Override
-  public RawErasureEncoder createEncoder() {
-    return new JRSRawEncoder();
-  }
-
-  @Override
-  public RawErasureDecoder createDecoder() {
-    return new JRSRawDecoder();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
new file mode 100644
index 0000000..24fa637
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.rawcoder.util.RSUtil;
+
+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.
+ */
+public class RSRawDecoder extends AbstractRawErasureDecoder {
+  // To describe and calculate the needed Vandermonde matrix
+  private int[] errSignature;
+  private int[] primitivePower;
+
+  @Override
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
+    super.initialize(numDataUnits, numParityUnits, chunkSize);
+    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
+
+    this.errSignature = new int[getNumParityUnits()];
+    this.primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
+        getNumParityUnits());
+  }
+
+  @Override
+  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                          ByteBuffer[] outputs) {
+    for (int i = 0; i < erasedIndexes.length; i++) {
+      errSignature[i] = primitivePower[erasedIndexes[i]];
+      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
+    }
+
+    int dataLen = inputs[0].remaining();
+    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
+        erasedIndexes.length, dataLen);
+  }
+
+  @Override
+  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
+                          byte[][] outputs) {
+    for (int i = 0; i < erasedIndexes.length; i++) {
+      errSignature[i] = primitivePower[erasedIndexes[i]];
+      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
+    }
+
+    int dataLen = inputs[0].length;
+    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
+        erasedIndexes.length, dataLen);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
new file mode 100644
index 0000000..7b501ce
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.rawcoder.util.RSUtil;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw erasure encoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible.
+ */
+public class RSRawEncoder extends AbstractRawErasureEncoder {
+  private int[] generatingPolynomial;
+
+  @Override
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
+    super.initialize(numDataUnits, numParityUnits, chunkSize);
+    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
+
+    int[] primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
+        getNumParityUnits());
+    // compute generating polynomial
+    int[] gen = {1};
+    int[] poly = new int[2];
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      poly[0] = primitivePower[i];
+      poly[1] = 1;
+      gen = RSUtil.GF.multiply(gen, poly);
+    }
+    // generating polynomial has all generating roots
+    generatingPolynomial = gen;
+  }
+
+  @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];
+    }
+
+    // Compute the remainder
+    RSUtil.GF.remainder(data, 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];
+    }
+
+    // Compute the remainder
+    RSUtil.GF.remainder(data, generatingPolynomial);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
new file mode 100644
index 0000000..19a95af
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+/**
+ * A raw coder factory for raw Reed-Solomon coder in Java.
+ */
+public class RSRawErasureCoderFactory implements RawErasureCoderFactory {
+
+  @Override
+  public RawErasureEncoder createEncoder() {
+    return new RSRawEncoder();
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder() {
+    return new RSRawDecoder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
new file mode 100644
index 0000000..b6b1633
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.nio.ByteBuffer;
+
+/**
+ * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ */
+public class XORRawDecoder extends AbstractRawErasureDecoder {
+
+  @Override
+  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                          ByteBuffer[] outputs) {
+    assert(erasedIndexes.length == outputs.length);
+    assert(erasedIndexes.length <= 1);
+
+    int bufSize = inputs[0].remaining();
+    int erasedIdx = erasedIndexes[0];
+
+    // Set the output to zeros.
+    for (int j = 0; j < bufSize; j++) {
+      outputs[0].put(j, (byte) 0);
+    }
+
+    // Process the inputs.
+    for (int i = 0; i < inputs.length; i++) {
+      // Skip the erased location.
+      if (i == erasedIdx) {
+        continue;
+      }
+
+      for (int j = 0; j < bufSize; j++) {
+        outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j)));
+      }
+    }
+  }
+
+  @Override
+  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
+                          byte[][] outputs) {
+    assert(erasedIndexes.length == outputs.length);
+    assert(erasedIndexes.length <= 1);
+
+    int bufSize = inputs[0].length;
+    int erasedIdx = erasedIndexes[0];
+
+    // Set the output to zeros.
+    for (int j = 0; j < bufSize; j++) {
+      outputs[0][j] = 0;
+    }
+
+    // Process the inputs.
+    for (int i = 0; i < inputs.length; i++) {
+      // Skip the erased location.
+      if (i == erasedIdx) {
+        continue;
+      }
+
+      for (int j = 0; j < bufSize; j++) {
+        outputs[0][j] ^= inputs[i][j];
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
new file mode 100644
index 0000000..dbfab5d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ */
+public class XORRawEncoder extends AbstractRawErasureEncoder {
+
+  @Override
+  protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+    int bufSize = inputs[0].remaining();
+
+    // Get the first buffer's data.
+    for (int j = 0; j < bufSize; j++) {
+      outputs[0].put(j, inputs[0].get(j));
+    }
+
+    // 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)));
+      }
+    }
+  }
+
+  @Override
+  protected void doEncode(byte[][] inputs, byte[][] outputs) {
+    int bufSize = inputs[0].length;
+
+    // Get the first buffer's data.
+    for (int j = 0; j < bufSize; j++) {
+      outputs[0][j] = inputs[0][j];
+    }
+
+    // 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];
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
new file mode 100644
index 0000000..67f45c1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+/**
+ * A raw coder factory for raw XOR coder.
+ */
+public class XORRawErasureCoderFactory implements RawErasureCoderFactory {
+
+  @Override
+  public RawErasureEncoder createEncoder() {
+    return new XORRawEncoder();
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder() {
+    return new XORRawDecoder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
deleted file mode 100644
index 98307a7..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java
+++ /dev/null
@@ -1,81 +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.io.erasurecode.rawcoder;
-
-import java.nio.ByteBuffer;
-
-/**
- * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
- */
-public class XorRawDecoder extends AbstractRawErasureDecoder {
-
-  @Override
-  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
-                          ByteBuffer[] outputs) {
-    assert(erasedIndexes.length == outputs.length);
-    assert(erasedIndexes.length <= 1);
-
-    int bufSize = inputs[0].remaining();
-    int erasedIdx = erasedIndexes[0];
-
-    // Set the output to zeros.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0].put(j, (byte) 0);
-    }
-
-    // Process the inputs.
-    for (int i = 0; i < inputs.length; i++) {
-      // Skip the erased location.
-      if (i == erasedIdx) {
-        continue;
-      }
-
-      for (int j = 0; j < bufSize; j++) {
-        outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j)));
-      }
-    }
-  }
-
-  @Override
-  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
-                          byte[][] outputs) {
-    assert(erasedIndexes.length == outputs.length);
-    assert(erasedIndexes.length <= 1);
-
-    int bufSize = inputs[0].length;
-    int erasedIdx = erasedIndexes[0];
-
-    // Set the output to zeros.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0][j] = 0;
-    }
-
-    // Process the inputs.
-    for (int i = 0; i < inputs.length; i++) {
-      // Skip the erased location.
-      if (i == erasedIdx) {
-        continue;
-      }
-
-      for (int j = 0; j < bufSize; j++) {
-        outputs[0][j] ^= inputs[i][j];
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
deleted file mode 100644
index 99b20b9..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java
+++ /dev/null
@@ -1,61 +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.io.erasurecode.rawcoder;
-
-import java.nio.ByteBuffer;
-
-/**
- * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
- */
-public class XorRawEncoder extends AbstractRawErasureEncoder {
-
-  @Override
-  protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    int bufSize = inputs[0].remaining();
-
-    // Get the first buffer's data.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0].put(j, inputs[0].get(j));
-    }
-
-    // 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)));
-      }
-    }
-  }
-
-  @Override
-  protected void doEncode(byte[][] inputs, byte[][] outputs) {
-    int bufSize = inputs[0].length;
-
-    // Get the first buffer's data.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0][j] = inputs[0][j];
-    }
-
-    // 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];
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
deleted file mode 100644
index 751d16f..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.io.erasurecode.rawcoder;
-
-/**
- * A raw coder factory for raw XOR coder.
- */
-public class XorRawErasureCoderFactory implements RawErasureCoderFactory {
-
-  @Override
-  public RawErasureEncoder createEncoder() {
-    return new XorRawEncoder();
-  }
-
-  @Override
-  public RawErasureDecoder createDecoder() {
-    return new XorRawDecoder();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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 8a7561c..3507dd2 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
@@ -19,7 +19,7 @@ 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.rawcoder.JRSRawErasureCoderFactory;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -58,7 +58,7 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
      */
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        JRSRawErasureCoderFactory.class.getCanonicalName());
+        RSRawErasureCoderFactory.class.getCanonicalName());
     conf.setBoolean(
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false);
     prepare(conf, 10, 4, null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
new file mode 100644
index 0000000..109e46e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test XOR encoding and decoding.
+ */
+public class TestXORCoder extends TestErasureCoderBase {
+
+  @Before
+  public void setup() {
+    this.encoderClass = XORErasureEncoder.class;
+    this.decoderClass = XORErasureDecoder.class;
+
+    this.numDataUnits = 10;
+    this.numParityUnits = 1;
+    this.erasedDataIndexes = new int[] {0};
+
+    this.numChunksInBlock = 10;
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer() {
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer() {
+    testCoding(true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
deleted file mode 100644
index d46fe06..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.io.erasurecode.coder;
-
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test XOR encoding and decoding.
- */
-public class TestXorCoder extends TestErasureCoderBase {
-
-  @Before
-  public void setup() {
-    this.encoderClass = XorErasureEncoder.class;
-    this.decoderClass = XorErasureDecoder.class;
-
-    this.numDataUnits = 10;
-    this.numParityUnits = 1;
-    this.erasedDataIndexes = new int[] {0};
-
-    this.numChunksInBlock = 10;
-  }
-
-  @Test
-  public void testCodingNoDirectBuffer() {
-    testCoding(false);
-  }
-
-  @Test
-  public void testCodingDirectBuffer() {
-    testCoding(true);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
deleted file mode 100644
index 39e5deb..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.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.
- */
-public class TestJRSRawCoder 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);
-  }
-
-  @Before
-  public void setup() {
-    this.encoderClass = JRSRawEncoder.class;
-    this.decoderClass = JRSRawDecoder.class;
-  }
-
-  @Test
-  public void testCodingNoDirectBuffer_10x4() {
-    prepare(null, 10, 4, null);
-    testCoding(false);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_10x4() {
-    prepare(null, 10, 4, null);
-    testCoding(true);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
-    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});
-    testCoding(true);
-  }
-
-  @Test
-  public void testCodingNoDirectBuffer_3x3() {
-    prepare(null, 3, 3, null);
-    testCoding(false);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_3x3() {
-    prepare(null, 3, 3, null);
-    testCoding(true);
-  }
-
-  @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/f24e2442/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
new file mode 100644
index 0000000..8bb5d0f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 raw Reed-solomon encoding and decoding.
+ */
+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);
+  }
+
+  @Before
+  public void setup() {
+    this.encoderClass = RSRawEncoder.class;
+    this.decoderClass = RSRawDecoder.class;
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_10x4() {
+    prepare(null, 10, 4, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4() {
+    prepare(null, 10, 4, null);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
+    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});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_3x3() {
+    prepare(null, 3, 3, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_3x3() {
+    prepare(null, 3, 3, null);
+    testCoding(true);
+  }
+
+  @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/f24e2442/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
new file mode 100644
index 0000000..e66e48d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test XOR encoding and decoding.
+ */
+public class TestXORRawCoder extends TestRawCoderBase {
+
+  @Before
+  public void setup() {
+    this.encoderClass = XORRawEncoder.class;
+    this.decoderClass = XORRawDecoder.class;
+
+    this.numDataUnits = 10;
+    this.numParityUnits = 1;
+
+    this.erasedDataIndexes = new int[] {0};
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer() {
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer() {
+    testCoding(true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24e2442/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
deleted file mode 100644
index ff48586..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
+++ /dev/null
@@ -1,51 +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.io.erasurecode.rawcoder;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.Random;
-
-/**
- * Test XOR encoding and decoding.
- */
-public class TestXorRawCoder extends TestRawCoderBase {
-
-  @Before
-  public void setup() {
-    this.encoderClass = XorRawEncoder.class;
-    this.decoderClass = XorRawDecoder.class;
-
-    this.numDataUnits = 10;
-    this.numParityUnits = 1;
-
-    this.erasedDataIndexes = new int[] {0};
-  }
-
-  @Test
-  public void testCodingNoDirectBuffer() {
-    testCoding(false);
-  }
-
-  @Test
-  public void testCodingDirectBuffer() {
-    testCoding(true);
-  }
-
-}


[15/50] hadoop git commit: HDFS-8074 Define a system-wide default EC schema. Contributed by Kai Zheng

Posted by ji...@apache.org.
HDFS-8074 Define a system-wide default EC schema. Contributed by Kai Zheng


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

Branch: refs/heads/HDFS-7285
Commit: 44aa26ac4f1fee981f36d9f6a34776283147230a
Parents: a7a65f2
Author: Kai Zheng <ka...@intel.com>
Authored: Thu Apr 9 01:30:02 2015 +0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:14:10 2015 -0700

----------------------------------------------------------------------
 .../src/main/conf/ecschema-def.xml              |  5 --
 .../apache/hadoop/io/erasurecode/ECSchema.java  | 57 +++++++++++++++++-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  4 +-
 .../hdfs/server/namenode/ECSchemaManager.java   | 62 ++++++++++++++++++++
 4 files changed, 120 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/44aa26ac/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
index e619485..e36d386 100644
--- a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
+++ b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
@@ -27,11 +27,6 @@ You can modify and remove those not used yet, or add new ones.
 -->
 
 <schemas>
-  <schema name="RS-6-3">
-    <k>6</k>
-    <m>3</m>
-    <codec>RS</codec>
-  </schema>
   <schema name="RS-10-4">
     <k>10</k>
     <m>4</m>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44aa26ac/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 27be00e..8c3310e 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
@@ -23,12 +23,12 @@ import java.util.Map;
 /**
  * Erasure coding schema to housekeeper relevant information.
  */
-public class ECSchema {
+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 = 64 * 1024; // 64K
+  public static final int DEFAULT_CHUNK_SIZE = 256 * 1024; // 256K
 
   private String schemaName;
   private String codecName;
@@ -82,6 +82,18 @@ public class ECSchema {
   }
 
   /**
+   * Constructor with key parameters provided.
+   * @param schemaName
+   * @param codecName
+   * @param numDataUnits
+   * @param numParityUnits
+   */
+  public ECSchema(String schemaName, String codecName,
+                  int numDataUnits, int numParityUnits) {
+    this(schemaName, codecName, numDataUnits, numParityUnits, null);
+  }
+
+  /**
    * Constructor with key parameters provided. Note the options may contain
    * additional information for the erasure codec to interpret further.
    * @param schemaName
@@ -200,4 +212,45 @@ public class ECSchema {
 
     return sb.toString();
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    ECSchema ecSchema = (ECSchema) o;
+
+    if (numDataUnits != ecSchema.numDataUnits) {
+      return false;
+    }
+    if (numParityUnits != ecSchema.numParityUnits) {
+      return false;
+    }
+    if (chunkSize != ecSchema.chunkSize) {
+      return false;
+    }
+    if (!schemaName.equals(ecSchema.schemaName)) {
+      return false;
+    }
+    if (!codecName.equals(ecSchema.codecName)) {
+      return false;
+    }
+    return options.equals(ecSchema.options);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = schemaName.hashCode();
+    result = 31 * result + codecName.hashCode();
+    result = 31 * result + options.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/44aa26ac/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 7423033..5078a15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -52,4 +52,6 @@
     manage EC zones (Zhe Zhang)
 
     HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from
-    NameNode (vinayakumarb)
\ No newline at end of file
+    NameNode (vinayakumarb)
+    
+    HDFS-8074. Define a system-wide default EC schema. (Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44aa26ac/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
new file mode 100644
index 0000000..b001c57
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+/**
+ * This manages EC schemas predefined and activated in the system. It loads from
+ * predefined ones in XML and syncs with persisted ones in NameNode image.
+ *
+ * This class is instantiated by the FSNamesystem.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+public final class ECSchemaManager {
+
+  private static final int DEFAULT_DATA_BLOCKS = 6;
+  private static final int DEFAULT_PARITY_BLOCKS = 3;
+  private static final String DEFAULT_CODEC_NAME = "rs";
+  private static final String DEFAULT_SCHEMA_NAME = "SYS-DEFAULT-RS-6-3";
+
+  private static ECSchema SYS_DEFAULT_SCHEMA = new ECSchema(DEFAULT_SCHEMA_NAME,
+      DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS);
+
+  /**
+   * 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, but for safety we check all properties.
+    return SYS_DEFAULT_SCHEMA.equals(schema);
+  }
+}


[11/50] hadoop git commit: HDFS-8077. Erasure coding: fix bugs in EC zone and symlinks. Contributed by Jing Zhao and Zhe Zhang.

Posted by ji...@apache.org.
HDFS-8077. Erasure coding: fix bugs in EC zone and symlinks. Contributed by Jing Zhao and Zhe Zhang.


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

Branch: refs/heads/HDFS-7285
Commit: 19f5d1fa54ff605594ff15b45f845f2cecd32f7b
Parents: 17c271d
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Apr 9 17:53:22 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:14:10 2015 -0700

----------------------------------------------------------------------
 .../BlockInfoStripedUnderConstruction.java         |  2 +-
 .../hdfs/server/blockmanagement/BlockManager.java  | 12 ++++++------
 .../server/namenode/ErasureCodingZoneManager.java  |  7 +++++++
 .../hadoop/hdfs/server/namenode/FSDirectory.java   |  4 ++--
 .../hdfs/server/namenode/FSEditLogLoader.java      | 11 ++++++-----
 .../hdfs/server/namenode/FSImageSerialization.java |  4 ++--
 .../hadoop/hdfs/server/namenode/INodeFile.java     | 17 ++++-------------
 .../hdfs/server/namenode/TestFSEditLogLoader.java  |  4 ++--
 .../hadoop/hdfs/server/namenode/TestFSImage.java   |  2 +-
 .../server/namenode/TestRecoverStripedBlocks.java  |  2 +-
 10 files changed, 32 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f5d1fa/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 cfaf3a0..0373314 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
@@ -96,7 +96,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
     for(int i = 0; i < numLocations; i++) {
       // when creating a new block we simply sequentially assign block index to
       // each storage
-      Block blk = new Block(this.getBlockId() + i, this.getGenerationStamp(), 0);
+      Block blk = new Block(this.getBlockId() + i, 0, this.getGenerationStamp());
       replicas[i] = new ReplicaUnderConstruction(blk, targets[i],
           ReplicaState.RBW);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f5d1fa/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 90ec426..d46af17 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
@@ -2460,12 +2460,12 @@ public class BlockManager {
       case COMMITTED:
         if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
           final long reportedGS = reported.getGenerationStamp();
-          return new BlockToMarkCorrupt(reported, storedBlock, reportedGS,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS,
               "block is " + ucState + " and reported genstamp " + reportedGS
               + " does not match genstamp in block map "
               + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
         } else if (storedBlock.getNumBytes() != reported.getNumBytes()) {
-          return new BlockToMarkCorrupt(reported, storedBlock,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
               "block is " + ucState + " and reported length " +
               reported.getNumBytes() + " does not match " +
               "length in block map " + storedBlock.getNumBytes(),
@@ -2476,7 +2476,7 @@ public class BlockManager {
       case UNDER_CONSTRUCTION:
         if (storedBlock.getGenerationStamp() > reported.getGenerationStamp()) {
           final long reportedGS = reported.getGenerationStamp();
-          return new BlockToMarkCorrupt(reported, storedBlock, reportedGS,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS,
               "block is " + ucState + " and reported state " + reportedState
               + ", But reported genstamp " + reportedGS
               + " does not match genstamp in block map "
@@ -2492,7 +2492,7 @@ public class BlockManager {
         return null; // not corrupt
       } else if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
         final long reportedGS = reported.getGenerationStamp();
-        return new BlockToMarkCorrupt(reported, storedBlock, reportedGS,
+        return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS,
             "reported " + reportedState + " replica with genstamp " + reportedGS
             + " does not match COMPLETE block's genstamp in block map "
             + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
@@ -2507,7 +2507,7 @@ public class BlockManager {
               "complete with the same genstamp");
           return null;
         } else {
-          return new BlockToMarkCorrupt(reported, storedBlock,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
               "reported replica has invalid state " + reportedState,
               Reason.INVALID_STATE);
         }
@@ -2520,7 +2520,7 @@ public class BlockManager {
       " on " + dn + " size " + storedBlock.getNumBytes();
       // log here at WARN level since this is really a broken HDFS invariant
       LOG.warn(msg);
-      return new BlockToMarkCorrupt(reported, storedBlock, msg,
+      return new BlockToMarkCorrupt(new Block(reported), storedBlock, msg,
           Reason.INVALID_STATE);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f5d1fa/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 d4ff7c5..606e804 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
@@ -59,6 +59,13 @@ public class ErasureCodingZoneManager {
       if (inode == null) {
         continue;
       }
+      // We don't allow symlinks in an EC zone, or pointing to a file/dir in
+      // an EC. Therefore if a symlink is encountered, the dir shouldn't have
+      // EC
+      // TODO: properly support symlinks in EC zones
+      if (inode.isSymlink()) {
+        return false;
+      }
       final List<XAttr> xAttrs = inode.getXAttrFeature() == null ?
           new ArrayList<XAttr>(0)
           : inode.getXAttrFeature().getXAttrs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f5d1fa/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 a07ff23..f11b6a4 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
@@ -468,8 +468,8 @@ public class FSDirectory implements Closeable {
     try {
       INodesInPath iip = addINode(existing, newNode);
       if (iip != null) {
-        // TODO: we will no longer use storage policy for "Erasure Coding Zone"
-        if (newNode.isStriped()) {
+        // check if the file is in an EC zone
+        if (getECPolicy(iip)) {
           newNode.addStripedBlocksFeature();
         }
         if (aclEntries != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f5d1fa/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 8bea5d3..de930e6 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
@@ -418,7 +418,7 @@ public class FSEditLogLoader {
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, addCloseOp, iip, newFile, newFile.isStriped());
+      updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.getECPolicy(iip));
       break;
     }
     case OP_CLOSE: {
@@ -439,7 +439,7 @@ public class FSEditLogLoader {
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, addCloseOp, iip, file, file.isStriped());
+      updateBlocks(fsDir, addCloseOp, iip, file, fsDir.getECPolicy(iip));
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -498,7 +498,7 @@ public class FSEditLogLoader {
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, updateOp, iip, oldFile, oldFile.isStriped());
+      updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.getECPolicy(iip));
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
@@ -512,10 +512,11 @@ public class FSEditLogLoader {
         FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " new block id : " + addBlockOp.getLastBlock().getBlockId());
       }
-      INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
+      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, oldFile.isStriped());
+      addNewBlock(addBlockOp, oldFile, fsDir.getECPolicy(iip));
       break;
     }
     case OP_SET_REPLICATION: {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f5d1fa/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 1e58858..58244e5 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
@@ -207,7 +207,7 @@ public class FSImageSerialization {
     out.writeLong(cons.getModificationTime());
     out.writeLong(cons.getPreferredBlockSize());
     // whether the file has striped blocks
-    out.writeBoolean(cons.isWithStripedBlocks());
+    out.writeBoolean(cons.isStriped());
     writeBlocks(cons.getBlocks(), out);
     cons.getPermissionStatus().write(out);
 
@@ -233,7 +233,7 @@ public class FSImageSerialization {
     out.writeLong(file.getAccessTime());
     out.writeLong(file.getPreferredBlockSize());
     // whether the file has striped blocks
-    out.writeBoolean(file.isWithStripedBlocks());
+    out.writeBoolean(file.isStriped());
     writeBlocks(file.getBlocks(), out);
     SnapshotFSImageFormat.saveFileDiffList(file, out);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f5d1fa/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 deb89d7..53caa8d 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
@@ -185,17 +185,13 @@ public class INodeFile extends INodeWithAdditionalFields
   public FileWithStripedBlocksFeature addStripedBlocksFeature() {
     assert blocks == null || blocks.length == 0:
         "The file contains contiguous blocks";
-    assert !isWithStripedBlocks();
+    assert !isStriped();
     this.setFileReplication((short) 0);
     FileWithStripedBlocksFeature sb = new FileWithStripedBlocksFeature();
     addFeature(sb);
     return sb;
   }
 
-  public boolean isWithStripedBlocks() {
-    return getStripedBlocksFeature() != null;
-  }
-
   /** Used to make sure there is no contiguous block related info */
   private boolean hasNoContiguousBlock() {
     return (blocks == null || blocks.length == 0) && getFileReplication() == 0;
@@ -431,7 +427,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication,
       int latestSnapshotId) throws QuotaExceededException {
-    Preconditions.checkState(!isWithStripedBlocks(),
+    Preconditions.checkState(!isStriped(),
         "Cannot set replication to a file with striped blocks");
     recordModification(latestSnapshotId);
     setFileReplication(replication);
@@ -653,7 +649,7 @@ public class INodeFile extends INodeWithAdditionalFields
     long nsDelta = 1;
     final long ssDeltaNoReplication;
     short replication;
-    if (isWithStripedBlocks()) {
+    if (isStriped()) {
       return computeQuotaUsageWithStriped(bsps, counts);
     }
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
@@ -695,11 +691,6 @@ public class INodeFile extends INodeWithAdditionalFields
 
   /**
    * Compute quota of striped file
-   * @param bsps
-   * @param counts
-   * @param useCache
-   * @param lastSnapshotId
-   * @return quota counts
    */
   public final QuotaCounts computeQuotaUsageWithStriped(
       BlockStoragePolicySuite bsps, QuotaCounts counts) {
@@ -828,7 +819,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * Use preferred block size for the last block if it is under construction.
    */
   public final long storagespaceConsumed() {
-    if (isWithStripedBlocks()) {
+    if (isStriped()) {
       return storagespaceConsumedWithStriped();
     } else {
       return storagespaceConsumedNoReplication() * getBlockReplication();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f5d1fa/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 407d07e..0eeb7f8 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
@@ -472,7 +472,7 @@ public class TestFSEditLogLoader {
       INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory()
           .getINode(testFilePath);
 
-      assertTrue(inodeLoaded.isWithStripedBlocks());
+      assertTrue(inodeLoaded.isStriped());
 
       BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
       assertEquals(1, blks.length);
@@ -551,7 +551,7 @@ public class TestFSEditLogLoader {
       INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory()
           .getINode(testFilePath);
 
-      assertTrue(inodeLoaded.isWithStripedBlocks());
+      assertTrue(inodeLoaded.isStriped());
 
       BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
       assertEquals(1, blks.length);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f5d1fa/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 83f01c6..a456cad 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
@@ -199,7 +199,7 @@ public class TestFSImage {
     assertEquals(mtime, fileByLoaded.getModificationTime());
     assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime());
     assertEquals(0, fileByLoaded.getContiguousBlocks().length);
-    assertEquals(0, fileByLoaded.getBlockReplication());
+    assertEquals(0, fileByLoaded.getFileReplication());
     assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
 
     //check the BlockInfoStriped

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f5d1fa/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
index b2ff6c8..4292f9a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -84,7 +84,7 @@ public class TestRecoverStripedBlocks {
     final INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
         .getINode4Write(filePath.toString()).asFile();
     assertFalse(fileNode.isUnderConstruction());
-    assertTrue(fileNode.isWithStripedBlocks());
+    assertTrue(fileNode.isStriped());
     BlockInfo[] blocks = fileNode.getBlocks();
     assertEquals(numBlocks, blocks.length);
     for (BlockInfo blk : blocks) {


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

Posted by ji...@apache.org.
HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to create BlockReader. Contributed by Tsz Wo Nicholas Sze.


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

Branch: refs/heads/HDFS-7285
Commit: dbeb2c9981a3339a2cc3a620e9bd55151b9a6804
Parents: 4b0ad40
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Apr 21 20:56:39 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:55 2015 -0700

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


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

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

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

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


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

Posted by ji...@apache.org.
HDFS-8166. DFSStripedOutputStream should not create empty blocks. Contributed by Jing Zhao.


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

Branch: refs/heads/HDFS-7285
Commit: 4921a4efd32a70e2cbd0e4e79111529a20002efc
Parents: 54cad2d
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Apr 17 17:55:19 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:54 2015 -0700

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


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

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

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

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


[06/50] hadoop git commit: HADOOP-11740. Combine erasure encoder and decoder interfaces. Contributed by Zhe Zhang. Updated CHANGES-HDFS-EC-7285.txt

Posted by ji...@apache.org.
HADOOP-11740. Combine erasure encoder and decoder interfaces. Contributed by Zhe Zhang.
Updated CHANGES-HDFS-EC-7285.txt


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

Branch: refs/heads/HDFS-7285
Commit: b95f2335b5b31b66391b0701288a0e8c98c28a21
Parents: c0c6534
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 7 15:35:18 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:06:50 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt | 2 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt     | 5 +----
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b95f2335/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 68d1d32..7716728 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -33,5 +33,7 @@
     HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin
     ( Xinwei Qin via Kai Zheng )
 
+    HADOOP-11740. Combine erasure encoder and decoder interfaces (Zhe Zhang)
+
     HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng
     ( Kai Zheng )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b95f2335/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 3874cb4..9927ccf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -49,7 +49,4 @@
     (Hui Zheng via Zhe Zhang)
 
     HDFS-7839. Erasure coding: implement facilities in NameNode to create and
-    manage EC zones (Zhe Zhang)
-
-    HADOOP-11740. Combine erasure encoder and decoder interfaces (Zhe Zhang)
-
+    manage EC zones (Zhe Zhang)
\ No newline at end of file


[03/50] hadoop git commit: HADOOP-11740. Combine erasure encoder and decoder interfaces. Contributed by Zhe Zhang.

Posted by ji...@apache.org.
HADOOP-11740. Combine erasure encoder and decoder interfaces. 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/b0ef98ea
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b0ef98ea
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b0ef98ea

Branch: refs/heads/HDFS-7285
Commit: b0ef98ea335c319dcebb2b59004feb033dd3050a
Parents: d717dc3
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Apr 3 15:22:50 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:06:49 2015 -0700

----------------------------------------------------------------------
 .../coder/AbstractErasureDecoder.java           |  7 ++--
 .../coder/AbstractErasureEncoder.java           |  7 ++--
 .../io/erasurecode/coder/ErasureCoder.java      | 12 ++++++
 .../io/erasurecode/coder/ErasureDecoder.java    | 41 --------------------
 .../io/erasurecode/coder/ErasureEncoder.java    | 39 -------------------
 .../erasurecode/coder/TestErasureCoderBase.java | 20 +++++-----
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        | 14 ++++++-
 7 files changed, 41 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0ef98ea/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 54a6d1e..cd31294 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
@@ -23,13 +23,12 @@ import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 /**
  * An abstract erasure decoder that's to be inherited by new decoders.
  *
- * It implements the {@link ErasureDecoder} interface.
+ * It implements the {@link ErasureCoder} interface.
  */
-public abstract class AbstractErasureDecoder extends AbstractErasureCoder
-    implements ErasureDecoder {
+public abstract class AbstractErasureDecoder extends AbstractErasureCoder {
 
   @Override
-  public ErasureCodingStep decode(ECBlockGroup blockGroup) {
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
     // We may have more than this when considering complicate cases. HADOOP-11550
     return prepareDecodingStep(blockGroup);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0ef98ea/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 09b31e5..a836b75 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
@@ -23,13 +23,12 @@ import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 /**
  * An abstract erasure encoder that's to be inherited by new encoders.
  *
- * It implements the {@link ErasureEncoder} interface.
+ * It implements the {@link ErasureCoder} interface.
  */
-public abstract class AbstractErasureEncoder extends AbstractErasureCoder
-    implements ErasureEncoder {
+public abstract class AbstractErasureEncoder extends AbstractErasureCoder {
 
   @Override
-  public ErasureCodingStep encode(ECBlockGroup blockGroup) {
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
     // We may have more than this when considering complicate cases. HADOOP-11550
     return prepareEncodingStep(blockGroup);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0ef98ea/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 c5922f3..fb90156 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 
 /**
  * An erasure coder to perform encoding or decoding given a group. Generally it
@@ -64,6 +65,17 @@ public interface ErasureCoder extends Configurable {
   public int getChunkSize();
 
   /**
+   * Calculate the encoding or decoding steps given a block blockGroup.
+   *
+   * Note, currently only one coding step is supported. Will support complex
+   * cases of multiple coding steps.
+   *
+   * @param blockGroup the erasure coding block group containing all necessary
+   *                   information for codec calculation
+   */
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup);
+
+  /**
    * 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.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0ef98ea/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
deleted file mode 100644
index dfd9e54..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.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.io.erasurecode.coder;
-
-
-import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-
-/**
- * Erasure decoder interface to perform decoding given a block group.
- *
- * It extends {@link ErasureCoder}.
- */
-public interface ErasureDecoder extends ErasureCoder {
-
-  /**
-   * Perform the decoding given a blockGroup. By default it will try the best to
-   * attempt to recover all the missing blocks according to the codec logic.
-   *
-   * Note, currently only one coding step is supported. Will support complex
-   * cases of multiple coding steps.
-   *
-   * @param blockGroup
-   */
-  public ErasureCodingStep decode(ECBlockGroup blockGroup);
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0ef98ea/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
deleted file mode 100644
index e837d22..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.io.erasurecode.coder;
-
-import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-
-/**
- * Erasure encoder interface to perform encoding given a block group.
- *
- * It extends {@link ErasureCoder}.
- */
-public interface ErasureEncoder extends ErasureCoder {
-
-  /**
-   * Calculate the encoding steps given a block blockGroup.
-   *
-   * Note, currently only one coding step is supported. Will support complex
-   * cases of multiple coding steps.
-   *
-   * @param blockGroup
-   */
-  public ErasureCodingStep encode(ECBlockGroup blockGroup);
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0ef98ea/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 b963a59..05a62a7 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
@@ -26,8 +26,8 @@ import org.apache.hadoop.io.erasurecode.TestCoderBase;
  * Erasure coder test base with utilities.
  */
 public abstract class TestErasureCoderBase extends TestCoderBase {
-  protected Class<? extends ErasureEncoder> encoderClass;
-  protected Class<? extends ErasureDecoder> decoderClass;
+  protected Class<? extends ErasureCoder> encoderClass;
+  protected Class<? extends ErasureCoder> decoderClass;
 
   protected int numChunksInBlock = 16;
 
@@ -55,7 +55,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   protected void testCoding(boolean usingDirectBuffer) {
     this.usingDirectBuffer = usingDirectBuffer;
 
-    ErasureEncoder encoder = createEncoder();
+    ErasureCoder encoder = createEncoder();
 
     // Generate data and encode
     ECBlockGroup blockGroup = prepareBlockGroupForEncoding();
@@ -68,7 +68,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
 
     ErasureCodingStep codingStep;
     try {
-      codingStep = encoder.encode(blockGroup);
+      codingStep = encoder.calculateCoding(blockGroup);
       performCodingStep(codingStep);
     } finally {
       encoder.release();
@@ -78,9 +78,9 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
 
     //Decode
     blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
-    ErasureDecoder decoder = createDecoder();
+    ErasureCoder decoder = createDecoder();
     try {
-      codingStep = decoder.decode(blockGroup);
+      codingStep = decoder.calculateCoding(blockGroup);
       performCodingStep(codingStep);
     } finally {
       decoder.release();
@@ -138,8 +138,8 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
    * Create erasure encoder for test.
    * @return
    */
-  private ErasureEncoder createEncoder() {
-    ErasureEncoder encoder;
+  private ErasureCoder createEncoder() {
+    ErasureCoder encoder;
     try {
       encoder = encoderClass.newInstance();
     } catch (Exception e) {
@@ -155,8 +155,8 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
    * Create the erasure decoder for the test.
    * @return
    */
-  private ErasureDecoder createDecoder() {
-    ErasureDecoder decoder;
+  private ErasureCoder createDecoder() {
+    ErasureCoder decoder;
     try {
       decoder = decoderClass.newInstance();
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0ef98ea/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 a686315..4e60a7c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -43,4 +43,16 @@
     blocks in NameNode (Jing Zhao)
 
     HDFS-8005. Erasure Coding: simplify striped block recovery work computation
-    and add tests (Jing Zhao)
\ No newline at end of file
+    and add tests (Jing Zhao)
+
+    HDFS-7617. Add unit tests for editlog transactions for EC 
+    (Hui Zheng via Zhe Zhang)
+
+    HADOOP-11782. Correct two thrown messages in ECSchema class
+    (Xinwei Qin via Kai Zheng)
+
+    HDFS-7839. Erasure coding: implement facilities in NameNode to create and
+    manage EC zones (Zhe Zhang)
+
+    HADOOP-11740. Combine erasure encoder and decoder interfaces (Zhe Zhang)
+


[12/50] hadoop git commit: HADOOP-11818 Minor improvements for erasurecode classes. Contributed by Rakesh R

Posted by ji...@apache.org.
HADOOP-11818 Minor improvements for erasurecode classes. 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/17c271dd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/17c271dd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/17c271dd

Branch: refs/heads/HDFS-7285
Commit: 17c271dd85811c7d97dcb117e3f0be0ad11f5db2
Parents: ae85757
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Apr 10 04:31:48 2015 +0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:14:10 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt       |  2 ++
 .../hadoop/io/erasurecode/SchemaLoader.java      | 12 ++++++------
 .../io/erasurecode/coder/RSErasureDecoder.java   | 19 ++++++++++++++++++-
 .../io/erasurecode/coder/RSErasureEncoder.java   | 19 ++++++++++++++++++-
 .../io/erasurecode/coder/XORErasureDecoder.java  |  2 +-
 .../io/erasurecode/rawcoder/util/RSUtil.java     | 17 +++++++++++++++++
 6 files changed, 62 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/17c271dd/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 c72394e..b850e11 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -40,3 +40,5 @@
 
     HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code
     ( Kai Zheng via vinayakumarb )
+  
+    HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/17c271dd/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
index c51ed37..75dd03a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.io.erasurecode;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.w3c.dom.*;
@@ -36,7 +36,7 @@ import java.util.*;
  * A EC schema loading utility that loads predefined EC schemas from XML file
  */
 public class SchemaLoader {
-  private static final Log LOG = LogFactory.getLog(SchemaLoader.class.getName());
+  private static final Logger LOG = LoggerFactory.getLogger(SchemaLoader.class.getName());
 
   /**
    * Load predefined ec schemas from configuration file. This file is
@@ -63,7 +63,7 @@ public class SchemaLoader {
   private List<ECSchema> loadSchema(File schemaFile)
       throws ParserConfigurationException, IOException, SAXException {
 
-    LOG.info("Loading predefined EC schema file " + schemaFile);
+    LOG.info("Loading predefined EC schema file {}", schemaFile);
 
     // Read and parse the schema file.
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
@@ -87,7 +87,7 @@ public class SchemaLoader {
           ECSchema schema = loadSchema(element);
             schemas.add(schema);
         } else {
-          LOG.warn("Bad element in EC schema configuration file: " +
+          LOG.warn("Bad element in EC schema configuration file: {}",
               element.getTagName());
         }
       }
@@ -109,7 +109,7 @@ public class SchemaLoader {
       URL url = Thread.currentThread().getContextClassLoader()
           .getResource(schemaFilePath);
       if (url == null) {
-        LOG.warn(schemaFilePath + " not found on the classpath.");
+        LOG.warn("{} not found on the classpath.", schemaFilePath);
         schemaFile = null;
       } else if (! url.getProtocol().equalsIgnoreCase("file")) {
         throw new RuntimeException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/17c271dd/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 e2c5051..fc664a5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -1,3 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.coder;
 
 import org.apache.hadoop.conf.Configuration;
@@ -11,7 +28,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 /**
  * Reed-Solomon erasure decoder that decodes a block group.
  *
- * It implements {@link ErasureDecoder}.
+ * It implements {@link ErasureCoder}.
  */
 public class RSErasureDecoder extends AbstractErasureDecoder {
   private RawErasureDecoder rsRawDecoder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/17c271dd/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 a7d02b5..18ca5ac 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
@@ -1,3 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.coder;
 
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -9,7 +26,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 /**
  * Reed-Solomon erasure encoder that encodes a block group.
  *
- * It implements {@link ErasureEncoder}.
+ * It implements {@link ErasureCoder}.
  */
 public class RSErasureEncoder extends AbstractErasureEncoder {
   private RawErasureEncoder rawEncoder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/17c271dd/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 6f4b423..0672549 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
@@ -25,7 +25,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 /**
  * Xor erasure decoder that decodes a block group.
  *
- * It implements {@link ErasureDecoder}.
+ * It implements {@link ErasureCoder}.
  */
 public class XORErasureDecoder extends AbstractErasureDecoder {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/17c271dd/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
index 33ba561..8badf02 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
@@ -1,3 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util;
 
 /**


[20/50] hadoop git commit: HDFS-8122. Erasure Coding: Support specifying ECSchema during creation of ECZone. Contributed by Vinayakumar B.

Posted by ji...@apache.org.
HDFS-8122. Erasure Coding: Support specifying ECSchema during creation of 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/5d78eec3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5d78eec3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5d78eec3

Branch: refs/heads/HDFS-7285
Commit: 5d78eec334f5e452a7630bec4a93d0f28ea65129
Parents: 87e09b7
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 13 11:08:57 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:15:36 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  6 ++--
 .../hadoop/hdfs/DistributedFileSystem.java      | 33 ++++++++++++++++++++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  6 ++--
 ...tNamenodeProtocolServerSideTranslatorPB.java |  4 ++-
 .../ClientNamenodeProtocolTranslatorPB.java     |  5 ++-
 .../namenode/ErasureCodingZoneManager.java      | 30 +++++++++++++-----
 .../hdfs/server/namenode/FSDirectory.java       | 22 ++++++++-----
 .../hdfs/server/namenode/FSNamesystem.java      | 19 ++++++-----
 .../hdfs/server/namenode/NameNodeRpcServer.java |  6 ++--
 .../src/main/proto/ClientNamenodeProtocol.proto |  1 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  2 +-
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |  2 +-
 .../hadoop/hdfs/TestErasureCodingZones.java     | 18 +++++------
 .../server/namenode/TestAddStripedBlocks.java   |  2 +-
 .../server/namenode/TestFSEditLogLoader.java    |  4 +--
 .../hdfs/server/namenode/TestFSImage.java       |  4 +--
 16 files changed, 112 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 994d5ac..a254485 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
@@ -1319,7 +1319,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                              Progressable progress,
                              int buffersize,
                              ChecksumOpt checksumOpt) throws IOException {
-    return create(src, permission, flag, createParent, replication, blockSize, 
+    return create(src, permission, flag, createParent, replication, blockSize,
         progress, buffersize, checksumOpt, null);
   }
 
@@ -2970,12 +2970,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return new EncryptionZoneIterator(namenode, traceSampler);
   }
 
-  public void createErasureCodingZone(String src)
+  public void createErasureCodingZone(String src, ECSchema schema)
       throws IOException {
     checkOpen();
     TraceScope scope = getPathTraceScope("createErasureCodingZone", src);
     try {
-      namenode.createErasureCodingZone(src);
+      namenode.createErasureCodingZone(src, schema);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
           SafeModeException.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 4ca6d57..4c3e0a5 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
@@ -86,6 +86,7 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Credentials;
@@ -2264,4 +2265,36 @@ public class DistributedFileSystem extends FileSystem {
       throws IOException {
     return dfs.getInotifyEventStream(lastReadTxid);
   }
+
+  /**
+   * Create the erasurecoding zone
+   * 
+   * @param path Directory to create the ec zone
+   * @param schema ECSchema for the zone. If not specified default will be used.
+   * @throws IOException
+   */
+  public void createErasureCodingZone(final Path path, final ECSchema schema)
+      throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        dfs.createErasureCodingZone(getPathName(p), null);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p) throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          myDfs.createErasureCodingZone(p, schema);
+          return null;
+        }
+        throw new UnsupportedOperationException(
+            "Cannot createErasureCodingZone through a symlink to a "
+                + "non-DistributedFileSystem: " + path + " -> " + p);
+      }
+    }.resolve(this, absF);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 7f5ac49..0c04ca9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1364,11 +1364,11 @@ public interface ClientProtocol {
       long prevId) throws IOException;
 
   /**
-   * Create an erasure coding zone (currently with hardcoded schema)
-   * TODO: Configurable and pluggable schemas (HDFS-7337)
+   * Create an erasure coding zone with specified schema, if any, otherwise
+   * default
    */
   @Idempotent
-  public void createErasureCodingZone(String src)
+  public void createErasureCodingZone(String src, ECSchema schema)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 79ab939..48f0efd 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
@@ -1403,7 +1403,9 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, CreateErasureCodingZoneRequestProto req)
       throws ServiceException {
     try {
-      server.createErasureCodingZone(req.getSrc());
+      ECSchema schema = req.hasSchema() ? PBHelper.convertECSchema(req
+          .getSchema()) : null;
+      server.createErasureCodingZone(req.getSrc(), schema);
       return CreateErasureCodingZoneResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 0211522..2e17823 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
@@ -1420,11 +1420,14 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void createErasureCodingZone(String src)
+  public void createErasureCodingZone(String src, ECSchema schema)
       throws IOException {
     final CreateErasureCodingZoneRequestProto.Builder builder =
         CreateErasureCodingZoneRequestProto.newBuilder();
     builder.setSrc(src);
+    if (schema != null) {
+      builder.setSchema(PBHelper.convertECSchema(schema));
+    }
     CreateErasureCodingZoneRequestProto req = builder.build();
     try {
       rpcProxy.createErasureCodingZone(null, req);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 606e804..c7daa2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
@@ -22,6 +22,9 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.XAttrHelper;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -50,7 +53,11 @@ public class ErasureCodingZoneManager {
     this.dir = dir;
   }
 
-  boolean getECPolicy(INodesInPath iip) {
+  boolean getECPolicy(INodesInPath iip) throws IOException {
+    return getECSchema(iip) != null;
+  }
+
+  ECSchema getECSchema(INodesInPath iip) throws IOException{
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     List<INode> inodes = iip.getReadOnlyINodes();
@@ -64,21 +71,23 @@ public class ErasureCodingZoneManager {
       // EC
       // TODO: properly support symlinks in EC zones
       if (inode.isSymlink()) {
-        return false;
+        return null;
       }
       final List<XAttr> xAttrs = inode.getXAttrFeature() == null ?
           new ArrayList<XAttr>(0)
           : inode.getXAttrFeature().getXAttrs();
       for (XAttr xAttr : xAttrs) {
         if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
-          return true;
+          ECSchemaProto ecSchemaProto;
+          ecSchemaProto = ECSchemaProto.parseFrom(xAttr.getValue());
+          return PBHelper.convertECSchema(ecSchemaProto);
         }
       }
     }
-    return false;
+    return null;
   }
 
-  XAttr createErasureCodingZone(String src)
+  XAttr createErasureCodingZone(String src, ECSchema schema)
       throws IOException {
     assert dir.hasWriteLock();
     final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false);
@@ -97,8 +106,15 @@ public class ErasureCodingZoneManager {
       throw new IOException("Directory " + src + " is already in an " +
           "erasure coding zone.");
     }
-    final XAttr ecXAttr = XAttrHelper
-        .buildXAttr(XATTR_ERASURECODING_ZONE, null);
+    // TODO HDFS-7859 Need to persist the schema in xattr in efficient way
+    // As of now storing the protobuf format
+    if (schema == null) {
+      schema = ECSchemaManager.getSystemDefaultSchema();
+    }
+    ECSchemaProto schemaProto = PBHelper.convertECSchema(schema);
+    byte[] schemaBytes = schemaProto.toByteArray();
+    final XAttr ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_ZONE,
+        schemaBytes);
     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/5d78eec3/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 f11b6a4..ab4d160 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
@@ -66,6 +66,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.util.ByteArray;
 import org.apache.hadoop.hdfs.util.EnumCounters;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
@@ -414,11 +415,12 @@ public class FSDirectory implements Closeable {
    * Add the given filename to the fs.
    * @return the new INodesInPath instance that contains the new INode
    */
-  INodesInPath addFile(INodesInPath existing, String localName, PermissionStatus
-      permissions, short replication, long preferredBlockSize,
+  INodesInPath addFile(INodesInPath existing, String localName,
+      PermissionStatus permissions, short replication, long preferredBlockSize,
       String clientName, String clientMachine)
-    throws FileAlreadyExistsException, QuotaExceededException,
-      UnresolvedLinkException, SnapshotAccessControlException, AclException {
+      throws FileAlreadyExistsException, QuotaExceededException,
+      UnresolvedLinkException, SnapshotAccessControlException, AclException,
+      IOException {
 
     long modTime = now();
     INodeFile newNode = newINodeFile(allocateNewInodeId(), permissions, modTime,
@@ -1400,20 +1402,24 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  XAttr createErasureCodingZone(String src)
+  XAttr createErasureCodingZone(String src, ECSchema schema)
       throws IOException {
     writeLock();
     try {
-      return ecZoneManager.createErasureCodingZone(src);
+      return ecZoneManager.createErasureCodingZone(src, schema);
     } finally {
       writeUnlock();
     }
   }
 
-  public boolean getECPolicy(INodesInPath iip) {
+  public boolean getECPolicy(INodesInPath iip) throws IOException {
+    return getECSchema(iip) != null;
+  }
+
+  ECSchema getECSchema(INodesInPath iip) throws IOException {
     readLock();
     try {
-      return ecZoneManager.getECPolicy(iip);
+      return ecZoneManager.getECSchema(iip);
     } finally {
       readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 78c82ce..7c1fd16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -8120,16 +8120,16 @@ 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
    *                erasure coding zone. The directory must be empty.
+   *
    * @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 boolean logRetryCache)
-      throws IOException, UnresolvedLinkException,
+  void createErasureCodingZone(final String srcArg, final ECSchema schema,
+      final boolean logRetryCache) throws IOException, UnresolvedLinkException,
       SafeModeException, AccessControlException {
     String src = srcArg;
     HdfsFileStatus resultingStat = null;
@@ -8145,7 +8145,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);
+      final XAttr ecXAttr = dir.createErasureCodingZone(src, schema);
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
       xAttrs.add(ecXAttr);
       getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
@@ -8175,11 +8175,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (isPermissionEnabled) {
         dir.checkPathAccess(pc, iip, FsAction.READ);
       }
-      if (dir.getECPolicy(iip)) {
-        // TODO HDFS-8074 and HDFS-7859 : To get from loaded schemas
-        Map<String, String> options = new HashMap<String, String>();
-        ECSchema defaultSchema = new ECSchema("RS-6-3", "rs", 6, 3, options);
-        return new ECInfo(src, defaultSchema);
+      // Get schema set for the zone
+      ECSchema schema = dir.getECSchema(iip);
+      if (schema != null) {
+        return new ECInfo(src, schema);
       }
     } finally {
       readUnlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 54e31b0..23e87a8 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
@@ -1845,8 +1845,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public void createErasureCodingZone(String src)
-    throws IOException {
+  public void createErasureCodingZone(String src, ECSchema schema)
+      throws IOException {
     checkNNStartup();
     final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
@@ -1854,7 +1854,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
     boolean success = false;
     try {
-      namesystem.createErasureCodingZone(src, cacheEntry != null);
+      namesystem.createErasureCodingZone(src, schema, cacheEntry != null);
     } finally {
       RetryCache.setState(cacheEntry, success);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 3389a22..c9059bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -716,6 +716,7 @@ message GetEditsFromTxidResponseProto {
 
 message CreateErasureCodingZoneRequestProto {
   required string src = 1;
+  optional ECSchemaProto schema = 2;
 }
 
 message CreateErasureCodingZoneResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 90984e4..aaacf23 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
@@ -1815,7 +1815,7 @@ public class DFSTestUtil {
       int numBlocks, int numStripesPerBlk) throws Exception {
     DistributedFileSystem dfs = cluster.getFileSystem();
     dfs.mkdirs(dir);
-    dfs.getClient().createErasureCodingZone(dir.toString());
+    dfs.getClient().createErasureCodingZone(dir.toString(), null);
 
     FSDataOutputStream out = null;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 ee6998b..c78922e 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
@@ -50,7 +50,7 @@ public class TestDFSStripedOutputStream {
     Configuration conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, cellsInBlock * cellSize);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
-    cluster.getFileSystem().getClient().createErasureCodingZone("/");
+    cluster.getFileSystem().getClient().createErasureCodingZone("/", null);
     fs = cluster.getFileSystem();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 bdca915..699df4e 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
@@ -64,7 +64,7 @@ public class TestErasureCodingZones {
     fs.mkdir(testDir, FsPermission.getDirDefault());
 
     /* Normal creation of an erasure coding zone */
-    fs.getClient().createErasureCodingZone(testDir.toString());
+    fs.getClient().createErasureCodingZone(testDir.toString(), null);
 
     /* Verify files under the zone are striped */
     final Path ECFilePath = new Path(testDir, "foo");
@@ -77,7 +77,7 @@ public class TestErasureCodingZones {
     fs.mkdir(notEmpty, FsPermission.getDirDefault());
     fs.create(new Path(notEmpty, "foo"));
     try {
-      fs.getClient().createErasureCodingZone(notEmpty.toString());
+      fs.getClient().createErasureCodingZone(notEmpty.toString(), null);
       fail("Erasure coding zone on non-empty dir");
     } catch (IOException e) {
       assertExceptionContains("erasure coding zone for a non-empty directory", e);
@@ -87,10 +87,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());
+    fs.getClient().createErasureCodingZone(zone1.toString(), null);
     fs.mkdir(zone2, FsPermission.getDirDefault());
     try {
-      fs.getClient().createErasureCodingZone(zone2.toString());
+      fs.getClient().createErasureCodingZone(zone2.toString(), null);
       fail("Nested erasure coding zones");
     } catch (IOException e) {
       assertExceptionContains("already in an erasure coding zone", e);
@@ -100,7 +100,7 @@ public class TestErasureCodingZones {
     final Path fPath = new Path("/file");
     fs.create(fPath);
     try {
-      fs.getClient().createErasureCodingZone(fPath.toString());
+      fs.getClient().createErasureCodingZone(fPath.toString(), null);
       fail("Erasure coding zone on file");
     } catch (IOException e) {
       assertExceptionContains("erasure coding zone for a file", e);
@@ -113,8 +113,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());
-    fs.getClient().createErasureCodingZone(dstECDir.toString());
+    fs.getClient().createErasureCodingZone(srcECDir.toString(), null);
+    fs.getClient().createErasureCodingZone(dstECDir.toString(), null);
     final Path srcFile = new Path(srcECDir, "foo");
     fs.create(srcFile);
 
@@ -158,7 +158,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);
+    fs.getClient().createErasureCodingZone(src, null);
     verifyErasureCodingInfo(src);
     fs.create(new Path(ecDir, "/child1")).close();
     // verify for the files in ec zone
@@ -182,4 +182,4 @@ public class TestErasureCodingZones {
     assertEquals("Default chunkSize should be used",
         ECSchema.DEFAULT_CHUNK_SIZE, schema.getChunkSize());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 c3c8239..27df1cd 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
@@ -68,7 +68,7 @@ public class TestAddStripedBlocks {
         .numDataNodes(GROUP_SIZE).build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
-    dfs.getClient().createErasureCodingZone("/");
+    dfs.getClient().createErasureCodingZone("/", null);
   }
 
   @After

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d78eec3/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 a456cad..fe130a9 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
@@ -135,7 +135,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("/", false);
+    fsn.createErasureCodingZone("/", null, false);
     long id = 123456789;
     byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
     PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
@@ -397,7 +397,7 @@ public class TestFSImage {
           .build();
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
-      fs.getClient().getNamenode().createErasureCodingZone("/");
+      fs.getClient().getNamenode().createErasureCodingZone("/", null);
       Path file = new Path("/striped");
       FSDataOutputStream out = fs.create(file);
       byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE);


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 4b0ad40441f6ea56f63c57b371d1837547b2b2b1
Parents: df77601
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Apr 20 17:42:02 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:55 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b0ad404/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
index 2368021..d622d4d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@ -25,6 +25,8 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Utility class for analyzing striped block groups
  */
@@ -81,46 +83,43 @@ public class StripedBlockUtil {
   /**
    * Get the size of an internal block at the given index of a block group
    *
-   * @param numBytesInGroup Size of the block group only counting data blocks
+   * @param dataSize Size of the block group only counting data blocks
    * @param cellSize The size of a striping cell
-   * @param dataBlkNum The number of data blocks
-   * @param idxInGroup The logical index in the striped block group
+   * @param numDataBlocks The number of data blocks
+   * @param i The logical index in the striped block group
    * @return The size of the internal block at the specified index
    */
-  public static long getInternalBlockLength(long numBytesInGroup,
-      int cellSize, int dataBlkNum, int idxInGroup) {
+  public static long getInternalBlockLength(long dataSize,
+      int cellSize, int numDataBlocks, int i) {
+    Preconditions.checkArgument(dataSize >= 0);
+    Preconditions.checkArgument(cellSize > 0);
+    Preconditions.checkArgument(numDataBlocks > 0);
+    Preconditions.checkArgument(i >= 0);
     // Size of each stripe (only counting data blocks)
-    final long numBytesPerStripe = cellSize * dataBlkNum;
-    assert numBytesPerStripe  > 0:
-        "getInternalBlockLength should only be called on valid striped blocks";
+    final int stripeSize = cellSize * numDataBlocks;
     // If block group ends at stripe boundary, each internal block has an equal
     // share of the group
-    if (numBytesInGroup % numBytesPerStripe == 0) {
-      return numBytesInGroup / dataBlkNum;
+    final int lastStripeDataLen = (int)(dataSize % stripeSize);
+    if (lastStripeDataLen == 0) {
+      return dataSize / numDataBlocks;
     }
 
-    int numStripes = (int) ((numBytesInGroup - 1) / numBytesPerStripe + 1);
-    assert numStripes >= 1 : "There should be at least 1 stripe";
-
-    // All stripes but the last one are full stripes. The block should at least
-    // contain (numStripes - 1) full cells.
-    long blkSize = (numStripes - 1) * cellSize;
-
-    long lastStripeLen = numBytesInGroup % numBytesPerStripe;
-    // Size of parity cells should equal the size of the first cell, if it
-    // is not full.
-    long lastParityCellLen = Math.min(cellSize, lastStripeLen);
-
-    if (idxInGroup >= dataBlkNum) {
-      // for parity blocks
-      blkSize += lastParityCellLen;
-    } else {
-      // for data blocks
-      blkSize +=  Math.min(cellSize,
-          Math.max(0, lastStripeLen - cellSize * idxInGroup));
+    final int numStripes = (int) ((dataSize - 1) / stripeSize + 1);
+    return (numStripes - 1L)*cellSize
+        + lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i);
+  }
+  
+  private static int lastCellSize(int size, int cellSize, int numDataBlocks,
+      int i) {
+    if (i < numDataBlocks) {
+      // parity block size (i.e. i >= numDataBlocks) is the same as 
+      // the first data block size (i.e. i = 0).
+      size -= i*cellSize;
+      if (size < 0) {
+        size = 0;
+      }
     }
-
-    return blkSize;
+    return size > cellSize? cellSize: size;
   }
 
   /**

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


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

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


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

Branch: refs/heads/HDFS-7285
Commit: de6b66d6341471589ae702a1d36127acf79da64a
Parents: 3297989
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Apr 15 12:09:16 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:15:36 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de6b66d6/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index c11b963..a13a2bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -343,6 +343,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>hdfs.proto</include>
                   <include>encryption.proto</include>
                   <include>inotify.proto</include>
+                  <include>erasurecoding.proto</include>
                 </includes>
               </source>
               <output>${project.build.directory}/generated-sources/java</output>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de6b66d6/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 48f0efd..169ea2d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -107,12 +107,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
@@ -197,14 +193,18 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de6b66d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 2e17823..44d1258 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -107,11 +107,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
@@ -165,13 +161,16 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Trunca
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.*;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de6b66d6/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 dd86e9d..a323534 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -131,6 +131,9 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDele
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -151,9 +154,6 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeLocalInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de6b66d6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
index c7daa2b..5320c1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
@@ -22,7 +22,7 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.XAttrHelper;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de6b66d6/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
new file mode 100644
index 0000000..4d5731b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ 
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "ErasureCodingProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+
+/**
+ * ECSchema options entry
+ */
+message ECSchemaOptionEntryProto {
+  required string key = 1;
+  required string value = 2;
+}
+
+/**
+ * ECSchema for erasurecoding
+ */
+message ECSchemaProto {
+  required string schemaName = 1;
+  required string codecName = 2;
+  required uint32 dataUnits = 3;
+  required uint32 parityUnits = 4;
+  repeated ECSchemaOptionEntryProto options = 5;
+}
+
+/**
+ * ECInfo
+ */
+message ECInfoProto {
+ required string src = 1;
+ required ECSchemaProto schema = 2;
+}
+
+
+message CreateErasureCodingZoneRequestProto {
+  required string src = 1;
+  optional ECSchemaProto schema = 2;
+}
+
+message CreateErasureCodingZoneResponseProto {
+}
+
+message GetErasureCodingInfoRequestProto {
+  required string src = 1;
+}
+
+message GetErasureCodingInfoResponseProto {
+  optional ECInfoProto ECInfo = 1;
+}
+
+message GetECSchemasRequestProto { // void request
+}
+
+message GetECSchemasResponseProto {
+  repeated ECSchemaProto schemas = 1;
+}

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


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

Posted by ji...@apache.org.
HDFS-8228. Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause block id conflicts. Contributed by Jing Zhao.


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

Branch: refs/heads/HDFS-7285
Commit: d1e7dfa0e0e9bf76f4925936f27b1768339cb5dd
Parents: f24c2c0
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Apr 24 09:30:38 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:56 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1e7dfa0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
index e9e22ee..de8e379 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
@@ -19,9 +19,11 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.util.SequentialNumber;
 
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_GROUP_INDEX_MASK;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_BLOCKS_IN_GROUP;
+
 /**
  * Generate the next valid block group ID by incrementing the maximum block
  * group ID allocated so far, with the first 2^10 block group IDs reserved.
@@ -34,6 +36,9 @@ import org.apache.hadoop.util.SequentialNumber;
  * bits (n+2) to (64-m) represent the ID of its block group, while the last m
  * bits represent its index of the group. The value m is determined by the
  * maximum number of blocks in a group (MAX_BLOCKS_IN_GROUP).
+ *
+ * Note that the {@link #nextValue()} methods requires external lock to
+ * guarantee IDs have no conflicts.
  */
 @InterfaceAudience.Private
 public class SequentialBlockGroupIdGenerator extends SequentialNumber {
@@ -47,32 +52,30 @@ public class SequentialBlockGroupIdGenerator extends SequentialNumber {
 
   @Override // NumberGenerator
   public long nextValue() {
-    // Skip to next legitimate block group ID based on the naming protocol
-    while (super.getCurrentValue() % HdfsConstants.MAX_BLOCKS_IN_GROUP > 0) {
-      super.nextValue();
-    }
+    skipTo((getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK) + MAX_BLOCKS_IN_GROUP);
     // Make sure there's no conflict with existing random block IDs
-    while (hasValidBlockInRange(super.getCurrentValue())) {
-      super.skipTo(super.getCurrentValue() +
-          HdfsConstants.MAX_BLOCKS_IN_GROUP);
+    final Block b = new Block(getCurrentValue());
+    while (hasValidBlockInRange(b)) {
+      skipTo(getCurrentValue() + MAX_BLOCKS_IN_GROUP);
+      b.setBlockId(getCurrentValue());
     }
-    if (super.getCurrentValue() >= 0) {
-      BlockManager.LOG.warn("All negative block group IDs are used, " +
-          "growing into positive IDs, " +
-          "which might conflict with non-erasure coded blocks.");
+    if (b.getBlockId() >= 0) {
+      throw new IllegalStateException("All negative block group IDs are used, "
+          + "growing into positive IDs, "
+          + "which might conflict with non-erasure coded blocks.");
     }
-    return super.getCurrentValue();
+    return getCurrentValue();
   }
 
   /**
-   *
-   * @param id The starting ID of the range
+   * @param b A block object whose id is set to the starting point for check
    * @return true if any ID in the range
    *      {id, id+HdfsConstants.MAX_BLOCKS_IN_GROUP} is pointed-to by a file
    */
-  private boolean hasValidBlockInRange(long id) {
-    for (int i = 0; i < HdfsConstants.MAX_BLOCKS_IN_GROUP; i++) {
-      Block b = new Block(id + i);
+  private boolean hasValidBlockInRange(Block b) {
+    final long id = b.getBlockId();
+    for (int i = 0; i < MAX_BLOCKS_IN_GROUP; i++) {
+      b.setBlockId(id + i);
       if (blockManager.getBlockCollection(b) != null) {
         return true;
       }

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1e7dfa0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index 6bb1162..d03e938 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -82,6 +82,27 @@ public class TestAddStripedBlocks {
     }
   }
 
+  /**
+   * Make sure the IDs of striped blocks do not conflict
+   */
+  @Test
+  public void testAllocateBlockId() throws Exception {
+    Path testPath = new Path("/testfile");
+    // create a file while allocates a new block
+    DFSTestUtil.writeFile(dfs, testPath, "hello, world!");
+    LocatedBlocks lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0);
+    final long firstId = lb.get(0).getBlock().getBlockId();
+    // delete the file
+    dfs.delete(testPath, true);
+
+    // allocate a new block, and make sure the new block's id does not conflict
+    // with the previous one
+    DFSTestUtil.writeFile(dfs, testPath, "hello again");
+    lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0);
+    final long secondId = lb.get(0).getBlock().getBlockId();
+    Assert.assertEquals(firstId + HdfsConstants.MAX_BLOCKS_IN_GROUP, secondId);
+  }
+
   @Test
   public void testAddStripedBlock() throws Exception {
     final Path file = new Path("/file1");


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

Posted by ji...@apache.org.
HDFS-8181. createErasureCodingZone sets retryCache state as false always (Contributed by Uma Maheswara Rao G)


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

Branch: refs/heads/HDFS-7285
Commit: 80e516fcc977de4b05f6a0cfbbf0258749a543af
Parents: 92aadcd
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Apr 20 15:04:49 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:55 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80e516fc/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 7bbf358..36510f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -1856,6 +1856,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     boolean success = false;
     try {
       namesystem.createErasureCodingZone(src, schema, cacheEntry != null);
+      success = true;
     } finally {
       RetryCache.setState(cacheEntry, success);
     }


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 3102e6af1c6a7a0ac52d5a52447a5e4690ec9eb4
Parents: 6452510
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Apr 28 13:42:24 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:17:52 2015 -0700

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


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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3102e6af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
index fe9e101..f6f7ed2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@ -134,11 +134,12 @@ public class DFSStripedInputStream extends DFSInputStream {
   private final short parityBlkNum;
   private final ECInfo ecInfo;
 
-  DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum)
+  DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, ECInfo info)
       throws IOException {
     super(dfsClient, src, verifyChecksum);
     // ECInfo is restored from NN just before reading striped file.
-    ecInfo = dfsClient.getErasureCodingInfo(src);
+    assert info != null;
+    ecInfo = info;
     cellSize = ecInfo.getSchema().getChunkSize();
     dataBlkNum = (short)ecInfo.getSchema().getNumDataUnits();
     parityBlkNum = (short)ecInfo.getSchema().getNumParityUnits();

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

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


[10/50] hadoop git commit: HDFS-7782. Erasure coding: pread from files in striped layout. Contributed by Zhe Zhang and Jing Zhao

Posted by ji...@apache.org.
HDFS-7782. Erasure coding: pread from files in striped layout. 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/593bbfd5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/593bbfd5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/593bbfd5

Branch: refs/heads/HDFS-7285
Commit: 593bbfd53a8aa7561acc74bb1a382ee7b68b5859
Parents: 347619a
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Apr 7 11:20:13 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:14:09 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/LocatedBlock.java      |   4 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  55 +++
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   8 +-
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  80 +++-
 .../hadoop/hdfs/DFSStripedInputStream.java      | 367 +++++++++++++++++++
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   2 +-
 .../hdfs/protocol/LocatedStripedBlock.java      |   5 +
 .../blockmanagement/BlockInfoStriped.java       |   6 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  92 ++++-
 .../apache/hadoop/hdfs/TestReadStripedFile.java | 304 +++++++++++++++
 .../namenode/TestRecoverStripedBlocks.java      |  88 +----
 11 files changed, 897 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
index 4e8f202..a9596bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
@@ -203,4 +203,8 @@ public class LocatedBlock {
         + "; locs=" + Arrays.asList(locs)
         + "}";
   }
+
+  public boolean isStriped() {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/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 da3b0e5..ff8bad0 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
@@ -236,6 +236,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
       new DFSHedgedReadMetrics();
   private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
+  private static volatile ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
   private final Sampler<?> traceSampler;
 
   public DfsClientConf getConf() {
@@ -371,6 +372,19 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) {
       this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize());
     }
+    numThreads = conf.getInt(
+        DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE,
+        DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE);
+    if (numThreads <= 0) {
+      LOG.warn("The value of "
+          + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE
+          + " must be greater than 0. The current setting is " + numThreads
+          + ". Reset it to the default value "
+          + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE);
+      numThreads =
+          DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE;
+    }
+    this.initThreadsNumForStripedReads(numThreads);
     this.saslClient = new SaslDataTransferClient(
       conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf),
       TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth);
@@ -3151,11 +3165,52 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       LOG.debug("Using hedged reads; pool threads=" + num);
     }
   }
+  
+  /**
+   * Create thread pool for parallel reading in striped layout,
+   * STRIPED_READ_THREAD_POOL, if it does not already exist.
+   * @param num Number of threads for striped reads thread pool.
+   */
+  private void initThreadsNumForStripedReads(int num) {
+    assert num > 0;
+    if (STRIPED_READ_THREAD_POOL != null) {
+      return;
+    }
+    synchronized (DFSClient.class) {
+      if (STRIPED_READ_THREAD_POOL == null) {
+        STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
+            TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new Daemon.DaemonFactory() {
+          private final AtomicInteger threadIndex = new AtomicInteger(0);
+
+          @Override
+          public Thread newThread(Runnable r) {
+            Thread t = super.newThread(r);
+            t.setName("stripedRead-" + threadIndex.getAndIncrement());
+            return t;
+          }
+        }, new ThreadPoolExecutor.CallerRunsPolicy() {
+          @Override
+          public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {
+            LOG.info("Execution for striped reading rejected, "
+                + "Executing in current thread");
+            // will run in the current thread
+            super.rejectedExecution(runnable, e);
+          }
+        });
+        STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
+      }
+    }
+  }
 
   ThreadPoolExecutor getHedgedReadsThreadPool() {
     return HEDGED_READ_THREAD_POOL;
   }
 
+  ThreadPoolExecutor getStripedReadsThreadPool() {
+    return STRIPED_READ_THREAD_POOL;
+  }
+
   boolean isHedgedReadsEnabled() {
     return (HEDGED_READ_THREAD_POOL != null) &&
       HEDGED_READ_THREAD_POOL.getMaximumPoolSize() > 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/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 5fe3ec7..3c83156 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
@@ -653,7 +653,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.reject-unresolved-dn-topology-mapping";
   public static final boolean DFS_REJECT_UNRESOLVED_DN_TOPOLOGY_MAPPING_DEFAULT =
       false;
-  
+
+  public static final String DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE =
+      "dfs.client.striped.read.threadpool.size";
+  // With default 3+2 schema, each normal read could span 3 DNs. So this
+  // default value accommodates 6 read streams
+  public static final int DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE = 18;
+
   // Slow io warning log threshold settings for dfsclient and datanode.
   public static final String DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY =
     "dfs.datanode.slow.io.warning.threshold.ms";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/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 3290223..79bbd54 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
@@ -43,6 +43,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ByteBufferReadable;
@@ -93,7 +94,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   @VisibleForTesting
   public static boolean tcpReadsDisabledForTesting = false;
   private long hedgedReadOpsLoopNumForTesting = 0;
-  private final DFSClient dfsClient;
+  protected final DFSClient dfsClient;
   private AtomicBoolean closed = new AtomicBoolean(false);
   private final String src;
   private final boolean verifyChecksum;
@@ -440,7 +441,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * @return located block
    * @throws IOException
    */
-  private LocatedBlock getBlockAt(long offset) throws IOException {
+  protected LocatedBlock getBlockAt(long offset) throws IOException {
     synchronized(infoLock) {
       assert (locatedBlocks != null) : "locatedBlocks is null";
 
@@ -712,7 +713,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Wraps different possible read implementations so that readBuffer can be
    * strategy-agnostic.
    */
-  private interface ReaderStrategy {
+  interface ReaderStrategy {
     public int doRead(BlockReader blockReader, int off, int len)
         throws ChecksumException, IOException;
   }
@@ -1055,7 +1056,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     return errMsgr.toString();
   }
 
-  private void fetchBlockByteRange(long blockStartOffset, long start, long end,
+  protected void fetchBlockByteRange(long blockStartOffset, long start, long end,
       byte[] buf, int offset,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
@@ -1097,13 +1098,42 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     };
   }
 
+  /**
+   * Used when reading contiguous blocks
+   */
   private void actualGetFromOneDataNode(final DNAddrPair datanode,
       long blockStartOffset, final long start, final long end, byte[] buf,
       int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
+    final int length = (int) (end - start + 1);
+    actualGetFromOneDataNode(datanode, block, start, end, buf,
+        new int[]{offset}, new int[]{length}, corruptedBlockMap);
+  }
+
+  /**
+   * Read data from one DataNode.
+   * @param datanode the datanode from which to read data
+   * @param block the block to read
+   * @param startInBlk the startInBlk offset of the block
+   * @param endInBlk the endInBlk offset of the block
+   * @param buf the given byte array into which the data is read
+   * @param offsets the data may be read into multiple segments of the buf
+   *                (when reading a striped block). this array indicates the
+   *                offset of each buf segment.
+   * @param lengths the length of each buf segment
+   * @param corruptedBlockMap map recording list of datanodes with corrupted
+   *                          block replica
+   */
+  void actualGetFromOneDataNode(final DNAddrPair datanode,
+      LocatedBlock block, final long startInBlk, final long endInBlk,
+      byte[] buf, int[] offsets, int[] lengths,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
     DFSClientFaultInjector.get().startFetchFromDatanode();
     int refetchToken = 1; // only need to get a new access token once
     int refetchEncryptionKey = 1; // only need to get a new encryption key once
+    final int len = (int) (endInBlk - startInBlk + 1);
+    checkReadPortions(offsets, lengths, len);
 
     while (true) {
       // cached block locations may have been updated by chooseDataNode()
@@ -1113,15 +1143,15 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       BlockReader reader = null;
       try {
         DFSClientFaultInjector.get().fetchFromDatanodeException();
-        int len = (int) (end - start + 1);
         reader = getBlockReader(block, start, len, datanode.addr,
             datanode.storageType, datanode.info);
-        int nread = reader.readAll(buf, offset, len);
-        updateReadStatistics(readStatistics, nread, reader);
-
-        if (nread != len) {
-          throw new IOException("truncated return from reader.read(): " +
-                                "excpected " + len + ", got " + nread);
+        for (int i = 0; i < offsets.length; i++) {
+          int nread = reader.readAll(buf, offsets[i], lengths[i]);
+          updateReadStatistics(readStatistics, nread, reader);
+          if (nread != len) {
+            throw new IOException("truncated return from reader.read(): " +
+                "excpected " + len + ", got " + nread);
+          }
         }
         DFSClientFaultInjector.get().readFromDatanodeDelay();
         return;
@@ -1166,7 +1196,26 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   }
 
   /**
-   * Like {@link #fetchBlockByteRange} except we start up a second, parallel,
+   * This method verifies that the read portions are valid and do not overlap
+   * with each other.
+   */
+  private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) {
+    Preconditions.checkArgument(offsets.length == lengths.length &&
+        offsets.length > 0);
+    int sum = 0;
+    for (int i = 0; i < lengths.length; i++) {
+      if (i > 0) {
+        int gap = offsets[i] - offsets[i - 1];
+        // make sure read portions do not overlap with each other
+        Preconditions.checkArgument(gap >= lengths[i - 1]);
+      }
+      sum += lengths[i];
+    }
+    Preconditions.checkArgument(sum == totalLen);
+  }
+
+  /**
+   * Like {@link #fetchBlockByteRange}except we start up a second, parallel,
    * 'hedged' read if the first read is taking longer than configured amount of
    * time. We then wait on which ever read returns first.
    */
@@ -1385,10 +1434,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       long targetStart = position - blk.getStartOffset();
       long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
       try {
-        if (dfsClient.isHedgedReadsEnabled()) {
+        if (dfsClient.isHedgedReadsEnabled() && !blk.isStriped()) {
           hedgedFetchBlockByteRange(blk.getStartOffset(), targetStart,
-              targetStart + bytesToRead - 1, buffer, offset,
-              corruptedBlockMap);
+              targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
         } else {
           fetchBlockByteRange(blk.getStartOffset(), targetStart,
               targetStart + bytesToRead - 1, buffer, offset,
@@ -1584,7 +1632,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   }
 
   /** Utility class to encapsulate data node info and its address. */
-  private static final class DNAddrPair {
+  static final class DNAddrPair {
     final DatanodeInfo info;
     final InetSocketAddress addr;
     final StorageType storageType;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/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
new file mode 100644
index 0000000..077b0f8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@ -0,0 +1,367 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutionException;
+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 #groupSize} 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: TODO: HDFS-8033
+ *   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
+ *****************************************************************************/
+public class DFSStripedInputStream extends DFSInputStream {
+  /**
+   * This method plans the read portion from each block in the stripe
+   * @param groupSize The size / width of the striping group
+   * @param cellSize The size of each striping cell
+   * @param startInBlk Starting offset in the striped block
+   * @param len Length of the read request
+   * @param bufOffset  Initial offset in the result buffer
+   * @return array of {@link ReadPortion}, each representing the portion of I/O
+   *         for an individual block in the group
+   */
+  @VisibleForTesting
+  static ReadPortion[] planReadPortions(final int groupSize,
+      final int cellSize, final long startInBlk, final int len, int bufOffset) {
+    ReadPortion[] results = new ReadPortion[groupSize];
+    for (int i = 0; i < groupSize; i++) {
+      results[i] = new ReadPortion();
+    }
+
+    // cellIdxInBlk is the index of the cell in the block
+    // E.g., cell_3 is the 2nd cell in blk_0
+    int cellIdxInBlk = (int) (startInBlk / (cellSize * groupSize));
+
+    // blkIdxInGroup is the index of the block in the striped block group
+    // E.g., blk_2 is the 3rd block in the group
+    final int blkIdxInGroup = (int) (startInBlk / cellSize % groupSize);
+    results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk +
+        startInBlk % cellSize;
+    boolean crossStripe = false;
+    for (int i = 1; i < groupSize; i++) {
+      if (blkIdxInGroup + i >= groupSize && !crossStripe) {
+        cellIdxInBlk++;
+        crossStripe = true;
+      }
+      results[(blkIdxInGroup + i) % groupSize].startOffsetInBlock =
+          cellSize * cellIdxInBlk;
+    }
+
+    int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len);
+    results[blkIdxInGroup].offsetsInBuf.add(bufOffset);
+    results[blkIdxInGroup].lengths.add(firstCellLen);
+    results[blkIdxInGroup].readLength += firstCellLen;
+
+    int i = (blkIdxInGroup + 1) % groupSize;
+    for (int done = firstCellLen; done < len; done += cellSize) {
+      ReadPortion rp = results[i];
+      rp.offsetsInBuf.add(done + bufOffset);
+      final int readLen = Math.min(len - done, cellSize);
+      rp.lengths.add(readLen);
+      rp.readLength += readLen;
+      i = (i + 1) % groupSize;
+    }
+    return results;
+  }
+
+  /**
+   * This method parses a striped block group into individual blocks.
+   *
+   * @param bg The striped block group
+   * @param dataBlkNum the number of data blocks
+   * @return An array containing the blocks in the group
+   */
+  @VisibleForTesting
+  static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg,
+      int dataBlkNum, int cellSize) {
+    int locatedBGSize = bg.getBlockIndices().length;
+    // TODO not considering missing blocks for now, only identify data blocks
+    LocatedBlock[] lbs = new LocatedBlock[dataBlkNum];
+    for (short i = 0; i < locatedBGSize; i++) {
+      final int idx = bg.getBlockIndices()[i];
+      if (idx < dataBlkNum && lbs[idx] == null) {
+        lbs[idx] = constructInternalBlock(bg, i, cellSize, idx);
+      }
+    }
+    return lbs;
+  }
+
+  private static LocatedBlock constructInternalBlock(LocatedStripedBlock bg,
+      int idxInReturnedLocs, int cellSize, int idxInBlockGroup) {
+    final ExtendedBlock blk = new ExtendedBlock(bg.getBlock());
+    blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup);
+    // TODO: fix the numBytes computation
+
+    return new LocatedBlock(blk,
+        new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
+        new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
+        new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
+        bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(),
+        null);
+  }
+
+
+  private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS;
+
+  DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum)
+      throws IOException {
+    super(dfsClient, src, verifyChecksum);
+    DFSClient.LOG.debug("Creating an striped input stream for file " + src);
+  }
+
+  @Override
+  public synchronized int read(final ByteBuffer buf) throws IOException {
+    throw new UnsupportedActionException("Stateful read is not supported");
+  }
+
+  @Override
+  public synchronized int read(final byte buf[], int off, int len)
+      throws IOException {
+    throw new UnsupportedActionException("Stateful read is not supported");
+  }
+
+  /**
+   * | <--------- LocatedStripedBlock (ID = 0) ---------> |
+   * LocatedBlock (0) | LocatedBlock (1) | LocatedBlock (2)
+   *                      ^
+   *                    offset
+   * On a striped file, the super method {@link DFSInputStream#getBlockAt}
+   * treats a striped block group as a single {@link LocatedBlock} object,
+   * which includes target in its range. This method adds the logic of:
+   *   1. Analyzing the index of required block based on offset
+   *   2. Parsing the block group to obtain the block location on that index
+   */
+  @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";
+
+    int idx = (int) (((blkStartOffset - lb.getStartOffset()) / cellSize)
+        % groupSize);
+    // If indexing information is returned, iterate through the index array
+    // to find the entry for position idx in the group
+    LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
+    int i = 0;
+    for (; i < lsb.getBlockIndices().length; i++) {
+      if (lsb.getBlockIndices()[i] == idx) {
+        break;
+      }
+    }
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("getBlockAt for striped blocks, offset="
+          + blkStartOffset + ". Obtained block " + lb + ", idx=" + idx);
+    }
+    return constructInternalBlock(lsb, i, cellSize, idx);
+  }
+
+  private LocatedBlock getBlockGroupAt(long offset) throws IOException {
+    return super.getBlockAt(offset);
+  }
+
+  /**
+   * Real implementation of pread.
+   */
+  @Override
+  protected void fetchBlockByteRange(LocatedBlock block, long start,
+      long end, byte[] buf, int offset,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    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
+    block = getBlockGroupAt(block.getStartOffset());
+    assert block instanceof LocatedStripedBlock : "NameNode" +
+        " should return a LocatedStripedBlock for a striped file";
+    LocatedStripedBlock blockGroup = (LocatedStripedBlock) block;
+
+    // Planning the portion of I/O for each shard
+    ReadPortion[] readPortions = planReadPortions(groupSize, cellSize, start,
+        len, offset);
+
+    // Parse group to get chosen DN location
+    LocatedBlock[] blks = parseStripedBlockGroup(blockGroup, groupSize, cellSize);
+
+    for (short i = 0; i < groupSize; i++) {
+      ReadPortion rp = readPortions[i];
+      if (rp.readLength <= 0) {
+        continue;
+      }
+      DatanodeInfo loc = blks[i].getLocations()[0];
+      StorageType type = blks[i].getStorageTypes()[0];
+      DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr(
+          loc.getXferAddr(dfsClient.getConf().connectToDnViaHostname)), type);
+      Callable<Void> readCallable = getFromOneDataNode(dnAddr, blks[i],
+          rp.startOffsetInBlock, rp.startOffsetInBlock + rp.readLength - 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);
+    }
+    while (!futures.isEmpty()) {
+      try {
+        waitNextCompletion(stripedReadsService, futures);
+      } catch (InterruptedException ie) {
+        // Ignore and retry
+      }
+    }
+  }
+
+  private Callable<Void> getFromOneDataNode(final DNAddrPair datanode,
+      final LocatedBlock block, final long start, final long end,
+      final byte[] buf, final int[] offsets, final int[] lengths,
+      final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
+      final int hedgedReadId) {
+    final Span parentSpan = Trace.currentSpan();
+    return new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        TraceScope scope =
+            Trace.startSpan("Parallel reading " + hedgedReadId, parentSpan);
+        try {
+          actualGetFromOneDataNode(datanode, block, start,
+              end, buf, offsets, lengths, corruptedBlockMap);
+        } finally {
+          scope.close();
+        }
+        return null;
+      }
+    };
+  }
+
+  private void waitNextCompletion(CompletionService<Void> stripedReadsService,
+      Map<Future<Void>, Integer> futures) throws InterruptedException {
+    if (futures.isEmpty()) {
+      throw new InterruptedException("Futures already empty");
+    }
+    Future<Void> future = null;
+    try {
+      future = stripedReadsService.take();
+      future.get();
+      futures.remove(future);
+    } catch (ExecutionException | CancellationException e) {
+      // already logged in the Callable
+      futures.remove(future);
+    }
+    throw new InterruptedException("let's retry");
+  }
+
+  public void setCellSize(int cellSize) {
+    this.cellSize = cellSize;
+  }
+
+  /**
+   * This class represents the portion of I/O associated with each block in the
+   * striped block group.
+   */
+  static class ReadPortion {
+    /**
+     * startOffsetInBlock
+     *     |
+     *     v
+     *     |<-lengths[0]->|<-  lengths[1]  ->|<-lengths[2]->|
+     * +------------------+------------------+----------------+
+     * |      cell_0      |      cell_3      |     cell_6     |  <- blk_0
+     * +------------------+------------------+----------------+
+     *   _/                \_______________________
+     *  |                                          |
+     *  v offsetsInBuf[0]                          v offsetsInBuf[1]
+     * +------------------------------------------------------+
+     * |  cell_0     |      cell_1 and cell_2      |cell_3 ...|   <- buf
+     * |  (partial)  |    (from blk_1 and blk_2)   |          |
+     * +------------------------------------------------------+
+     */
+    private long startOffsetInBlock = 0;
+    private long readLength = 0;
+    private final List<Integer> offsetsInBuf = new ArrayList<>();
+    private final List<Integer> lengths = new ArrayList<>();
+
+    int[] getOffsets() {
+      int[] offsets = new int[offsetsInBuf.size()];
+      for (int i = 0; i < offsets.length; i++) {
+        offsets[i] = offsetsInBuf.get(i);
+      }
+      return offsets;
+    }
+
+    int[] getLengths() {
+      int[] lens = new int[this.lengths.size()];
+      for (int i = 0; i < lens.length; i++) {
+        lens[i] = this.lengths.get(i);
+      }
+      return lens;
+    }
+
+    long getReadLength() {
+      return readLength;
+    }
+
+    long getStartOffsetInBlock() {
+      return startOffsetInBlock;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index 892c5f1..72b8f38 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -179,5 +179,5 @@ public class HdfsConstants {
   public static final byte MAX_BLOCKS_IN_GROUP = 16;
 
   // The chunk size for striped block which is used by erasure coding
-  public static final int BLOCK_STRIPED_CHUNK_SIZE = 64 * 1024;
+  public static final int BLOCK_STRIPED_CELL_SIZE = 128 * 1024;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
index 97e3a69..98614db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
@@ -65,4 +65,9 @@ public class LocatedStripedBlock extends LocatedBlock {
   public int[] getBlockIndices() {
     return this.blockIndices;
   }
+
+  @Override
+  public boolean isStriped() {
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/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 4a85efb..20b0c5c 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
@@ -23,7 +23,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 
 /**
  * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
@@ -203,8 +203,8 @@ public class BlockInfoStriped extends BlockInfo {
     // In case striped blocks, total usage by this striped blocks should
     // be the total of data blocks and parity blocks because
     // `getNumBytes` is the total of actual data block size.
-    return ((getNumBytes() - 1) / (dataBlockNum * BLOCK_STRIPED_CHUNK_SIZE) + 1)
-        * BLOCK_STRIPED_CHUNK_SIZE * parityBlockNum + getNumBytes();
+    return ((getNumBytes() - 1) / (dataBlockNum * BLOCK_STRIPED_CELL_SIZE) + 1)
+        * BLOCK_STRIPED_CELL_SIZE * parityBlockNum + getNumBytes();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/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 62d67d8..90984e4 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
@@ -64,6 +64,12 @@ import java.util.Random;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeoutException;
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
@@ -100,6 +106,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@@ -107,7 +114,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
@@ -122,8 +128,10 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
@@ -131,7 +139,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
-import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.IOUtils;
@@ -151,12 +158,8 @@ import org.apache.log4j.Level;
 import org.junit.Assume;
 import org.mockito.internal.util.reflection.Whitebox;
 
-import com.google.common.base.Charsets;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 
 /** Utilities for HDFS tests */
 public class DFSTestUtil {
@@ -1807,4 +1810,77 @@ public class DFSTestUtil {
     reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks);
     return reports;
   }
+
+  public static void createECFile(MiniDFSCluster cluster, Path file, Path dir,
+      int numBlocks, int numStripesPerBlk) throws Exception {
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    dfs.mkdirs(dir);
+    dfs.getClient().createErasureCodingZone(dir.toString());
+
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1); // create an empty file
+
+      FSNamesystem ns = cluster.getNamesystem();
+      FSDirectory fsdir = ns.getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+
+      ExtendedBlock previous = null;
+      for (int i = 0; i < numBlocks; i++) {
+        Block newBlock = createBlock(cluster.getDataNodes(), dfs, ns,
+            file.toString(), fileNode, dfs.getClient().getClientName(),
+            previous, numStripesPerBlk);
+        previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
+      }
+
+      dfs.getClient().namenode.complete(file.toString(),
+          dfs.getClient().getClientName(), previous, fileNode.getId());
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+  }
+
+  static Block createBlock(List<DataNode> dataNodes, DistributedFileSystem fs,
+      FSNamesystem ns, String file, INodeFile fileNode, String clientName,
+      ExtendedBlock previous, int numStripes) throws Exception {
+    fs.getClient().namenode.addBlock(file, clientName, previous, null,
+        fileNode.getId(), null);
+
+    final BlockInfo lastBlock = fileNode.getLastBlock();
+    final int groupSize = fileNode.getBlockReplication();
+    // 1. RECEIVING_BLOCK IBR
+    int i = 0;
+    for (DataNode dn : dataNodes) {
+      if (i < groupSize) {
+        final Block block = new Block(lastBlock.getBlockId() + i++, 0,
+            lastBlock.getGenerationStamp());
+        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+            .makeReportForReceivedBlock(block,
+                ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
+        for (StorageReceivedDeletedBlocks report : reports) {
+          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
+        }
+      }
+    }
+
+    // 2. RECEIVED_BLOCK IBR
+    i = 0;
+    for (DataNode dn : dataNodes) {
+      if (i < groupSize) {
+        final Block block = new Block(lastBlock.getBlockId() + i++,
+            numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp());
+        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+            .makeReportForReceivedBlock(block,
+                ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+        for (StorageReceivedDeletedBlocks report : reports) {
+          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
+        }
+      }
+    }
+
+    lastBlock.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS);
+    return lastBlock;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/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
new file mode 100644
index 0000000..0032bdd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
@@ -0,0 +1,304 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import static org.apache.hadoop.hdfs.DFSStripedInputStream.ReadPortion;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+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 final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS;
+  private final short TOTAL_SIZE = HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS;
+  private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final int NUM_STRIPE_PER_BLOCK = 2;
+  private final int BLOCKSIZE = 2 * GROUP_SIZE * CELLSIZE;
+
+  @Before
+  public void setup() throws IOException {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    SimulatedFSDataset.setFactory(conf);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(TOTAL_SIZE)
+        .build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private void testPlanReadPortions(int startInBlk, int length,
+      int bufferOffset, int[] readLengths, int[] offsetsInBlock,
+      int[][] bufferOffsets, int[][] bufferLengths) {
+    ReadPortion[] results = DFSStripedInputStream.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 DFSStripedInputStream#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}});
+  }
+
+  private LocatedStripedBlock createDummyLocatedBlock() {
+    final long blockGroupID = -1048576;
+    DatanodeInfo[] locs = new DatanodeInfo[TOTAL_SIZE];
+    String[] storageIDs = new String[TOTAL_SIZE];
+    StorageType[] storageTypes = new StorageType[TOTAL_SIZE];
+    int[] indices = new int[TOTAL_SIZE];
+    for (int i = 0; i < TOTAL_SIZE; i++) {
+      locs[i] = new DatanodeInfo(cluster.getDataNodes().get(i).getDatanodeId());
+      storageIDs[i] = cluster.getDataNodes().get(i).getDatanodeUuid();
+      storageTypes[i] = StorageType.DISK;
+      indices[i] = (i + 2) % GROUP_SIZE;
+    }
+    return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID),
+        locs, storageIDs, storageTypes, indices, 0, false, null);
+  }
+
+  @Test
+  public void testParseDummyStripedBlock() {
+    LocatedStripedBlock lsb = createDummyLocatedBlock();
+    LocatedBlock[] blocks = DFSStripedInputStream.parseStripedBlockGroup(
+        lsb, GROUP_SIZE, CELLSIZE);
+    assertEquals(GROUP_SIZE, blocks.length);
+    for (int j = 0; j < GROUP_SIZE; j++) {
+      assertFalse(blocks[j].isStriped());
+      assertEquals(j,
+          BlockIdManager.getBlockIndex(blocks[j].getBlock().getLocalBlock()));
+      assertEquals(j * CELLSIZE, blocks[j].getStartOffset());
+    }
+  }
+
+  @Test
+  public void testParseStripedBlock() throws Exception {
+    final int numBlocks = 4;
+    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
+        NUM_STRIPE_PER_BLOCK);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCKSIZE * numBlocks);
+
+    assertEquals(4, lbs.locatedBlockCount());
+    List<LocatedBlock> lbList = lbs.getLocatedBlocks();
+    for (LocatedBlock lb : lbList) {
+      assertTrue(lb.isStriped());
+    }
+
+    for (int i = 0; i < numBlocks; i++) {
+      LocatedStripedBlock lsb = (LocatedStripedBlock) (lbs.get(i));
+      LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb,
+          GROUP_SIZE, CELLSIZE);
+      assertEquals(GROUP_SIZE, blks.length);
+      for (int j = 0; j < GROUP_SIZE; j++) {
+        assertFalse(blks[j].isStriped());
+        assertEquals(j,
+            BlockIdManager.getBlockIndex(blks[j].getBlock().getLocalBlock()));
+        assertEquals(i * BLOCKSIZE + j * CELLSIZE, blks[j].getStartOffset());
+      }
+    }
+  }
+
+  /**
+   * Test {@link DFSStripedInputStream#getBlockAt(long)}
+   */
+  @Test
+  public void testGetBlock() throws Exception {
+    final int numBlocks = 4;
+    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
+        NUM_STRIPE_PER_BLOCK);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCKSIZE * numBlocks);
+    final DFSStripedInputStream in =
+        new DFSStripedInputStream(fs.getClient(), filePath.toString(), false);
+
+    List<LocatedBlock> lbList = lbs.getLocatedBlocks();
+    for (LocatedBlock aLbList : lbList) {
+      LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList;
+      LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb,
+          GROUP_SIZE, CELLSIZE);
+      for (int j = 0; j < GROUP_SIZE; j++) {
+        LocatedBlock 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 = 4;
+    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
+        NUM_STRIPE_PER_BLOCK);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCKSIZE);
+
+    assert lbs.get(0) instanceof LocatedStripedBlock;
+    LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0));
+    for (int i = 0; i < GROUP_SIZE; i++) {
+      Block blk = new Block(bg.getBlock().getBlockId() + i, BLOCKSIZE,
+          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);
+    in.setCellSize(CELLSIZE);
+    int readSize = BLOCKSIZE;
+    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
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/593bbfd5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
index d965ae7..b2ff6c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -18,15 +18,11 @@
 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.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
@@ -36,19 +32,14 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
-import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
-import org.apache.hadoop.io.IOUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.UUID;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
+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;
 import static org.junit.Assert.assertFalse;
@@ -84,83 +75,10 @@ public class TestRecoverStripedBlocks {
     }
   }
 
-  public static void createECFile(MiniDFSCluster cluster, Path file, Path dir,
-      int numBlocks) throws Exception {
-    DistributedFileSystem dfs = cluster.getFileSystem();
-    dfs.mkdirs(dir);
-    dfs.getClient().getNamenode().createErasureCodingZone(dir.toString());
-
-    FSDataOutputStream out = null;
-    try {
-      out = dfs.create(file, (short) 1); // create an empty file
-
-      FSNamesystem ns = cluster.getNamesystem();
-      FSDirectory fsdir = ns.getFSDirectory();
-      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
-
-      ExtendedBlock previous = null;
-      for (int i = 0; i < numBlocks; i++) {
-        Block newBlock = createBlock(cluster.getDataNodes(), ns,
-            file.toString(), fileNode, dfs.getClient().getClientName(),
-            previous);
-        previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
-      }
-
-      ns.completeFile(file.toString(), dfs.getClient().getClientName(),
-          previous, fileNode.getId());
-    } finally {
-      IOUtils.cleanup(null, out);
-    }
-  }
-
-  static Block createBlock(List<DataNode> dataNodes, FSNamesystem ns,
-      String file, INodeFile fileNode, String clientName,
-      ExtendedBlock previous) throws Exception {
-    ns.getAdditionalBlock(file, fileNode.getId(), clientName, previous, null,
-        null);
-
-    final BlockInfo lastBlock = fileNode.getLastBlock();
-    final int groupSize = fileNode.getBlockReplication();
-    // 1. RECEIVING_BLOCK IBR
-    int i = 0;
-    for (DataNode dn : dataNodes) {
-      if (i < groupSize) {
-        final Block block = new Block(lastBlock.getBlockId() + i++, 0,
-            lastBlock.getGenerationStamp());
-        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
-        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
-            .makeReportForReceivedBlock(block,
-                ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
-        for (StorageReceivedDeletedBlocks report : reports) {
-          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
-        }
-      }
-    }
-
-    // 2. RECEIVED_BLOCK IBR
-    i = 0;
-    for (DataNode dn : dataNodes) {
-      if (i < groupSize) {
-        final Block block = new Block(lastBlock.getBlockId() + i++,
-            BLOCK_STRIPED_CHUNK_SIZE, lastBlock.getGenerationStamp());
-        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
-        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
-            .makeReportForReceivedBlock(block,
-                ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
-        for (StorageReceivedDeletedBlocks report : reports) {
-          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
-        }
-      }
-    }
-
-    lastBlock.setNumBytes(BLOCK_STRIPED_CHUNK_SIZE * NUM_DATA_BLOCKS);
-    return lastBlock;
-  }
-
   @Test
   public void testMissingStripedBlock() throws Exception {
     final int numBlocks = 4;
-    createECFile(cluster, filePath, dirPath, numBlocks);
+    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, 1);
 
     // make sure the file is complete in NN
     final INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
@@ -172,7 +90,7 @@ public class TestRecoverStripedBlocks {
     for (BlockInfo blk : blocks) {
       assertTrue(blk.isStriped());
       assertTrue(blk.isComplete());
-      assertEquals(BLOCK_STRIPED_CHUNK_SIZE * NUM_DATA_BLOCKS, blk.getNumBytes());
+      assertEquals(BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS, blk.getNumBytes());
       final BlockInfoStriped sb = (BlockInfoStriped) blk;
       assertEquals(GROUP_SIZE, sb.numNodes());
     }


[16/50] hadoop git commit: HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (Contributed by Vinayakumar B) Added missed file

Posted by ji...@apache.org.
HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (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/a7a65f22
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a7a65f22
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a7a65f22

Branch: refs/heads/HDFS-7285
Commit: a7a65f225bbfca94a1baa383be634d687da782d2
Parents: 96173ce
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Apr 8 14:23:03 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:14:10 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/protocol/ECInfo.java | 41 ++++++++++++++++++++
 1 file changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7a65f22/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
new file mode 100644
index 0000000..ca642c2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.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 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;
+  }
+}


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

Posted by ji...@apache.org.
HDFS-8024. Erasure Coding: ECworker frame, basics, bootstraping and configuration. (Contributed by Uma Maheswara Rao G)


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

Branch: refs/heads/HDFS-7285
Commit: 12e161d382d6b72e8326039f8cbf3de391c75c0a
Parents: 7dfd6d4
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Wed Apr 22 19:30:14 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:55 2015 -0700

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


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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e161d3/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 0c6c97d..60e6d91 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -973,6 +973,8 @@ public class PBHelper {
       return REG_CMD;
     case BlockIdCommand:
       return PBHelper.convert(proto.getBlkIdCmd());
+    case BlockECRecoveryCommand:
+      return PBHelper.convert(proto.getBlkECRecoveryCmd());
     default:
       return null;
     }
@@ -1123,6 +1125,11 @@ public class PBHelper {
       builder.setCmdType(DatanodeCommandProto.Type.BlockIdCommand).
         setBlkIdCmd(PBHelper.convert((BlockIdCommand) datanodeCommand));
       break;
+    case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY:
+      builder.setCmdType(DatanodeCommandProto.Type.BlockECRecoveryCommand)
+          .setBlkECRecoveryCmd(
+              convert((BlockECRecoveryCommand) datanodeCommand));
+      break;
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     default:
       builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand);

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12e161d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
new file mode 100644
index 0000000..6430308
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.erasurecode;
+
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
+import org.apache.hadoop.io.erasurecode.coder.AbstractErasureCoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
+
+/**
+ * ErasureCodingWorker handles the erasure coding recovery work commands. These
+ * commands would be issued from Namenode as part of Datanode's heart beat
+ * response. BPOfferService delegates the work to this class for handling EC
+ * commands.
+ */
+public final class ErasureCodingWorker {
+
+  private Configuration conf;
+  RawErasureCoder rawEncoder = null;
+  RawErasureCoder rawDecoder = null;
+
+  public ErasureCodingWorker(Configuration conf) {
+    this.conf = conf;
+    initialize();
+  }
+
+  /**
+   * Initializes the required resources for handling the erasure coding recovery
+   * work.
+   */
+  public void initialize() {
+    // Right now directly used RS coder. Once other coders integration ready, we
+    // can load preferred codec here.
+    initializeErasureEncoder();
+    initializeErasureDecoder();
+  }
+
+  private void initializeErasureDecoder() {
+    rawDecoder = AbstractErasureCoder.createRawCoder(conf,
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, false);
+    if (rawDecoder == null) {
+      rawDecoder = new RSRawDecoder();
+    }
+  }
+
+  private void initializeErasureEncoder() {
+    rawEncoder = AbstractErasureCoder.createRawCoder(conf,
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, true);
+    if (rawEncoder == null) {
+      rawEncoder = new RSRawEncoder();
+    }
+  }
+
+  /**
+   * Handles the Erasure Coding recovery work commands.
+   * 
+   * @param ecTasks
+   *          BlockECRecoveryInfo
+   */
+  public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
+    // HDFS-7348 : Implement the actual recovery process
+  }
+}

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


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

Posted by ji...@apache.org.
HDFS-8233. Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last stripe is at the block group boundary. Contributed by Jing Zhao.


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

Branch: refs/heads/HDFS-7285
Commit: cdda9db0af89c1d19fbcb5e6bd7ef8aa5359141b
Parents: 5b9f873
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Apr 23 15:43:04 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:56 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdda9db0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index eeb9d7e..245dfc1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.protocol.ECInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.util.DataChecksum;
@@ -278,14 +277,6 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     return numDataBlocks * cellSize;
   }
 
-  private long getCurrentBlockGroupBytes() {
-    long sum = 0;
-    for (int i = 0; i < numDataBlocks; i++) {
-      sum += streamers.get(i).getBytesCurBlock();
-    }
-    return sum;
-  }
-
   private void notSupported(String headMsg)
       throws IOException{
       throw new IOException(
@@ -347,37 +338,43 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     }
   }
 
+  /**
+   * Simply add bytesCurBlock together. Note that this result is not accurately
+   * the size of the block group.
+   */
+  private long getCurrentSumBytes() {
+    long sum = 0;
+    for (int i = 0; i < numDataBlocks; i++) {
+      sum += streamers.get(i).getBytesCurBlock();
+    }
+    return sum;
+  }
+
   private void writeParityCellsForLastStripe() throws IOException {
-    final long currentBlockGroupBytes = getCurrentBlockGroupBytes();
-    long parityBlkSize = StripedBlockUtil.getInternalBlockLength(
-        currentBlockGroupBytes, cellSize, numDataBlocks,
-        numDataBlocks + 1);
-    if (parityBlkSize == 0 || currentBlockGroupBytes % stripeDataSize() == 0) {
+    final long currentBlockGroupBytes = getCurrentSumBytes();
+    if (currentBlockGroupBytes % stripeDataSize() == 0) {
       return;
     }
-    int parityCellSize = parityBlkSize % cellSize == 0 ? cellSize :
-                        (int) (parityBlkSize % cellSize);
+    long firstCellSize = getLeadingStreamer().getBytesCurBlock() % cellSize;
+    long parityCellSize = firstCellSize > 0 && firstCellSize < cellSize ?
+        firstCellSize : cellSize;
 
     for (int i = 0; i < numAllBlocks; i++) {
-      long internalBlkLen = StripedBlockUtil.getInternalBlockLength(
-          currentBlockGroupBytes, cellSize, numDataBlocks, i);
       // Pad zero bytes to make all cells exactly the size of parityCellSize
       // If internal block is smaller than parity block, pad zero bytes.
       // Also pad zero bytes to all parity cells
-      if (internalBlkLen < parityBlkSize || i >= numDataBlocks) {
-        int position = cellBuffers[i].position();
-        assert position <= parityCellSize : "If an internal block is smaller" +
-            " than parity block, then its last cell should be small than last" +
-            " parity cell";
-        for (int j = 0; j < parityCellSize - position; j++) {
-          cellBuffers[i].put((byte) 0);
-        }
+      int position = cellBuffers[i].position();
+      assert position <= parityCellSize : "If an internal block is smaller" +
+          " than parity block, then its last cell should be small than last" +
+          " parity cell";
+      for (int j = 0; j < parityCellSize - position; j++) {
+        cellBuffers[i].put((byte) 0);
       }
       cellBuffers[i].flip();
     }
     encode(cellBuffers);
 
-    //write parity cells
+    // write parity cells
     curIdx = numDataBlocks;
     refreshStreamer();
     for (int i = numDataBlocks; i < numAllBlocks; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdda9db0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index 26f6d2c..5ce94ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -115,6 +115,12 @@ public class TestDFSStripedOutputStream {
   }
 
   @Test
+  public void testFileLessThanFullBlockGroup() throws IOException {
+    testOneFile("/LessThanFullBlockGroup",
+        cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize);
+  }
+
+  @Test
   public void testFileFullBlockGroup() throws IOException {
     testOneFile("/FullBlockGroup", blockSize * dataBlocks);
   }


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 3297989c20dd296222423474d43d955284e5e552
Parents: 5d78eec
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 13 11:15:02 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:15:36 2015 -0700

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


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


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

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


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

Branch: refs/heads/HDFS-7285
Commit: f3d0e5588114d6fb2c37c42981180125f19a6ba3
Parents: 3102e6a
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Apr 29 11:35:58 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:35:58 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3d0e558/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 6eb25d0..bef4da0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1116,7 +1116,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   /**
    * Read data from one DataNode.
    * @param datanode the datanode from which to read data
-   * @param block the block to read
+   * @param blockStartOffset starting offset in the file
    * @param startInBlk the startInBlk offset of the block
    * @param endInBlk the endInBlk offset of the block
    * @param buf the given byte array into which the data is read
@@ -1146,7 +1146,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       BlockReader reader = null;
       try {
         DFSClientFaultInjector.get().fetchFromDatanodeException();
-        reader = getBlockReader(block, start, len, datanode.addr,
+        reader = getBlockReader(block, startInBlk, len, datanode.addr,
             datanode.storageType, datanode.info);
         for (int i = 0; i < offsets.length; i++) {
           int nread = reader.readAll(buf, offsets[i], lengths[i]);
@@ -1203,8 +1203,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * with each other.
    */
   private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) {
-    Preconditions.checkArgument(offsets.length == lengths.length &&
-        offsets.length > 0);
+    Preconditions.checkArgument(offsets.length == lengths.length && offsets.length > 0);
     int sum = 0;
     for (int i = 0; i < lengths.length; i++) {
       if (i > 0) {

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3d0e558/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
index 19c205e..ef7e2a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -58,9 +58,10 @@ public class StripedDataStreamer extends DataStreamer {
                       Progressable progress, DataChecksum checksum,
                       AtomicReference<CachingStrategy> cachingStrategy,
                       ByteArrayManager byteArrayManage, short index,
-                      List<BlockingQueue<LocatedBlock>> stripedBlocks) {
-    super(stat,block, dfsClient, src, progress, checksum, cachingStrategy,
-        byteArrayManage);
+                      List<BlockingQueue<LocatedBlock>> stripedBlocks,
+                      String[] favoredNodes) {
+    super(stat, block, dfsClient, src, progress, checksum, cachingStrategy,
+        byteArrayManage, favoredNodes);
     this.index = index;
     this.stripedBlocks = stripedBlocks;
   }


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 54cad2d4dcb880e887786340f9dd155fbe842f58
Parents: 94d0e9a
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Apr 17 16:07:07 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:54 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54cad2d4/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
deleted file mode 100644
index e36d386..0000000
--- a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
+++ /dev/null
@@ -1,35 +0,0 @@
-<?xml version="1.0"?>
-
-<!--
- 
- Licensed to the Apache Software Foundation (ASF) under one
- or more contributor license agreements.  See the NOTICE file
- distributed with this work for additional information
- regarding copyright ownership.  The ASF licenses this file
- to you under the Apache License, Version 2.0 (the
- "License"); you may not use this file except in compliance
- with the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
--->
-
-<!--
-Please define your EC schemas here. Note, once these schemas are loaded
-and referenced by EC storage policies, any change to them will be ignored.
-You can modify and remove those not used yet, or add new ones.
--->
-
-<schemas>
-  <schema name="RS-10-4">
-    <k>10</k>
-    <m>4</m>
-    <codec>RS</codec>
-  </schema>
-</schemas>
\ No newline at end of file

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

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

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


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 5fbe138326e1196b85bfdaaa67f923b90ed0e015
Parents: 4921a4e
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Apr 17 18:07:07 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:54 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5fbe1383/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
new file mode 100644
index 0000000..d251c30
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+import org.junit.Test;
+
+/**
+ * This class tests INodeFile with striped feature.
+ */
+public class TestStripedINodeFile {
+  public static final Log LOG = LogFactory.getLog(TestINodeFile.class);
+
+  private static final PermissionStatus perm = new PermissionStatus(
+      "userName", null, FsPermission.getDefault());
+
+  private static INodeFile createStripedINodeFile() {
+    return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+        null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID);
+  }
+
+  @Test
+  public void testBlockStripedFeature()
+      throws IOException, InterruptedException{
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    assertTrue(inf.isStriped());
+  }
+
+  @Test
+  public void testBlockStripedTotalBlockCount() {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    assertEquals(9, blockInfoStriped.getTotalBlockNum());
+  }
+
+  @Test
+  public void testBlockStripedLength()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    inf.addBlock(blockInfoStriped);
+    assertEquals(1, inf.getBlocks().length);
+  }
+
+  @Test
+  public void testBlockStripedConsumedSpace()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    blockInfoStriped.setNumBytes(1);
+    inf.addBlock(blockInfoStriped);
+    //   0. Calculate the total bytes per stripes <Num Bytes per Stripes>
+    //   1. Calculate the number of stripes in this block group. <Num Stripes>
+    //   2. Calculate the last remaining length which does not make a stripe. <Last Stripe Length>
+    //   3. Total consumed space is the total of
+    //     a. The total of the full cells of data blocks and parity blocks.
+    //     b. The remaining of data block which does not make a stripe.
+    //     c. The last parity block cells. These size should be same
+    //        to the first cell in this stripe.
+    // So the total consumed space is the sum of
+    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
+    //  b. <Num Bytes> % <Num Bytes per Stripes> = 1
+    //  c. <Last Stripe Length> * <Parity Block Num> = 1 * 3
+    assertEquals(4, inf.storagespaceConsumedWithStriped());
+    assertEquals(4, inf.storagespaceConsumed());
+  }
+
+  @Test
+  public void testMultipleBlockStripedConsumedSpace()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk1 = new Block(1);
+    BlockInfoStriped blockInfoStriped1
+        = new BlockInfoStriped(blk1,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    blockInfoStriped1.setNumBytes(1);
+    Block blk2 = new Block(2);
+    BlockInfoStriped blockInfoStriped2
+        = new BlockInfoStriped(blk2,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    blockInfoStriped2.setNumBytes(1);
+    inf.addBlock(blockInfoStriped1);
+    inf.addBlock(blockInfoStriped2);
+    // This is the double size of one block in above case.
+    assertEquals(4 * 2, inf.storagespaceConsumedWithStriped());
+    assertEquals(4 * 2, inf.storagespaceConsumed());
+  }
+
+  @Test
+  public void testBlockStripedFileSize()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    blockInfoStriped.setNumBytes(100);
+    inf.addBlock(blockInfoStriped);
+    // Compute file size should return actual data
+    // size which is retained by this file.
+    assertEquals(100, inf.computeFileSize());
+    assertEquals(100, inf.computeFileSize(false, false));
+  }
+
+  @Test
+  public void testBlockStripedUCFileSize()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStripedUnderConstruction bInfoStripedUC
+        = new BlockInfoStripedUnderConstruction(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    bInfoStripedUC.setNumBytes(100);
+    inf.addBlock(bInfoStripedUC);
+    assertEquals(100, inf.computeFileSize());
+    assertEquals(0, inf.computeFileSize(false, false));
+  }
+
+  @Test
+  public void testBlockStripedComputeQuotaUsage()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    blockInfoStriped.setNumBytes(100);
+    inf.addBlock(blockInfoStriped);
+
+    BlockStoragePolicySuite suite =
+        BlockStoragePolicySuite.createDefaultSuite();
+    QuotaCounts counts =
+        inf.computeQuotaUsageWithStriped(suite,
+            new QuotaCounts.Builder().build());
+    assertEquals(1, counts.getNameSpace());
+    // The total consumed space is the sum of
+    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
+    //  b. <Num Bytes> % <Num Bytes per Stripes> = 100
+    //  c. <Last Stripe Length> * <Parity Block Num> = 100 * 3
+    assertEquals(400, counts.getStorageSpace());
+  }
+
+  @Test
+  public void testBlockStripedUCComputeQuotaUsage()
+      throws IOException, InterruptedException {
+    ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+    INodeFile inf = createStripedINodeFile();
+    inf.addStripedBlocksFeature();
+    Block blk = new Block(1);
+    BlockInfoStripedUnderConstruction bInfoStripedUC
+        = new BlockInfoStripedUnderConstruction(blk,
+            (short)defaultSchema.getNumDataUnits(),
+            (short)defaultSchema.getNumParityUnits());
+    bInfoStripedUC.setNumBytes(100);
+    inf.addBlock(bInfoStripedUC);
+
+    BlockStoragePolicySuite suite
+        = BlockStoragePolicySuite.createDefaultSuite();
+    QuotaCounts counts
+        = inf.computeQuotaUsageWithStriped(suite,
+              new QuotaCounts.Builder().build());
+    assertEquals(1024, inf.getPreferredBlockSize());
+    assertEquals(1, counts.getNameSpace());
+    // Consumed space in the case of BlockInfoStripedUC can be calculated
+    // by using preferred block size. This is 1024 and total block num
+    // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216.
+    assertEquals(9216, counts.getStorageSpace());
+  }
+}


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

Posted by ji...@apache.org.
HDFS-8145. Fix the editlog corruption exposed by failed TestAddStripedBlocks. Contributed by Jing Zhao.


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

Branch: refs/heads/HDFS-7285
Commit: 1c0a214928c02e98d0e5a5d91952efbec41fa679
Parents: 5fbe138
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Apr 17 18:13:47 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:54 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c0a2149/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 9f2f5ba..23e3153 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -244,13 +244,6 @@ public class BlockInfoStriped extends BlockInfo {
     return num;
   }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeShort(dataBlockNum);
-    out.writeShort(parityBlockNum);
-    super.write(out);
-  }
-
   /**
    * Convert a complete block to an under construction block.
    * @return BlockInfoUnderConstruction -  an under construction block.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c0a2149/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
index 0a84083..3f94227 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
@@ -54,10 +54,6 @@ public class ErasureCodingZoneManager {
     this.dir = dir;
   }
 
-  boolean getECPolicy(INodesInPath iip) throws IOException {
-    return getECSchema(iip) != null;
-  }
-
   ECSchema getECSchema(INodesInPath iip) throws IOException {
     ECZoneInfo ecZoneInfo = getECZoneInfo(iip);
     return ecZoneInfo == null ? null : ecZoneInfo.getSchema();
@@ -109,7 +105,7 @@ public class ErasureCodingZoneManager {
       throw new IOException("Attempt to create an erasure coding zone " +
           "for a file.");
     }
-    if (getECPolicy(srcIIP)) {
+    if (getECSchema(srcIIP) != null) {
       throw new IOException("Directory " + src + " is already in an " +
           "erasure coding zone.");
     }
@@ -132,8 +128,10 @@ public class ErasureCodingZoneManager {
   void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src)
       throws IOException {
     assert dir.hasReadLock();
-    if (getECPolicy(srcIIP)
-        != getECPolicy(dstIIP)) {
+    final ECSchema srcSchema = getECSchema(srcIIP);
+    final ECSchema dstSchema = getECSchema(dstIIP);
+    if ((srcSchema != null && !srcSchema.equals(dstSchema)) ||
+        (dstSchema != null && !dstSchema.equals(srcSchema))) {
       throw new IOException(
           src + " can't be moved because the source and destination have " +
               "different erasure coding policies.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c0a2149/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 f51934d..611a8ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -433,7 +433,7 @@ public class FSDirectory implements Closeable {
     writeLock();
     try {
       newiip = addINode(existing, newNode);
-      if (newiip != null && getECPolicy(newiip)) {
+      if (newiip != null && isInECZone(newiip)) {
         newNode.addStripedBlocksFeature();
       }
     } finally {
@@ -472,7 +472,7 @@ public class FSDirectory implements Closeable {
       INodesInPath iip = addINode(existing, newNode);
       if (iip != null) {
         // check if the file is in an EC zone
-        if (getECPolicy(iip)) {
+        if (isInECZone(iip)) {
           newNode.addStripedBlocksFeature();
         }
         if (aclEntries != null) {
@@ -1413,7 +1413,7 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  public boolean getECPolicy(INodesInPath iip) throws IOException {
+  public boolean isInECZone(INodesInPath iip) throws IOException {
     return getECSchema(iip) != null;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c0a2149/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 2c5566c..29c0080 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
@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -418,7 +419,7 @@ public class FSEditLogLoader {
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.getECPolicy(iip));
+      updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.isInECZone(iip));
       break;
     }
     case OP_CLOSE: {
@@ -439,7 +440,7 @@ public class FSEditLogLoader {
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, addCloseOp, iip, file, fsDir.getECPolicy(iip));
+      updateBlocks(fsDir, addCloseOp, iip, file, fsDir.isInECZone(iip));
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -498,7 +499,7 @@ public class FSEditLogLoader {
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.getECPolicy(iip));
+      updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.isInECZone(iip));
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
@@ -516,7 +517,7 @@ public class FSEditLogLoader {
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // add the new block to the INodeFile
       // TODO whether the file is striped should later be retrieved from iip
-      addNewBlock(addBlockOp, oldFile, fsDir.getECPolicy(iip));
+      addNewBlock(addBlockOp, oldFile, fsDir.isInECZone(iip));
       break;
     }
     case OP_SET_REPLICATION: {
@@ -1079,7 +1080,9 @@ public class FSEditLogLoader {
           // is only executed when loading edits written by prior
           // versions of Hadoop. Current versions always log
           // OP_ADD operations as each block is allocated.
-          newBI = new BlockInfoContiguous(newBlock, file.getBlockReplication());
+          newBI = isStriped ? new BlockInfoStriped(newBlock,
+              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS) :
+              new BlockInfoContiguous(newBlock, file.getBlockReplication());
         }
         fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);
         file.addBlock(newBI);

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c0a2149/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 568f49d..ad8868f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -158,7 +158,7 @@ public class TestFSImage {
     for (int i = 0; i < stripedBlks.length; i++) {
       stripedBlks[i] = new BlockInfoStriped(
               new Block(stripedBlkId + i, preferredBlockSize, timestamp),
-              (short) 6, (short) 3);
+              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
       file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
     }
 


[08/50] hadoop git commit: Updated CHANGES-HDFS-EC-7285.txt

Posted by ji...@apache.org.
Updated CHANGES-HDFS-EC-7285.txt


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

Branch: refs/heads/HDFS-7285
Commit: 99901eb3d91d052db246cd945a52821940a5403e
Parents: f24e244
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Apr 8 01:31:46 2015 +0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:06:50 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99901eb3/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 01280db..68d1d32 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -32,3 +32,6 @@
 
     HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin
     ( Xinwei Qin via Kai Zheng )
+
+    HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng
+    ( Kai Zheng )


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

Posted by ji...@apache.org.
HDFS-8027. Erasure Coding: Update CHANGES-HDFS-7285.txt with branch commits (Vinayakumar B)


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

Branch: refs/heads/HDFS-7285
Commit: 39af9f1df648596a24362de96126e192ac23113d
Parents: de6b66d
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Apr 15 12:23:07 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:15:36 2015 -0700

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


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


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 94d0e9a0b7d536480b5c5c4491f9e9b79e4581f9
Parents: d5683a3
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Apr 17 12:05:31 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:54 2015 -0700

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


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

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


[07/50] hadoop git commit: HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin Updated CHANGES-HDFS-EC-7285.txt

Posted by ji...@apache.org.
HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin
Updated CHANGES-HDFS-EC-7285.txt


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

Branch: refs/heads/HDFS-7285
Commit: c0c653430246aefc74aaefa89904fb4c79b1d1f2
Parents: 99901eb
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 7 15:34:37 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:06:50 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0c65343/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 4e60a7c..3874cb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -48,9 +48,6 @@
     HDFS-7617. Add unit tests for editlog transactions for EC 
     (Hui Zheng via Zhe Zhang)
 
-    HADOOP-11782. Correct two thrown messages in ECSchema class
-    (Xinwei Qin via Kai Zheng)
-
     HDFS-7839. Erasure coding: implement facilities in NameNode to create and
     manage EC zones (Zhe Zhang)
 


[46/50] hadoop git commit: Addendum fix for HDFS-7749 to be compatible with HDFS-7993

Posted by ji...@apache.org.
Addendum fix for HDFS-7749 to be compatible with HDFS-7993


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

Branch: refs/heads/HDFS-7285
Commit: b65f508988a7a3df0b7f48f7487759d1993294a9
Parents: f827630
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 27 11:08:16 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:17:52 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/namenode/NamenodeFsck.java  | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b65f5089/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 0c564bd..b108cd6 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
@@ -629,9 +629,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         report.append(" repl=" + liveReplicas);
         if (showLocations || showRacks || showReplicaDetails) {
           StringBuilder sb = new StringBuilder("[");
-          Iterable<DatanodeStorageInfo> storages = bm.getStorages(block.getLocalBlock());
-          for (Iterator<DatanodeStorageInfo> iterator = storages.iterator(); iterator.hasNext();) {
-            DatanodeStorageInfo storage = iterator.next();
+          DatanodeStorageInfo[] storages = bm.getStorages(storedBlock);
+          for (int i = 0; i < storages.length; i++) {
+            DatanodeStorageInfo storage = storages[i];
             DatanodeDescriptor dnDesc = storage.getDatanodeDescriptor();
             if (showRacks) {
               sb.append(NodeBase.getPath(dnDesc));
@@ -640,7 +640,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
                   .getStorageType()));
             }
             if (showReplicaDetails) {
-              LightWeightLinkedSet<Block> blocksExcess =
+              LightWeightLinkedSet<BlockInfo> blocksExcess =
                   bm.excessReplicateMap.get(dnDesc.getDatanodeUuid());
               Collection<DatanodeDescriptor> corruptReplicas =
                   bm.getCorruptReplicas(block.getLocalBlock());
@@ -661,7 +661,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
                 sb.append("LIVE)");
               }
             }
-            if (iterator.hasNext()) {
+            if (i < storages.length - 1) {
               sb.append(", ");
             }
           }


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

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


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

Branch: refs/heads/HDFS-7285
Commit: e6a925338c0a99816b6e63e32b816a8416a271da
Parents: 12e161d
Author: Zhe Zhang <zh...@apache.org>
Authored: Wed Apr 22 14:48:54 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:55 2015 -0700

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


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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6a92533/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 60e6d91..e418c5a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -3143,7 +3143,7 @@ public class PBHelper {
         .setCodecName(schema.getCodecName())
         .setDataUnits(schema.getNumDataUnits())
         .setParityUnits(schema.getNumParityUnits());
-    Set<Entry<String, String>> entrySet = schema.getOptions().entrySet();
+    Set<Entry<String, String>> entrySet = schema.getExtraOptions().entrySet();
     for (Entry<String, String> entry : entrySet) {
       builder.addOptions(ECSchemaOptionEntryProto.newBuilder()
           .setKey(entry.getKey()).setValue(entry.getValue()).build());

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

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

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

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

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


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

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


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

Branch: refs/heads/HDFS-7285
Commit: df776013d54e0ee971e97c700ff2c4f1bb6626ce
Parents: 80e516f
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 20 14:19:12 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:55 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/df776013/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 86c8a87..dc2f1d5 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
@@ -176,6 +176,18 @@ public interface HdfsClientConfigKeys {
     int     THREADPOOL_SIZE_DEFAULT = 0;
   }
 
+  /** dfs.client.read.striped configuration properties */
+  interface StripedRead {
+    String PREFIX = Read.PREFIX + "striped.";
+
+    String  THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size";
+    /**
+     * With default 6+3 schema, each normal read could span 6 DNs. So this
+     * default value accommodates 3 read streams
+     */
+    int     THREADPOOL_SIZE_DEFAULT = 18;
+  }
+
   /** dfs.http.client configuration properties */
   interface HttpClient {
     String  PREFIX = "dfs.http.client.";

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df776013/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 a4444e2..61c34b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -873,7 +873,7 @@ public class BlockManager {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
-        return new LocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
+        return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
             false);
       } else {
         assert blk instanceof BlockInfoContiguousUnderConstruction;
@@ -882,13 +882,8 @@ public class BlockManager {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
             blk);
-        return new LocatedBlock(eb, storages, pos, false);
+        return newLocatedBlock(eb, storages, pos, false);
       }
-      final BlockInfoContiguousUnderConstruction uc =
-          (BlockInfoContiguousUnderConstruction) blk;
-      final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-      final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
-      return newLocatedBlock(eb, storages, pos, false);
     }
 
     // get block locations
@@ -930,7 +925,7 @@ public class BlockManager {
     final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
     return blockIndices == null ?
         newLocatedBlock(eb, machines, pos, isCorrupt) :
-        new LocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
+        newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
   }
 
   /** Create a LocatedBlocks. */
@@ -3480,7 +3475,7 @@ public class BlockManager {
     if (pendingReplicationBlocksCount == 0 &&
         underReplicatedBlocksCount == 0) {
       LOG.info("Node {} is dead and there are no under-replicated" +
-          " blocks or blocks pending replication. Safe to decommission.", 
+          " blocks or blocks pending replication. Safe to decommission.",
           node);
       return true;
     }
@@ -3886,6 +3881,18 @@ public class BlockManager {
         null);
   }
 
+  public static LocatedStripedBlock newLocatedStripedBlock(
+      ExtendedBlock b, DatanodeStorageInfo[] storages,
+      int[] indices, long startOffset, boolean corrupt) {
+    // startOffset is unknown
+    return new LocatedStripedBlock(
+        b, DatanodeStorageInfo.toDatanodeInfos(storages),
+        DatanodeStorageInfo.toStorageIDs(storages),
+        DatanodeStorageInfo.toStorageTypes(storages),
+        indices, startOffset, corrupt,
+        null);
+  }
+
   /**
    * This class is used internally by {@link this#computeRecoveryWorkForBlocks}
    * to represent a task to recover a block through replication or erasure

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df776013/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
index d251c30..4a6d6cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -45,7 +46,7 @@ public class TestStripedINodeFile {
       "userName", null, FsPermission.getDefault());
 
   private static INodeFile createStripedINodeFile() {
-    return new INodeFile(INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+    return new INodeFile(HdfsConstantsClient.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID);
   }
 


[05/50] hadoop git commit: HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code ( Contributed by Kai Zheng)

Posted by ji...@apache.org.
HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code ( 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/347619a4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/347619a4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/347619a4

Branch: refs/heads/HDFS-7285
Commit: 347619a406415dd0dff186e43a7f871ca13b6748
Parents: b95f233
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 7 16:05:22 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:06:50 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  3 +
 .../hadoop/io/erasurecode/ECBlockGroup.java     | 18 ++++
 .../erasurecode/codec/AbstractErasureCodec.java | 88 +++++++++++++++++++
 .../io/erasurecode/codec/ErasureCodec.java      | 56 ++++++++++++
 .../io/erasurecode/codec/RSErasureCodec.java    | 38 +++++++++
 .../io/erasurecode/codec/XORErasureCodec.java   | 45 ++++++++++
 .../erasurecode/coder/AbstractErasureCoder.java |  7 ++
 .../io/erasurecode/coder/ErasureCoder.java      |  7 ++
 .../io/erasurecode/grouper/BlockGrouper.java    | 90 ++++++++++++++++++++
 9 files changed, 352 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/347619a4/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 7716728..c72394e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -37,3 +37,6 @@
 
     HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng
     ( Kai Zheng )
+
+    HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code
+    ( Kai Zheng via vinayakumarb )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/347619a4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
index 2c851a5..0a86907 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
@@ -79,4 +79,22 @@ public class ECBlockGroup {
     return false;
   }
 
+  /**
+   * Get erased blocks count
+   * @return
+   */
+  public int getErasedCount() {
+    int erasedCount = 0;
+
+    for (ECBlock dataBlock : dataBlocks) {
+      if (dataBlock.isErased()) erasedCount++;
+    }
+
+    for (ECBlock parityBlock : parityBlocks) {
+      if (parityBlock.isErased()) erasedCount++;
+    }
+
+    return erasedCount;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/347619a4/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
new file mode 100644
index 0000000..9993786
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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;
+
+/**
+ * Abstract Erasure Codec that implements {@link ErasureCodec}.
+ */
+public abstract class AbstractErasureCodec extends Configured
+    implements ErasureCodec {
+
+  private ECSchema schema;
+
+  @Override
+  public void setSchema(ECSchema schema) {
+    this.schema = schema;
+  }
+
+  public String getName() {
+    return schema.getCodecName();
+  }
+
+  protected ECSchema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public BlockGrouper createBlockGrouper() {
+    BlockGrouper blockGrouper = new BlockGrouper();
+    blockGrouper.setSchema(getSchema());
+
+    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/347619a4/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
new file mode 100644
index 0000000..e639484
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.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.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;
+
+/**
+ * Erasure Codec API that's to cover the essential specific aspects of a code.
+ * Currently it cares only block grouper and erasure coder. In future we may
+ * add more aspects here to make the behaviors customizable.
+ */
+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
+   */
+  public BlockGrouper createBlockGrouper();
+
+  /**
+   * Create Erasure Encoder
+   * @return erasure encoder
+   */
+  public ErasureCoder createEncoder();
+
+  /**
+   * Create Erasure Decoder
+   * @return erasure decoder
+   */
+  public ErasureCoder createDecoder();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/347619a4/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
new file mode 100644
index 0000000..9e91b60
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.codec;
+
+import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.io.erasurecode.coder.RSErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.RSErasureEncoder;
+
+/**
+ * A Reed-Solomon erasure codec.
+ */
+public class RSErasureCodec extends AbstractErasureCodec {
+
+  @Override
+  protected ErasureCoder doCreateEncoder() {
+    return new RSErasureEncoder();
+  }
+
+  @Override
+  protected ErasureCoder doCreateDecoder() {
+    return new RSErasureDecoder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/347619a4/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
new file mode 100644
index 0000000..0f726d7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.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.XORErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.XORErasureEncoder;
+
+/**
+ * A XOR erasure codec.
+ */
+public class XORErasureCodec extends AbstractErasureCodec {
+
+  @Override
+  public void setSchema(ECSchema schema) {
+    super.setSchema(schema);
+    assert(schema.getNumParityUnits() == 1);
+  }
+
+  @Override
+  protected ErasureCoder doCreateEncoder() {
+    return new XORErasureEncoder();
+  }
+
+  @Override
+  protected ErasureCoder doCreateDecoder() {
+    return new XORErasureDecoder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/347619a4/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 0e4de89..e5bf11a 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
@@ -105,6 +106,12 @@ public abstract class AbstractErasureCoder
   }
 
   @Override
+  public void initialize(ECSchema schema) {
+      initialize(schema.getNumDataUnits(), schema.getNumParityUnits(),
+          schema.getChunkSize());
+  }
+
+  @Override
   public int getNumDataUnits() {
     return numDataUnits;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/347619a4/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 fb90156..64a82ea 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,6 +19,7 @@ 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
@@ -45,6 +46,12 @@ public interface ErasureCoder extends Configurable {
   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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/347619a4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java
new file mode 100644
index 0000000..bdc1624
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.grouper;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+/**
+ * As part of a codec, to handle how to form a block group for encoding
+ * and provide instructions on how to recover erased blocks from a block group
+ */
+public class BlockGrouper {
+
+  private ECSchema schema;
+
+  /**
+   * Set EC schema.
+   * @param schema
+   */
+  public void setSchema(ECSchema schema) {
+    this.schema = schema;
+  }
+
+  /**
+   * Get EC schema.
+   * @return
+   */
+  protected ECSchema getSchema() {
+    return schema;
+  }
+
+  /**
+   * Get required data blocks count in a BlockGroup.
+   * @return count of required data blocks
+   */
+  public int getRequiredNumDataBlocks() {
+    return schema.getNumDataUnits();
+  }
+
+  /**
+   * Get required parity blocks count in a BlockGroup.
+   * @return count of required parity blocks
+   */
+  public int getRequiredNumParityBlocks() {
+    return schema.getNumParityUnits();
+  }
+
+  /**
+   * Calculating and organizing BlockGroup, to be called by ECManager
+   * @param dataBlocks Data blocks to compute parity blocks against
+   * @param parityBlocks To be computed parity blocks
+   * @return
+   */
+  public ECBlockGroup makeBlockGroup(ECBlock[] dataBlocks,
+                                     ECBlock[] parityBlocks) {
+
+    ECBlockGroup blockGroup = new ECBlockGroup(dataBlocks, parityBlocks);
+    return blockGroup;
+  }
+
+  /**
+   * Given a BlockGroup, tell if any of the missing blocks can be recovered,
+   * to be called by ECManager
+   * @param blockGroup a blockGroup that may contain erased blocks but not sure
+   *                   recoverable or not
+   * @return true if any erased block recoverable, false otherwise
+   */
+  public boolean anyRecoverable(ECBlockGroup blockGroup) {
+    int erasedCount = blockGroup.getErasedCount();
+
+    return erasedCount > 0 && erasedCount <= getRequiredNumParityBlocks();
+  }
+
+}


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 186f139ced853e54572a8a568d4d7073dcd96e68
Parents: 39af9f1
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Apr 15 16:38:22 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:15:36 2015 -0700

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


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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/186f139c/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 a254485..0e84db1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -118,6 +118,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -3304,4 +3305,21 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     return scope;
   }
+
+  /**
+   * Get the erasure coding zone information for the specified path
+   * 
+   * @param src path to get the information for
+   * @return Returns the zone information if path is in EC Zone, null otherwise
+   * @throws IOException
+   */
+  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+    checkOpen();
+    try {
+      return namenode.getErasureCodingZoneInfo(src);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(FileNotFoundException.class,
+          AccessControlException.class, UnresolvedPathException.class);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/186f139c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 4c3e0a5..4c8fff3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -72,6 +72,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -2297,4 +2298,35 @@ public class DistributedFileSystem extends FileSystem {
       }
     }.resolve(this, absF);
   }
+
+  /**
+   * Get ErasureCoding zone information for the specified path
+   * 
+   * @param path
+   * @return Returns the zone information if path is in EC zone, null otherwise
+   * @throws IOException
+   */
+  public ECZoneInfo getErasureCodingZoneInfo(final Path path)
+      throws IOException {
+    Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<ECZoneInfo>() {
+      @Override
+      public ECZoneInfo doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        return dfs.getErasureCodingZoneInfo(getPathName(p));
+      }
+
+      @Override
+      public ECZoneInfo next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          return myDfs.getErasureCodingZoneInfo(p);
+        }
+        throw new UnsupportedOperationException(
+            "Cannot getErasureCodingZoneInfo through a symlink to a "
+                + "non-DistributedFileSystem: " + path + " -> " + p);
+      }
+    }.resolve(this, absF);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/186f139c/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 0c04ca9..c257cc1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1484,4 +1484,13 @@ public interface ClientProtocol {
    */
   @Idempotent
   public ECSchema[] getECSchemas() throws IOException;
+
+  /**
+   * Get the information about the EC zone for the path
+   * 
+   * @param src path to get the info for
+   * @throws IOException
+   */
+  @Idempotent
+  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException;
 }

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/186f139c/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 169ea2d..36e5492 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -201,6 +202,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptio
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
@@ -1551,4 +1554,19 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetECZoneInfoResponseProto getErasureCodingZoneInfo(RpcController controller,
+      GetECZoneInfoRequestProto request) throws ServiceException {
+    try {
+      ECZoneInfo ecZoneInfo = server.getErasureCodingZoneInfo(request.getSrc());
+      GetECZoneInfoResponseProto.Builder builder = GetECZoneInfoResponseProto.newBuilder();
+      if (ecZoneInfo != null) {
+        builder.setECZoneInfo(PBHelper.convertECZoneInfo(ecZoneInfo));
+      }
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/186f139c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 44d1258..6c1c971 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -167,6 +168,8 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathR
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECSchemasResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECZoneInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CreateErasureCodingZoneRequestProto;
@@ -1576,4 +1579,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+    GetECZoneInfoRequestProto req = GetECZoneInfoRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      GetECZoneInfoResponseProto response = rpcProxy.getErasureCodingZoneInfo(
+          null, req);
+      if (response.hasECZoneInfo()) {
+        return PBHelper.convertECZoneInfo(response.getECZoneInfo());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/186f139c/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 a323534..b9d87aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
@@ -134,6 +135,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportC
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECZoneInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -3138,4 +3140,14 @@ public class PBHelper {
     }
     return builder.build();
   }
+
+  public static ECZoneInfoProto convertECZoneInfo(ECZoneInfo ecZoneInfo) {
+    return ECZoneInfoProto.newBuilder().setDir(ecZoneInfo.getDir())
+        .setSchema(convertECSchema(ecZoneInfo.getSchema())).build();
+  }
+
+  public static ECZoneInfo convertECZoneInfo(ECZoneInfoProto ecZoneInfoProto) {
+    return new ECZoneInfo(ecZoneInfoProto.getDir(),
+        convertECSchema(ecZoneInfoProto.getSchema()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/186f139c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
index 5320c1c..0a84083 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
@@ -22,6 +22,7 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.XAttrHelper;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.io.erasurecode.ECSchema;
@@ -57,7 +58,12 @@ public class ErasureCodingZoneManager {
     return getECSchema(iip) != null;
   }
 
-  ECSchema getECSchema(INodesInPath iip) throws IOException{
+  ECSchema getECSchema(INodesInPath iip) throws IOException {
+    ECZoneInfo ecZoneInfo = getECZoneInfo(iip);
+    return ecZoneInfo == null ? null : ecZoneInfo.getSchema();
+  }
+
+  ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
     List<INode> inodes = iip.getReadOnlyINodes();
@@ -80,7 +86,8 @@ public class ErasureCodingZoneManager {
         if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
           ECSchemaProto ecSchemaProto;
           ecSchemaProto = ECSchemaProto.parseFrom(xAttr.getValue());
-          return PBHelper.convertECSchema(ecSchemaProto);
+          ECSchema schema = PBHelper.convertECSchema(ecSchemaProto);
+          return new ECZoneInfo(inode.getFullPathName(), schema);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/186f139c/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 ab4d160..f51934d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
@@ -1425,6 +1426,15 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  ECZoneInfo getECZoneInfo(INodesInPath iip) throws IOException {
+    readLock();
+    try {
+      return ecZoneManager.getECZoneInfo(iip);
+    } finally {
+      readUnlock();
+    }
+  }
+
   static INode resolveLastINode(INodesInPath iip) throws FileNotFoundException {
     INode inode = iip.getLastINode();
     if (inode == null) {

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/186f139c/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 23e87a8..7bbf358 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ECInfo;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
@@ -2065,4 +2066,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
     checkNNStartup();
     return namesystem.getECSchemas();
   }
+
+  @Override // ClientProtocol
+  public ECZoneInfo getErasureCodingZoneInfo(String src) throws IOException {
+    checkNNStartup();
+    return namesystem.getErasureCodingZoneInfo(src);
+  }
 }

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/186f139c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
new file mode 100644
index 0000000..84c2275
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.tools.erasurecode;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.Command;
+import org.apache.hadoop.fs.shell.CommandFactory;
+import org.apache.hadoop.fs.shell.PathData;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.ECZoneInfo;
+import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Erasure Coding CLI commands
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class ECCommand extends Command {
+
+  public static void registerCommands(CommandFactory factory) {
+    // Register all commands of Erasure CLI, with a '-' at the beginning in name
+    // of the command.
+    factory.addClass(CreateECZoneCommand.class, "-" + CreateECZoneCommand.NAME);
+    factory.addClass(GetECZoneInfoCommand.class, "-"
+        + GetECZoneInfoCommand.NAME);
+    factory.addClass(ListECSchemas.class, "-" + ListECSchemas.NAME);
+  }
+
+  @Override
+  public String getCommandName() {
+    return getName();
+  }
+
+  @Override
+  protected void run(Path path) throws IOException {
+    throw new RuntimeException("Not suppose to get here");
+  }
+
+  @Deprecated
+  @Override
+  public int runAll() {
+    return run(args);
+  }
+
+  @Override
+  protected void processPath(PathData item) throws IOException {
+    if (!(item.fs instanceof DistributedFileSystem)) {
+      throw new UnsupportedActionException(
+          "Erasure commands are only supported for the HDFS paths");
+    }
+  }
+
+  /**
+   * Create EC encoding zone command. Zones are created to use specific EC
+   * encoding schema, other than default while encoding the files under some
+   * specific directory.
+   */
+  static class CreateECZoneCommand extends ECCommand {
+    public static final String NAME = "createZone";
+    public static final String USAGE = "[-s <schemaName>] <path>";
+    public static final String DESCRIPTION = 
+        "Create a zone to encode files using a specified schema\n"
+        + "Options :\n"
+        + "  -s <schemaName> : EC schema name to encode files. "
+        + "If not passed default schema will be used\n"
+        + "  <path>  : Path to an empty directory. Under this directory "
+        + "files will be encoded using specified schema";
+    private String schemaName;
+    private ECSchema schema = null;
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      schemaName = StringUtils.popOptionWithArgument("-s", args);
+      if (args.isEmpty()) {
+        throw new HadoopIllegalArgumentException("<path> is missing");
+      }
+      if (args.size() > 1) {
+        throw new HadoopIllegalArgumentException("Too many arguments");
+      }
+    }
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      super.processPath(item);
+      DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
+      try {
+        if (schemaName != null) {
+          ECSchema[] ecSchemas = dfs.getClient().getECSchemas();
+          for (ECSchema ecSchema : ecSchemas) {
+            if (schemaName.equals(ecSchema.getSchemaName())) {
+              schema = ecSchema;
+              break;
+            }
+          }
+          if (schema == null) {
+            StringBuilder sb = new StringBuilder();
+            sb.append("Schema '");
+            sb.append(schemaName);
+            sb.append("' does not match any of the supported schemas.");
+            sb.append("Please select any one of [");
+            for (ECSchema ecSchema : ecSchemas) {
+              sb.append(ecSchema.getSchemaName());
+              sb.append(", ");
+            }
+            throw new HadoopIllegalArgumentException(sb.toString());
+          }
+        }
+        dfs.createErasureCodingZone(item.path, schema);
+        out.println("EC Zone created successfully at " + item.path);
+      } catch (IOException e) {
+        throw new IOException("Unable to create EC zone for the path "
+            + item.path, e);
+      }
+    }
+  }
+
+  /**
+   * Get the information about the zone
+   */
+  static class GetECZoneInfoCommand extends ECCommand {
+    public static final String NAME = "getZoneInfo";
+    public static final String USAGE = "<path>";
+    public static final String DESCRIPTION =
+        "Get information about the EC zone at specified path\n";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      if (args.isEmpty()) {
+        throw new HadoopIllegalArgumentException("<path> is missing");
+      }
+      if (args.size() > 1) {
+        throw new HadoopIllegalArgumentException("Too many arguments");
+      }
+    }
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      super.processPath(item);
+      DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
+      try {
+        ECZoneInfo ecZoneInfo = dfs.getErasureCodingZoneInfo(item.path);
+        out.println(ecZoneInfo.toString());
+      } catch (IOException e) {
+        throw new IOException("Unable to create EC zone for the path "
+            + item.path, e);
+      }
+    }
+  }
+
+  /**
+   * List all supported EC Schemas
+   */
+  static class ListECSchemas extends ECCommand {
+    public static final String NAME = "listSchemas";
+    public static final String USAGE = "";
+    public static final String DESCRIPTION = 
+        "Get the list of ECSchemas supported\n";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      if (!args.isEmpty()) {
+        throw new HadoopIllegalArgumentException("Too many parameters");
+      }
+
+      FileSystem fs = FileSystem.get(getConf());
+      if (fs instanceof DistributedFileSystem == false) {
+        throw new UnsupportedActionException(
+            "Erasure commands are only supported for the HDFS");
+      }
+      DistributedFileSystem dfs = (DistributedFileSystem) fs;
+
+      ECSchema[] ecSchemas = dfs.getClient().getECSchemas();
+      StringBuilder sb = new StringBuilder();
+      int i = 0;
+      while (i < ecSchemas.length) {
+        ECSchema ecSchema = ecSchemas[i];
+        sb.append(ecSchema.getSchemaName());
+        i++;
+        if (i < ecSchemas.length) {
+          sb.append(", ");
+        }
+      }
+      out.println(sb.toString());
+    }
+  }
+}
\ No newline at end of file

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

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


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

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


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

Branch: refs/heads/HDFS-7285
Commit: f24c2c064b0a3b5005843c96c19f5549ddb158fb
Parents: cdda9db
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Apr 23 15:48:21 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:56 2015 -0700

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


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

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

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


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

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


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

Branch: refs/heads/HDFS-7285
Commit: f8276301453cf10c6fe0e9cc01a23a61a8ab03ac
Parents: d1e7dfa
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Apr 24 22:36:15 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:17:52 2015 -0700

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


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

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

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

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

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


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

Posted by ji...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit is for HDFS-8035). 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/979f453b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/979f453b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/979f453b

Branch: refs/heads/HDFS-7285
Commit: 979f453ba297213cd47670f1049d56f3c59aba5f
Parents: b0ef98e
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 6 10:37:23 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:06:49 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/blockmanagement/BlockManager.java | 11 +++++------
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java |  8 ++++----
 2 files changed, 9 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/979f453b/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 9bd687a..90ec426 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
@@ -3550,13 +3550,12 @@ public class BlockManager {
       String src, BlockInfo[] blocks) {
     for (BlockInfo b: blocks) {
       if (!b.isComplete()) {
-        final BlockInfoContiguousUnderConstruction uc =
-            (BlockInfoContiguousUnderConstruction)b;
         final int numNodes = b.numNodes();
-        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
-          + uc.getBlockUCState() + ", replication# = " + numNodes
-          + (numNodes < minReplication ? " < ": " >= ")
-          + " minimum = " + minReplication + ") in file " + src);
+        final int min = getMinStorageNum(b);
+        final BlockUCState state = b.getBlockUCState();
+        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " + state
+            + ", replication# = " + numNodes + (numNodes < min ? " < " : " >= ")
+            + " minimum = " + min + ") in file " + src);
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/979f453b/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 1740365..9c71574 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
@@ -3140,7 +3140,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   LocatedBlock storeAllocatedBlock(String src, long fileId, String clientName,
       ExtendedBlock previous, DatanodeStorageInfo[] targets) throws IOException {
-    BlockInfo newBlockInfo = null;
+    Block newBlock = null;
     long offset;
     checkOperation(OperationCategory.WRITE);
     waitForLoadingFSImage();
@@ -3173,8 +3173,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                                 ExtendedBlock.getLocalBlock(previous));
 
       // allocate new block, record block locations in INode.
-      Block newBlock = createNewBlock(isStriped);
-      newBlockInfo = saveAllocatedBlock(src, fileState.iip, newBlock, targets,
+      newBlock = createNewBlock(isStriped);
+      saveAllocatedBlock(src, fileState.iip, newBlock, targets,
           isStriped);
 
       persistNewBlock(src, pendingFile);
@@ -3185,7 +3185,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     getEditLog().logSync();
 
     // Return located block
-    return makeLocatedBlock(newBlockInfo, targets, offset);
+    return makeLocatedBlock(getStoredBlock(newBlock), targets, offset);
   }
 
   /*


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

Posted by ji...@apache.org.
HDFS-8146. Protobuf changes for BlockECRecoveryCommand and its fields for making it ready for transfer to DN (Contributed by Uma Maheswara Rao G)


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

Branch: refs/heads/HDFS-7285
Commit: 92aadcd7fa0c5fee199d3a3a87854c0c3dee232a
Parents: 1c0a214
Author: Vinayakumar B <vi...@apache.org>
Authored: Sat Apr 18 23:20:45 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:54 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92aadcd7/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 b9d87aa..0c6c97d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -28,6 +28,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
@@ -100,7 +101,7 @@ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.AclEntryTyp
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto.FsActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
+import org.apache.hadoop.hdfs.protocol.proto.*;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
@@ -121,6 +122,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmI
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
@@ -132,11 +134,11 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDele
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaOptionEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.ECZoneInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
@@ -184,7 +186,6 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StripedBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
@@ -204,8 +205,10 @@ import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@@ -3150,4 +3153,132 @@ public class PBHelper {
     return new ECZoneInfo(ecZoneInfoProto.getDir(),
         convertECSchema(ecZoneInfoProto.getSchema()));
   }
+  
+  public static BlockECRecoveryInfo convertBlockECRecoveryInfo(
+      BlockECRecoveryInfoProto blockEcRecoveryInfoProto) {
+    ExtendedBlockProto blockProto = blockEcRecoveryInfoProto.getBlock();
+    ExtendedBlock block = convert(blockProto);
+
+    DatanodeInfosProto sourceDnInfosProto = blockEcRecoveryInfoProto
+        .getSourceDnInfos();
+    DatanodeInfo[] sourceDnInfos = convert(sourceDnInfosProto);
+
+    DatanodeInfosProto targetDnInfosProto = blockEcRecoveryInfoProto
+        .getTargetDnInfos();
+    DatanodeInfo[] targetDnInfos = convert(targetDnInfosProto);
+
+    StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto
+        .getTargetStorageUuids();
+    String[] targetStorageUuids = convert(targetStorageUuidsProto);
+
+    StorageTypesProto targetStorageTypesProto = blockEcRecoveryInfoProto
+        .getTargetStorageTypes();
+    StorageType[] convertStorageTypes = convertStorageTypes(
+        targetStorageTypesProto.getStorageTypesList(), targetStorageTypesProto
+            .getStorageTypesList().size());
+
+    List<Integer> liveBlockIndicesList = blockEcRecoveryInfoProto
+        .getLiveBlockIndicesList();
+    short[] liveBlkIndices = new short[liveBlockIndicesList.size()];
+    for (int i = 0; i < liveBlockIndicesList.size(); i++) {
+      liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue();
+    }
+
+    return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos,
+        targetStorageUuids, convertStorageTypes, liveBlkIndices);
+  }
+
+  public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo(
+      BlockECRecoveryInfo blockEcRecoveryInfo) {
+    BlockECRecoveryInfoProto.Builder builder = BlockECRecoveryInfoProto
+        .newBuilder();
+    builder.setBlock(convert(blockEcRecoveryInfo.getExtendedBlock()));
+
+    DatanodeInfo[] sourceDnInfos = blockEcRecoveryInfo.getSourceDnInfos();
+    builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos));
+
+    DatanodeInfo[] targetDnInfos = blockEcRecoveryInfo.getTargetDnInfos();
+    builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos));
+
+    String[] targetStorageIDs = blockEcRecoveryInfo.getTargetStorageIDs();
+    builder.setTargetStorageUuids(convertStorageIDs(targetStorageIDs));
+
+    StorageType[] targetStorageTypes = blockEcRecoveryInfo
+        .getTargetStorageTypes();
+    builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes));
+
+    short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices();
+    builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices));
+
+    return builder.build();
+  }
+
+  private static List<Integer> convertIntArray(short[] liveBlockIndices) {
+    List<Integer> liveBlockIndicesList = new ArrayList<Integer>();
+    for (short s : liveBlockIndices) {
+      liveBlockIndicesList.add((int) s);
+    }
+    return liveBlockIndicesList;
+  }
+
+  private static StorageTypesProto convertStorageTypesProto(
+      StorageType[] targetStorageTypes) {
+    StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
+    for (StorageType storageType : targetStorageTypes) {
+      builder.addStorageTypes(convertStorageType(storageType));
+    }
+    return builder.build();
+  }
+
+  private static StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) {
+    StorageUuidsProto.Builder builder = StorageUuidsProto.newBuilder();
+    for (String storageUuid : targetStorageIDs) {
+      builder.addStorageUuids(storageUuid);
+    }
+    return builder.build();
+  }
+
+  private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) {
+    DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder();
+    for (DatanodeInfo datanodeInfo : dnInfos) {
+      builder.addDatanodes(convert(datanodeInfo));
+    }
+    return builder.build();
+  }
+
+  private static String[] convert(StorageUuidsProto targetStorageUuidsProto) {
+    List<String> storageUuidsList = targetStorageUuidsProto
+        .getStorageUuidsList();
+    String[] storageUuids = new String[storageUuidsList.size()];
+    for (int i = 0; i < storageUuidsList.size(); i++) {
+      storageUuids[i] = storageUuidsList.get(i);
+    }
+    return storageUuids;
+  }
+  
+  public static BlockECRecoveryCommandProto convert(
+      BlockECRecoveryCommand blkECRecoveryCmd) {
+    BlockECRecoveryCommandProto.Builder builder = BlockECRecoveryCommandProto
+        .newBuilder();
+    Collection<BlockECRecoveryInfo> blockECRecoveryInfos = blkECRecoveryCmd
+        .getECTasks();
+    for (BlockECRecoveryInfo blkECRecoveryInfo : blockECRecoveryInfos) {
+      builder
+          .addBlockECRecoveryinfo(convertBlockECRecoveryInfo(blkECRecoveryInfo));
+    }
+    return builder.build();
+  }
+  
+  public static BlockECRecoveryCommand convert(
+      BlockECRecoveryCommandProto blkECRecoveryCmdProto) {
+    Collection<BlockECRecoveryInfo> blkECRecoveryInfos = new ArrayList<BlockECRecoveryInfo>();
+    List<BlockECRecoveryInfoProto> blockECRecoveryinfoList = blkECRecoveryCmdProto
+        .getBlockECRecoveryinfoList();
+    for (BlockECRecoveryInfoProto blockECRecoveryInfoProto : blockECRecoveryinfoList) {
+      blkECRecoveryInfos
+          .add(convertBlockECRecoveryInfo(blockECRecoveryInfoProto));
+    }
+    return new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY,
+        blkECRecoveryInfos);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92aadcd7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 7ec71a2..35cc31b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -32,8 +32,8 @@ import java.util.Set;
 import java.util.Arrays;
 
 import com.google.common.annotations.VisibleForTesting;
-
 import com.google.common.collect.ImmutableList;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -99,34 +100,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  /** Block and targets pair */
-  @InterfaceAudience.Private
-  @InterfaceStability.Evolving
-  public static class BlockECRecoveryInfo {
-    public final ExtendedBlock block;
-    public final DatanodeDescriptor[] sources;
-    public final DatanodeStorageInfo[] targets;
-    public final short[] liveBlockIndices;
-
-    BlockECRecoveryInfo(ExtendedBlock block, DatanodeDescriptor[] sources,
-        DatanodeStorageInfo[] targets, short[] liveBlockIndices) {
-      this.block = block;
-      this.sources = sources;
-      this.targets = targets;
-      this.liveBlockIndices = liveBlockIndices;
-    }
-
-    @Override
-    public String toString() {
-      return new StringBuilder().append("BlockECRecoveryInfo(\n  ").
-          append("Recovering ").append(block).
-          append(" From: ").append(Arrays.asList(sources)).
-          append(" To: ").append(Arrays.asList(targets)).append(")\n").
-          append(" Block Indices: ").append(Arrays.asList(liveBlockIndices)).
-          toString();
-    }
-  }
-
   /** A BlockTargetPair queue. */
   private static class BlockQueue<E> {
     private final Queue<E> blockq = new LinkedList<E>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92aadcd7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 3fcbfb8..bb3e645 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -34,12 +34,12 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.protocol.*;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
 import org.apache.hadoop.ipc.Server;
@@ -1445,7 +1445,7 @@ public class DatanodeManager {
         List<BlockECRecoveryInfo> pendingECList =
             nodeinfo.getErasureCodeCommand(maxTransfers);
         if (pendingECList != null) {
-          cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_CODEC,
+          cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY,
               pendingECList));
         }
         //check block invalidation

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92aadcd7/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
index d888f71..59bd949 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/erasurecoding.proto
@@ -21,6 +21,7 @@ option java_outer_classname = "ErasureCodingProtos";
 option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
+import "hdfs.proto";
 
 /**
  * ECSchema options entry
@@ -86,4 +87,16 @@ message GetECZoneInfoRequestProto {
 
 message GetECZoneInfoResponseProto {
   optional ECZoneInfoProto ECZoneInfo = 1;
+}
+
+/**
+ * Block erasure coding recovery info
+ */
+message BlockECRecoveryInfoProto {
+  required ExtendedBlockProto block = 1;
+  required DatanodeInfosProto sourceDnInfos = 2;
+  required DatanodeInfosProto targetDnInfos = 3;
+  required StorageUuidsProto targetStorageUuids = 4;
+  required StorageTypesProto targetStorageTypes = 5;
+  repeated uint32 liveBlockIndices = 6;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92aadcd7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
index 4b42f4c..4ec4ea5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@ -24,6 +24,8 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -40,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
@@ -63,15 +66,20 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -639,4 +647,84 @@ public class TestPBHelper {
         .build();
     Assert.assertEquals(s, PBHelper.convert(PBHelper.convert(s)));
   }
+  
+  @Test
+  public void testBlockECRecoveryCommand() {
+    DatanodeInfo[] dnInfos0 = new DatanodeInfo[] {
+        DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
+    DatanodeStorageInfo targetDnInfos_0 = BlockManagerTestUtil
+        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
+            new DatanodeStorage("s00"));
+    DatanodeStorageInfo targetDnInfos_1 = BlockManagerTestUtil
+        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
+            new DatanodeStorage("s01"));
+    DatanodeStorageInfo[] targetDnInfos0 = new DatanodeStorageInfo[] {
+        targetDnInfos_0, targetDnInfos_1 };
+    short[] liveBlkIndices0 = new short[2];
+    BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo(
+        new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0,
+        liveBlkIndices0);
+    DatanodeInfo[] dnInfos1 = new DatanodeInfo[] {
+        DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
+    DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil
+        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
+            new DatanodeStorage("s02"));
+    DatanodeStorageInfo targetDnInfos_3 = BlockManagerTestUtil
+        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
+            new DatanodeStorage("s03"));
+    DatanodeStorageInfo[] targetDnInfos1 = new DatanodeStorageInfo[] {
+        targetDnInfos_2, targetDnInfos_3 };
+    short[] liveBlkIndices1 = new short[2];
+    BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo(
+        new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1,
+        liveBlkIndices1);
+    List<BlockECRecoveryInfo> blkRecoveryInfosList = new ArrayList<BlockECRecoveryInfo>();
+    blkRecoveryInfosList.add(blkECRecoveryInfo0);
+    blkRecoveryInfosList.add(blkECRecoveryInfo1);
+    BlockECRecoveryCommand blkECRecoveryCmd = new BlockECRecoveryCommand(
+        DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY, blkRecoveryInfosList);
+    BlockECRecoveryCommandProto blkECRecoveryCmdProto = PBHelper
+        .convert(blkECRecoveryCmd);
+    blkECRecoveryCmd = PBHelper.convert(blkECRecoveryCmdProto);
+    Iterator<BlockECRecoveryInfo> iterator = blkECRecoveryCmd.getECTasks()
+        .iterator();
+    assertBlockECRecoveryInfoEquals(blkECRecoveryInfo0, iterator.next());
+    assertBlockECRecoveryInfoEquals(blkECRecoveryInfo1, iterator.next());
+  }
+
+  private void assertBlockECRecoveryInfoEquals(
+      BlockECRecoveryInfo blkECRecoveryInfo1,
+      BlockECRecoveryInfo blkECRecoveryInfo2) {
+    assertEquals(blkECRecoveryInfo1.getExtendedBlock(),
+        blkECRecoveryInfo2.getExtendedBlock());
+
+    DatanodeInfo[] sourceDnInfos1 = blkECRecoveryInfo1.getSourceDnInfos();
+    DatanodeInfo[] sourceDnInfos2 = blkECRecoveryInfo2.getSourceDnInfos();
+    assertDnInfosEqual(sourceDnInfos1, sourceDnInfos2);
+
+    DatanodeInfo[] targetDnInfos1 = blkECRecoveryInfo1.getTargetDnInfos();
+    DatanodeInfo[] targetDnInfos2 = blkECRecoveryInfo2.getTargetDnInfos();
+    assertDnInfosEqual(targetDnInfos1, targetDnInfos2);
+
+    String[] targetStorageIDs1 = blkECRecoveryInfo1.getTargetStorageIDs();
+    String[] targetStorageIDs2 = blkECRecoveryInfo2.getTargetStorageIDs();
+    assertEquals(targetStorageIDs1.length, targetStorageIDs2.length);
+    for (int i = 0; i < targetStorageIDs1.length; i++) {
+      assertEquals(targetStorageIDs1[i], targetStorageIDs2[i]);
+    }
+
+    short[] liveBlockIndices1 = blkECRecoveryInfo1.getLiveBlockIndices();
+    short[] liveBlockIndices2 = blkECRecoveryInfo2.getLiveBlockIndices();
+    for (int i = 0; i < liveBlockIndices1.length; i++) {
+      assertEquals(liveBlockIndices1[i], liveBlockIndices2[i]);
+    }
+  }
+
+  private void assertDnInfosEqual(DatanodeInfo[] dnInfos1,
+      DatanodeInfo[] dnInfos2) {
+    assertEquals(dnInfos1.length, dnInfos2.length);
+    for (int i = 0; i < dnInfos1.length; i++) {
+      compare(dnInfos1[i], dnInfos2[i]);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92aadcd7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
index ea18c3e..ca4fbbc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -29,9 +29,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -115,10 +115,10 @@ public class TestRecoverStripedBlocks {
         last.getNumberOfBlocksToBeErasureCoded());
     List<BlockECRecoveryInfo> recovery = last.getErasureCodeCommand(numBlocks);
     for (BlockECRecoveryInfo info : recovery) {
-      assertEquals(1, info.targets.length);
-      assertEquals(last, info.targets[0].getDatanodeDescriptor());
-      assertEquals(GROUP_SIZE - 1, info.sources.length);
-      assertEquals(GROUP_SIZE - 1, info.liveBlockIndices.length);
+      assertEquals(1, info.getTargetDnInfos().length);
+      assertEquals(last, info.getTargetDnInfos()[0]);
+      assertEquals(GROUP_SIZE - 1, info.getSourceDnInfos().length);
+      assertEquals(GROUP_SIZE - 1, info.getLiveBlockIndices().length);
     }
   }
 }


[25/50] hadoop git commit: HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all ECSchemas loaded in Namenode. (Contributed by Vinayakumar B)

Posted by ji...@apache.org.
HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all ECSchemas loaded in Namenode. (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/824683d1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/824683d1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/824683d1

Branch: refs/heads/HDFS-7285
Commit: 824683d13b71aed0438e2f89d840de170e556b4e
Parents: 1ce17bc
Author: Vinayakumar B <vi...@apache.org>
Authored: Fri Apr 10 15:07:32 2015 +0530
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:15:36 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/io/erasurecode/ECSchema.java  |  4 +-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  5 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 11 ++++
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 10 ++++
 ...tNamenodeProtocolServerSideTranslatorPB.java | 19 +++++++
 .../ClientNamenodeProtocolTranslatorPB.java     | 26 ++++++++-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  5 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 17 ++++++
 .../hdfs/server/namenode/NameNodeRpcServer.java |  9 +++-
 .../src/main/proto/ClientNamenodeProtocol.proto |  9 ++++
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |  3 +-
 .../org/apache/hadoop/hdfs/TestECSchemas.java   | 57 ++++++++++++++++++++
 12 files changed, 164 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 8c3310e..32077f6 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
@@ -123,12 +123,12 @@ public final class ECSchema {
 
     this.chunkSize = DEFAULT_CHUNK_SIZE;
     try {
-      if (options.containsKey(CHUNK_SIZE_KEY)) {
+      if (this.options.containsKey(CHUNK_SIZE_KEY)) {
         this.chunkSize = Integer.parseInt(options.get(CHUNK_SIZE_KEY));
       }
     } catch (NumberFormatException e) {
       throw new IllegalArgumentException("Option value " +
-          options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
+          this.options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
           " is found. It should be an integer");
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 753795a..5250dfa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -58,4 +58,7 @@
 
     HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng)
 
-    HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng)
\ No newline at end of file
+    HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng)
+
+    HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all
+    ECSchemas loaded in Namenode. (vinayakumarb)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 16f876c..994d5ac 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
@@ -163,6 +163,7 @@ import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
@@ -3109,6 +3110,16 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  public ECSchema[] getECSchemas() throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("getECSchemas", traceSampler);
+    try {
+      return namenode.getECSchemas();
+    } finally {
+      scope.close();
+    }
+  }
+
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
     return new DFSInotifyEventInputStream(traceSampler, namenode);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 45d92f3..7f5ac49 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
@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.retry.AtMostOnce;
 import org.apache.hadoop.io.retry.Idempotent;
 import org.apache.hadoop.security.AccessControlException;
@@ -1474,4 +1475,13 @@ public interface ClientProtocol {
    */
   @Idempotent
   public ECInfo getErasureCodingInfo(String src) throws IOException;
+
+  /**
+   * Gets list of ECSchemas loaded in Namenode
+   *
+   * @return Returns the list of ECSchemas loaded at Namenode
+   * @throws IOException
+   */
+  @Idempotent
+  public ECSchema[] getECSchemas() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 7569819..79ab939 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -107,6 +107,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
@@ -218,6 +220,7 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
@@ -1530,4 +1533,20 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetECSchemasResponseProto getECSchemas(RpcController controller,
+      GetECSchemasRequestProto request) throws ServiceException {
+    try {
+      ECSchema[] ecSchemas = server.getECSchemas();
+      GetECSchemasResponseProto.Builder resBuilder = GetECSchemasResponseProto
+          .newBuilder();
+      for (ECSchema ecSchema : ecSchemas) {
+        resBuilder.addSchemas(PBHelper.convertECSchema(ecSchema));
+      }
+      return resBuilder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 568da68..0211522 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -107,6 +107,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto;
@@ -165,10 +167,11 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
+import org.apache.hadoop.hdfs.protocol.proto.*;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.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;
@@ -180,6 +183,7 @@ import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolTranslator;
@@ -237,6 +241,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
   VOID_GET_STORAGE_POLICIES_REQUEST =
       GetStoragePoliciesRequestProto.newBuilder().build();
 
+  private final static GetECSchemasRequestProto
+  VOID_GET_ECSCHEMAS_REQUEST = GetECSchemasRequestProto
+      .newBuilder().build();
+
   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
     rpcProxy = proxy;
   }
@@ -1550,4 +1558,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public ECSchema[] getECSchemas() throws IOException {
+    try {
+      GetECSchemasResponseProto response = rpcProxy.getECSchemas(null,
+          VOID_GET_ECSCHEMAS_REQUEST);
+      ECSchema[] schemas = new ECSchema[response.getSchemasCount()];
+      int i = 0;
+      for (ECSchemaProto schemaProto : response.getSchemasList()) {
+        schemas[i++] = PBHelper.convertECSchema(schemaProto);
+      }
+      return schemas;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 9aff943..dd86e9d 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
@@ -3121,8 +3121,6 @@ public class PBHelper {
     for (ECSchemaOptionEntryProto option : optionsList) {
       options.put(option.getKey(), option.getValue());
     }
-    // include chunksize in options.
-    options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(schema.getChunkSize()));
     return new ECSchema(schema.getSchemaName(), schema.getCodecName(),
         schema.getDataUnits(), schema.getParityUnits(), options);
   }
@@ -3132,8 +3130,7 @@ public class PBHelper {
         .setSchemaName(schema.getSchemaName())
         .setCodecName(schema.getCodecName())
         .setDataUnits(schema.getNumDataUnits())
-        .setParityUnits(schema.getNumParityUnits())
-        .setChunkSize(schema.getChunkSize());
+        .setParityUnits(schema.getNumParityUnits());
     Set<Entry<String, String>> entrySet = schema.getOptions().entrySet();
     for (Entry<String, String> entry : entrySet) {
       builder.addOptions(ECSchemaOptionEntryProto.newBuilder()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 5eb84c7..af4962b 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
@@ -8186,6 +8186,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return null;
   }
 
+  /**
+   * Get available ECSchemas
+   */
+  ECSchema[] getECSchemas() throws IOException {
+    checkOperation(OperationCategory.READ);
+    waitForLoadingFSImage();
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      // TODO HDFS-7866 Need to return all schemas maintained by Namenode
+      ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+      return new ECSchema[] { defaultSchema };
+    } finally {
+      readUnlock();
+    }
+  }
+
   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
                 boolean logRetryCache)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 d2b20a8..54e31b0 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
@@ -143,6 +143,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RetryCache;
@@ -2053,9 +2054,15 @@ class NameNodeRpcServer implements NamenodeProtocols {
     nn.spanReceiverHost.removeSpanReceiver(id);
   }
 
-  @Override // ClientNameNodeProtocol
+  @Override // ClientProtocol
   public ECInfo getErasureCodingInfo(String src) throws IOException {
     checkNNStartup();
     return namesystem.getErasureCodingInfo(src);
   }
+
+  @Override // ClientProtocol
+  public ECSchema[] getECSchemas() throws IOException {
+    checkNNStartup();
+    return namesystem.getECSchemas();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 9488aed..3389a22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -729,6 +729,13 @@ message GetErasureCodingInfoResponseProto {
   optional ECInfoProto ECInfo = 1;
 }
 
+message GetECSchemasRequestProto { // void request
+}
+
+message GetECSchemasResponseProto {
+  repeated ECSchemaProto schemas = 1;
+}
+
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -879,4 +886,6 @@ service ClientNamenodeProtocol {
       returns(GetEditsFromTxidResponseProto);
   rpc getErasureCodingInfo(GetErasureCodingInfoRequestProto)
       returns(GetErasureCodingInfoResponseProto);
+  rpc getECSchemas(GetECSchemasRequestProto)
+      returns(GetECSchemasResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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 1314ea0..0507538 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -637,8 +637,7 @@ message ECSchemaProto {
   required string codecName = 2;
   required uint32 dataUnits = 3;
   required uint32 parityUnits = 4;
-  required uint32 chunkSize = 5;
-  repeated ECSchemaOptionEntryProto options = 6;
+  repeated ECSchemaOptionEntryProto options = 5;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/824683d1/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
new file mode 100644
index 0000000..07e1359
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestECSchemas {
+  private MiniDFSCluster cluster;
+
+  @Before
+  public void before() throws IOException {
+    cluster = new MiniDFSCluster.Builder(new Configuration()).numDataNodes(0)
+        .build();
+    cluster.waitActive();
+  }
+
+  @After
+  public void after() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testGetECSchemas() throws Exception {
+    ECSchema[] ecSchemas = cluster.getFileSystem().getClient().getECSchemas();
+    // TODO update assertion after HDFS-7866
+    assertNotNull(ecSchemas);
+    assertEquals("Should have only one ecSchema", 1, ecSchemas.length);
+    assertEquals("Returned schemas should have only default schema",
+        ECSchemaManager.getSystemDefaultSchema(), ecSchemas[0]);
+  }
+}


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 5b9f8733fa786f026bdc0750752030efbeb5b0c8
Parents: e6a9253
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Apr 24 00:19:12 2015 +0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:56 2015 -0700

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


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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b9f8733/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index c213183..26f6d2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -159,7 +159,7 @@ public class TestDFSStripedOutputStream {
     // check file length
     FileStatus status = fs.getFileStatus(testPath);
     Assert.assertEquals(writeBytes, status.getLen());
-    
+
     checkData(src, writeBytes);
   }
 
@@ -236,7 +236,7 @@ public class TestDFSStripedOutputStream {
           cellSize, dataBlockBytes, parityBlockBytes);
     }
   }
-    
+
   static void verifyParity(final long size, final int cellSize,
       byte[][] dataBytes, byte[][] parityBytes) {
     // verify the parity blocks

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


[21/50] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit mainly addresses HDFS-8081 and HDFS-8048. Contributed by Zhe Zhang.

Posted by ji...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit mainly addresses HDFS-8081 and HDFS-8048. 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/87e09b71
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/87e09b71
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/87e09b71

Branch: refs/heads/HDFS-7285
Commit: 87e09b71aa12341ce613edfe74dc92e0c9fb4ff1
Parents: 824683d
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 13 10:56:24 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:15:36 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSInputStream.java |  4 ++--
 .../apache/hadoop/hdfs/DFSStripedInputStream.java   | 16 +++++++++-------
 .../apache/hadoop/hdfs/DFSStripedOutputStream.java  |  3 ++-
 .../hadoop/hdfs/server/namenode/FSNamesystem.java   |  5 +++--
 .../hadoop/hdfs/TestDFSStripedOutputStream.java     |  3 ++-
 5 files changed, 18 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/87e09b71/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 79bbd54..9104f84 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
@@ -1106,7 +1106,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
     final int length = (int) (end - start + 1);
-    actualGetFromOneDataNode(datanode, block, start, end, buf,
+    actualGetFromOneDataNode(datanode, blockStartOffset, start, end, buf,
         new int[]{offset}, new int[]{length}, corruptedBlockMap);
   }
 
@@ -1125,7 +1125,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    *                          block replica
    */
   void actualGetFromOneDataNode(final DNAddrPair datanode,
-      LocatedBlock block, final long startInBlk, final long endInBlk,
+      long blockStartOffset, final long startInBlk, final long endInBlk,
       byte[] buf, int[] offsets, int[] lengths,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87e09b71/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 077b0f8..8a431b1 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
@@ -224,7 +224,7 @@ public class DFSStripedInputStream extends DFSInputStream {
    * Real implementation of pread.
    */
   @Override
-  protected void fetchBlockByteRange(LocatedBlock block, long start,
+  protected void fetchBlockByteRange(long blockStartOffset, long start,
       long end, byte[] buf, int offset,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
@@ -234,7 +234,7 @@ public class DFSStripedInputStream extends DFSInputStream {
     int len = (int) (end - start + 1);
 
     // Refresh the striped block group
-    block = getBlockGroupAt(block.getStartOffset());
+    LocatedBlock block = getBlockGroupAt(blockStartOffset);
     assert block instanceof LocatedStripedBlock : "NameNode" +
         " should return a LocatedStripedBlock for a striped file";
     LocatedStripedBlock blockGroup = (LocatedStripedBlock) block;
@@ -254,9 +254,11 @@ public class DFSStripedInputStream extends DFSInputStream {
       DatanodeInfo loc = blks[i].getLocations()[0];
       StorageType type = blks[i].getStorageTypes()[0];
       DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr(
-          loc.getXferAddr(dfsClient.getConf().connectToDnViaHostname)), type);
-      Callable<Void> readCallable = getFromOneDataNode(dnAddr, blks[i],
-          rp.startOffsetInBlock, rp.startOffsetInBlock + rp.readLength - 1, buf,
+          loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())),
+          type);
+      Callable<Void> readCallable = getFromOneDataNode(dnAddr,
+          blks[i].getStartOffset(), rp.startOffsetInBlock,
+          rp.startOffsetInBlock + rp.readLength - 1, buf,
           rp.getOffsets(), rp.getLengths(), corruptedBlockMap, i);
       Future<Void> getFromDNRequest = stripedReadsService.submit(readCallable);
       DFSClient.LOG.debug("Submitting striped read request for " + blks[i]);
@@ -272,7 +274,7 @@ public class DFSStripedInputStream extends DFSInputStream {
   }
 
   private Callable<Void> getFromOneDataNode(final DNAddrPair datanode,
-      final LocatedBlock block, final long start, final long end,
+      final long blockStartOffset, final long start, final long end,
       final byte[] buf, final int[] offsets, final int[] lengths,
       final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
       final int hedgedReadId) {
@@ -283,7 +285,7 @@ public class DFSStripedInputStream extends DFSInputStream {
         TraceScope scope =
             Trace.startSpan("Parallel reading " + hedgedReadId, parentSpan);
         try {
-          actualGetFromOneDataNode(datanode, block, start,
+          actualGetFromOneDataNode(datanode, blockStartOffset, start,
               end, buf, offsets, lengths, corruptedBlockMap);
         } finally {
           scope.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87e09b71/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 aded4fe..1d0e1be 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
@@ -284,7 +284,8 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     }
     for (StripedDataStreamer streamer : streamers) {
       streamer.setLastException(new IOException("Lease timeout of "
-          + (dfsClient.getHdfsTimeout()/1000) + " seconds expired."));
+          + (dfsClient.getConf().getHdfsTimeout()/1000) +
+          " seconds expired."));
     }
     closeThreads(true);
     dfsClient.endFileLease(fileId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87e09b71/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 af4962b..78c82ce 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
@@ -3074,7 +3074,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final long blockSize;
     final short numTargets;
     final byte storagePolicyID;
-    final boolean isStriped;
     Node clientNode = null;
     String clientMachine = null;
 
@@ -3116,7 +3115,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       clientNode = blockManager.getDatanodeManager().getDatanodeByHost(
           clientMachine);
       // TODO: make block group size configurable (HDFS-7337)
-      isStriped = pendingFile.isStriped();
+      boolean isStriped = pendingFile.isStriped();
       numTargets = isStriped ?
           HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS :
           pendingFile.getFileReplication();
@@ -3145,6 +3144,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       ExtendedBlock previous, DatanodeStorageInfo[] targets) throws IOException {
     Block newBlock = null;
     long offset;
+    boolean isStriped;
     checkOperation(OperationCategory.WRITE);
     waitForLoadingFSImage();
     writeLock();
@@ -3175,6 +3175,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       commitOrCompleteLastBlock(pendingFile, fileState.iip,
                                 ExtendedBlock.getLocalBlock(previous));
 
+      isStriped = pendingFile.isStriped();
       // allocate new block, record block locations in INode.
       newBlock = createNewBlock(isStriped);
       saveAllocatedBlock(src, fileState.iip, newBlock, targets,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87e09b71/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 f5a37f3..ee6998b 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
@@ -5,6 +5,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -241,7 +242,7 @@ public class TestDFSStripedOutputStream {
         }
 
         block.setNumBytes(lenOfBlock);
-        BlockReader blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)).
+        BlockReader blockReader = new BlockReaderFactory(new DfsClientConf(conf)).
             setFileName(src).
             setBlock(block).
             setBlockToken(lblock.getBlockToken()).