You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zh...@apache.org on 2015/03/16 21:18:08 UTC

[01/50] [abbrv] hadoop git commit: HDFS-7491. Add incremental blockreport latency to DN metrics. Contributed by Ming Ma.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 5ff0e73ce -> 4a6ab3758 (forced update)


HDFS-7491. Add incremental blockreport latency to DN metrics. Contributed by Ming Ma.


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

Branch: refs/heads/HDFS-7285
Commit: fb34f45727e63ea55377fe90241328025307d818
Parents: 344d7cb
Author: cnauroth <cn...@apache.org>
Authored: Wed Mar 11 14:11:19 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Wed Mar 11 14:11:19 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Metrics.md                  | 2 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                     | 3 +++
 .../org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java  | 2 ++
 .../hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java    | 5 +++++
 .../apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java | 2 ++
 5 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb34f457/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
index 0e0fc09..4a10a00 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -303,6 +303,8 @@ Each metrics record contains tags such as SessionId and Hostname as additional i
 | `HeartbeatsAvgTime` | Average heartbeat time in milliseconds |
 | `BlockReportsNumOps` | Total number of block report operations |
 | `BlockReportsAvgTime` | Average time of block report operations in milliseconds |
+| `IncrementalBlockReportsNumOps` | Total number of incremental block report operations |
+| `IncrementalBlockReportsAvgTime` | Average time of incremental block report operations in milliseconds |
 | `CacheReportsNumOps` | Total number of cache report operations |
 | `CacheReportsAvgTime` | Average time of cache report operations in milliseconds |
 | `PacketAckRoundTripTimeNanosNumOps` | Total number of ack round trip |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb34f457/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 0ba34a4..07213dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -740,6 +740,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-6806. HDFS Rolling upgrade document should mention the versions
     available. (J.Andreina via aajisaka)
 
+    HDFS-7491. Add incremental blockreport latency to DN metrics.
+    (Ming Ma via cnauroth)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb34f457/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index ff1ad78..3ba2f54 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -291,12 +291,14 @@ class BPServiceActor implements Runnable {
 
     // Send incremental block reports to the Namenode outside the lock
     boolean success = false;
+    final long startTime = Time.monotonicNow();
     try {
       bpNamenode.blockReceivedAndDeleted(bpRegistration,
           bpos.getBlockPoolId(),
           reports.toArray(new StorageReceivedDeletedBlocks[reports.size()]));
       success = true;
     } finally {
+      dn.getMetrics().addIncrementalBlockReport(Time.monotonicNow()-startTime);
       if (!success) {
         synchronized (pendingIncrementalBRperStorage) {
           for (StorageReceivedDeletedBlocks report : reports) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb34f457/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
index 0fbc2ee..2e8eb22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
@@ -107,6 +107,7 @@ public class DataNodeMetrics {
   @Metric MutableRate replaceBlockOp;
   @Metric MutableRate heartbeats;
   @Metric MutableRate blockReports;
+  @Metric MutableRate incrementalBlockReports;
   @Metric MutableRate cacheReports;
   @Metric MutableRate packetAckRoundTripTimeNanos;
   final MutableQuantiles[] packetAckRoundTripTimeNanosQuantiles;
@@ -201,6 +202,10 @@ public class DataNodeMetrics {
     blockReports.add(latency);
   }
 
+  public void addIncrementalBlockReport(long latency) {
+    incrementalBlockReports.add(latency);
+  }
+
   public void addCacheReport(long latency) {
     cacheReports.add(latency);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb34f457/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java
index 8a2bacf..5d27fe6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java
@@ -72,6 +72,8 @@ public class TestDataNodeMetrics {
       DataNode datanode = datanodes.get(0);
       MetricsRecordBuilder rb = getMetrics(datanode.getMetrics().name());
       assertCounter("BytesWritten", LONG_FILE_LEN, rb);
+      assertTrue("Expected non-zero number of incremental block reports",
+          getLongCounter("IncrementalBlockReportsNumOps", rb) > 0);
     } finally {
       if (cluster != null) {cluster.shutdown();}
     }


[43/50] [abbrv] hadoop git commit: HDFS-7749. Erasure Coding: Add striped block support in INodeFile. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-7749. Erasure Coding: Add striped block support in INodeFile. 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/5e90e36b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5e90e36b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5e90e36b

Branch: refs/heads/HDFS-7285
Commit: 5e90e36b4d50654cee8c8ed165d0a1e014cc7e24
Parents: 03778d4
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Feb 25 22:10:26 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:10:18 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  17 ++
 .../server/blockmanagement/BlockCollection.java |  13 +-
 .../hdfs/server/blockmanagement/BlockInfo.java  |  88 ++++++-
 .../BlockInfoContiguousUnderConstruction.java   |   6 +-
 .../blockmanagement/BlockInfoStriped.java       |  31 +++
 .../BlockInfoStripedUnderConstruction.java      | 240 ++++++++++++++++++
 .../server/blockmanagement/BlockManager.java    | 147 +++++------
 .../CacheReplicationMonitor.java                |  16 +-
 .../hdfs/server/namenode/FSDirConcatOp.java     |   9 +-
 .../hdfs/server/namenode/FSDirectory.java       |   5 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   8 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  16 +-
 .../hdfs/server/namenode/FSImageFormat.java     |   7 +-
 .../server/namenode/FSImageFormatPBINode.java   |  46 +++-
 .../hdfs/server/namenode/FSNamesystem.java      | 130 ++++++----
 .../namenode/FileUnderConstructionFeature.java  |  15 +-
 .../namenode/FileWithStripedBlocksFeature.java  | 112 ++++++++
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 254 +++++++++++++------
 .../hdfs/server/namenode/LeaseManager.java      |   6 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   4 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |   3 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |   7 +-
 .../server/namenode/snapshot/FileDiffList.java  |   9 +-
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |   5 +
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |  10 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   3 +-
 .../blockmanagement/TestReplicationPolicy.java  |   4 +-
 .../hdfs/server/namenode/TestAddBlock.java      |  12 +-
 .../hdfs/server/namenode/TestAddBlockgroup.java |   3 +-
 .../namenode/TestBlockUnderConstruction.java    |   6 +-
 .../hdfs/server/namenode/TestFSImage.java       |   4 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |   4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   4 +-
 .../snapshot/TestSnapshotBlocksMap.java         |  24 +-
 .../namenode/snapshot/TestSnapshotDeletion.java |  16 +-
 35 files changed, 963 insertions(+), 321 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 c428c2b..eaa26bc 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
@@ -170,6 +170,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
 import org.apache.hadoop.hdfs.protocol.proto.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;
@@ -182,6 +183,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
@@ -427,6 +429,21 @@ public class PBHelper {
     return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
   }
 
+  public static BlockInfoStriped convert(StripedBlockProto p) {
+    return new BlockInfoStriped(convert(p.getBlock()),
+        (short) p.getDataBlockNum(), (short) p.getParityBlockNum());
+  }
+
+  public static StripedBlockProto convert(BlockInfoStriped blk) {
+    BlockProto bp = BlockProto.newBuilder().setBlockId(blk.getBlockId())
+        .setGenStamp(blk.getGenerationStamp()).setNumBytes(blk.getNumBytes())
+        .build();
+    return StripedBlockProto.newBuilder()
+        .setDataBlockNum(blk.getDataBlockNum())
+        .setParityBlockNum(blk.getParityBlockNum())
+        .setBlock(bp).build();
+  }
+
   public static BlockWithLocationsProto convert(BlockWithLocations blk) {
     return BlockWithLocationsProto.newBuilder()
         .setBlock(convert(blk.getBlock()))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
index 974cac3..1c753de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
@@ -31,7 +31,7 @@ public interface BlockCollection {
   /**
    * Get the last block of the collection.
    */
-  public BlockInfoContiguous getLastBlock();
+  public BlockInfo getLastBlock();
 
   /** 
    * Get content summary.
@@ -44,9 +44,9 @@ public interface BlockCollection {
   public int numBlocks();
 
   /**
-   * Get the blocks or block groups.
+   * Get the blocks (striped or contiguous).
    */
-  public BlockInfoContiguous[] getBlocks();
+  public BlockInfo[] getBlocks();
 
   /**
    * Get preferred block size for the collection 
@@ -71,16 +71,15 @@ public interface BlockCollection {
   public String getName();
 
   /**
-   * Set the block/block-group at the given index.
+   * Set the block (contiguous or striped) at the given index.
    */
-  public void setBlock(int index, BlockInfoContiguous blk);
+  public void setBlock(int index, BlockInfo blk);
 
   /**
    * Convert the last block of the collection to an under-construction block
    * and set the locations.
    */
-  public BlockInfoContiguousUnderConstruction setLastBlock(
-      BlockInfoContiguous lastBlock,
+  public void convertLastBlockToUC(BlockInfo lastBlock,
       DatanodeStorageInfo[] targets) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index f19ad32..d15cbec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.LightWeightGSet;
 
+import java.io.IOException;
 import java.util.LinkedList;
 
 /**
@@ -289,8 +290,9 @@ public abstract class BlockInfo extends Block
 
   /**
    * BlockInfo represents a block that is not being constructed.
-   * In order to start modifying the block, the BlockInfo should be converted
-   * to {@link BlockInfoContiguousUnderConstruction}.
+   * In order to start modifying the block, the BlockInfo should be converted to
+   * {@link BlockInfoContiguousUnderConstruction} or
+   * {@link BlockInfoStripedUnderConstruction}.
    * @return {@link HdfsServerConstants.BlockUCState#COMPLETE}
    */
   public HdfsServerConstants.BlockUCState getBlockUCState() {
@@ -336,4 +338,86 @@ public abstract class BlockInfo extends Block
       return new BlockInfoStriped((BlockInfoStriped) b);
     }
   }
+
+  static BlockInfo convertToCompleteBlock(BlockInfo blk) throws IOException {
+    if (blk instanceof BlockInfoContiguousUnderConstruction) {
+      return ((BlockInfoContiguousUnderConstruction) blk)
+          .convertToCompleteBlock();
+    } else if (blk instanceof BlockInfoStripedUnderConstruction) {
+      return ((BlockInfoStripedUnderConstruction) blk).convertToCompleteBlock();
+    } else {
+      return blk;
+    }
+  }
+
+  static void commitBlock(BlockInfo blockInfo, Block reported)
+      throws IOException {
+    if (blockInfo instanceof BlockInfoContiguousUnderConstruction) {
+      ((BlockInfoContiguousUnderConstruction) blockInfo).commitBlock(reported);
+    } else if (blockInfo instanceof BlockInfoStripedUnderConstruction) {
+      ((BlockInfoStripedUnderConstruction) blockInfo).commitBlock(reported);
+    }
+  }
+
+  static void addReplica(BlockInfo ucBlock, DatanodeStorageInfo storageInfo,
+      Block reportedBlock, HdfsServerConstants.ReplicaState reportedState) {
+    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
+        ucBlock instanceof BlockInfoStripedUnderConstruction;
+    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
+      ((BlockInfoContiguousUnderConstruction) ucBlock).addReplicaIfNotPresent(
+          storageInfo, reportedBlock, reportedState);
+    } else { // StripedUC
+      ((BlockInfoStripedUnderConstruction) ucBlock).addReplicaIfNotPresent(
+          storageInfo, reportedBlock, reportedState);
+    }
+  }
+
+  static int getNumExpectedLocations(BlockInfo ucBlock) {
+    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
+        ucBlock instanceof BlockInfoStripedUnderConstruction;
+    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
+      return ((BlockInfoContiguousUnderConstruction) ucBlock)
+          .getNumExpectedLocations();
+    } else { // StripedUC
+      return ((BlockInfoStripedUnderConstruction) ucBlock)
+          .getNumExpectedLocations();
+    }
+  }
+
+  public static DatanodeStorageInfo[] getExpectedStorageLocations(
+      BlockInfo ucBlock) {
+    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
+        ucBlock instanceof BlockInfoStripedUnderConstruction;
+    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
+      return ((BlockInfoContiguousUnderConstruction) ucBlock)
+          .getExpectedStorageLocations();
+    } else { // StripedUC
+      return ((BlockInfoStripedUnderConstruction) ucBlock)
+          .getExpectedStorageLocations();
+    }
+  }
+
+  public static void setExpectedLocations(BlockInfo ucBlock,
+      DatanodeStorageInfo[] targets) {
+    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
+        ucBlock instanceof BlockInfoStripedUnderConstruction;
+    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
+      ((BlockInfoContiguousUnderConstruction) ucBlock)
+          .setExpectedLocations(targets);
+    } else { // StripedUC
+      ((BlockInfoStripedUnderConstruction) ucBlock)
+          .setExpectedLocations(targets);
+    }
+  }
+
+  public static long getBlockRecoveryId(BlockInfo ucBlock) {
+    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
+        ucBlock instanceof BlockInfoStripedUnderConstruction;
+    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
+      return ((BlockInfoContiguousUnderConstruction) ucBlock)
+          .getBlockRecoveryId();
+    } else { // StripedUC
+      return ((BlockInfoStripedUnderConstruction) ucBlock).getBlockRecoveryId();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 e75c5d7..e45cd4e 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
@@ -74,7 +74,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
       BlockUCState state, DatanodeStorageInfo[] targets) {
     super(blk, replication);
     assert getBlockUCState() != BlockUCState.COMPLETE :
-      "BlockInfoUnderConstruction cannot be in COMPLETE state";
+      "BlockInfoContiguousUnderConstruction cannot be in COMPLETE state";
     this.blockUCState = state;
     setExpectedLocations(targets);
   }
@@ -82,7 +82,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   /**
    * Convert an under construction block to a complete block.
    * 
-   * @return BlockInfo - a complete block.
+   * @return BlockInfoContiguous - a complete block.
    * @throws IOException if the state of the block 
    * (the generation stamp and the length) has not been committed by 
    * the client or it does not have at least a minimal number of replicas 
@@ -197,7 +197,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     blockRecoveryId = recoveryId;
     if (replicas.size() == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK*"
-        + " BlockInfoUnderConstruction.initLeaseRecovery:"
+        + " BlockInfoContiguousUnderConstruction.initLeaseRecovery:"
         + " No blocks found, lease removed.");
     }
     boolean allLiveReplicasTriedAsPrimary = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 5fff41e..57de772 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
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
 /**
  * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
@@ -59,6 +61,14 @@ public class BlockInfoStriped extends BlockInfo {
     return (short) (dataBlockNum + parityBlockNum);
   }
 
+  public short getDataBlockNum() {
+    return dataBlockNum;
+  }
+
+  public short getParityBlockNum() {
+    return parityBlockNum;
+  }
+
   private void initIndices() {
     for (int i = 0; i < indices.length; i++) {
       indices[i] = -1;
@@ -176,4 +186,25 @@ public class BlockInfoStriped extends BlockInfo {
     }
     return num;
   }
+
+  /**
+   * Convert a complete block to an under construction block.
+   * @return BlockInfoUnderConstruction -  an under construction block.
+   */
+  public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction(
+      BlockUCState s, DatanodeStorageInfo[] targets) {
+    final BlockInfoStripedUnderConstruction ucBlock;
+    if(isComplete()) {
+      ucBlock = new BlockInfoStripedUnderConstruction(this, getDataBlockNum(),
+              getParityBlockNum(),  s, targets);
+      ucBlock.setBlockCollection(getBlockCollection());
+    } else {
+      // the block is already under construction
+      ucBlock = (BlockInfoStripedUnderConstruction) this;
+      ucBlock.setBlockUCState(s);
+      ucBlock.setExpectedLocations(targets);
+      ucBlock.setBlockCollection(getBlockCollection());
+    }
+    return ucBlock;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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
new file mode 100644
index 0000000..151241b2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
@@ -0,0 +1,240 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
+
+/**
+ * Represents a striped block that is currently being constructed.
+ * This is usually the last block of a file opened for write or append.
+ */
+public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
+  private BlockUCState blockUCState;
+
+  /**
+   * Block replicas as assigned when the block was allocated.
+   *
+   * TODO: we need to update this attribute, along with the return type of
+   * getExpectedStorageLocations and LocatedBlock. For striped blocks, clients
+   * need to understand the index of each striped block in the block group.
+   */
+  private List<ReplicaUnderConstruction> replicas;
+
+  /**
+   * The new generation stamp, which this block will have
+   * after the recovery succeeds. Also used as a recovery id to identify
+   * the right recovery if any of the abandoned recoveries re-appear.
+   */
+  private long blockRecoveryId = 0;
+
+  /**
+   * Constructor with null storage targets.
+   */
+  public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum,
+      short parityBlockNum) {
+    this(blk, dataBlockNum, parityBlockNum, UNDER_CONSTRUCTION, null);
+  }
+
+  /**
+   * Create a striped block that is currently being constructed.
+   */
+  public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum,
+      short parityBlockNum, BlockUCState state, DatanodeStorageInfo[] targets) {
+    super(blk, dataBlockNum, parityBlockNum);
+    assert getBlockUCState() != COMPLETE :
+      "BlockInfoStripedUnderConstruction cannot be in COMPLETE state";
+    this.blockUCState = state;
+    setExpectedLocations(targets);
+  }
+
+  /**
+   * Convert an under construction striped block to a complete striped block.
+   * 
+   * @return BlockInfoStriped - a complete block.
+   * @throws IOException if the state of the block 
+   * (the generation stamp and the length) has not been committed by 
+   * the client or it does not have at least a minimal number of replicas 
+   * reported from data-nodes. 
+   */
+  BlockInfoStriped convertToCompleteBlock() throws IOException {
+    assert getBlockUCState() != COMPLETE :
+      "Trying to convert a COMPLETE block";
+    return new BlockInfoStriped(this);
+  }
+
+  /** Set expected locations */
+  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
+    int numLocations = targets == null ? 0 : targets.length;
+    this.replicas = new ArrayList<>(numLocations);
+    for(int i = 0; i < numLocations; i++) {
+      replicas.add(new ReplicaUnderConstruction(this, targets[i],
+          ReplicaState.RBW));
+    }
+  }
+
+  /**
+   * Create array of expected replica locations
+   * (as has been assigned by chooseTargets()).
+   */
+  public DatanodeStorageInfo[] getExpectedStorageLocations() {
+    int numLocations = getNumExpectedLocations();
+    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
+    for (int i = 0; i < numLocations; i++) {
+      storages[i] = replicas.get(i).getExpectedStorageLocation();
+    }
+    return storages;
+  }
+
+  /** Get the number of expected locations */
+  public int getNumExpectedLocations() {
+    return replicas == null ? 0 : replicas.size();
+  }
+
+  /**
+   * Return the state of the block under construction.
+   * @see BlockUCState
+   */
+  @Override // BlockInfo
+  public BlockUCState getBlockUCState() {
+    return blockUCState;
+  }
+
+  void setBlockUCState(BlockUCState s) {
+    blockUCState = s;
+  }
+
+  /** Get block recovery ID */
+  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.
+   */
+  public void setGenerationStampAndVerifyReplicas(long genStamp) {
+    // Set the generation stamp for the block.
+    setGenerationStamp(genStamp);
+    if (replicas == null)
+      return;
+
+    // Remove the replicas with wrong gen stamp.
+    // The replica list is unchanged.
+    for (ReplicaUnderConstruction r : replicas) {
+      if (genStamp != r.getGenerationStamp()) {
+        r.getExpectedStorageLocation().removeBlock(this);
+        NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
+            + "from location: {}", r.getExpectedStorageLocation());
+      }
+    }
+  }
+
+  /**
+   * Commit block's length and generation stamp as reported by the client.
+   * Set block state to {@link BlockUCState#COMMITTED}.
+   * @param block - contains client reported block length and generation
+   */
+  void commitBlock(Block block) throws IOException {
+    if (getBlockId() != block.getBlockId()) {
+      throw new IOException("Trying to commit inconsistent block: id = "
+          + block.getBlockId() + ", expected id = " + getBlockId());
+    }
+    blockUCState = BlockUCState.COMMITTED;
+    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
+    // Sort out invalid replicas.
+    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
+  }
+
+  /**
+   * Initialize lease recovery for this striped block.
+   */
+  public void initializeBlockRecovery(long recoveryId) {
+    setBlockUCState(BlockUCState.UNDER_RECOVERY);
+    blockRecoveryId = recoveryId;
+    if (replicas == null || replicas.size() == 0) {
+      NameNode.blockStateChangeLog.warn("BLOCK*" +
+          " BlockInfoUnderConstruction.initLeaseRecovery:" +
+          " No blocks found, lease removed.");
+    }
+    // TODO we need to implement different recovery logic here
+  }
+
+  void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block block,
+      ReplicaState rState) {
+    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
+    while (it.hasNext()) {
+      ReplicaUnderConstruction r = it.next();
+      DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
+      if (expectedLocation == storage) {
+        // Record the gen stamp from the report
+        r.setGenerationStamp(block.getGenerationStamp());
+        return;
+      } else if (expectedLocation != null &&
+          expectedLocation.getDatanodeDescriptor() ==
+              storage.getDatanodeDescriptor()) {
+        // The Datanode reported that the block is on a different storage
+        // than the one chosen by BlockPlacementPolicy. This can occur as
+        // we allow Datanodes to choose the target storage. Update our
+        // state by removing the stale entry and adding a new one.
+        it.remove();
+        break;
+      }
+    }
+    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder b = new StringBuilder(100);
+    appendStringTo(b);
+    return b.toString();
+  }
+
+  @Override
+  public void appendStringTo(StringBuilder sb) {
+    super.appendStringTo(sb);
+    appendUCParts(sb);
+  }
+
+  private void appendUCParts(StringBuilder sb) {
+    sb.append("{UCState=").append(blockUCState).append(", replicas=[");
+    if (replicas != null) {
+      Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
+      if (iter.hasNext()) {
+        iter.next().appendStringTo(sb);
+        while (iter.hasNext()) {
+          sb.append(", ");
+          iter.next().appendStringTo(sb);
+        }
+      }
+    }
+    sb.append("]}");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 d9f2772..a898696 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
@@ -539,8 +539,8 @@ public class BlockManager {
     int usableReplicas = numReplicas.liveReplicas() +
                          numReplicas.decommissionedReplicas();
     
-    if (block instanceof BlockInfoContiguous) {
-      BlockCollection bc = ((BlockInfoContiguous) block).getBlockCollection();
+    if (block instanceof BlockInfo) {
+      BlockCollection bc = ((BlockInfo) block).getBlockCollection();
       String fileName = (bc == null) ? "[orphaned]" : bc.getName();
       out.print(fileName + ": ");
     }
@@ -594,15 +594,14 @@ public class BlockManager {
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  private static boolean commitBlock(
-      final BlockInfoContiguousUnderConstruction block,
+  private static boolean commitBlock(final BlockInfo block,
       final Block commitBlock) throws IOException {
     if (block.getBlockUCState() == BlockUCState.COMMITTED)
       return false;
     assert block.getNumBytes() <= commitBlock.getNumBytes() :
       "commitBlock length is less than the stored one "
       + commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
-    block.commitBlock(commitBlock);
+    BlockInfo.commitBlock(block, commitBlock);
     return true;
   }
   
@@ -620,16 +619,16 @@ public class BlockManager {
       Block commitBlock) throws IOException {
     if(commitBlock == null)
       return false; // not committing, this is a block allocation retry
-    BlockInfoContiguous lastBlock = bc.getLastBlock();
+    BlockInfo lastBlock = bc.getLastBlock();
     if(lastBlock == null)
       return false; // no blocks in file yet
     if(lastBlock.isComplete())
       return false; // already completed (e.g. by syncBlock)
     
-    final boolean b = commitBlock(
-        (BlockInfoContiguousUnderConstruction)lastBlock, commitBlock);
-    if(countNodes(lastBlock).liveReplicas() >= minReplication)
-      completeBlock(bc, bc.numBlocks()-1, false);
+    final boolean b = commitBlock(lastBlock, commitBlock);
+    if (countNodes(lastBlock).liveReplicas() >= minReplication) {
+      completeBlock(bc, bc.numBlocks() - 1, false);
+    }
     return b;
   }
 
@@ -642,22 +641,25 @@ public class BlockManager {
    */
   private BlockInfo completeBlock(final BlockCollection bc,
       final int blkIndex, boolean force) throws IOException {
-    if(blkIndex < 0)
+    if (blkIndex < 0) {
       return null;
-    BlockInfoContiguous curBlock = bc.getBlocks()[blkIndex];
-    if (curBlock.isComplete())
+    }
+    BlockInfo curBlock = bc.getBlocks()[blkIndex];
+    if (curBlock.isComplete()) {
       return curBlock;
-    // TODO: support BlockInfoStripedUC
-    BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction)curBlock;
-    int numNodes = ucBlock.numNodes();
-    if (!force && numNodes < minReplication)
+    }
+
+    int numNodes = curBlock.numNodes();
+    if (!force && numNodes < minReplication) {
       throw new IOException("Cannot complete block: " +
           "block does not satisfy minimal replication requirement.");
-    if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
+    }
+    if (!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
       throw new IOException(
           "Cannot complete block: block has not been COMMITTED by the client");
-    BlockInfoContiguous completeBlock = ucBlock.convertToCompleteBlock();
+    }
+
+    final BlockInfo completeBlock = BlockInfo.convertToCompleteBlock(curBlock);
     // replace penultimate block in file
     bc.setBlock(blkIndex, completeBlock);
     
@@ -675,10 +677,9 @@ public class BlockManager {
     return blocksMap.replaceBlock(completeBlock);
   }
 
-  // TODO: support BlockInfoStrippedUC
   private BlockInfo completeBlock(final BlockCollection bc,
       final BlockInfo block, boolean force) throws IOException {
-    BlockInfoContiguous[] fileBlocks = bc.getBlocks();
+    BlockInfo[] fileBlocks = bc.getBlocks();
     for (int idx = 0; idx < fileBlocks.length; idx++) {
       if (fileBlocks[idx] == block) {
         return completeBlock(bc, idx, force);
@@ -694,6 +695,7 @@ public class BlockManager {
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
       final BlockInfoContiguousUnderConstruction block) throws IOException {
+    // TODO: support BlockInfoStripedUC for editlog
     block.commitBlock(block);
     return completeBlock(bc, block, true);
   }
@@ -715,7 +717,7 @@ public class BlockManager {
    */
   public LocatedBlock convertLastBlockToUnderConstruction(
       BlockCollection bc, long bytesToRemove) throws IOException {
-    BlockInfoContiguous oldBlock = bc.getLastBlock();
+    BlockInfo oldBlock = bc.getLastBlock();
     if(oldBlock == null ||
        bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
       return null;
@@ -724,8 +726,10 @@ public class BlockManager {
 
     DatanodeStorageInfo[] targets = getStorages(oldBlock);
 
-    BlockInfoContiguousUnderConstruction ucBlock = bc.setLastBlock(oldBlock,
-        targets);
+    // convert the last block to UC
+    bc.convertLastBlockToUC(oldBlock, targets);
+    // get the new created uc block
+    BlockInfo ucBlock = bc.getLastBlock();
     blocksMap.replaceBlock(ucBlock);
 
     // Remove block from replication queue.
@@ -767,11 +771,10 @@ public class BlockManager {
     return locations;
   }
   
-  private List<LocatedBlock> createLocatedBlockList(
-      final BlockInfoContiguous[] blocks,
+  private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
       final long offset, final long length, final int nrBlocksToReturn,
       final AccessMode mode) throws IOException {
-    int curBlk = 0;
+    int curBlk;
     long curPos = 0, blkSize = 0;
     int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
     for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
@@ -784,10 +787,10 @@ public class BlockManager {
     }
 
     if (nrBlocks > 0 && curBlk == nrBlocks)   // offset >= end of file
-      return Collections.<LocatedBlock>emptyList();
+      return Collections.emptyList();
 
     long endOff = offset + length;
-    List<LocatedBlock> results = new ArrayList<LocatedBlock>(blocks.length);
+    List<LocatedBlock> results = new ArrayList<>(blocks.length);
     do {
       results.add(createLocatedBlock(blocks[curBlk], curPos, mode));
       curPos += blocks[curBlk].getNumBytes();
@@ -798,9 +801,9 @@ public class BlockManager {
     return results;
   }
 
-  private LocatedBlock createLocatedBlock(final BlockInfoContiguous[] blocks,
+  private LocatedBlock createLocatedBlock(final BlockInfo[] blocks,
       final long endPos, final AccessMode mode) throws IOException {
-    int curBlk = 0;
+    int curBlk;
     long curPos = 0;
     int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
     for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
@@ -814,8 +817,9 @@ public class BlockManager {
     return createLocatedBlock(blocks[curBlk], curPos, mode);
   }
   
-  private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos,
-    final BlockTokenSecretManager.AccessMode mode) throws IOException {
+  private LocatedBlock createLocatedBlock(final BlockInfo blk,
+      final long pos, final BlockTokenSecretManager.AccessMode mode)
+      throws IOException {
     final LocatedBlock lb = createLocatedBlock(blk, pos);
     if (mode != null) {
       setBlockToken(lb, mode);
@@ -824,8 +828,8 @@ public class BlockManager {
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos
-      ) throws IOException {
+  private LocatedBlock createLocatedBlock(final BlockInfo blk,
+      final long pos) throws IOException {
     if (blk instanceof BlockInfoContiguousUnderConstruction) {
       if (blk.isComplete()) {
         throw new IOException(
@@ -838,6 +842,7 @@ public class BlockManager {
       final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
       return new LocatedBlock(eb, storages, pos, false);
     }
+    // TODO support BlockInfoStripedUC
 
     // get block locations
     final int numCorruptNodes = countNodes(blk).corruptReplicas();
@@ -872,7 +877,7 @@ public class BlockManager {
   }
 
   /** Create a LocatedBlocks. */
-  public LocatedBlocks createLocatedBlocks(final BlockInfoContiguous[] blocks,
+  public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks,
       final long fileSizeExcludeBlocksUnderConstruction,
       final boolean isFileUnderConstruction, final long offset,
       final long length, final boolean needBlockToken,
@@ -895,7 +900,7 @@ public class BlockManager {
       final LocatedBlock lastlb;
       final boolean isComplete;
       if (!inSnapshot) {
-        final BlockInfoContiguous last = blocks[blocks.length - 1];
+        final BlockInfo last = blocks[blocks.length - 1];
         final long lastPos = last.isComplete()?
             fileSizeExcludeBlocksUnderConstruction - last.getNumBytes()
             : fileSizeExcludeBlocksUnderConstruction;
@@ -1710,12 +1715,15 @@ public class BlockManager {
    * reported by the datanode in the block report. 
    */
   static class StatefulBlockInfo {
-    final BlockInfoContiguousUnderConstruction storedBlock;
+    final BlockInfo storedBlock; // should be UC block
     final Block reportedBlock;
     final ReplicaState reportedState;
     
-    StatefulBlockInfo(BlockInfoContiguousUnderConstruction storedBlock,
+    StatefulBlockInfo(BlockInfo storedBlock,
         Block reportedBlock, ReplicaState reportedState) {
+      Preconditions.checkArgument(
+          storedBlock instanceof BlockInfoContiguousUnderConstruction ||
+          storedBlock instanceof BlockInfoStripedUnderConstruction);
       this.storedBlock = storedBlock;
       this.reportedBlock = reportedBlock;
       this.reportedState = reportedState;
@@ -2021,15 +2029,12 @@ public class BlockManager {
       
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-        ((BlockInfoContiguousUnderConstruction) storedBlock)
-            .addReplicaIfNotPresent(storageInfo, iblk, reportedState);
+        BlockInfo.addReplica(storedBlock, storageInfo, iblk, reportedState);
         // OpenFileBlocks only inside snapshots also will be added to safemode
         // threshold. So we need to update such blocks to safemode
         // refer HDFS-5283
-        BlockInfoContiguousUnderConstruction blockUC =
-            (BlockInfoContiguousUnderConstruction) storedBlock;
-        if (namesystem.isInSnapshot(blockUC)) {
-          int numOfReplicas = blockUC.getNumExpectedLocations();
+        if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) {
+          int numOfReplicas = BlockInfo.getNumExpectedLocations(storedBlock);
           namesystem.incrementSafeBlockCount(numOfReplicas);
         }
         //and fall through to next clause
@@ -2052,7 +2057,7 @@ public class BlockManager {
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
     Block delimiterBlock = new Block();
-    BlockInfoContiguous delimiter = new BlockInfoContiguous(delimiterBlock,
+    BlockInfo delimiter = new BlockInfoContiguous(delimiterBlock,
         (short) 1);
     AddBlockResult result = storageInfo.addBlock(delimiter, delimiterBlock);
     assert result == AddBlockResult.ADDED 
@@ -2182,9 +2187,8 @@ public class BlockManager {
     }
 
     if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-      toUC.add(new StatefulBlockInfo(
-          (BlockInfoContiguousUnderConstruction) storedBlock,
-          new Block(block), reportedState));
+      toUC.add(new StatefulBlockInfo(storedBlock, new Block(block),
+          reportedState));
       return storedBlock;
     }
 
@@ -2368,9 +2372,8 @@ public class BlockManager {
 
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
       DatanodeStorageInfo storageInfo) throws IOException {
-    BlockInfoContiguousUnderConstruction block = ucBlock.storedBlock;
-    block.addReplicaIfNotPresent(
-        storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
+    BlockInfo block = ucBlock.storedBlock;
+    BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
         !block.findDatanode(storageInfo.getDatanodeDescriptor())) {
@@ -2430,7 +2433,8 @@ public class BlockManager {
     assert block != null && namesystem.hasWriteLock();
     BlockInfo storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
-    if (block instanceof BlockInfoContiguousUnderConstruction) {
+    if (block instanceof BlockInfoContiguousUnderConstruction ||
+        block instanceof BlockInfoStripedUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
       storedBlock = getStoredBlock(block);
     } else {
@@ -2446,7 +2450,6 @@ public class BlockManager {
       return block;
     }
     BlockCollection bc = storedBlock.getBlockCollection();
-    assert bc != null : "Block must belong to a file";
 
     // add block to the datanode
     AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock);
@@ -3407,7 +3410,7 @@ public class BlockManager {
     if (!this.shouldCheckForEnoughRacks) {
       return true;
     }
-    boolean enoughRacks = false;;
+    boolean enoughRacks = false;
     Collection<DatanodeDescriptor> corruptNodes = 
                                   corruptReplicas.getNodes(b);
     int numExpectedReplicas = getReplication(b);
@@ -3453,21 +3456,15 @@ public class BlockManager {
     return this.neededReplications.getCorruptReplOneBlockSize();
   }
 
-  public BlockInfoContiguous addBlockCollection(BlockInfoContiguous block,
+  public BlockInfo addBlockCollection(BlockInfo block,
       BlockCollection bc) {
-    // TODO
-    return (BlockInfoContiguous) blocksMap.addBlockCollection(block, bc);
+    return blocksMap.addBlockCollection(block, bc);
   }
 
   public BlockCollection getBlockCollection(Block b) {
     return blocksMap.getBlockCollection(b);
   }
 
-  /** @return an iterator of the datanodes. */
-  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
-    return blocksMap.getStorages(block);
-  }
-
   public int numCorruptReplicas(Block block) {
     return corruptReplicas.numCorruptReplicas(block);
   }
@@ -3481,26 +3478,6 @@ public class BlockManager {
   public int getCapacity() {
     return blocksMap.getCapacity();
   }
-  
-  /**
-   * Return a range of corrupt replica block ids. Up to numExpectedBlocks 
-   * blocks starting at the next block after startingBlockId are returned
-   * (fewer if numExpectedBlocks blocks are unavailable). If startingBlockId 
-   * is null, up to numExpectedBlocks blocks are returned from the beginning.
-   * If startingBlockId cannot be found, null is returned.
-   *
-   * @param numExpectedBlocks Number of block ids to return.
-   *  0 <= numExpectedBlocks <= 100
-   * @param startingBlockId Block id from which to start. If null, start at
-   *  beginning.
-   * @return Up to numExpectedBlocks blocks from startingBlockId if it exists
-   *
-   */
-  public long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
-                                   Long startingBlockId) {
-    return corruptReplicas.getCorruptReplicaBlockIds(numExpectedBlocks,
-                                                     startingBlockId);
-  }
 
   /**
    * Return an iterator over the set of blocks for which there are no replicas.
@@ -3614,7 +3591,7 @@ public class BlockManager {
     datanodeManager.clearPendingQueues();
     postponedMisreplicatedBlocks.clear();
     postponedMisreplicatedBlocksCount.set(0);
-  };
+  }
   
 
   private static class ReplicationWork {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
index 79d7713..a1290a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
@@ -369,7 +369,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
    * @param file The file.
    */
   private void rescanFile(CacheDirective directive, INodeFile file) {
-    BlockInfoContiguous[] blockInfos = file.getBlocks();
+    BlockInfo[] blockInfos = file.getBlocks();
 
     // Increment the "needed" statistics
     directive.addFilesNeeded(1);
@@ -394,7 +394,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
     }
 
     long cachedTotal = 0;
-    for (BlockInfoContiguous blockInfo : blockInfos) {
+    for (BlockInfo blockInfo : blockInfos) {
       if (!blockInfo.getBlockUCState().equals(BlockUCState.COMPLETE)) {
         // We don't try to cache blocks that are under construction.
         LOG.trace("Directive {}: can't cache block {} because it is in state "
@@ -452,8 +452,8 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
         file.getFullPathName(), cachedTotal, neededTotal);
   }
 
-  private String findReasonForNotCaching(CachedBlock cblock, 
-          BlockInfoContiguous blockInfo) {
+  private String findReasonForNotCaching(CachedBlock cblock,
+      BlockInfo blockInfo) {
     if (blockInfo == null) {
       // Somehow, a cache report with the block arrived, but the block
       // reports from the DataNode haven't (yet?) described such a block.
@@ -513,7 +513,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
           iter.remove();
         }
       }
-      BlockInfoContiguous blockInfo = namesystem.getStoredBlock(new Block(cblock.getBlockId()));
+      BlockInfo blockInfo = namesystem.getStoredBlock(new Block(cblock.getBlockId()));
       String reason = findReasonForNotCaching(cblock, blockInfo);
       int neededCached = 0;
       if (reason != null) {
@@ -627,7 +627,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       List<DatanodeDescriptor> pendingCached) {
     // To figure out which replicas can be cached, we consult the
     // blocksMap.  We don't want to try to cache a corrupt replica, though.
-    BlockInfoContiguous blockInfo = namesystem.getStoredBlock(new Block(cachedBlock.getBlockId()));
+    BlockInfo blockInfo = namesystem.getStoredBlock(new Block(cachedBlock.getBlockId()));
     if (blockInfo == null) {
       LOG.debug("Block {}: can't add new cached replicas," +
           " because there is no record of this block " +
@@ -665,7 +665,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       Iterator<CachedBlock> it = datanode.getPendingCached().iterator();
       while (it.hasNext()) {
         CachedBlock cBlock = it.next();
-        BlockInfoContiguous info =
+        BlockInfo info =
             namesystem.getStoredBlock(new Block(cBlock.getBlockId()));
         if (info != null) {
           pendingBytes -= info.getNumBytes();
@@ -675,7 +675,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       // Add pending uncached blocks from effective capacity
       while (it.hasNext()) {
         CachedBlock cBlock = it.next();
-        BlockInfoContiguous info =
+        BlockInfo info =
             namesystem.getStoredBlock(new Block(cBlock.getBlockId()));
         if (info != null) {
           pendingBytes += info.getNumBytes();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
index 5ccd3ea..7fe8339 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
@@ -127,10 +127,15 @@ class FSDirConcatOp {
         throw new HadoopIllegalArgumentException("concat: the src file " + src
             + " is the same with the target file " + targetIIP.getPath());
       }
-      if(srcINodeFile.isUnderConstruction() || srcINodeFile.numBlocks() == 0) {
+      if (srcINodeFile.isUnderConstruction() || srcINodeFile.numBlocks() == 0) {
         throw new HadoopIllegalArgumentException("concat: source file " + src
             + " is invalid or empty or underConstruction");
       }
+      // TODO currently we do not support concatenating EC files
+      if (srcINodeFile.isStriped()) {
+        throw new HadoopIllegalArgumentException("concat: the src file " + src
+            + " is with striped blocks");
+      }
       si.add(srcINodeFile);
     }
 
@@ -206,7 +211,7 @@ class FSDirConcatOp {
     int count = 0;
     for (INodeFile nodeToRemove : srcList) {
       if(nodeToRemove != null) {
-        nodeToRemove.setBlocks(null);
+        nodeToRemove.setContiguousBlocks(null);
         nodeToRemove.getParent().removeChild(nodeToRemove);
         fsd.getINodeMap().remove(nodeToRemove);
         count++;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 4e1f63a..0c1032e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -1057,7 +1058,7 @@ public class FSDirectory implements Closeable {
         unprotectedTruncate(iip, newLength, collectedBlocks, mtime);
 
     if(! onBlockBoundary) {
-      BlockInfoContiguous oldBlock = file.getLastBlock();
+      BlockInfo oldBlock = file.getLastBlock();
       Block tBlk =
       getFSNamesystem().prepareFileForTruncate(iip,
           clientName, clientMachine, file.computeFileSize() - newLength,
@@ -1066,7 +1067,7 @@ public class FSDirectory implements Closeable {
           tBlk.getNumBytes() == truncateBlock.getNumBytes() :
           "Should be the same block.";
       if(oldBlock.getBlockId() != tBlk.getBlockId() &&
-         !file.isBlockInLatestSnapshot(oldBlock)) {
+         !file.isBlockInLatestSnapshot((BlockInfoContiguous) oldBlock)) {
         getBlockManager().removeBlockFromMap(oldBlock);
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 9d487e5..eca4739 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -45,7 +45,7 @@ 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.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -773,10 +773,10 @@ public class FSEditLog implements LogsPurgeable {
   
   public void logAddBlock(String path, INodeFile file) {
     Preconditions.checkArgument(file.isUnderConstruction());
-    BlockInfoContiguous[] blocks = file.getBlocks();
+    BlockInfo[] blocks = file.getBlocks();
     Preconditions.checkState(blocks != null && blocks.length > 0);
-    BlockInfoContiguous pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
-    BlockInfoContiguous lastBlock = blocks[blocks.length - 1];
+    BlockInfo pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
+    BlockInfo lastBlock = blocks[blocks.length - 1];
     AddBlockOp op = AddBlockOp.getInstance(cache.get()).setPath(path)
         .setPenultimateBlock(pBlock).setLastBlock(lastBlock);
     logEdit(op);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 51c167a..869f3d4 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
@@ -35,6 +35,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -507,7 +508,7 @@ public class FSEditLogLoader {
       }
       INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
       // add the new block to the INodeFile
-      addNewBlock(fsDir, addBlockOp, oldFile);
+      addNewBlock(addBlockOp, oldFile);
       break;
     }
     case OP_SET_REPLICATION: {
@@ -936,15 +937,15 @@ public class FSEditLogLoader {
 
   /**
    * Add a new block into the given INodeFile
+   * TODO support adding striped block
    */
-  private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file)
-      throws IOException {
-    BlockInfoContiguous[] oldBlocks = file.getBlocks();
+  private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException {
+    BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
     Block newBlock= op.getLastBlock();
     
     if (pBlock != null) { // the penultimate block is not null
-      Preconditions.checkState(oldBlocks != null && oldBlocks.length > 0);
+      assert oldBlocks != null && oldBlocks.length > 0;
       // compare pBlock with the last block of oldBlocks
       Block oldLastBlock = oldBlocks[oldBlocks.length - 1];
       if (oldLastBlock.getBlockId() != pBlock.getBlockId()
@@ -974,12 +975,13 @@ public class FSEditLogLoader {
   
   /**
    * Update in-memory data structures with new block information.
+   * TODO support adding striped block
    * @throws IOException
    */
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
       INodesInPath iip, INodeFile file) throws IOException {
     // Update its block list
-    BlockInfoContiguous[] oldBlocks = file.getBlocks();
+    BlockInfo[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
     String path = op.getPath();
     
@@ -988,7 +990,7 @@ public class FSEditLogLoader {
     
     // First, update blocks in common
     for (int i = 0; i < oldBlocks.length && i < newBlocks.length; i++) {
-      BlockInfoContiguous oldBlock = oldBlocks[i];
+      BlockInfo oldBlock = oldBlocks[i];
       Block newBlock = newBlocks[i];
       
       boolean isLastBlock = i == newBlocks.length - 1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 956a219..7601ffa 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
@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -683,7 +684,7 @@ public class FSImageFormat {
 
     public void updateBlocksMap(INodeFile file) {
       // Add file->block mapping
-      final BlockInfoContiguous[] blocks = file.getBlocks();
+      final BlockInfo[] blocks = file.getBlocks();
       if (blocks != null) {
         final BlockManager bm = namesystem.getBlockManager();
         for (int i = 0; i < blocks.length; i++) {
@@ -951,9 +952,9 @@ public class FSImageFormat {
         FileUnderConstructionFeature uc = cons.getFileUnderConstructionFeature();
         oldnode.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         if (oldnode.numBlocks() > 0) {
-          BlockInfoContiguous ucBlock = cons.getLastBlock();
+          BlockInfo ucBlock = cons.getLastBlock();
           // we do not replace the inode, just replace the last block of oldnode
-          BlockInfoContiguous info = namesystem.getBlockManager().addBlockCollection(
+          BlockInfo info = namesystem.getBlockManager().addBlockCollection(
               ucBlock, oldnode);
           oldnode.setBlock(oldnode.numBlocks() - 1, info);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 b758458..a025bb0 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
@@ -41,9 +41,13 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StripedBlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@@ -52,6 +56,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructio
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.StripedBlocksFeature;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrCompactProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.QuotaByStorageTypeEntryProto;
@@ -210,7 +215,7 @@ public final class FSImageFormatPBINode {
 
     public static void updateBlocksMap(INodeFile file, BlockManager bm) {
       // Add file->block mapping
-      final BlockInfoContiguous[] blocks = file.getBlocks();
+      final BlockInfo[] blocks = file.getBlocks();
       if (blocks != null) {
         for (int i = 0; i < blocks.length; i++) {
           file.setBlock(i, bm.addBlockCollection(blocks[i], file));
@@ -345,16 +350,30 @@ public final class FSImageFormatPBINode {
             loadXAttrs(f.getXAttrs(), state.getStringTable())));
       }
 
+      FileWithStripedBlocksFeature stripeFeature = null;
+      if (f.hasStripedBlocks()) {
+        StripedBlocksFeature sb = f.getStripedBlocks();
+        stripeFeature = file.addStripedBlocksFeature();
+        for (StripedBlockProto sp : sb.getBlocksList()) {
+          stripeFeature.addBlock(PBHelper.convert(sp));
+        }
+      }
+
       // under-construction information
       if (f.hasFileUC()) {
         INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
         file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
-        if (blocks.length > 0) {
-          BlockInfoContiguous lastBlk = file.getLastBlock();
-          // replace the last block of file
-          file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction(
-              lastBlk, replication));
+        BlockInfo lastBlk = file.getLastBlock();
+        // replace the last block of file
+        final BlockInfo ucBlk;
+        if (stripeFeature != null) {
+          BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
+          ucBlk = new BlockInfoStripedUnderConstruction(striped,
+              striped.getDataBlockNum(), striped.getParityBlockNum());
+        } else {
+          ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, replication);
         }
+        file.setBlock(file.numBlocks() - 1, ucBlk);
       }
       return file;
     }
@@ -617,6 +636,19 @@ public final class FSImageFormatPBINode {
         }
       }
 
+      FileWithStripedBlocksFeature sb = n.getStripedBlocksFeature();
+      if (sb != null) {
+        StripedBlocksFeature.Builder builder =
+            StripedBlocksFeature.newBuilder();
+        BlockInfoStriped[] sblocks = sb.getBlocks();
+        if (sblocks != null) {
+          for (BlockInfoStriped sblk : sblocks) {
+            builder.addBlocks(PBHelper.convert(sblk));
+          }
+        }
+        b.setStripedBlocks(builder.build());
+      }
+
       FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature();
       if (uc != null) {
         INodeSection.FileUnderConstructionFeature f =
@@ -645,7 +677,7 @@ public final class FSImageFormatPBINode {
       r.writeDelimitedTo(out);
     }
 
-    private final INodeSection.INode.Builder buildINodeCommon(INode n) {
+    private INodeSection.INode.Builder buildINodeCommon(INode n) {
       return INodeSection.INode.newBuilder()
           .setId(n.getId())
           .setName(ByteString.copyFrom(n.getLocalNameBytes()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 f9939b9..fba8784 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
@@ -202,8 +202,10 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -1961,6 +1963,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final BlockStoragePolicy lpPolicy =
         blockManager.getStoragePolicy("LAZY_PERSIST");
 
+    // not support truncating file with striped blocks
+    if (file.isStriped()) {
+      throw new UnsupportedOperationException(
+          "Cannot truncate file with striped block " + src);
+    }
     if (lpPolicy != null &&
         lpPolicy.getId() == file.getStoragePolicyID()) {
       throw new UnsupportedOperationException(
@@ -2033,8 +2040,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     leaseManager.addLease(
         file.getFileUnderConstructionFeature().getClientName(), src);
     boolean shouldRecoverNow = (newBlock == null);
-    BlockInfoContiguous oldBlock = file.getLastBlock();
-    boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock);
+
+    BlockInfo oldBlock = file.getLastBlock();
+    assert oldBlock instanceof BlockInfoContiguous;
+
+    boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file,
+        (BlockInfoContiguous) oldBlock);
     if(newBlock == null) {
       newBlock = (shouldCopyOnTruncate) ? createNewBlock(file.isStriped()) :
           new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
@@ -2049,7 +2060,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           file.getBlockReplication());
       truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
       truncatedBlockUC.setTruncateBlock(oldBlock);
-      file.setLastBlock(truncatedBlockUC, blockManager.getStorages(oldBlock));
+      file.convertLastBlockToUC(truncatedBlockUC,
+          blockManager.getStorages(oldBlock));
       getBlockManager().addBlockCollection(truncatedBlockUC, file);
 
       NameNode.stateChangeLog.info("BLOCK* prepareFileForTruncate: "
@@ -2633,6 +2645,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           + src + " for client " + clientMachine);
       }
       INodeFile myFile = INodeFile.valueOf(inode, src, true);
+
+      // not support appending file with striped blocks
+      if (myFile.isStriped()) {
+        throw new UnsupportedOperationException(
+            "Cannot truncate file with striped block " + src);
+      }
+
       final BlockStoragePolicy lpPolicy =
           blockManager.getStoragePolicy("LAZY_PERSIST");
       if (lpPolicy != null &&
@@ -2644,7 +2663,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE,
           iip, src, holder, clientMachine, false);
       
-      final BlockInfoContiguous lastBlock = myFile.getLastBlock();
+      final BlockInfoContiguous lastBlock =
+          (BlockInfoContiguous) myFile.getLastBlock();
       // Check that the block has at least minimum replication.
       if(lastBlock != null && lastBlock.isComplete() &&
           !getBlockManager().isSufficientlyReplicated(lastBlock)) {
@@ -2693,7 +2713,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         dir.updateSpaceConsumed(iip, 0, diff, file.getBlockReplication());
       }
     } else {
-      BlockInfoContiguous lastBlock = file.getLastBlock();
+      BlockInfo lastBlock = file.getLastBlock();
       if (lastBlock != null) {
         ExtendedBlock blk = new ExtendedBlock(this.getBlockPoolId(), lastBlock);
         ret = new LocatedBlock(blk, new DatanodeInfo[0]);
@@ -2826,7 +2846,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 op.getExceptionMessage(src, holder, clientMachine,
                     "lease recovery is in progress. Try again later."));
         } else {
-          final BlockInfoContiguous lastBlock = file.getLastBlock();
+          final BlockInfo lastBlock = file.getLastBlock();
           if (lastBlock != null
               && lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
             throw new RecoveryInProgressException(
@@ -3013,9 +3033,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           return onRetryBlock[0];
         } else {
           // add new chosen targets to already allocated block and return
-          BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
-          ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
-              .setExpectedLocations(targets);
+          BlockInfo lastBlockInFile = pendingFile.getLastBlock();
+          BlockInfo.setExpectedLocations(lastBlockInFile, targets);
           offset = pendingFile.computeFileSize();
           return makeLocatedBlock(lastBlockInFile, targets, offset);
         }
@@ -3105,7 +3124,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
     }
     final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
-    BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
+    BlockInfo lastBlockInFile = pendingFile.getLastBlock();
     if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
       // The block that the client claims is the current last block
       // doesn't match up with what we think is the last block. There are
@@ -3133,7 +3152,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       //    changed the namesystem state yet.
       //    We run this analysis again in Part II where case 4 is impossible.
 
-      BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
+      BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
       if (previous == null &&
           lastBlockInFile != null &&
           lastBlockInFile.getNumBytes() == pendingFile.getPreferredBlockSize() &&
@@ -3160,8 +3179,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             src + ". Returning previously allocated block " + lastBlockInFile);
         long offset = pendingFile.computeFileSize();
         onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
-            ((BlockInfoContiguousUnderConstruction)lastBlockInFile)
-                .getExpectedStorageLocations(), offset);
+            BlockInfo.getExpectedStorageLocations(lastBlockInFile), offset);
         return new FileState(pendingFile, src, iip);
       } else {
         // Case 3
@@ -3452,6 +3470,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param isStriped is the file under striping or contigunous layout?
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
+  // TODO: support striped block
   BlockInfoContiguous saveAllocatedBlock(String src, INodesInPath inodesInPath,
       Block newBlock, DatanodeStorageInfo[] targets, boolean isStriped)
           throws IOException {
@@ -3485,16 +3504,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       if (checkall) {
         // check all blocks of the file.
-        for (BlockInfoContiguous block: v.getBlocks()) {
-          if (!isCompleteBlock(src, block, blockManager.minReplication)) {
+        for (BlockInfo block: v.getBlocks()) {
+          if (!isCompleteBlock(src, block)) {
             return false;
           }
         }
       } else {
         // check the penultimate block of this file
-        BlockInfoContiguous b = v.getPenultimateBlock();
+        BlockInfo b = v.getPenultimateBlock();
         if (b != null
-            && !isCompleteBlock(src, b, blockManager.minReplication)) {
+            && !isCompleteBlock(src, b)) {
           return false;
         }
       }
@@ -3504,16 +3523,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  private static boolean isCompleteBlock(String src, BlockInfoContiguous b,
-      int minRepl) {
+  private boolean isCompleteBlock(String src, BlockInfo b) {
     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 < minRepl? " < ": " >= ")
-          + " minimum = " + minRepl + ") in file " + src);
+      final int min;
+      final BlockUCState state = b.getBlockUCState();
+      if (b instanceof BlockInfoStripedUnderConstruction) {
+        min = ((BlockInfoStripedUnderConstruction) b).getDataBlockNum();
+      } else {
+        min = blockManager.minReplication;
+      }
+      LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " + state
+          + ", replication# = " + numNodes + (numNodes < min ? " < " : " >= ")
+          + " minimum = " + min + ") in file " + src);
       return false;
     }
     return true;
@@ -3698,7 +3720,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     for (Block b : blocks.getToDeleteList()) {
       if (trackBlockCounts) {
-        BlockInfoContiguous bi = getStoredBlock(b);
+        BlockInfo bi = getStoredBlock(b);
         if (bi.isComplete()) {
           numRemovedComplete++;
           if (bi.numNodes() >= blockManager.minReplication) {
@@ -3922,10 +3944,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     final INodeFile pendingFile = iip.getLastINode().asFile();
     int nrBlocks = pendingFile.numBlocks();
-    BlockInfoContiguous[] blocks = pendingFile.getBlocks();
+    BlockInfo[] blocks = pendingFile.getBlocks();
 
     int nrCompleteBlocks;
-    BlockInfoContiguous curBlock = null;
+    BlockInfo curBlock = null;
     for(nrCompleteBlocks = 0; nrCompleteBlocks < nrBlocks; nrCompleteBlocks++) {
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
@@ -3960,12 +3982,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // The last block is not COMPLETE, and
     // that the penultimate block if exists is either COMPLETE or COMMITTED
-    final BlockInfoContiguous lastBlock = pendingFile.getLastBlock();
+    final BlockInfo lastBlock = pendingFile.getLastBlock();
     BlockUCState lastBlockState = lastBlock.getBlockUCState();
-    BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
+    BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
 
     // If penultimate block doesn't exist then its minReplication is met
-    boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
+    boolean penultimateBlockMinReplication = penultimateBlock == null ||
         blockManager.checkMinReplication(penultimateBlock);
 
     switch(lastBlockState) {
@@ -3995,6 +4017,7 @@ 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;
       // determine if last block was intended to be truncated
@@ -4106,14 +4129,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     blockManager.checkReplication(pendingFile);
   }
 
-  public BlockInfoContiguous getStoredBlock(Block block) {
-    return (BlockInfoContiguous) blockManager.getStoredBlock(block);
+  public BlockInfo getStoredBlock(Block block) {
+    return blockManager.getStoredBlock(block);
   }
   
   @Override
-  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC) {
+  public boolean isInSnapshot(BlockCollection bc) {
     assert hasReadLock();
-    final BlockCollection bc = blockUC.getBlockCollection();
     if (bc == null || !(bc instanceof INodeFile)
         || !bc.isUnderConstruction()) {
       return false;
@@ -4164,7 +4186,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   
       checkNameNodeSafeMode(
           "Cannot commitBlockSynchronization while in safe mode");
-      final BlockInfoContiguous storedBlock = getStoredBlock(
+      final BlockInfo storedBlock = getStoredBlock(
           ExtendedBlock.getLocalBlock(oldBlock));
       if (storedBlock == null) {
         if (deleteblock) {
@@ -4212,11 +4234,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         return;
       }
 
-      BlockInfoContiguousUnderConstruction truncatedBlock =
-          (BlockInfoContiguousUnderConstruction) iFile.getLastBlock();
-      long recoveryId = truncatedBlock.getBlockRecoveryId();
-      boolean copyTruncate =
-          truncatedBlock.getBlockId() != storedBlock.getBlockId();
+      BlockInfo ucBlock = iFile.getLastBlock();
+      long recoveryId = BlockInfo.getBlockRecoveryId(ucBlock);
+      boolean copyTruncate = ucBlock.getBlockId() != storedBlock.getBlockId();
       if(recoveryId != newgenerationstamp) {
         throw new IOException("The recovery id " + newgenerationstamp
                               + " does not match current recovery id "
@@ -4229,8 +4249,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         if (remove) {
           blockManager.removeBlock(storedBlock);
         }
-      }
-      else {
+      } else {
         // update last block
         if(!copyTruncate) {
           storedBlock.setGenerationStamp(newgenerationstamp);
@@ -4266,7 +4285,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i));
             if (storageInfo != null) {
               if(copyTruncate) {
-                storageInfo.addBlock(truncatedBlock, truncatedBlock);
+                storageInfo.addBlock(ucBlock, ucBlock);
               } else {
                 storageInfo.addBlock(storedBlock, storedBlock);
               }
@@ -4280,16 +4299,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]),
                 trimmedStorages.toArray(new String[trimmedStorages.size()]));
         if(copyTruncate) {
-          iFile.setLastBlock(truncatedBlock, trimmedStorageInfos);
+          iFile.convertLastBlockToUC(ucBlock, trimmedStorageInfos);
         } else {
-          iFile.setLastBlock(storedBlock, trimmedStorageInfos);
+          iFile.convertLastBlockToUC(storedBlock, trimmedStorageInfos);
         }
       }
 
       if (closeFile) {
         if(copyTruncate) {
-          src = closeFileCommitBlocks(iFile, truncatedBlock);
-          if(!iFile.isBlockInLatestSnapshot(storedBlock)) {
+          src = closeFileCommitBlocks(iFile, ucBlock);
+          if(!iFile.isBlockInLatestSnapshot((BlockInfoContiguous) storedBlock)) {
             blockManager.removeBlock(storedBlock);
           }
         } else {
@@ -4322,7 +4341,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @throws IOException on error
    */
   @VisibleForTesting
-  String closeFileCommitBlocks(INodeFile pendingFile, BlockInfoContiguous storedBlock)
+  String closeFileCommitBlocks(INodeFile pendingFile, BlockInfo storedBlock)
       throws IOException {
     final INodesInPath iip = INodesInPath.fromINode(pendingFile);
     final String src = iip.getPath();
@@ -4632,7 +4651,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
         while (it.hasNext()) {
           Block b = it.next();
-          BlockInfoContiguous blockInfo = getStoredBlock(b);
+          BlockInfo blockInfo = getStoredBlock(b);
           if (blockInfo.getBlockCollection().getStoragePolicyID() == lpPolicy.getId()) {
             filesToDelete.add(blockInfo.getBlockCollection());
           }
@@ -5579,7 +5598,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
       return;
-    BlockInfoContiguous storedBlock = getStoredBlock(b);
+    BlockInfo storedBlock = getStoredBlock(b);
     if (storedBlock.isComplete()) {
       safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas());
     }
@@ -6124,7 +6143,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         + "access token for block " + block);
     
     // check stored block state
-    BlockInfoContiguous storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
+    BlockInfo storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
     if (storedBlock == null || 
         storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) {
         throw new IOException(block + 
@@ -6253,8 +6272,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     assert hasWriteLock();
     // check the vadility of the block and lease holder name
     final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
-    final BlockInfoContiguousUnderConstruction blockinfo
-        = (BlockInfoContiguousUnderConstruction)pendingFile.getLastBlock();
+    final BlockInfo lastBlock = pendingFile.getLastBlock();
+    // when updating pipeline, the last block must be contiguous block
+    assert lastBlock instanceof BlockInfoContiguousUnderConstruction;
+    BlockInfoContiguousUnderConstruction blockinfo =
+        (BlockInfoContiguousUnderConstruction) lastBlock;
 
     // check new GS & length: this is not expected
     if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() ||

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
index 1ebdde6..a8e2e00 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+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.namenode.INode.BlocksMapUpdateInfo;
 
 /**
@@ -58,12 +60,12 @@ public class FileUnderConstructionFeature implements INode.Feature {
    */
   void updateLengthOfLastBlock(INodeFile f, long lastBlockLength)
       throws IOException {
-    BlockInfoContiguous lastBlock = f.getLastBlock();
+    BlockInfo lastBlock = f.getLastBlock();
     assert (lastBlock != null) : "The last block for path "
         + f.getFullPathName() + " is null when updating its length";
-    assert (lastBlock instanceof BlockInfoContiguousUnderConstruction)
+    assert !lastBlock.isComplete()
         : "The last block for path " + f.getFullPathName()
-            + " is not a BlockInfoUnderConstruction when updating its length";
+        + " is not a BlockInfoUnderConstruction when updating its length";
     lastBlock.setNumBytes(lastBlockLength);
   }
 
@@ -74,11 +76,10 @@ public class FileUnderConstructionFeature implements INode.Feature {
    */
   void cleanZeroSizeBlock(final INodeFile f,
       final BlocksMapUpdateInfo collectedBlocks) {
-    final BlockInfoContiguous[] blocks = f.getBlocks();
+    final BlockInfo[] blocks = f.getBlocks();
     if (blocks != null && blocks.length > 0
-        && blocks[blocks.length - 1] instanceof BlockInfoContiguousUnderConstruction) {
-      BlockInfoContiguousUnderConstruction lastUC =
-          (BlockInfoContiguousUnderConstruction) blocks[blocks.length - 1];
+        && !blocks[blocks.length - 1].isComplete()) {
+      BlockInfo lastUC = blocks[blocks.length - 1];
       if (lastUC.getNumBytes() == 0) {
         // this is a 0-sized block. do not need check its UC state here
         collectedBlocks.addDeleteBlock(lastUC);


[28/50] [abbrv] hadoop git commit: YARN-1453. [JDK8] Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA, Andrew Purtell, and Allen Wittenauer.

Posted by zh...@apache.org.
YARN-1453. [JDK8] Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA, Andrew Purtell, and Allen Wittenauer.


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

Branch: refs/heads/HDFS-7285
Commit: 3da9a97cfbcc3a1c50aaf85b1a129d4d269cd5fd
Parents: 3ff1ba2
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon Mar 16 23:19:05 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Mon Mar 16 23:19:05 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../yarn/api/ApplicationBaseProtocol.java       | 44 ++++---------
 .../yarn/api/ApplicationClientProtocol.java     |  3 -
 .../api/protocolrecords/AllocateRequest.java    | 25 ++++---
 .../api/protocolrecords/AllocateResponse.java   | 68 ++++++++++----------
 .../FinishApplicationMasterRequest.java         | 25 ++++---
 .../FinishApplicationMasterResponse.java        |  7 +-
 .../protocolrecords/GetApplicationsRequest.java |  1 -
 .../GetClusterMetricsResponse.java              |  4 +-
 .../GetContainerStatusesRequest.java            |  2 -
 .../GetContainerStatusesResponse.java           |  2 -
 .../protocolrecords/GetQueueInfoRequest.java    |  2 +-
 .../protocolrecords/GetQueueInfoResponse.java   | 11 ++--
 .../KillApplicationResponse.java                |  9 ++-
 .../RegisterApplicationMasterRequest.java       | 33 +++++-----
 .../RegisterApplicationMasterResponse.java      | 11 ++--
 .../protocolrecords/StartContainerRequest.java  |  9 +--
 .../api/records/ApplicationAttemptReport.java   | 23 +++----
 .../yarn/api/records/ApplicationReport.java     | 47 +++++++-------
 .../records/ApplicationSubmissionContext.java   | 50 +++++++-------
 .../hadoop/yarn/api/records/Container.java      | 49 +++++++-------
 .../api/records/ContainerLaunchContext.java     | 35 +++++-----
 .../yarn/api/records/ContainerReport.java       | 29 ++++-----
 .../yarn/api/records/ContainerStatus.java       | 21 +++---
 .../yarn/api/records/LocalResourceType.java     | 32 ++++-----
 .../api/records/LocalResourceVisibility.java    | 31 +++++----
 .../yarn/api/records/LogAggregationContext.java | 39 ++++++-----
 .../hadoop/yarn/api/records/NodeReport.java     | 25 ++++---
 .../yarn/api/records/PreemptionMessage.java     | 32 ++++-----
 .../hadoop/yarn/api/records/QueueACL.java       | 13 ++--
 .../hadoop/yarn/api/records/QueueInfo.java      | 25 ++++---
 .../hadoop/yarn/api/records/QueueState.java     | 15 ++---
 .../yarn/api/records/ReservationRequest.java    | 17 ++---
 .../records/ReservationRequestInterpreter.java  | 38 +++++------
 .../yarn/api/records/ResourceRequest.java       | 51 +++++++--------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  5 +-
 .../UpdateNodeResourceRequest.java              |  4 +-
 .../hadoop/yarn/client/api/AHSClient.java       | 24 +++----
 .../hadoop/yarn/client/api/AMRMClient.java      |  4 +-
 .../apache/hadoop/yarn/client/api/NMClient.java |  4 +-
 .../hadoop/yarn/client/api/NMTokenCache.java    | 58 ++++++++---------
 .../hadoop/yarn/client/api/YarnClient.java      | 23 ++++---
 .../nodelabels/CommonNodeLabelsManager.java     |  6 +-
 .../hadoop/yarn/nodelabels/NodeLabelsStore.java |  3 +-
 .../server/security/ApplicationACLsManager.java |  1 -
 .../apache/hadoop/yarn/util/StringHelper.java   |  6 +-
 .../org/apache/hadoop/yarn/webapp/WebApps.java  |  4 +-
 .../registry/client/binding/RegistryUtils.java  |  8 +--
 .../client/impl/RegistryOperationsClient.java   |  2 +-
 .../client/impl/zk/ZookeeperConfigOptions.java  |  3 +-
 .../server/services/MicroZookeeperService.java  | 10 +--
 .../registry/server/services/package-info.java  |  9 ++-
 ...TimelineAuthenticationFilterInitializer.java | 13 ++--
 .../org/apache/hadoop/yarn/lib/ZKClient.java    |  2 +-
 .../RegisterNodeManagerRequest.java             |  3 +-
 .../server/api/records/NodeHealthStatus.java    | 24 ++++---
 .../server/nodemanager/ContainerExecutor.java   |  8 ++-
 .../util/NodeManagerHardwareUtils.java          |  8 +--
 .../rmapp/attempt/RMAppAttempt.java             | 11 ++--
 .../scheduler/SchedulerNode.java                |  2 +-
 .../scheduler/SchedulerUtils.java               |  3 +-
 .../fair/policies/ComputeFairShares.java        | 19 +++---
 .../security/DelegationTokenRenewer.java        |  2 -
 .../yarn/server/webproxy/ProxyUriUtils.java     |  2 +-
 64 files changed, 517 insertions(+), 585 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index bcab88c..26ef7d3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -769,6 +769,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3171. Sort by Application id, AppAttempt and ContainerID doesn't work
     in ATS / RM web ui. (Naganarasimha G R via xgong)
 
+    YARN-1453. [JDK8] Fix Javadoc errors caused by incorrect or illegal tags in 
+    doc comments. (Akira AJISAKA, Andrew Purtell, and Allen Wittenauer via ozawa)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
index 2a8a283..8234c2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
@@ -65,41 +65,31 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 public interface ApplicationBaseProtocol {
 
   /**
-   * <p>
    * The interface used by clients to get a report of an Application from the
    * <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>.
-   * </p>
-   *
    * <p>
    * The client, via {@link GetApplicationReportRequest} provides the
    * {@link ApplicationId} of the application.
-   * </p>
-   *
    * <p>
    * In secure mode,the <code>ResourceManager</code> or
    * <code>ApplicationHistoryServer</code> verifies access to the application,
    * queue etc. before accepting the request.
-   * </p>
-   *
    * <p>
    * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
    * responds with a {@link GetApplicationReportResponse} which includes the
    * {@link ApplicationReport} for the application.
-   * </p>
-   *
    * <p>
    * If the user does not have <code>VIEW_APP</code> access then the following
    * fields in the report will be set to stubbed values:
    * <ul>
-   * <li>host - set to "N/A"</li>
-   * <li>RPC port - set to -1</li>
-   * <li>client token - set to "N/A"</li>
-   * <li>diagnostics - set to "N/A"</li>
-   * <li>tracking URL - set to "N/A"</li>
-   * <li>original tracking URL - set to "N/A"</li>
-   * <li>resource usage report - all values are -1</li>
+   *   <li>host - set to "N/A"</li>
+   *   <li>RPC port - set to -1</li>
+   *   <li>client token - set to "N/A"</li>
+   *   <li>diagnostics - set to "N/A"</li>
+   *   <li>tracking URL - set to "N/A"</li>
+   *   <li>original tracking URL - set to "N/A"</li>
+   *   <li>resource usage report - all values are -1</li>
    * </ul>
-   * </p>
    *
    * @param request
    *          request for an application report
@@ -148,40 +138,30 @@ public interface ApplicationBaseProtocol {
           IOException;
 
   /**
-   * <p>
    * The interface used by clients to get a report of an Application Attempt
    * from the <code>ResourceManager</code> or
    * <code>ApplicationHistoryServer</code>
-   * </p>
-   *
    * <p>
    * The client, via {@link GetApplicationAttemptReportRequest} provides the
    * {@link ApplicationAttemptId} of the application attempt.
-   * </p>
-   *
    * <p>
    * In secure mode,the <code>ResourceManager</code> or
    * <code>ApplicationHistoryServer</code> verifies access to the method before
    * accepting the request.
-   * </p>
-   *
    * <p>
    * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
    * responds with a {@link GetApplicationAttemptReportResponse} which includes
    * the {@link ApplicationAttemptReport} for the application attempt.
-   * </p>
-   *
    * <p>
    * If the user does not have <code>VIEW_APP</code> access then the following
    * fields in the report will be set to stubbed values:
    * <ul>
-   * <li>host</li>
-   * <li>RPC port</li>
-   * <li>client token</li>
-   * <li>diagnostics - set to "N/A"</li>
-   * <li>tracking URL</li>
+   *   <li>host</li>
+   *   <li>RPC port</li>
+   *   <li>client token</li>
+   *   <li>diagnostics - set to "N/A"</li>
+   *   <li>tracking URL</li>
    * </ul>
-   * </p>
    *
    * @param request
    *          request for an application attempt report

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
index 0a7d415..8b9937b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
@@ -135,9 +135,6 @@ public interface ApplicationClientProtocol extends ApplicationBaseProtocol {
    * @return (empty) response on accepting the submission
    * @throws YarnException
    * @throws IOException
-   * @throws InvalidResourceRequestException
-   *           The exception is thrown when a {@link ResourceRequest} is out of
-   *           the range of the configured lower and upper resource boundaries.
    * @see #getNewApplication(GetNewApplicationRequest)
    */
   @Public

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
index 62316a6..2458d9b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
@@ -35,19 +35,18 @@ import org.apache.hadoop.yarn.util.Records;
  * <code>ResourceManager</code> to obtain resources in the cluster.</p> 
  *
  * <p>The request includes:
- *   <ul>
- *     <li>A response id to track duplicate responses.</li>
- *     <li>Progress information.</li>
- *     <li>
- *       A list of {@link ResourceRequest} to inform the 
- *       <code>ResourceManager</code> about the application's 
- *       resource requirements.
- *     </li>
- *     <li>
- *       A list of unused {@link Container} which are being returned. 
- *     </li>
- *   </ul>
- * </p>
+ * <ul>
+ *   <li>A response id to track duplicate responses.</li>
+ *   <li>Progress information.</li>
+ *   <li>
+ *     A list of {@link ResourceRequest} to inform the
+ *     <code>ResourceManager</code> about the application's
+ *     resource requirements.
+ *   </li>
+ *   <li>
+ *     A list of unused {@link Container} which are being returned.
+ *   </li>
+ * </ul>
  * 
  * @see ApplicationMasterProtocol#allocate(AllocateRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
index 46ac642..c4fdb79 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
@@ -39,27 +39,27 @@ import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The response sent by the <code>ResourceManager</code> the  
- * <code>ApplicationMaster</code> during resource negotiation.</p>
- *
- * <p>The response, includes:
- *   <ul>
- *     <li>Response ID to track duplicate responses.</li>
- *     <li>
- *       An AMCommand sent by ResourceManager to let the <code>ApplicationMaster</code>
- *       take some actions (resync, shutdown etc.).
- *     <li>A list of newly allocated {@link Container}.</li>
- *     <li>A list of completed {@link Container}s' statuses.</li>
- *     <li>
- *       The available headroom for resources in the cluster for the
- *       application. 
- *     </li>
- *     <li>A list of nodes whose status has been updated.</li>
- *     <li>The number of available nodes in a cluster.</li>
- *     <li>A description of resources requested back by the cluster</li>
- *     <li>AMRMToken, if AMRMToken has been rolled over</li>
- *   </ul>
- * </p>
+ * The response sent by the <code>ResourceManager</code> the
+ * <code>ApplicationMaster</code> during resource negotiation.
+ * <p>
+ * The response, includes:
+ * <ul>
+ *   <li>Response ID to track duplicate responses.</li>
+ *   <li>
+ *     An AMCommand sent by ResourceManager to let the
+ *     {@code ApplicationMaster} take some actions (resync, shutdown etc.).
+ *   </li>
+ *   <li>A list of newly allocated {@link Container}.</li>
+ *   <li>A list of completed {@link Container}s' statuses.</li>
+ *   <li>
+ *     The available headroom for resources in the cluster for the
+ *     application.
+ *   </li>
+ *   <li>A list of nodes whose status has been updated.</li>
+ *   <li>The number of available nodes in a cluster.</li>
+ *   <li>A description of resources requested back by the cluster</li>
+ *   <li>AMRMToken, if AMRMToken has been rolled over</li>
+ * </ul>
  * 
  * @see ApplicationMasterProtocol#allocate(AllocateRequest)
  */
@@ -220,16 +220,16 @@ public abstract class AllocateResponse {
   public abstract void setNumClusterNodes(int numNodes);
 
   /**
-   * <p>Get the description of containers owned by the AM, but requested back by
+   * Get the description of containers owned by the AM, but requested back by
    * the cluster. Note that the RM may have an inconsistent view of the
    * resources owned by the AM. These messages are advisory, and the AM may
-   * elect to ignore them.<p>
-   *
-   * <p>The message is a snapshot of the resources the RM wants back from the AM.
+   * elect to ignore them.
+   * <p>
+   * The message is a snapshot of the resources the RM wants back from the AM.
    * While demand persists, the RM will repeat its request; applications should
-   * not interpret each message as a request for <em>additional<em>
+   * not interpret each message as a request for <em>additional</em>
    * resources on top of previous messages. Resources requested consistently
-   * over some duration may be forcibly killed by the RM.<p>
+   * over some duration may be forcibly killed by the RM.
    *
    * @return A specification of the resources to reclaim from this AM.
    */
@@ -242,15 +242,17 @@ public abstract class AllocateResponse {
   public abstract void setPreemptionMessage(PreemptionMessage request);
 
   /**
-   * <p>Get the list of NMTokens required for communicating with NM. New NMTokens
-   * issued only if<p>
-   * <p>1) AM is receiving first container on underlying NodeManager.<br>
+   * Get the list of NMTokens required for communicating with NM. New NMTokens
+   * issued only if
+   * <p>
+   * 1) AM is receiving first container on underlying NodeManager.<br>
    * OR<br>
    * 2) NMToken master key rolled over in ResourceManager and AM is getting new
-   * container on the same underlying NodeManager.<p>
-   * <p>AM will receive one NMToken per NM irrespective of the number of containers
+   * container on the same underlying NodeManager.
+   * <p>
+   * AM will receive one NMToken per NM irrespective of the number of containers
    * issued on same NM. AM is expected to store these tokens until issued a
-   * new token for the same NM.<p>
+   * new token for the same NM.
    */
   @Public
   @Stable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
index 15c3680..cbbe9c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
@@ -25,19 +25,18 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The finalization request sent by the <code>ApplicationMaster</code> to
- * inform the <code>ResourceManager</code> about its completion.</p>
- *
- * <p>The final request includes details such:
- *   <ul>
- *     <li>Final state of the <code>ApplicationMaster</code></li>
- *     <li>
- *       Diagnostic information in case of failure of the
- *       <code>ApplicationMaster</code>
- *     </li>
- *     <li>Tracking URL</li>
- *   </ul>
- * </p>
+ * The finalization request sent by the {@code ApplicationMaster} to
+ * inform the {@code ResourceManager} about its completion.
+ * <p>
+ * The final request includes details such:
+ * <ul>
+ *   <li>Final state of the {@code ApplicationMaster}</li>
+ *   <li>
+ *     Diagnostic information in case of failure of the
+ *     {@code ApplicationMaster}
+ *   </li>
+ *   <li>Tracking URL</li>
+ * </ul>
  *
  * @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java
index 8de2c73..6647a10 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterResponse.java
@@ -26,22 +26,19 @@ import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
  * The response sent by the <code>ResourceManager</code> to a
  * <code>ApplicationMaster</code> on it's completion.
- * </p>
- * 
  * <p>
  * The response, includes:
  * <ul>
  * <li>A flag which indicates that the application has successfully unregistered
  * with the RM and the application can safely stop.</li>
  * </ul>
- * </p>
+ * <p>
  * Note: The flag indicates whether the application has successfully
  * unregistered and is safe to stop. The application may stop after the flag is
  * true. If the application stops before the flag is true then the RM may retry
- * the application .
+ * the application.
  * 
  * @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationsRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationsRequest.java
index 7fc58d6..35392e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationsRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationsRequest.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.yarn.util.Records;
  * <p>The request from clients to get a report of Applications
  * in the cluster from the <code>ResourceManager</code>.</p>
  *
- *
  * @see ApplicationClientProtocol#getApplications(GetApplicationsRequest)
  */
 @Public

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java
index 6329aac..18a0807 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterMetricsResponse.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The response sent by the <code>ResourceManager</code> to a client
- * requesting cluster metrics.<p>
+ * The response sent by the <code>ResourceManager</code> to a client
+ * requesting cluster metrics.
  * 
  * @see YarnClusterMetrics
  * @see ApplicationClientProtocol#getClusterMetrics(GetClusterMetricsRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java
index f9f77a3..60c63ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java
@@ -28,11 +28,9 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
  * The request sent by the <code>ApplicationMaster</code> to the
  * <code>NodeManager</code> to get {@link ContainerStatus} of requested
  * containers.
- * </p>
  * 
  * @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java
index b0a0f0e..68e6a8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java
@@ -32,11 +32,9 @@ import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
  * The response sent by the <code>NodeManager</code> to the
  * <code>ApplicationMaster</code> when asked to obtain the
  * <code>ContainerStatus</code> of requested containers.
- * </p>
  * 
  * @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java
index df3342f..0e33e21 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoRequest.java
@@ -63,7 +63,7 @@ public abstract class GetQueueInfoRequest {
   public abstract void setQueueName(String queueName);
 
   /**
-   * Is information about <em>active applications<e/m> required?
+   * Is information about <em>active applications</em> required?
    * @return <code>true</code> if applications' information is to be included,
    *         else <code>false</code>
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java
index ea6cb7b..b97a5a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetQueueInfoResponse.java
@@ -27,12 +27,11 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The response sent by the <code>ResourceManager</code> to a client
- * requesting information about queues in the system.</p>
- *
- * <p>The response includes a {@link QueueInfo} which has details such as
- * queue name, used/total capacities, running applications, child queues etc
- * .</p>
+ * The response sent by the {@code ResourceManager} to a client
+ * requesting information about queues in the system.
+ * <p>
+ * The response includes a {@link QueueInfo} which has details such as
+ * queue name, used/total capacities, running applications, child queues etc.
  * 
  * @see QueueInfo
  * @see ApplicationClientProtocol#getQueueInfo(GetQueueInfoRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java
index 77bb71d..7225bf5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/KillApplicationResponse.java
@@ -26,21 +26,20 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
  * The response sent by the <code>ResourceManager</code> to the client aborting
  * a submitted application.
- * </p>
  * <p>
  * The response, includes:
  * <ul>
- * <li>A flag which indicates that the process of killing the application is
- * completed or not.</li>
+ *   <li>
+ *     A flag which indicates that the process of killing the application is
+ *     completed or not.
+ *   </li>
  * </ul>
  * Note: user is recommended to wait until this flag becomes true, otherwise if
  * the <code>ResourceManager</code> crashes before the process of killing the
  * application is completed, the <code>ResourceManager</code> may retry this
  * application on recovery.
- * </p>
  * 
  * @see ApplicationClientProtocol#forceKillApplication(KillApplicationRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
index 6b01854..395e190 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterRequest.java
@@ -24,16 +24,15 @@ import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The request sent by the <code>ApplicationMaster</code> to 
- * <code>ResourceManager</code> on registration.</p>
- * 
- * <p>The registration includes details such as:
- *   <ul>
- *     <li>Hostname on which the AM is running.</li>
- *     <li>RPC Port</li>
- *     <li>Tracking URL</li>
- *   </ul>
- * </p>
+ * The request sent by the {@code ApplicationMaster} to {@code ResourceManager}
+ * on registration.
+ * <p>
+ * The registration includes details such as:
+ * <ul>
+ *   <li>Hostname on which the AM is running.</li>
+ *   <li>RPC Port</li>
+ *   <li>Tracking URL</li>
+ * </ul>
  * 
  * @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)
  */
@@ -83,20 +82,20 @@ public abstract class RegisterApplicationMasterRequest {
   public abstract void setHost(String host);
 
   /**
-   * Get the <em>RPC port</em> on which the <code>ApplicationMaster</code> 
-   * is responding. 
-   * @return the <em>RPC port<em> on which the <code>ApplicationMaster</code> is 
-   *         responding
+   * Get the <em>RPC port</em> on which the {@code ApplicationMaster} is
+   * responding.
+   * @return the <em>RPC port</em> on which the {@code ApplicationMaster}
+   *         is responding
    */
   @Public
   @Stable
   public abstract int getRpcPort();
   
   /**
-   * Set the <em>RPC port<em> on which the <code>ApplicationMaster</code> is 
+   * Set the <em>RPC port</em> on which the {@code ApplicationMaster} is
    * responding.
-   * @param port <em>RPC port<em> on which the <code>ApplicationMaster</code> is 
-   *             responding
+   * @param port <em>RPC port</em> on which the {@code ApplicationMaster}
+   *             is responding
    */
   @Public
   @Stable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java
index 33daf28..1a51ba6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java
@@ -36,16 +36,15 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The response sent by the <code>ResourceManager</code> to a new 
- * <code>ApplicationMaster</code> on registration.</p>
- * 
- * <p>The response contains critical details such as:
+ * The response sent by the {@code ResourceManager} to a new
+ * {@code ApplicationMaster} on registration.
+ * <p>
+ * The response contains critical details such as:
  * <ul>
  *   <li>Maximum capability for allocated resources in the cluster.</li>
- *   <li><code>ApplicationACL</code>s for the application.</li>
+ *   <li>{@code ApplicationACL}s for the application.</li>
  *   <li>ClientToAMToken master key.</li>
  * </ul>
- * </p>
  * 
  * @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
index 1dcefb2..50179a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
@@ -74,10 +74,11 @@ public abstract class StartContainerRequest {
   public abstract void setContainerLaunchContext(ContainerLaunchContext context);
 
   /**
-   * <p>Get the container token to be used for authorization during starting
-   * container.</p>
-   * <p>Note: {@link NMToken} will be used for authenticating communication with </code>
-   * NodeManager</code>.</p>
+   * Get the container token to be used for authorization during starting
+   * container.
+   * <p>
+   * Note: {@link NMToken} will be used for authenticating communication with
+   * {@code NodeManager}.
    * @return the container token to be used for authorization during starting
    * container.
    * @see NMToken

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
index 53c18ae..b7f9c1b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
@@ -24,24 +24,19 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
- * <code>ApplicationAttemptReport</code> is a report of an application attempt.
- * </p>
- * 
+ * {@code ApplicationAttemptReport} is a report of an application attempt.
  * <p>
  * It includes details such as:
  * <ul>
- * <li>{@link ApplicationAttemptId} of the application.</li>
- * <li>Host on which the <code>ApplicationMaster</code> of this attempt is
- * running.</li>
- * <li>RPC port of the <code>ApplicationMaster</code> of this attempt.</li>
- * <li>Tracking URL.</li>
- * <li>Diagnostic information in case of errors.</li>
- * <li>{@link YarnApplicationAttemptState} of the application attempt.</li>
- * <li>{@link ContainerId} of the master Container.</li>
+ *   <li>{@link ApplicationAttemptId} of the application.</li>
+ *   <li>Host on which the <code>ApplicationMaster</code> of this attempt is
+ *   running.</li>
+ *   <li>RPC port of the <code>ApplicationMaster</code> of this attempt.</li>
+ *   <li>Tracking URL.</li>
+ *   <li>Diagnostic information in case of errors.</li>
+ *   <li>{@link YarnApplicationAttemptState} of the application attempt.</li>
+ *   <li>{@link ContainerId} of the master Container.</li>
  * </ul>
- * </p>
- * 
  */
 @Public
 @Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
index 412c22b..ff4fb52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
@@ -28,23 +28,22 @@ import org.apache.hadoop.yarn.util.Records;
 import java.util.Set;
 
 /**
- * <p><code>ApplicationReport</code> is a report of an application.</p>
- *
- * <p>It includes details such as:
- *   <ul>
- *     <li>{@link ApplicationId} of the application.</li>
- *     <li>Applications user.</li>
- *     <li>Application queue.</li>
- *     <li>Application name.</li>
- *     <li>Host on which the <code>ApplicationMaster</code> is running.</li>
- *     <li>RPC port of the <code>ApplicationMaster</code>.</li>
- *     <li>Tracking URL.</li>
- *     <li>{@link YarnApplicationState} of the application.</li>
- *     <li>Diagnostic information in case of errors.</li>
- *     <li>Start time of the application.</li>
- *     <li>Client {@link Token} of the application (if security is enabled).</li>
- *   </ul>
- * </p>
+ * {@code ApplicationReport} is a report of an application.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>{@link ApplicationId} of the application.</li>
+ *   <li>Applications user.</li>
+ *   <li>Application queue.</li>
+ *   <li>Application name.</li>
+ *   <li>Host on which the <code>ApplicationMaster</code> is running.</li>
+ *   <li>RPC port of the <code>ApplicationMaster</code>.</li>
+ *   <li>Tracking URL.</li>
+ *   <li>{@link YarnApplicationState} of the application.</li>
+ *   <li>Diagnostic information in case of errors.</li>
+ *   <li>Start time of the application.</li>
+ *   <li>Client {@link Token} of the application (if security is enabled).</li>
+ * </ul>
  *
  * @see ApplicationClientProtocol#getApplicationReport(org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest)
  */
@@ -341,20 +340,20 @@ public abstract class ApplicationReport {
 
   /**
    * Get the AMRM token of the application.
-   * <p/>
+   * <p>
    * The AMRM token is required for AM to RM scheduling operations. For 
    * managed Application Masters Yarn takes care of injecting it. For unmanaged
    * Applications Masters, the token must be obtained via this method and set
    * in the {@link org.apache.hadoop.security.UserGroupInformation} of the
    * current user.
-   * <p/>
+   * <p>
    * The AMRM token will be returned only if all the following conditions are
    * met:
-   * <li>
-   *   <ul>the requester is the owner of the ApplicationMaster</ul>
-   *   <ul>the application master is an unmanaged ApplicationMaster</ul>
-   *   <ul>the application master is in ACCEPTED state</ul>
-   * </li>
+   * <ul>
+   *   <li>the requester is the owner of the ApplicationMaster</li>
+   *   <li>the application master is an unmanaged ApplicationMaster</li>
+   *   <li>the application master is in ACCEPTED state</li>
+   * </ul>
    * Else this method returns NULL.
    * 
    * @return the AM to RM token if available.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
index c4014fc..21cd1bb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
@@ -33,32 +33,34 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>ApplicationSubmissionContext</code> represents all of the
- * information needed by the <code>ResourceManager</code> to launch 
- * the <code>ApplicationMaster</code> for an application.</p>
- * 
- * <p>It includes details such as:
- *   <ul>
- *     <li>{@link ApplicationId} of the application.</li>
- *     <li>Application user.</li>
- *     <li>Application name.</li>
- *     <li>{@link Priority} of the application.</li>
- *     <li>
- *       {@link ContainerLaunchContext} of the container in which the 
- *       <code>ApplicationMaster</code> is executed.
- *     </li>
- *     <li>maxAppAttempts. The maximum number of application attempts.
+ * {@code ApplicationSubmissionContext} represents all of the
+ * information needed by the {@code ResourceManager} to launch
+ * the {@code ApplicationMaster} for an application.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>{@link ApplicationId} of the application.</li>
+ *   <li>Application user.</li>
+ *   <li>Application name.</li>
+ *   <li>{@link Priority} of the application.</li>
+ *   <li>
+ *     {@link ContainerLaunchContext} of the container in which the
+ *     <code>ApplicationMaster</code> is executed.
+ *   </li>
+ *   <li>
+ *     maxAppAttempts. The maximum number of application attempts.
  *     It should be no larger than the global number of max attempts in the
- *     Yarn configuration.</li>
- *     <li>attemptFailuresValidityInterval. The default value is -1.
- *     when attemptFailuresValidityInterval in milliseconds is set to > 0,
- *     the failure number will no take failures which happen out of the
- *     validityInterval into failure count. If failure count reaches to
- *     maxAppAttempts, the application will be failed.
- *     </li>
+ *     Yarn configuration.
+ *   </li>
+ *   <li>
+ *     attemptFailuresValidityInterval. The default value is -1.
+ *     when attemptFailuresValidityInterval in milliseconds is set to
+ *     {@literal >} 0, the failure number will no take failures which happen
+ *     out of the validityInterval into failure count. If failure count
+ *     reaches to maxAppAttempts, the application will be failed.
+ *   </li>
  *   <li>Optional, application-specific {@link LogAggregationContext}</li>
- *   </ul>
- * </p>
+ * </ul>
  * 
  * @see ContainerLaunchContext
  * @see ApplicationClientProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
index 279f127..38fa8b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
@@ -27,34 +27,31 @@ import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>Container</code> represents an allocated resource in the cluster.
- * </p>
- * 
- * <p>The <code>ResourceManager</code> is the sole authority to allocate any
- * <code>Container</code> to applications. The allocated <code>Container</code>
+ * {@code Container} represents an allocated resource in the cluster.
+ * <p>
+ * The {@code ResourceManager} is the sole authority to allocate any
+ * {@code Container} to applications. The allocated {@code Container}
  * is always on a single node and has a unique {@link ContainerId}. It has
- * a specific amount of {@link Resource} allocated.</p>
- * 
- * <p>It includes details such as:
- *   <ul>
- *     <li>{@link ContainerId} for the container, which is globally unique.</li>
- *     <li>
- *       {@link NodeId} of the node on which it is allocated.
- *     </li>
- *     <li>HTTP uri of the node.</li>
- *     <li>{@link Resource} allocated to the container.</li>
- *     <li>{@link Priority} at which the container was allocated.</li>
- *     <li>
- *       Container {@link Token} of the container, used to securely verify
- *       authenticity of the allocation. 
- *     </li>
- *   </ul>
- * </p>
+ * a specific amount of {@link Resource} allocated.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>{@link ContainerId} for the container, which is globally unique.</li>
+ *   <li>
+ *     {@link NodeId} of the node on which it is allocated.
+ *   </li>
+ *   <li>HTTP uri of the node.</li>
+ *   <li>{@link Resource} allocated to the container.</li>
+ *   <li>{@link Priority} at which the container was allocated.</li>
+ *   <li>
+ *     Container {@link Token} of the container, used to securely verify
+ *     authenticity of the allocation.
+ *   </li>
+ * </ul>
  * 
- * <p>Typically, an <code>ApplicationMaster</code> receives the 
- * <code>Container</code> from the <code>ResourceManager</code> during
- * resource-negotiation and then talks to the <code>NodeManager</code> to 
- * start/stop containers.</p>
+ * Typically, an {@code ApplicationMaster} receives the {@code Container}
+ * from the {@code ResourceManager} during resource-negotiation and then
+ * talks to the {@code NodeManager} to start/stop containers.
  * 
  * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
  * @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
index a648fef..932945b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
@@ -30,24 +30,23 @@ import org.apache.hadoop.yarn.server.api.AuxiliaryService;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>ContainerLaunchContext</code> represents all of the information
- * needed by the <code>NodeManager</code> to launch a container.</p>
- * 
- * <p>It includes details such as:
- *   <ul>
- *     <li>{@link ContainerId} of the container.</li>
- *     <li>{@link Resource} allocated to the container.</li>
- *     <li>User to whom the container is allocated.</li>
- *     <li>Security tokens (if security is enabled).</li>
- *     <li>
- *       {@link LocalResource} necessary for running the container such
- *       as binaries, jar, shared-objects, side-files etc. 
- *     </li>
- *     <li>Optional, application-specific binary service data.</li>
- *     <li>Environment variables for the launched process.</li>
- *     <li>Command to launch the container.</li>
- *   </ul>
- * </p>
+ * {@code ContainerLaunchContext} represents all of the information
+ * needed by the {@code NodeManager} to launch a container.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>{@link ContainerId} of the container.</li>
+ *   <li>{@link Resource} allocated to the container.</li>
+ *   <li>User to whom the container is allocated.</li>
+ *   <li>Security tokens (if security is enabled).</li>
+ *   <li>
+ *     {@link LocalResource} necessary for running the container such
+ *     as binaries, jar, shared-objects, side-files etc.
+ *   </li>
+ *   <li>Optional, application-specific binary service data.</li>
+ *   <li>Environment variables for the launched process.</li>
+ *   <li>Command to launch the container.</li>
+ * </ul>
  * 
  * @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
index 72b8edf..11d7bca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
@@ -24,27 +24,22 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
- * <code>ContainerReport</code> is a report of an container.
- * </p>
- * 
+ * {@code ContainerReport} is a report of an container.
  * <p>
  * It includes details such as:
  * <ul>
- * <li>{@link ContainerId} of the container.</li>
- * <li>Allocated Resources to the container.</li>
- * <li>Assigned Node id.</li>
- * <li>Assigned Priority.</li>
- * <li>Creation Time.</li>
- * <li>Finish Time.</li>
- * <li>Container Exit Status.</li>
- * <li>{@link ContainerState} of the container.</li>
- * <li>Diagnostic information in case of errors.</li>
- * <li>Log URL.</li>
- * <li>nodeHttpAddress</li>
+ *   <li>{@link ContainerId} of the container.</li>
+ *   <li>Allocated Resources to the container.</li>
+ *   <li>Assigned Node id.</li>
+ *   <li>Assigned Priority.</li>
+ *   <li>Creation Time.</li>
+ *   <li>Finish Time.</li>
+ *   <li>Container Exit Status.</li>
+ *   <li>{@link ContainerState} of the container.</li>
+ *   <li>Diagnostic information in case of errors.</li>
+ *   <li>Log URL.</li>
+ *   <li>nodeHttpAddress</li>
  * </ul>
- * </p>
- * 
  */
 
 @Public

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
index 574373c..5ccf6dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
@@ -25,17 +25,16 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>ContainerStatus</code> represents the current status of a 
- * <code>Container</code>.</p>
- * 
- * <p>It provides details such as:
- *   <ul>
- *     <li><code>ContainerId</code> of the container.</li>
- *     <li><code>ContainerState</code> of the container.</li>
- *     <li><em>Exit status</em> of a completed container.</li>
- *     <li><em>Diagnostic</em> message for a failed container.</li>
- *   </ul>
- * </p>
+ * {@code ContainerStatus} represents the current status of a
+ * {@code Container}.
+ * <p>
+ * It provides details such as:
+ * <ul>
+ *   <li>{@code ContainerId} of the container.</li>
+ *   <li>{@code ContainerState} of the container.</li>
+ *   <li><em>Exit status</em> of a completed container.</li>
+ *   <li><em>Diagnostic</em> message for a failed container.</li>
+ * </ul>
  */
 @Public
 @Stable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java
index d1aa45b..1552cdf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java
@@ -23,22 +23,22 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 
 /**
- * <p><code>LocalResourceType</code> specifies the <em>type</em>
- * of a resource localized by the <code>NodeManager</code>.</p> 
- *
- * <p>The <em>type</em> can be one of:
- *   <ul>
- *     <li>
- *       {@link #FILE} - Regular file i.e. uninterpreted bytes. 
- *     </li>
- *     <li>
- *       {@link #ARCHIVE} - Archive, which is automatically unarchived by the 
- *       <code>NodeManager</code>.
- *     </li>
- *     <li>
- *       {@link #PATTERN} - A hybrid between {@link #ARCHIVE} and {@link #FILE}.
- *   </ul>
- * </p>
+ * {@code LocalResourceType} specifies the <em>type</em>
+ * of a resource localized by the {@code NodeManager}.
+ * <p>
+ * The <em>type</em> can be one of:
+ * <ul>
+ *   <li>
+ *     {@link #FILE} - Regular file i.e. uninterpreted bytes.
+ *   </li>
+ *   <li>
+ *     {@link #ARCHIVE} - Archive, which is automatically unarchived by the
+ *     <code>NodeManager</code>.
+ *   </li>
+ *   <li>
+ *     {@link #PATTERN} - A hybrid between {@link #ARCHIVE} and {@link #FILE}.
+ *   </li>
+ * </ul>
  *
  * @see LocalResource
  * @see ContainerLaunchContext

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java
index d368bfb..2b71991 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java
@@ -23,22 +23,21 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 
 /**
- * <p><code>LocalResourceVisibility</code> specifies the <em>visibility</em>
- * of a resource localized by the <code>NodeManager</code>.</p>
- * 
- * <p>The <em>visibility</em> can be one of:
- *   <ul>
- *     <li>{@link #PUBLIC} - Shared by all users on the node.</li>
- *     <li>
- *       {@link #PRIVATE} - Shared among all applications of the 
- *       <em>same user</em> on the node.
- *     </li>
- *     <li>
- *       {@link #APPLICATION} - Shared only among containers of the 
- *       <em>same application</em> on the node.
- *     </li>
- *   </ul>
- * </p>
+ * {@code LocalResourceVisibility} specifies the <em>visibility</em>
+ * of a resource localized by the {@code NodeManager}.
+ * <p>
+ * The <em>visibility</em> can be one of:
+ * <ul>
+ *   <li>{@link #PUBLIC} - Shared by all users on the node.</li>
+ *   <li>
+ *     {@link #PRIVATE} - Shared among all applications of the
+ *     <em>same user</em> on the node.
+ *   </li>
+ *   <li>
+ *     {@link #APPLICATION} - Shared only among containers of the
+ *     <em>same application</em> on the node.
+ *   </li>
+ * </ul>
  * 
  * @see LocalResource
  * @see ContainerLaunchContext

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
index e582d2c..9383004 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
@@ -24,30 +24,37 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>LogAggregationContext</code> represents all of the
- * information needed by the <code>NodeManager</code> to handle
- * the logs for an application.</p>
- *
- * <p>It includes details such as:
- *   <ul>
- *     <li>includePattern. It uses Java Regex to filter the log files
+ * {@code LogAggregationContext} represents all of the
+ * information needed by the {@code NodeManager} to handle
+ * the logs for an application.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>
+ *     includePattern. It uses Java Regex to filter the log files
  *     which match the defined include pattern and those log files
- *     will be uploaded when the application finishes. </li>
- *     <li>excludePattern. It uses Java Regex to filter the log files
+ *     will be uploaded when the application finishes.
+ *   </li>
+ *   <li>
+ *     excludePattern. It uses Java Regex to filter the log files
  *     which match the defined exclude pattern and those log files
  *     will not be uploaded when application finishes. If the log file
  *     name matches both the include and the exclude pattern, this file
- *     will be excluded eventually</li>
- *     <li>rolledLogsIncludePattern. It uses Java Regex to filter the log files
+ *     will be excluded eventually.
+ *   </li>
+ *   <li>
+ *     rolledLogsIncludePattern. It uses Java Regex to filter the log files
  *     which match the defined include pattern and those log files
- *     will be aggregated in a rolling fashion.</li>
- *     <li>rolledLogsExcludePattern. It uses Java Regex to filter the log files
+ *     will be aggregated in a rolling fashion.
+ *   </li>
+ *   <li>
+ *     rolledLogsExcludePattern. It uses Java Regex to filter the log files
  *     which match the defined exclude pattern and those log files
  *     will not be aggregated in a rolling fashion. If the log file
  *     name matches both the include and the exclude pattern, this file
- *     will be excluded eventually</li>
- *   </ul>
- * </p>
+ *     will be excluded eventually.
+ *   </li>
+ * </ul>
  *
  * @see ApplicationSubmissionContext
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
index 7ba866d..aac19bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
@@ -28,19 +28,18 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>NodeReport</code> is a summary of runtime information of a 
- * node in the cluster.</p>
- * 
- * <p>It includes details such as:
- *   <ul>
- *     <li>{@link NodeId} of the node.</li>
- *     <li>HTTP Tracking URL of the node.</li>
- *     <li>Rack name for the node.</li>
- *     <li>Used {@link Resource} on the node.</li>
- *     <li>Total available {@link Resource} of the node.</li>
- *     <li>Number of running containers on the node.</li>
- *   </ul>
- * </p>
+ * {@code NodeReport} is a summary of runtime information of a node
+ * in the cluster.
+ * <p>
+ * It includes details such as:
+ * <ul>
+ *   <li>{@link NodeId} of the node.</li>
+ *   <li>HTTP Tracking URL of the node.</li>
+ *   <li>Rack name for the node.</li>
+ *   <li>Used {@link Resource} on the node.</li>
+ *   <li>Total available {@link Resource} of the node.</li>
+ *   <li>Number of running containers on the node.</li>
+ * </ul>
  *
  * @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java
index 976d181..cdf34f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java
@@ -24,36 +24,36 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>A {@link PreemptionMessage} is part of the RM-AM protocol, and it is used by
+ * A {@link PreemptionMessage} is part of the RM-AM protocol, and it is used by
  * the RM to specify resources that the RM wants to reclaim from this
- * <code>ApplicationMaster</code> (AM). The AM receives a {@link
+ * {@code ApplicationMaster} (AM). The AM receives a {@link
  * StrictPreemptionContract} message encoding which containers the platform may
  * forcibly kill, granting it an opportunity to checkpoint state or adjust its
  * execution plan. The message may also include a {@link PreemptionContract}
  * granting the AM more latitude in selecting which resources to return to the
- * cluster.<p>
- *
- * <p>The AM should decode both parts of the message. The {@link
+ * cluster.
+ * <p>
+ * The AM should decode both parts of the message. The {@link
  * StrictPreemptionContract} specifies particular allocations that the RM
  * requires back. The AM can checkpoint containers' state, adjust its execution
  * plan to move the computation, or take no action and hope that conditions that
- * caused the RM to ask for the container will change.<p>
- *
- * <p>In contrast, the {@link PreemptionContract} also includes a description of
+ * caused the RM to ask for the container will change.
+ * <p>
+ * In contrast, the {@link PreemptionContract} also includes a description of
  * resources with a set of containers. If the AM releases containers matching
  * that profile, then the containers enumerated in {@link
- * PreemptionContract#getContainers()} may not be killed.<p>
- *
- * <p>Each preemption message reflects the RM's current understanding of the
- * cluster state, so a request to return <emph>N</emph> containers may not
+ * PreemptionContract#getContainers()} may not be killed.
+ * <p>
+ * Each preemption message reflects the RM's current understanding of the
+ * cluster state, so a request to return <em>N</em> containers may not
  * reflect containers the AM is releasing, recently exited containers the RM has
  * yet to learn about, or new containers allocated before the message was
  * generated. Conversely, an RM may request a different profile of containers in
- * subsequent requests.<p>
- *
- * <p>The policy enforced by the RM is part of the scheduler. Generally, only
+ * subsequent requests.
+ * <p>
+ * The policy enforced by the RM is part of the scheduler. Generally, only
  * containers that have been requested consistently should be killed, but the
- * details are not specified.<p>
+ * details are not specified.
  */
 @Public
 @Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java
index c6777db..585faf8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java
@@ -23,18 +23,15 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 
 /**
- * <p>
- * <code>QueueACL</code> enumerates the various ACLs for queues.
- * </p>
- * 
+ * {@code QueueACL} enumerates the various ACLs for queues.
  * <p>
  * The ACL is one of:
  * <ul>
- * <li>{@link #SUBMIT_APPLICATIONS} - ACL to submit applications to the
- * queue.</li>
- * <li>{@link #ADMINISTER_QUEUE} - ACL to administer the queue.</li>
+ *   <li>
+ *     {@link #SUBMIT_APPLICATIONS} - ACL to submit applications to the queue.
+ *   </li>
+ *   <li>{@link #ADMINISTER_QUEUE} - ACL to administer the queue.</li>
  * </ul>
- * </p>
  * 
  * @see QueueInfo
  * @see ApplicationClientProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java
index d762b41..bee5275 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java
@@ -29,19 +29,18 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>QueueInfo is a report of the runtime information of the queue.</p>
- * 
- * <p>It includes information such as:
- *   <ul>
- *     <li>Queue name.</li>
- *     <li>Capacity of the queue.</li>
- *     <li>Maximum capacity of the queue.</li>
- *     <li>Current capacity of the queue.</li>
- *     <li>Child queues.</li>
- *     <li>Running applications.</li>
- *     <li>{@link QueueState} of the queue.</li>
- *   </ul>
- * </p>
+ * QueueInfo is a report of the runtime information of the queue.
+ * <p>
+ * It includes information such as:
+ * <ul>
+ *   <li>Queue name.</li>
+ *   <li>Capacity of the queue.</li>
+ *   <li>Maximum capacity of the queue.</li>
+ *   <li>Current capacity of the queue.</li>
+ *   <li>Child queues.</li>
+ *   <li>Running applications.</li>
+ *   <li>{@link QueueState} of the queue.</li>
+ * </ul>
  *
  * @see QueueState
  * @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java
index 01698de..2bc0407 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java
@@ -23,14 +23,13 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 
 /**
- * <p>State of a Queue.</p>
- * 
- * <p>A queue is in one of:
- *   <ul>
- *     <li>{@link #RUNNING} - normal state.</li> 
- *     <li>{@link #STOPPED} - not accepting new application submissions.
- *   </ul>
- * </p>
+ * State of a Queue.
+ * <p>
+ * A queue is in one of:
+ * <ul>
+ *   <li>{@link #RUNNING} - normal state.</li>
+ *   <li>{@link #STOPPED} - not accepting new application submissions.</li>
+ * </ul>
  * 
  * @see QueueInfo
  * @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
index a4f0585..4ebe1c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequest.java
@@ -25,23 +25,18 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>
  * {@link ReservationRequest} represents the request made by an application to
  * the {@code ResourceManager} to reserve {@link Resource}s.
- * </p>
- * 
  * <p>
  * It includes:
  * <ul>
- * <li>{@link Resource} required for each request.</li>
- * <li>
- * Number of containers, of above specifications, which are required by the
- * application.</li>
- * <li>
- * Concurrency that indicates the gang size of the request.</li>
+ *   <li>{@link Resource} required for each request.</li>
+ *   <li>
+ *     Number of containers, of above specifications, which are required by the
+ *     application.
+ *   </li>
+ *   <li>Concurrency that indicates the gang size of the request.</li>
  * </ul>
- * </p>
- * 
  */
 @Public
 @Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
index 1ee96c2..3b82626 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationRequestInterpreter.java
@@ -33,14 +33,13 @@ public enum ReservationRequestInterpreter {
    * Requires that exactly ONE among the {@link ReservationRequest} submitted as
    * of a {@link ReservationDefinition} is satisfied to satisfy the overall
    * {@link ReservationDefinition}.
-   * 
+   * <p>
    * WHEN TO USE THIS: This is useful when the user have multiple equivalent
    * ways to run an application, and wants to expose to the ReservationAgent
-   * such flexibility. For example an application could use one <32GB,16core>
-   * container for 10min, or 16 <2GB,1core> containers for 15min, the
-   * ReservationAgent will decide which one of the two it is best for the system
-   * to place.
-   * 
+   * such flexibility. For example an application could use one
+   * {@literal <32GB,16core>} container for 10min, or 16 {@literal <2GB,1core>}
+   * containers for 15min, the ReservationAgent will decide which one of the
+   * two it is best for the system to place.
    */
   R_ANY,
 
@@ -49,16 +48,16 @@ public enum ReservationRequestInterpreter {
    * {@link ReservationDefinition} are satisfied for the overall
    * {@link ReservationDefinition} to be satisfied. No constraints are imposed
    * on the temporal ordering of the allocation used to satisfy the
-   * ResourceRequeusts.
-   * 
+   * ResourceRequests.
+   * <p>
    * WHEN TO USE THIS: This is useful to capture a scenario in which the user
    * cares for multiple ReservationDefinition to be all accepted, or none. For
-   * example, a user might want a reservation R1: with 10 x <8GB,4core> for
-   * 10min, and a reservation R2: with 2 <1GB,1core> for 1h, and only if both
-   * are satisfied the workflow run in this reservation succeeds. The key
-   * differentiator from ALL and ORDER, ORDER_NO_GAP, is that ALL imposes no
-   * restrictions on the relative allocations used to place R1 and R2 above.
-   * 
+   * example, a user might want a reservation R1: with 10 x
+   * {@literal <8GB,4core>} for 10min, and a reservation R2:
+   * with 2 {@literal <1GB,1core>} for 1h, and only if both are satisfied
+   * the workflow run in this reservation succeeds. The key differentiator
+   * from ALL and ORDER, ORDER_NO_GAP, is that ALL imposes no restrictions
+   * on the relative allocations used to place R1 and R2 above.
    */
   R_ALL,
 
@@ -73,15 +72,16 @@ public enum ReservationRequestInterpreter {
    * constraints are imposed on temporal gaps between subsequent allocations
    * (the last instant of the previous allocation can be an arbitrary long
    * period of time before the first instant of the subsequent allocation).
-   * 
+   * <p>
    * WHEN TO USE THIS: Like ALL this requires all ReservationDefinitions to be
    * placed, but it also imposes a time ordering on the allocations used. This
    * is important if the ReservationDefinition(s) are used to describe a
    * workflow with inherent inter-stage dependencies. For example, a first job
-   * runs in a ReservaitonDefinition R1 (10 x <1GB,1core> for 20min), and its
-   * output is consumed by a second job described by a ReservationDefinition R2
-   * (5 x <1GB,1core>) for 50min). R2 allocation cannot overlap R1, as R2 models
-   * a job depending on the output of the job modeled by R1.
+   * runs in a ReservaitonDefinition R1 (10 x {@literal <1GB,1core>}
+   * for 20min), and its output is consumed by a second job described by
+   * a ReservationDefinition R2 (5 x {@literal <1GB,1core>}) for 50min).
+   * R2 allocation cannot overlap R1, as R2 models a job depending on
+   * the output of the job modeled by R1.
    */
   R_ORDER,
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
index 2f17ac9..790120a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
@@ -27,31 +27,30 @@ import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>ResourceRequest</code> represents the request made by an
- * application to the <code>ResourceManager</code> to obtain various 
- * <code>Container</code> allocations.</p>
- * 
- * <p>It includes:
- *   <ul>
- *     <li>{@link Priority} of the request.</li>
- *     <li>
- *       The <em>name</em> of the machine or rack on which the allocation is 
- *       desired. A special value of <em>*</em> signifies that 
- *       <em>any</em> host/rack is acceptable to the application.
- *     </li>
- *     <li>{@link Resource} required for each request.</li>
- *     <li>
- *       Number of containers, of above specifications, which are required 
- *       by the application.
- *     </li>
- *     <li>
- *       A boolean <em>relaxLocality</em> flag, defaulting to <code>true</code>,
- *       which tells the <code>ResourceManager</code> if the application wants
- *       locality to be loose (i.e. allows fall-through to rack or <em>any</em>)
- *       or strict (i.e. specify hard constraint on resource allocation).
- *     </li>
- *   </ul>
- * </p>
+ * {@code ResourceRequest} represents the request made
+ * by an application to the {@code ResourceManager}
+ * to obtain various {@code Container} allocations.
+ * <p>
+ * It includes:
+ * <ul>
+ *   <li>{@link Priority} of the request.</li>
+ *   <li>
+ *     The <em>name</em> of the machine or rack on which the allocation is
+ *     desired. A special value of <em>*</em> signifies that
+ *     <em>any</em> host/rack is acceptable to the application.
+ *   </li>
+ *   <li>{@link Resource} required for each request.</li>
+ *   <li>
+ *     Number of containers, of above specifications, which are required
+ *     by the application.
+ *   </li>
+ *   <li>
+ *     A boolean <em>relaxLocality</em> flag, defaulting to {@code true},
+ *     which tells the {@code ResourceManager} if the application wants
+ *     locality to be loose (i.e. allows fall-through to rack or <em>any</em>)
+ *     or strict (i.e. specify hard constraint on resource allocation).
+ *   </li>
+ * </ul>
  * 
  * @see Resource
  * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
@@ -267,7 +266,7 @@ public abstract class ResourceRequest implements Comparable<ResourceRequest> {
   /**
    * Set node label expression of this resource request. Now only support
    * specifying a single node label. In the future we will support more complex
-   * node label expression specification like AND(&&), OR(||), etc.
+   * node label expression specification like {@code AND(&&), OR(||)}, etc.
    * 
    * Any please note that node label expression now can only take effect when
    * the resource request has resourceName = ANY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 8c83fea..f40c999 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -733,8 +733,9 @@ public class YarnConfiguration extends Configuration {
   
   /**
    * How long to wait between aggregated log retention checks. If set to
-   * a value <= 0 then the value is computed as one-tenth of the log retention
-   * setting. Be careful set this too small and you will spam the name node.
+   * a value {@literal <=} 0 then the value is computed as one-tenth of the
+   * log retention setting. Be careful set this too small and you will spam
+   * the name node.
    */
   public static final String LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS =
       YARN_PREFIX + "log-aggregation.retain-check-interval-seconds";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeResourceRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeResourceRequest.java
index d1ab781..d540cce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeResourceRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeResourceRequest.java
@@ -54,7 +54,7 @@ public abstract class UpdateNodeResourceRequest {
   
   /**
    * Get the map from <code>NodeId</code> to <code>ResourceOption</code>.
-   * @return the map of <NodeId, ResourceOption>
+   * @return the map of {@code <NodeId, ResourceOption>}
    */
   @Public
   @Evolving
@@ -62,7 +62,7 @@ public abstract class UpdateNodeResourceRequest {
   
   /**
    * Set the map from <code>NodeId</code> to <code>ResourceOption</code>.
-   * @param nodeResourceMap the map of <NodeId, ResourceOption>
+   * @param nodeResourceMap the map of {@code <NodeId, ResourceOption>}
    */
   @Public
   @Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java
index b3c5308..b590a51 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java
@@ -56,28 +56,22 @@ public abstract class AHSClient extends AbstractService {
   }
 
   /**
-   * <p>
    * Get a report of the given Application.
-   * </p>
-   * 
    * <p>
    * In secure mode, <code>YARN</code> verifies access to the application, queue
    * etc. before accepting the request.
-   * </p>
-   * 
    * <p>
    * If the user does not have <code>VIEW_APP</code> access then the following
    * fields in the report will be set to stubbed values:
    * <ul>
-   * <li>host - set to "N/A"</li>
-   * <li>RPC port - set to -1</li>
-   * <li>client token - set to "N/A"</li>
-   * <li>diagnostics - set to "N/A"</li>
-   * <li>tracking URL - set to "N/A"</li>
-   * <li>original tracking URL - set to "N/A"</li>
-   * <li>resource usage report - all values are -1</li>
+   *   <li>host - set to "N/A"</li>
+   *   <li>RPC port - set to -1</li>
+   *   <li>client token - set to "N/A"</li>
+   *   <li>diagnostics - set to "N/A"</li>
+   *   <li>tracking URL - set to "N/A"</li>
+   *   <li>original tracking URL - set to "N/A"</li>
+   *   <li>resource usage report - all values are -1</li>
    * </ul>
-   * </p>
    * 
    * @param appId
    *          {@link ApplicationId} of the application that needs a report
@@ -121,7 +115,7 @@ public abstract class AHSClient extends AbstractService {
    *          a report
    * @return application attempt report
    * @throws YarnException
-   * @throws {@link ApplicationAttemptNotFoundException} if application attempt
+   * @throws ApplicationAttemptNotFoundException if application attempt
    *         not found
    * @throws IOException
    */
@@ -157,7 +151,7 @@ public abstract class AHSClient extends AbstractService {
    *          {@link ContainerId} of the container that needs a report
    * @return container report
    * @throws YarnException
-   * @throws {@link ContainerNotFoundException} if container not found
+   * @throws ContainerNotFoundException if container not found
    * @throws IOException
    */
   public abstract ContainerReport getContainerReport(ContainerId containerId)


[22/50] [abbrv] hadoop git commit: HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Masatake Iwasaki.

Posted by zh...@apache.org.
HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Masatake Iwasaki.


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

Branch: refs/heads/HDFS-7285
Commit: 7da136ecca4dafc83ef69b5d9980fa5b67ada084
Parents: bd0a9ba
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sun Mar 15 14:17:35 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sun Mar 15 14:17:35 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt       |  3 +++
 .../src/site/markdown/SchedulerLoadSimulator.md       |  2 +-
 .../src/site/markdown/HadoopStreaming.md.vm           | 14 +++++++-------
 3 files changed, 11 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7da136ec/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 55028cb..e386723 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1100,6 +1100,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt
     synchronization. (Sean Busbey via yliu)
 
+    HADOOP-11558. Fix dead links to doc of hadoop-tools. (Masatake Iwasaki
+    via ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7da136ec/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
index ca179ee..2cffc86 100644
--- a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
+++ b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
@@ -43,7 +43,7 @@ The Yarn Scheduler Load Simulator (SLS) is such a tool, which can simulate large
 o
 The simulator will exercise the real Yarn `ResourceManager` removing the network factor by simulating `NodeManagers` and `ApplicationMasters` via handling and dispatching `NM`/`AMs` heartbeat events from within the same JVM. To keep tracking of scheduler behavior and performance, a scheduler wrapper will wrap the real scheduler.
 
-The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](https://hadoop.apache.org/docs/stable/rumen.html).
+The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](../hadoop-rumen/Rumen.html).
 
 The simulator will produce real time metrics while executing, including:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7da136ec/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
index 0b64586..b4c5e38 100644
--- a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
+++ b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
@@ -201,7 +201,7 @@ To specify additional local temp directories use:
      -D mapred.system.dir=/tmp/system
      -D mapred.temp.dir=/tmp/temp
 
-**Note:** For more details on job configuration parameters see: [mapred-default.xml](./mapred-default.xml)
+**Note:** For more details on job configuration parameters see: [mapred-default.xml](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/mapred-default.xml)
 
 $H4 Specifying Map-Only Jobs
 
@@ -322,7 +322,7 @@ More Usage Examples
 
 $H3 Hadoop Partitioner Class
 
-Hadoop has a library class, [KeyFieldBasedPartitioner](../../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
+Hadoop has a library class, [KeyFieldBasedPartitioner](../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D stream.map.output.field.separator=. \
@@ -372,7 +372,7 @@ Sorting within each partition for the reducer(all 4 fields used for sorting)
 
 $H3 Hadoop Comparator Class
 
-Hadoop has a library class, [KeyFieldBasedComparator](../../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
+Hadoop has a library class, [KeyFieldBasedComparator](../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.job.output.key.comparator.class=org.apache.hadoop.mapreduce.lib.partition.KeyFieldBasedComparator \
@@ -406,7 +406,7 @@ Sorting output for the reducer (where second field used for sorting)
 
 $H3 Hadoop Aggregate Package
 
-Hadoop has a library package called [Aggregate](../../org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
+Hadoop has a library package called [Aggregate](../api/org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
 
 To use Aggregate, simply specify "-reducer aggregate":
 
@@ -441,7 +441,7 @@ The python program myAggregatorForKeyCount.py looks like:
 
 $H3 Hadoop Field Selection Class
 
-Hadoop has a library class, [FieldSelectionMapReduce](../../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
+Hadoop has a library class, [FieldSelectionMapReduce](../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.map.output.key.field.separator=. \
@@ -480,7 +480,7 @@ As an example, consider the problem of zipping (compressing) a set of files acro
 
 $H3 How many reducers should I use?
 
-See MapReduce Tutorial for details: [Reducer](./MapReduceTutorial.html#Reducer)
+See MapReduce Tutorial for details: [Reducer](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Reducer)
 
 $H3 If I set up an alias in my shell script, will that work after -mapper?
 
@@ -556,4 +556,4 @@ A streaming process can use the stderr to emit status information. To set a stat
 
 $H3 How do I get the Job variables in a streaming job's mapper/reducer?
 
-See [Configured Parameters](./MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.
+See [Configured Parameters](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.


[06/50] [abbrv] hadoop git commit: HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt synchronization. (Sean Busbey via yliu)

Posted by zh...@apache.org.
HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt synchronization. (Sean Busbey via yliu)


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

Branch: refs/heads/HDFS-7285
Commit: a85291003cf3e3fd79b6addcf59d4f43dc72d356
Parents: 8212877
Author: yliu <yl...@apache.org>
Authored: Fri Mar 13 02:25:02 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Fri Mar 13 02:25:02 2015 +0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt  |  3 +++
 .../apache/hadoop/crypto/CryptoOutputStream.java | 19 ++++++++++++-------
 2 files changed, 15 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8529100/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 6970bad..55028cb 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1097,6 +1097,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11693. Azure Storage FileSystem rename operations are throttled too
     aggressively to complete HBase WAL archiving. (Duo Xu via cnauroth)
 
+    HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt
+    synchronization. (Sean Busbey via yliu)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8529100/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
index f1ea0fc..bc09b8c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
@@ -40,6 +40,9 @@ import com.google.common.base.Preconditions;
  * padding = pos%(algorithm blocksize); 
  * <p/>
  * The underlying stream offset is maintained as state.
+ *
+ * Note that while some of this class' methods are synchronized, this is just to
+ * match the threadsafety behavior of DFSOutputStream. See HADOOP-11710.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -126,7 +129,7 @@ public class CryptoOutputStream extends FilterOutputStream implements
    * @throws IOException
    */
   @Override
-  public void write(byte[] b, int off, int len) throws IOException {
+  public synchronized void write(byte[] b, int off, int len) throws IOException {
     checkStream();
     if (b == null) {
       throw new NullPointerException();
@@ -213,14 +216,16 @@ public class CryptoOutputStream extends FilterOutputStream implements
   }
   
   @Override
-  public void close() throws IOException {
+  public synchronized void close() throws IOException {
     if (closed) {
       return;
     }
-    
-    super.close();
-    freeBuffers();
-    closed = true;
+    try {
+      super.close();
+      freeBuffers();
+    } finally {
+      closed = true;
+    }
   }
   
   /**
@@ -228,7 +233,7 @@ public class CryptoOutputStream extends FilterOutputStream implements
    * underlying stream, then do the flush.
    */
   @Override
-  public void flush() throws IOException {
+  public synchronized void flush() throws IOException {
     checkStream();
     encrypt();
     super.flush();


[26/50] [abbrv] hadoop git commit: YARN-3171. Sort by Application id, AppAttempt and ContainerID doesn't work in ATS / RM web ui. Contributed by Naganarasimha G R

Posted by zh...@apache.org.
YARN-3171. Sort by Application id, AppAttempt and ContainerID doesn't
work in ATS / RM web ui. Contributed by Naganarasimha G R


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

Branch: refs/heads/HDFS-7285
Commit: 3ff1ba2a7b00fdf06270d00b2193bde4b56b06b3
Parents: bc9cb3e
Author: Xuan <xg...@apache.org>
Authored: Sun Mar 15 20:26:10 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Sun Mar 15 20:26:10 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                | 3 +++
 .../org/apache/hadoop/yarn/server/webapp/WebPageUtils.java     | 6 +++---
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ff1ba2a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 77f8819..bcab88c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -766,6 +766,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3267. Timelineserver applies the ACL rules after applying the limit on
     the number of records (Chang Li via jeagles)
 
+    YARN-3171. Sort by Application id, AppAttempt and ContainerID doesn't work
+    in ATS / RM web ui. (Naganarasimha G R via xgong)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ff1ba2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
index 384a976..5acabf5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
@@ -44,7 +44,7 @@ public class WebPageUtils {
     StringBuilder sb = new StringBuilder();
     return sb
       .append("[\n")
-      .append("{'sType':'numeric', 'aTargets': [0]")
+      .append("{'sType':'string', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }")
       .append("\n, {'sType':'numeric', 'aTargets': " +
           (isFairSchedulerPage ? "[6, 7]": "[5, 6]"))
@@ -63,7 +63,7 @@ public class WebPageUtils {
 
   private static String getAttemptsTableColumnDefs() {
     StringBuilder sb = new StringBuilder();
-    return sb.append("[\n").append("{'sType':'numeric', 'aTargets': [0]")
+    return sb.append("[\n").append("{'sType':'string', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }")
       .append("\n, {'sType':'numeric', 'aTargets': [1]")
       .append(", 'mRender': renderHadoopDate }]").toString();
@@ -79,7 +79,7 @@ public class WebPageUtils {
 
   private static String getContainersTableColumnDefs() {
     StringBuilder sb = new StringBuilder();
-    return sb.append("[\n").append("{'sType':'numeric', 'aTargets': [0]")
+    return sb.append("[\n").append("{'sType':'string', 'aTargets': [0]")
       .append(", 'mRender': parseHadoopID }]").toString();
   }
 


[37/50] [abbrv] hadoop git commit: HADOOP-11541. Raw XOR coder

Posted by zh...@apache.org.
HADOOP-11541. Raw XOR coder


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

Branch: refs/heads/HDFS-7285
Commit: 435d7b0d1681546efdaafd18935af2d76f313241
Parents: 52c4219
Author: Kai Zheng <dr...@apache.org>
Authored: Sun Feb 8 01:40:27 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:08:49 2015 -0700

----------------------------------------------------------------------
 .../io/erasurecode/rawcoder/XorRawDecoder.java  |  81 ++++++
 .../io/erasurecode/rawcoder/XorRawEncoder.java  |  61 +++++
 .../hadoop/io/erasurecode/TestCoderBase.java    | 262 +++++++++++++++++++
 .../erasurecode/rawcoder/TestRawCoderBase.java  |  96 +++++++
 .../erasurecode/rawcoder/TestXorRawCoder.java   |  52 ++++
 5 files changed, 552 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/435d7b0d/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..98307a7
--- /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/435d7b0d/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..99b20b9
--- /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/435d7b0d/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
new file mode 100644
index 0000000..9482b43
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -0,0 +1,262 @@
+/**
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import static org.junit.Assert.assertArrayEquals;
+
+/**
+ * Test base of common utilities for tests not only raw coders but also block
+ * coders.
+ */
+public abstract class TestCoderBase {
+  protected static Random RAND = new Random();
+
+  protected int numDataUnits;
+  protected int numParityUnits;
+  protected int chunkSize = 16 * 1024;
+
+  // Indexes of erased data units. Will also support test of erasing
+  // parity units
+  protected int[] erasedDataIndexes = new int[] {0};
+
+  // Data buffers are either direct or on-heap, for performance the two cases
+  // may go to different coding implementations.
+  protected boolean usingDirectBuffer = true;
+
+  /**
+   * Compare and verify if erased chunks are equal to recovered chunks
+   * @param erasedChunks
+   * @param recoveredChunks
+   */
+  protected void compareAndVerify(ECChunk[] erasedChunks,
+                                  ECChunk[] recoveredChunks) {
+    byte[][] erased = ECChunk.toArray(erasedChunks);
+    byte[][] recovered = ECChunk.toArray(recoveredChunks);
+    for (int i = 0; i < erasedChunks.length; ++i) {
+      assertArrayEquals("Decoding and comparing failed.", erased[i],
+          recovered[i]);
+    }
+  }
+
+  /**
+   * Adjust and return erased indexes based on the array of the input chunks (
+   * parity chunks + data chunks).
+   * @return
+   */
+  protected int[] getErasedIndexesForDecoding() {
+    int[] erasedIndexesForDecoding = new int[erasedDataIndexes.length];
+    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+      erasedIndexesForDecoding[i] = erasedDataIndexes[i] + numParityUnits;
+    }
+    return erasedIndexesForDecoding;
+  }
+
+  /**
+   * Return input chunks for decoding, which is parityChunks + dataChunks.
+   * @param dataChunks
+   * @param parityChunks
+   * @return
+   */
+  protected ECChunk[] prepareInputChunksForDecoding(ECChunk[] dataChunks,
+                                                  ECChunk[] parityChunks) {
+    ECChunk[] inputChunks = new ECChunk[numParityUnits + numDataUnits];
+    
+    int idx = 0;
+    for (int i = 0; i < numParityUnits; i++) {
+      inputChunks[idx ++] = parityChunks[i];
+    }
+    for (int i = 0; i < numDataUnits; i++) {
+      inputChunks[idx ++] = dataChunks[i];
+    }
+    
+    return inputChunks;
+  }
+
+  /**
+   * Have a copy of the data chunks that's to be erased thereafter. The copy
+   * will be used to compare and verify with the to be recovered chunks.
+   * @param dataChunks
+   * @return
+   */
+  protected ECChunk[] copyDataChunksToErase(ECChunk[] dataChunks) {
+    ECChunk[] copiedChunks = new ECChunk[erasedDataIndexes.length];
+
+    int j = 0;
+    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+      copiedChunks[j ++] = cloneChunkWithData(dataChunks[erasedDataIndexes[i]]);
+    }
+
+    return copiedChunks;
+  }
+
+  /**
+   * Erase some data chunks to test the recovering of them
+   * @param dataChunks
+   */
+  protected void eraseSomeDataBlocks(ECChunk[] dataChunks) {
+    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+      eraseDataFromChunk(dataChunks[erasedDataIndexes[i]]);
+    }
+  }
+
+  /**
+   * Erase data from the specified chunks, putting ZERO bytes to the buffers.
+   * @param chunks
+   */
+  protected void eraseDataFromChunks(ECChunk[] chunks) {
+    for (int i = 0; i < chunks.length; ++i) {
+      eraseDataFromChunk(chunks[i]);
+    }
+  }
+
+  /**
+   * Erase data from the specified chunk, putting ZERO bytes to the buffer.
+   * @param chunk
+   */
+  protected void eraseDataFromChunk(ECChunk chunk) {
+    ByteBuffer chunkBuffer = chunk.getBuffer();
+    // erase the data
+    chunkBuffer.position(0);
+    for (int i = 0; i < chunkSize; ++i) {
+      chunkBuffer.put((byte) 0);
+    }
+    chunkBuffer.flip();
+  }
+
+  /**
+   * Clone chunks along with copying the associated data. It respects how the
+   * chunk buffer is allocated, direct or non-direct. It avoids affecting the
+   * original chunk buffers.
+   * @param chunks
+   * @return
+   */
+  protected static ECChunk[] cloneChunksWithData(ECChunk[] chunks) {
+    ECChunk[] results = new ECChunk[chunks.length];
+    for (int i = 0; i < chunks.length; ++i) {
+      results[i] = cloneChunkWithData(chunks[i]);
+    }
+
+    return results;
+  }
+
+  /**
+   * Clone chunk along with copying the associated data. It respects how the
+   * chunk buffer is allocated, direct or non-direct. It avoids affecting the
+   * original chunk.
+   * @param chunk
+   * @return a new chunk
+   */
+  protected static ECChunk cloneChunkWithData(ECChunk chunk) {
+    ByteBuffer srcBuffer = chunk.getBuffer();
+    ByteBuffer destBuffer;
+
+    byte[] bytesArr = new byte[srcBuffer.remaining()];
+    srcBuffer.mark();
+    srcBuffer.get(bytesArr);
+    srcBuffer.reset();
+
+    if (srcBuffer.hasArray()) {
+      destBuffer = ByteBuffer.wrap(bytesArr);
+    } else {
+      destBuffer = ByteBuffer.allocateDirect(srcBuffer.remaining());
+      destBuffer.put(bytesArr);
+      destBuffer.flip();
+    }
+
+    return new ECChunk(destBuffer);
+  }
+
+  /**
+   * Allocate a chunk for output or writing.
+   * @return
+   */
+  protected ECChunk allocateOutputChunk() {
+    ByteBuffer buffer = allocateOutputBuffer();
+
+    return new ECChunk(buffer);
+  }
+
+  /**
+   * Allocate a buffer for output or writing.
+   * @return
+   */
+  protected ByteBuffer allocateOutputBuffer() {
+    ByteBuffer buffer = usingDirectBuffer ?
+        ByteBuffer.allocateDirect(chunkSize) : ByteBuffer.allocate(chunkSize);
+
+    return buffer;
+  }
+
+  /**
+   * Prepare data chunks for each data unit, by generating random data.
+   * @return
+   */
+  protected ECChunk[] prepareDataChunksForEncoding() {
+    ECChunk[] chunks = new ECChunk[numDataUnits];
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = generateDataChunk();
+    }
+
+    return chunks;
+  }
+
+  /**
+   * Generate data chunk by making random data.
+   * @return
+   */
+  protected ECChunk generateDataChunk() {
+    ByteBuffer buffer = allocateOutputBuffer();
+    for (int i = 0; i < chunkSize; i++) {
+      buffer.put((byte) RAND.nextInt(256));
+    }
+    buffer.flip();
+
+    return new ECChunk(buffer);
+  }
+
+  /**
+   * Prepare parity chunks for encoding, each chunk for each parity unit.
+   * @return
+   */
+  protected ECChunk[] prepareParityChunksForEncoding() {
+    ECChunk[] chunks = new ECChunk[numParityUnits];
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = allocateOutputChunk();
+    }
+
+    return chunks;
+  }
+
+  /**
+   * Prepare output chunks for decoding, each output chunk for each erased
+   * chunk.
+   * @return
+   */
+  protected ECChunk[] prepareOutputChunksForDecoding() {
+    ECChunk[] chunks = new ECChunk[erasedDataIndexes.length];
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = allocateOutputChunk();
+    }
+
+    return chunks;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/435d7b0d/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
new file mode 100644
index 0000000..9119211
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
@@ -0,0 +1,96 @@
+/**
+ * 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.TestCoderBase;
+
+/**
+ * Raw coder test base with utilities.
+ */
+public abstract class TestRawCoderBase extends TestCoderBase {
+  protected Class<? extends RawErasureEncoder> encoderClass;
+  protected Class<? extends RawErasureDecoder> decoderClass;
+
+  /**
+   * Generating source data, encoding, recovering and then verifying.
+   * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
+   * it supports two kinds of ByteBuffers, one is array backed, the other is
+   * direct ByteBuffer. Have usingDirectBuffer to indicate which case to test.
+   * @param usingDirectBuffer
+   */
+  protected void testCoding(boolean usingDirectBuffer) {
+    // Generate data and encode
+    ECChunk[] dataChunks = prepareDataChunksForEncoding();
+    ECChunk[] parityChunks = prepareParityChunksForEncoding();
+    RawErasureEncoder encoder = createEncoder();
+
+    // Backup all the source chunks for later recovering because some coders
+    // may affect the source data.
+    ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks);
+    // Make a copy of a strip for later comparing
+    ECChunk[] toEraseDataChunks = copyDataChunksToErase(clonedDataChunks);
+
+    encoder.encode(dataChunks, parityChunks);
+    // Erase the copied sources
+    eraseSomeDataBlocks(clonedDataChunks);
+
+    //Decode
+    ECChunk[] inputChunks = prepareInputChunksForDecoding(clonedDataChunks,
+        parityChunks);
+    ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
+    RawErasureDecoder decoder = createDecoder();
+    decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks);
+
+    //Compare
+    compareAndVerify(toEraseDataChunks, recoveredChunks);
+  }
+
+  /**
+   * Create the raw erasure encoder to test
+   * @return
+   */
+  protected RawErasureEncoder createEncoder() {
+    RawErasureEncoder encoder;
+    try {
+      encoder = encoderClass.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create encoder", e);
+    }
+
+    encoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    return encoder;
+  }
+
+  /**
+   * create the raw erasure decoder to test
+   * @return
+   */
+  protected RawErasureDecoder createDecoder() {
+    RawErasureDecoder decoder;
+    try {
+      decoder = decoderClass.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create decoder", e);
+    }
+
+    decoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    return decoder;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/435d7b0d/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..8e59b8a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
@@ -0,0 +1,52 @@
+/**
+ * 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 {
+  private static Random RAND = new Random();
+
+  @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);
+  }
+
+}


[20/50] [abbrv] hadoop git commit: MAPREDUCE-6265. Make ContainerLauncherImpl.INITIAL_POOL_SIZE configurable to better control to launch/kill containers. Contributed by Zhihai Xu

Posted by zh...@apache.org.
MAPREDUCE-6265. Make ContainerLauncherImpl.INITIAL_POOL_SIZE configurable to better control to launch/kill containers. Contributed by Zhihai Xu


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

Branch: refs/heads/HDFS-7285
Commit: 9d38520c8e42530a817a7f69c9aa73a9ad40639c
Parents: 32741cf
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sat Mar 14 16:44:02 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sat Mar 14 16:44:02 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 +++
 .../v2/app/launcher/ContainerLauncherImpl.java  | 14 +++++++++----
 .../v2/app/launcher/TestContainerLauncher.java  | 21 +++++++++++++++-----
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  8 ++++++++
 .../src/main/resources/mapred-default.xml       |  8 ++++++++
 5 files changed, 45 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d38520c/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 0bbe85c..ab6eef5 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -340,6 +340,9 @@ Release 2.7.0 - UNRELEASED
     MAPREDUCE-6263. Configurable timeout between YARNRunner terminate the 
     application and forcefully kill. (Eric Payne via junping_du)
 
+    MAPREDUCE-6265. Make ContainerLauncherImpl.INITIAL_POOL_SIZE configurable 
+    to better control to launch/kill containers. (Zhihai Xu via ozawa)
+
   OPTIMIZATIONS
 
     MAPREDUCE-6169. MergeQueue should release reference to the current item 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d38520c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
index 666f757..9c1125d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
@@ -70,7 +70,7 @@ public class ContainerLauncherImpl extends AbstractService implements
     new ConcurrentHashMap<ContainerId, Container>(); 
   private final AppContext context;
   protected ThreadPoolExecutor launcherPool;
-  protected static final int INITIAL_POOL_SIZE = 10;
+  protected int initialPoolSize;
   private int limitOnPoolSize;
   private Thread eventHandlingThread;
   protected BlockingQueue<ContainerLauncherEvent> eventQueue =
@@ -246,6 +246,12 @@ public class ContainerLauncherImpl extends AbstractService implements
         MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT,
         MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT);
     LOG.info("Upper limit on the thread pool size is " + this.limitOnPoolSize);
+
+    this.initialPoolSize = conf.getInt(
+        MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE,
+        MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE);
+    LOG.info("The thread pool initial size is " + this.initialPoolSize);
+
     super.serviceInit(conf);
     cmProxy = new ContainerManagementProtocolProxy(conf);
   }
@@ -256,7 +262,7 @@ public class ContainerLauncherImpl extends AbstractService implements
         "ContainerLauncher #%d").setDaemon(true).build();
 
     // Start with a default core-pool size of 10 and change it dynamically.
-    launcherPool = new ThreadPoolExecutor(INITIAL_POOL_SIZE,
+    launcherPool = new ThreadPoolExecutor(initialPoolSize,
         Integer.MAX_VALUE, 1, TimeUnit.HOURS,
         new LinkedBlockingQueue<Runnable>(),
         tf);
@@ -289,11 +295,11 @@ public class ContainerLauncherImpl extends AbstractService implements
             int idealPoolSize = Math.min(limitOnPoolSize, numNodes);
 
             if (poolSize < idealPoolSize) {
-              // Bump up the pool size to idealPoolSize+INITIAL_POOL_SIZE, the
+              // Bump up the pool size to idealPoolSize+initialPoolSize, the
               // later is just a buffer so we are not always increasing the
               // pool-size
               int newPoolSize = Math.min(limitOnPoolSize, idealPoolSize
-                  + INITIAL_POOL_SIZE);
+                  + initialPoolSize);
               LOG.info("Setting ContainerLauncher pool size to " + newPoolSize
                   + " as number-of-nodes to talk to is " + numNodes);
               launcherPool.setCorePoolSize(newPoolSize);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d38520c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
index dc1d72f..41ee65d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
@@ -90,7 +90,7 @@ public class TestContainerLauncher {
 
   static final Log LOG = LogFactory.getLog(TestContainerLauncher.class);
 
-  @Test (timeout = 5000)
+  @Test (timeout = 10000)
   public void testPoolSize() throws InterruptedException {
 
     ApplicationId appId = ApplicationId.newInstance(12345, 67);
@@ -108,12 +108,14 @@ public class TestContainerLauncher {
     ThreadPoolExecutor threadPool = containerLauncher.getThreadPool();
 
     // No events yet
+    Assert.assertEquals(containerLauncher.initialPoolSize,
+        MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE);
     Assert.assertEquals(0, threadPool.getPoolSize());
-    Assert.assertEquals(ContainerLauncherImpl.INITIAL_POOL_SIZE,
+    Assert.assertEquals(containerLauncher.initialPoolSize,
       threadPool.getCorePoolSize());
     Assert.assertNull(containerLauncher.foundErrors);
 
-    containerLauncher.expectedCorePoolSize = ContainerLauncherImpl.INITIAL_POOL_SIZE;
+    containerLauncher.expectedCorePoolSize = containerLauncher.initialPoolSize;
     for (int i = 0; i < 10; i++) {
       ContainerId containerId = ContainerId.newContainerId(appAttemptId, i);
       TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, i);
@@ -152,7 +154,7 @@ public class TestContainerLauncher {
     // Different hosts, there should be an increase in core-thread-pool size to
     // 21(11hosts+10buffer)
     // Core pool size should be 21 but the live pool size should be only 11.
-    containerLauncher.expectedCorePoolSize = 11 + ContainerLauncherImpl.INITIAL_POOL_SIZE;
+    containerLauncher.expectedCorePoolSize = 11 + containerLauncher.initialPoolSize;
     containerLauncher.finishEventHandling = false;
     ContainerId containerId = ContainerId.newContainerId(appAttemptId, 21);
     TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, 21);
@@ -164,6 +166,15 @@ public class TestContainerLauncher {
     Assert.assertNull(containerLauncher.foundErrors);
 
     containerLauncher.stop();
+
+    // change configuration MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE
+    // and verify initialPoolSize value.
+    Configuration conf = new Configuration();
+    conf.setInt(MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE,
+        20);
+    containerLauncher = new CustomContainerLauncher(context);
+    containerLauncher.init(conf);
+    Assert.assertEquals(containerLauncher.initialPoolSize, 20);
   }
 
   @Test(timeout = 5000)
@@ -187,7 +198,7 @@ public class TestContainerLauncher {
     ThreadPoolExecutor threadPool = containerLauncher.getThreadPool();
 
     // 10 different hosts
-    containerLauncher.expectedCorePoolSize = ContainerLauncherImpl.INITIAL_POOL_SIZE;
+    containerLauncher.expectedCorePoolSize = containerLauncher.initialPoolSize;
     for (int i = 0; i < 10; i++) {
       containerLauncher.handle(new ContainerLauncherEvent(taskAttemptId,
         containerId, "host" + i + ":1234", null,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d38520c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 9f671cd..3aa304a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -504,6 +504,14 @@ public interface MRJobConfig {
   public static final int DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT = 
       500;
 
+  /**
+   * The initial size of thread pool to launch containers in the app master
+   */
+  public static final String MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE =
+      MR_AM_PREFIX+"containerlauncher.threadpool-initial-size";
+  public static final int DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE =
+      10;
+
   /** Number of threads to handle job client RPC requests.*/
   public static final String MR_AM_JOB_CLIENT_THREAD_COUNT =
     MR_AM_PREFIX + "job.client.thread-count";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d38520c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index d7bec9c..820c1ac 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -1694,4 +1694,12 @@
     calculated as (heapSize / mapreduce.heap.memory-mb.ratio).
   </description>
 </property>
+
+<property>
+  <name>yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size</name>
+  <value>10</value>
+  <description>The initial size of thread pool to launch containers in the
+    app master.
+  </description>
+</property>
 </configuration>


[30/50] [abbrv] hadoop git commit: HADOOP-9477. Amendment to CHANGES.txt.

Posted by zh...@apache.org.
HADOOP-9477. Amendment to CHANGES.txt.


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

Branch: refs/heads/HDFS-7285
Commit: d1eebd9c9c1fed5877ef2665959e9bd1485d080c
Parents: 03b77ed
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Mon Mar 16 09:16:57 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Mon Mar 16 09:16:57 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1eebd9c/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index e161d7d..a43a153 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -37,9 +37,6 @@ Trunk (Unreleased)
 
     HADOOP-11565. Add --slaves shell option (aw)
 
-    HADOOP-9477. Add posixGroups support for LDAP groups mapping service.
-    (Dapeng Sun via Yongjun Zhang)
-
   IMPROVEMENTS
 
     HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution
@@ -447,6 +444,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11226. Add a configuration to set ipc.Client's traffic class with
     IPTOS_LOWDELAY|IPTOS_RELIABILITY. (Gopal V via ozawa)
 
+    HADOOP-9477. Add posixGroups support for LDAP groups mapping service.
+    (Dapeng Sun via Yongjun Zhang)
+
   IMPROVEMENTS
 
     HADOOP-11692. Improve authentication failure WARN message to avoid user


[10/50] [abbrv] hadoop git commit: HADOOP-9477. Add posixGroups support for LDAP groups mapping service. (Dapeng Sun via Yongjun Zhang)

Posted by zh...@apache.org.
HADOOP-9477. Add posixGroups support for LDAP groups mapping service. (Dapeng Sun via Yongjun Zhang)


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

Branch: refs/heads/HDFS-7285
Commit: 82128774156c30a535b62d764bb6cf9c8d2f3a3b
Parents: 863079b
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Thu Mar 12 14:42:34 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Thu Mar 12 14:52:36 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../hadoop/security/LdapGroupsMapping.java      |  54 ++++++++--
 .../hadoop/security/TestLdapGroupsMapping.java  |  42 +-------
 .../security/TestLdapGroupsMappingBase.java     |  77 ++++++++++++++
 .../TestLdapGroupsMappingWithPosixGroup.java    | 103 +++++++++++++++++++
 5 files changed, 229 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/82128774/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 40b5cd0..6970bad 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -37,6 +37,9 @@ Trunk (Unreleased)
 
     HADOOP-11565. Add --slaves shell option (aw)
 
+    HADOOP-9477. Add posixGroups support for LDAP groups mapping service.
+    (Dapeng Sun via Yongjun Zhang)
+
   IMPROVEMENTS
 
     HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82128774/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
index d463ac7..df91b70 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
@@ -150,6 +150,14 @@ public class LdapGroupsMapping
   public static final String GROUP_NAME_ATTR_DEFAULT = "cn";
 
   /*
+   * Posix attributes
+   */
+  public static final String POSIX_UIDNUMBER = "uidNumber";
+  public static final String POSIX_GIDNUMBER = "gidNumber";
+  public static final String POSIX_GROUP = "posixGroup";
+  public static final String POSIX_ACCOUNT = "posixAccount";
+
+  /*
    * LDAP {@link SearchControls} attribute to set the time limit
    * for an invoked directory search. Prevents infinite wait cases.
    */
@@ -178,6 +186,7 @@ public class LdapGroupsMapping
   private String userSearchFilter;
   private String groupMemberAttr;
   private String groupNameAttr;
+  private boolean isPosix;
 
   public static int RECONNECT_RETRY_COUNT = 3;
   
@@ -242,15 +251,40 @@ public class LdapGroupsMapping
       SearchResult result = results.nextElement();
       String userDn = result.getNameInNamespace();
 
-      NamingEnumeration<SearchResult> groupResults =
-          ctx.search(baseDN,
-              "(&" + groupSearchFilter + "(" + groupMemberAttr + "={0}))",
-              new Object[]{userDn},
-              SEARCH_CONTROLS);
-      while (groupResults.hasMoreElements()) {
-        SearchResult groupResult = groupResults.nextElement();
-        Attribute groupName = groupResult.getAttributes().get(groupNameAttr);
-        groups.add(groupName.get().toString());
+      NamingEnumeration<SearchResult> groupResults = null;
+
+      if (isPosix) {
+        String gidNumber = null;
+        String uidNumber = null;
+        Attribute gidAttribute = result.getAttributes().get(POSIX_GIDNUMBER);
+        Attribute uidAttribute = result.getAttributes().get(POSIX_UIDNUMBER);
+        if (gidAttribute != null) {
+          gidNumber = gidAttribute.get().toString();
+        }
+        if (uidAttribute != null) {
+          uidNumber = uidAttribute.get().toString();
+        }
+        if (uidNumber != null && gidNumber != null) {
+          groupResults =
+              ctx.search(baseDN,
+                  "(&"+ groupSearchFilter + "(|(" + POSIX_GIDNUMBER + "={0})" +
+                      "(" + groupMemberAttr + "={1})))",
+                  new Object[] { gidNumber, uidNumber },
+                  SEARCH_CONTROLS);
+        }
+      } else {
+        groupResults =
+            ctx.search(baseDN,
+                "(&" + groupSearchFilter + "(" + groupMemberAttr + "={0}))",
+                new Object[]{userDn},
+                SEARCH_CONTROLS);
+      }
+      if (groupResults != null) {
+        while (groupResults.hasMoreElements()) {
+          SearchResult groupResult = groupResults.nextElement();
+          Attribute groupName = groupResult.getAttributes().get(groupNameAttr);
+          groups.add(groupName.get().toString());
+        }
       }
     }
 
@@ -334,6 +368,8 @@ public class LdapGroupsMapping
         conf.get(GROUP_SEARCH_FILTER_KEY, GROUP_SEARCH_FILTER_DEFAULT);
     userSearchFilter =
         conf.get(USER_SEARCH_FILTER_KEY, USER_SEARCH_FILTER_DEFAULT);
+    isPosix = groupSearchFilter.contains(POSIX_GROUP) && userSearchFilter
+        .contains(POSIX_ACCOUNT);
     groupMemberAttr =
         conf.get(GROUP_MEMBERSHIP_ATTR_KEY, GROUP_MEMBERSHIP_ATTR_DEFAULT);
     groupNameAttr =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82128774/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
index a021a8a..17a14d1 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
@@ -29,13 +29,7 @@ import java.util.Arrays;
 import java.util.List;
 
 import javax.naming.CommunicationException;
-import javax.naming.NamingEnumeration;
 import javax.naming.NamingException;
-import javax.naming.directory.Attribute;
-import javax.naming.directory.Attributes;
-import javax.naming.directory.BasicAttribute;
-import javax.naming.directory.BasicAttributes;
-import javax.naming.directory.DirContext;
 import javax.naming.directory.SearchControls;
 import javax.naming.directory.SearchResult;
 
@@ -49,46 +43,12 @@ import org.junit.Before;
 import org.junit.Test;
 
 @SuppressWarnings("unchecked")
-public class TestLdapGroupsMapping {
-  private DirContext mockContext;
-  
-  private LdapGroupsMapping mappingSpy = spy(new LdapGroupsMapping());
-  private NamingEnumeration mockUserNamingEnum = mock(NamingEnumeration.class);
-  private NamingEnumeration mockGroupNamingEnum = mock(NamingEnumeration.class);
-  private String[] testGroups = new String[] {"group1", "group2"};
-  
+public class TestLdapGroupsMapping extends TestLdapGroupsMappingBase {
   @Before
   public void setupMocks() throws NamingException {
-    mockContext = mock(DirContext.class);
-    doReturn(mockContext).when(mappingSpy).getDirContext();
-            
     SearchResult mockUserResult = mock(SearchResult.class);
-    // We only ever call hasMoreElements once for the user NamingEnum, so 
-    // we can just have one return value
-    when(mockUserNamingEnum.hasMoreElements()).thenReturn(true);
     when(mockUserNamingEnum.nextElement()).thenReturn(mockUserResult);
     when(mockUserResult.getNameInNamespace()).thenReturn("CN=some_user,DC=test,DC=com");
-    
-    SearchResult mockGroupResult = mock(SearchResult.class);
-    // We're going to have to define the loop here. We want two iterations,
-    // to get both the groups
-    when(mockGroupNamingEnum.hasMoreElements()).thenReturn(true, true, false);
-    when(mockGroupNamingEnum.nextElement()).thenReturn(mockGroupResult);
-    
-    // Define the attribute for the name of the first group
-    Attribute group1Attr = new BasicAttribute("cn");
-    group1Attr.add(testGroups[0]);
-    Attributes group1Attrs = new BasicAttributes();
-    group1Attrs.put(group1Attr);
-    
-    // Define the attribute for the name of the second group
-    Attribute group2Attr = new BasicAttribute("cn");
-    group2Attr.add(testGroups[1]);
-    Attributes group2Attrs = new BasicAttributes();
-    group2Attrs.put(group2Attr);
-    
-    // This search result gets reused, so return group1, then group2
-    when(mockGroupResult.getAttributes()).thenReturn(group1Attrs, group2Attrs);
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82128774/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java
new file mode 100644
index 0000000..c54ac4c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingBase.java
@@ -0,0 +1,77 @@
+/**
+ * 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.security;
+
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.Attributes;
+import javax.naming.directory.BasicAttribute;
+import javax.naming.directory.BasicAttributes;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.SearchResult;
+
+import org.junit.Before;
+
+public class TestLdapGroupsMappingBase {
+  protected DirContext mockContext;
+
+  protected LdapGroupsMapping mappingSpy = spy(new LdapGroupsMapping());
+  protected NamingEnumeration mockUserNamingEnum =
+      mock(NamingEnumeration.class);
+  protected NamingEnumeration mockGroupNamingEnum =
+      mock(NamingEnumeration.class);
+  protected String[] testGroups = new String[] {"group1", "group2"};
+
+  @Before
+  public void setupMocksBase() throws NamingException {
+    mockContext = mock(DirContext.class);
+    doReturn(mockContext).when(mappingSpy).getDirContext();
+
+    // We only ever call hasMoreElements once for the user NamingEnum, so
+    // we can just have one return value
+    when(mockUserNamingEnum.hasMoreElements()).thenReturn(true);
+
+    SearchResult mockGroupResult = mock(SearchResult.class);
+    // We're going to have to define the loop here. We want two iterations,
+    // to get both the groups
+    when(mockGroupNamingEnum.hasMoreElements()).thenReturn(true, true, false);
+    when(mockGroupNamingEnum.nextElement()).thenReturn(mockGroupResult);
+
+    // Define the attribute for the name of the first group
+    Attribute group1Attr = new BasicAttribute("cn");
+    group1Attr.add(testGroups[0]);
+    Attributes group1Attrs = new BasicAttributes();
+    group1Attrs.put(group1Attr);
+
+    // Define the attribute for the name of the second group
+    Attribute group2Attr = new BasicAttribute("cn");
+    group2Attr.add(testGroups[1]);
+    Attributes group2Attrs = new BasicAttributes();
+    group2Attrs.put(group2Attr);
+
+    // This search result gets reused, so return group1, then group2
+    when(mockGroupResult.getAttributes()).thenReturn(group1Attrs, group2Attrs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82128774/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
new file mode 100644
index 0000000..1d1a354
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
@@ -0,0 +1,103 @@
+/**
+ * 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.security;
+
+import static org.mockito.Matchers.anyString;
+
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.contains;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.Attributes;
+import javax.naming.directory.SearchControls;
+import javax.naming.directory.SearchResult;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+@SuppressWarnings("unchecked")
+public class TestLdapGroupsMappingWithPosixGroup
+  extends TestLdapGroupsMappingBase {
+
+  @Before
+  public void setupMocks() throws NamingException {
+    SearchResult mockUserResult = mock(SearchResult.class);
+    when(mockUserNamingEnum.nextElement()).thenReturn(mockUserResult);
+
+    Attribute mockUidAttr = mock(Attribute.class);
+    Attribute mockGidAttr = mock(Attribute.class);
+    Attributes mockAttrs = mock(Attributes.class);
+
+    when(mockUidAttr.get()).thenReturn("700");
+    when(mockGidAttr.get()).thenReturn("600");
+    when(mockAttrs.get(eq("uidNumber"))).thenReturn(mockUidAttr);
+    when(mockAttrs.get(eq("gidNumber"))).thenReturn(mockGidAttr);
+
+    when(mockUserResult.getAttributes()).thenReturn(mockAttrs);
+  }
+
+  @Test
+  public void testGetGroups() throws IOException, NamingException {
+    // The search functionality of the mock context is reused, so we will
+    // return the user NamingEnumeration first, and then the group
+    when(mockContext.search(anyString(), contains("posix"),
+        any(Object[].class), any(SearchControls.class)))
+        .thenReturn(mockUserNamingEnum, mockGroupNamingEnum);
+
+    doTestGetGroups(Arrays.asList(testGroups), 2);
+  }
+
+  private void doTestGetGroups(List<String> expectedGroups, int searchTimes)
+      throws IOException, NamingException {
+    Configuration conf = new Configuration();
+    // Set this, so we don't throw an exception
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
+    conf.set(LdapGroupsMapping.GROUP_SEARCH_FILTER_KEY,
+        "(objectClass=posixGroup)(cn={0})");
+    conf.set(LdapGroupsMapping.USER_SEARCH_FILTER_KEY,
+        "(objectClass=posixAccount)");
+    conf.set(LdapGroupsMapping.GROUP_MEMBERSHIP_ATTR_KEY, "memberUid");
+    conf.set(LdapGroupsMapping.GROUP_NAME_ATTR_KEY, "cn");
+
+    mappingSpy.setConf(conf);
+    // Username is arbitrary, since the spy is mocked to respond the same,
+    // regardless of input
+    List<String> groups = mappingSpy.getGroups("some_user");
+
+    Assert.assertEquals(expectedGroups, groups);
+
+    // We should have searched for a user, and then two groups
+    verify(mockContext, times(searchTimes)).search(anyString(),
+                                         anyString(),
+                                         any(Object[].class),
+                                         any(SearchControls.class));
+  }
+}


[27/50] [abbrv] hadoop git commit: YARN-1453. [JDK8] Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA, Andrew Purtell, and Allen Wittenauer.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index 9923806..bfe10d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -349,7 +349,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
    * Set the NM token cache for the <code>AMRMClient</code>. This cache must
    * be shared with the {@link NMClient} used to manage containers for the
    * <code>AMRMClient</code>
-   * <p/>
+   * <p>
    * If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
    * singleton instance will be used.
    *
@@ -363,7 +363,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
    * Get the NM token cache of the <code>AMRMClient</code>. This cache must be
    * shared with the {@link NMClient} used to manage containers for the
    * <code>AMRMClient</code>.
-   * <p/>
+   * <p>
    * If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
    * singleton instance will be used.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
index 721728e..08b911b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMClient.java
@@ -125,7 +125,7 @@ public abstract class NMClient extends AbstractService {
    * Set the NM Token cache of the <code>NMClient</code>. This cache must be
    * shared with the {@link AMRMClient} that requested the containers managed
    * by this <code>NMClient</code>
-   * <p/>
+   * <p>
    * If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
    * singleton instance will be used.
    *
@@ -139,7 +139,7 @@ public abstract class NMClient extends AbstractService {
    * Get the NM token cache of the <code>NMClient</code>. This cache must be
    * shared with the {@link AMRMClient} that requested the containers managed
    * by this <code>NMClient</code>
-   * <p/>
+   * <p>
    * If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
    * singleton instance will be used.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMTokenCache.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMTokenCache.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMTokenCache.java
index 0e7356f..0c349cc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMTokenCache.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/NMTokenCache.java
@@ -34,26 +34,26 @@ import com.google.common.annotations.VisibleForTesting;
 /**
  * NMTokenCache manages NMTokens required for an Application Master
  * communicating with individual NodeManagers.
- * <p/>
+ * <p>
  * By default Yarn client libraries {@link AMRMClient} and {@link NMClient} use
  * {@link #getSingleton()} instance of the cache.
  * <ul>
- * <li>Using the singleton instance of the cache is appropriate when running a
- * single ApplicationMaster in the same JVM.</li>
- * <li>When using the singleton, users don't need to do anything special,
- * {@link AMRMClient} and {@link NMClient} are already set up to use the default
- * singleton {@link NMTokenCache}</li>
+ *   <li>
+ *     Using the singleton instance of the cache is appropriate when running a
+ *     single ApplicationMaster in the same JVM.
+ *   </li>
+ *   <li>
+ *     When using the singleton, users don't need to do anything special,
+ *     {@link AMRMClient} and {@link NMClient} are already set up to use the
+ *     default singleton {@link NMTokenCache}
+ *     </li>
  * </ul>
- * <p/>
  * If running multiple Application Masters in the same JVM, a different cache
  * instance should be used for each Application Master.
- * <p/>
  * <ul>
- * <li>
- * If using the {@link AMRMClient} and the {@link NMClient}, setting up and using
- * an instance cache is as follows:
- * <p/>
- * 
+ *   <li>
+ *     If using the {@link AMRMClient} and the {@link NMClient}, setting up
+ *     and using an instance cache is as follows:
  * <pre>
  *   NMTokenCache nmTokenCache = new NMTokenCache();
  *   AMRMClient rmClient = AMRMClient.createAMRMClient();
@@ -61,12 +61,10 @@ import com.google.common.annotations.VisibleForTesting;
  *   nmClient.setNMTokenCache(nmTokenCache);
  *   ...
  * </pre>
- * </li>
- * <li>
- * If using the {@link AMRMClientAsync} and the {@link NMClientAsync}, setting up
- * and using an instance cache is as follows:
- * <p/>
- * 
+ *   </li>
+ *   <li>
+ *     If using the {@link AMRMClientAsync} and the {@link NMClientAsync},
+ *     setting up and using an instance cache is as follows:
  * <pre>
  *   NMTokenCache nmTokenCache = new NMTokenCache();
  *   AMRMClient rmClient = AMRMClient.createAMRMClient();
@@ -76,13 +74,11 @@ import com.google.common.annotations.VisibleForTesting;
  *   NMClientAsync nmClientAsync = new NMClientAsync("nmClient", nmClient, [NM_CALLBACK]);
  *   ...
  * </pre>
- * </li>
- * <li>
- * If using {@link ApplicationMasterProtocol} and
- * {@link ContainerManagementProtocol} directly, setting up and using an
- * instance cache is as follows:
- * <p/>
- * 
+ *   </li>
+ *   <li>
+ *     If using {@link ApplicationMasterProtocol} and
+ *     {@link ContainerManagementProtocol} directly, setting up and using an
+ *     instance cache is as follows:
  * <pre>
  *   NMTokenCache nmTokenCache = new NMTokenCache();
  *   ...
@@ -100,12 +96,12 @@ import com.google.common.annotations.VisibleForTesting;
  *   nmPro.startContainer(container, containerContext);
  *   ...
  * </pre>
- * </li>
+ *   </li>
  * </ul>
- * It is also possible to mix the usage of a client (<code>AMRMClient</code> or
- * <code>NMClient</code>, or the async versions of them) with a protocol proxy (
- * <code>ContainerManagementProtocolProxy</code> or
- * <code>ApplicationMasterProtocol</code>).
+ * It is also possible to mix the usage of a client ({@code AMRMClient} or
+ * {@code NMClient}, or the async versions of them) with a protocol proxy
+ * ({@code ContainerManagementProtocolProxy} or
+ * {@code ApplicationMasterProtocol}).
  */
 @Public
 @Evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
index d96761a..f617731 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
@@ -32,14 +32,12 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -58,8 +56,10 @@ import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationIdNotProvidedException;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
+import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 
@@ -171,7 +171,6 @@ public abstract class YarnClient extends AbstractService {
    * <li>original tracking URL - set to "N/A"</li>
    * <li>resource usage report - all values are -1</li>
    * </ul>
-   * </p>
    * 
    * @param appId
    *          {@link ApplicationId} of the application that needs a report
@@ -184,20 +183,20 @@ public abstract class YarnClient extends AbstractService {
 
   /**
    * Get the AMRM token of the application.
-   * <p/>
+   * <p>
    * The AMRM token is required for AM to RM scheduling operations. For 
    * managed Application Masters Yarn takes care of injecting it. For unmanaged
    * Applications Masters, the token must be obtained via this method and set
    * in the {@link org.apache.hadoop.security.UserGroupInformation} of the
    * current user.
-   * <p/>
+   * <p>
    * The AMRM token will be returned only if all the following conditions are
    * met:
-   * <li>
-   *   <ul>the requester is the owner of the ApplicationMaster</ul>
-   *   <ul>the application master is an unmanaged ApplicationMaster</ul>
-   *   <ul>the application master is in ACCEPTED state</ul>
-   * </li>
+   * <ul>
+   *   <li>the requester is the owner of the ApplicationMaster</li>
+   *   <li>the application master is an unmanaged ApplicationMaster</li>
+   *   <li>the application master is in ACCEPTED state</li>
+   * </ul>
    * Else this method returns NULL.
    *
    * @param appId {@link ApplicationId} of the application to get the AMRM token
@@ -415,7 +414,7 @@ public abstract class YarnClient extends AbstractService {
    *          a report
    * @return application attempt report
    * @throws YarnException
-   * @throws {@link ApplicationAttemptNotFoundException} if application attempt
+   * @throws ApplicationAttemptNotFoundException if application attempt
    *         not found
    * @throws IOException
    */
@@ -450,7 +449,7 @@ public abstract class YarnClient extends AbstractService {
    *          {@link ContainerId} of the container that needs a report
    * @return container report
    * @throws YarnException
-   * @throws {@link ContainerNotFoundException} if container not found.
+   * @throws ContainerNotFoundException if container not found.
    * @throws IOException
    */
   public abstract ContainerReport getContainerReport(ContainerId containerId)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
index e2da664..67a5494 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
@@ -344,7 +344,7 @@ public class CommonNodeLabelsManager extends AbstractService {
   /**
    * add more labels to nodes
    * 
-   * @param addedLabelsToNode node -> labels map
+   * @param addedLabelsToNode node {@literal ->} labels map
    */
   public void addLabelsToNode(Map<NodeId, Set<String>> addedLabelsToNode)
       throws IOException {
@@ -614,7 +614,7 @@ public class CommonNodeLabelsManager extends AbstractService {
    * remove labels from nodes, labels being removed most be contained by these
    * nodes
    * 
-   * @param removeLabelsFromNode node -> labels map
+   * @param removeLabelsFromNode node {@literal ->} labels map
    */
   public void
       removeLabelsFromNode(Map<NodeId, Set<String>> removeLabelsFromNode)
@@ -668,7 +668,7 @@ public class CommonNodeLabelsManager extends AbstractService {
   /**
    * replace labels to nodes
    * 
-   * @param replaceLabelsToNode node -> labels map
+   * @param replaceLabelsToNode node {@literal ->} labels map
    */
   public void replaceLabelsOnNode(Map<NodeId, Set<String>> replaceLabelsToNode)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
index 857d81b..4a34a09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
@@ -35,7 +35,7 @@ public abstract class NodeLabelsStore implements Closeable {
   }
   
   /**
-   * Store node -> label
+   * Store node {@literal ->} label
    */
   public abstract void updateNodeToLabelsMappings(
       Map<NodeId, Set<String>> nodeToLabels) throws IOException;
@@ -54,7 +54,6 @@ public abstract class NodeLabelsStore implements Closeable {
   
   /**
    * Recover labels and node to labels mappings from store
-   * @param conf
    */
   public abstract void recover() throws IOException;
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
index 4cacfca..4daaa68 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java
@@ -93,7 +93,6 @@ public class ApplicationACLsManager {
    * @param applicationAccessType
    * @param applicationOwner
    * @param applicationId
-   * @throws AccessControlException
    */
   public boolean checkAccess(UserGroupInformation callerUGI,
       ApplicationAccessType applicationAccessType, String applicationOwner,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
index e6f23f0..3343a17 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java
@@ -88,7 +88,7 @@ public final class StringHelper {
   }
 
   /**
-   * Join on slash & colon (e.g., path args in routing spec)
+   * Join on slash and colon (e.g., path args in routing spec)
    * @param args to join
    * @return args joined with /:
    */
@@ -116,7 +116,7 @@ public final class StringHelper {
   }
 
   /**
-   * Split on space & trim results.
+   * Split on space and trim results.
    * @param s the string to split
    * @return an iterable of strings
    */
@@ -125,7 +125,7 @@ public final class StringHelper {
   }
 
   /**
-   * Split on _ & trim results
+   * Split on _ and trim results
    * @param s the string to split
    * @return an iterable of strings
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
index 1b5840f..bda24aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
@@ -52,13 +52,13 @@ import com.google.inject.servlet.GuiceFilter;
 /**
  * Helpers to create an embedded webapp.
  *
- * <h4>Quick start:</h4>
+ * <b>Quick start:</b>
  * <pre>
  *   WebApp wa = WebApps.$for(myApp).start();</pre>
  * Starts a webapp with default routes binds to 0.0.0.0 (all network interfaces)
  * on an ephemeral port, which can be obtained with:<pre>
  *   int port = wa.port();</pre>
- * <h4>With more options:</h4>
+ * <b>With more options:</b>
  * <pre>
  *   WebApp wa = WebApps.$for(myApp).at(address, port).
  *                        with(configuration).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
index 06a56d8..858b6b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java
@@ -116,10 +116,10 @@ public class RegistryUtils {
   }
 
   /**
-   * Create a path to a service under a user & service class
+   * Create a path to a service under a user and service class
    * @param user username or ""
    * @param serviceClass service name
-   * @param serviceName service name unique for that user & service class
+   * @param serviceName service name unique for that user and service class
    * @return a full path
    */
   public static String servicePath(String user,
@@ -135,7 +135,7 @@ public class RegistryUtils {
    * Create a path for listing components under a service
    * @param user username or ""
    * @param serviceClass service name
-   * @param serviceName service name unique for that user & service class
+   * @param serviceName service name unique for that user and service class
    * @return a full path
    */
   public static String componentListPath(String user,
@@ -149,7 +149,7 @@ public class RegistryUtils {
    * Create the path to a service record for a component
    * @param user username or ""
    * @param serviceClass service name
-   * @param serviceName service name unique for that user & service class
+   * @param serviceName service name unique for that user and service class
    * @param componentName unique name/ID of the component
    * @return a full path
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
index db03936..44cefed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/RegistryOperationsClient.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.registry.client.impl.zk.RegistryOperationsService;
  *
  * For SASL, the client must be operating in the context of an authed user.
  *
- * For id:pass the client must have the relevant id & password, SASL is
+ * For id:pass the client must have the relevant id and password, SASL is
  * not used even if the client has credentials.
  *
  * For anonymous, nothing is used.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
index d81f24b..edcf085 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/ZookeeperConfigOptions.java
@@ -71,14 +71,13 @@ public interface ZookeeperConfigOptions {
    * The SASL client username: {@value}.
    * <p>
    * Set this to the <i>short</i> name of the client, e.g, "user",
-   * not <code>user/host</code>, or <code>user/host@REALM</code>
+   * not {@code user/host}, or {@code user/host@REALM}
    */
   String PROP_ZK_SASL_CLIENT_USERNAME = "zookeeper.sasl.client.username";
 
   /**
    * The SASL Server context, referring to a context in the JVM's
    * JAAS context file: {@value}
-   * <p>
    */
   String PROP_ZK_SERVER_SASL_CONTEXT =
       ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
index 3fa0c19..88e9d67 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/MicroZookeeperService.java
@@ -47,16 +47,16 @@ import java.net.UnknownHostException;
 /**
  * This is a small, localhost Zookeeper service instance that is contained
  * in a YARN service...it's been derived from Apache Twill.
- *
+ * <p>
  * It implements {@link RegistryBindingSource} and provides binding information,
- * <i>once started</i>. Until <code>start()</code> is called, the hostname &
+ * <i>once started</i>. Until {@link #start()} is called, the hostname and
  * port may be undefined. Accordingly, the service raises an exception in this
  * condition.
- *
+ * <p>
  * If you wish to chain together a registry service with this one under
- * the same <code>CompositeService</code>, this service must be added
+ * the same {@code CompositeService}, this service must be added
  * as a child first.
- *
+ * <p>
  * It also sets the configuration parameter
  * {@link RegistryConstants#KEY_REGISTRY_ZK_QUORUM}
  * to its connection string. Any code with access to the service configuration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
index 85d24b3..fe2a0a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/package-info.java
@@ -19,9 +19,10 @@
 /**
  * Basic services for the YARN registry
  * <ul>
- *   <li>The {@link org.apache.hadoop.registry.server.services.RegistryAdminService}</ol>
- *   extends the shared Yarn Registry client with registry setup and
- *   (potentially asynchronous) administrative actions.
+ *   <li>
+ *     The {@link org.apache.hadoop.registry.server.services.RegistryAdminService}
+ *     extends the shared Yarn Registry client with registry setup and
+ *     (potentially asynchronous) administrative actions.
  *   </li>
  *   <li>
  *     The {@link org.apache.hadoop.registry.server.services.MicroZookeeperService}
@@ -33,8 +34,6 @@
  *     extends the standard YARN composite service by making its add and remove
  *     methods public. It is a utility service used in parts of the codebase
  *   </li>
- *
  * </ul>
- *
  */
 package org.apache.hadoop.registry.server.services;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
index 39c10fb..1ee8181 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
@@ -45,17 +45,15 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import com.google.common.annotations.VisibleForTesting;
 
 /**
- * <p>
  * Initializes {@link TimelineAuthenticationFilter} which provides support for
  * Kerberos HTTP SPNEGO authentication.
- * <p/>
  * <p>
  * It enables Kerberos HTTP SPNEGO plus delegation token authentication for the
  * timeline server.
- * <p/>
- * Refer to the <code>core-default.xml</code> file, after the comment 'HTTP
+ * <p>
+ * Refer to the {@code core-default.xml} file, after the comment 'HTTP
  * Authentication' for details on the configuration options. All related
- * configuration properties have 'hadoop.http.authentication.' as prefix.
+ * configuration properties have {@code hadoop.http.authentication.} as prefix.
  */
 public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
 
@@ -71,14 +69,11 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
   Map<String, String> filterConfig;
 
   /**
-   * <p>
    * Initializes {@link TimelineAuthenticationFilter}
-   * <p/>
    * <p>
    * Propagates to {@link TimelineAuthenticationFilter} configuration all YARN
    * configuration properties prefixed with
-   * "yarn.timeline-service.authentication."
-   * </p>
+   * {@code yarn.timeline-service.authentication.}
    * 
    * @param container
    *          The filter container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java
index e675308..ba130c6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/lib/ZKClient.java
@@ -40,7 +40,7 @@ public class ZKClient {
    * the zookeeper client library to 
    * talk to zookeeper 
    * @param string the host
-   * @throws throws IOException
+   * @throws IOException
    */
   public ZKClient(String string) throws IOException {
     zkClient = new ZooKeeper(string, 30000, new ZKWatcher());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java
index 0e3d7e4..366c32c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java
@@ -52,7 +52,8 @@ public abstract class RegisterNodeManagerRequest {
    * We introduce this here because currently YARN RM doesn't persist nodes info
    * for application running. When RM restart happened, we cannot determinate if
    * a node should do application cleanup (like log-aggregation, status update,
-   * etc.) or not. <p/>
+   * etc.) or not.
+   * <p>
    * When we have this running application list in node manager register
    * request, we can recover nodes info for running applications. And then we
    * can take actions accordingly

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java
index bc5825a..b21b880 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeHealthStatus.java
@@ -26,19 +26,17 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p><code>NodeHealthStatus</code> is a summary of the health status of the
- * node.</p>
- *
- * <p>It includes information such as:
- *   <ul>
- *     <li>
- *       An indicator of whether the node is healthy, as determined by the 
- *       health-check script.
- *     </li>
- *     <li>The previous time at which the health status was reported.</li>
- *     <li>A diagnostic report on the health status.</li>
- *   </ul>
- * </p>
+ * {@code NodeHealthStatus} is a summary of the health status of the node.
+ * <p>
+ * It includes information such as:
+ * <ul>
+ *   <li>
+ *     An indicator of whether the node is healthy, as determined by the
+ *     health-check script.
+ *   </li>
+ *   <li>The previous time at which the health status was reported.</li>
+ *   <li>A diagnostic report on the health status.</li>
+ * </ul>
  * 
  * @see NodeReport
  * @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
index 248a393..377fd1d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
@@ -102,13 +102,15 @@ public abstract class ContainerExecutor implements Configurable {
   
   /**
    * Prepare the environment for containers in this application to execute.
+   * <pre>
    * For $x in local.dirs
    *   create $x/$user/$appId
-   * Copy $nmLocal/appTokens -> $N/$user/$appId
+   * Copy $nmLocal/appTokens {@literal ->} $N/$user/$appId
    * For $rsrc in private resources
-   *   Copy $rsrc -> $N/$user/filecache/[idef]
+   *   Copy $rsrc {@literal ->} $N/$user/filecache/[idef]
    * For $rsrc in job resources
-   *   Copy $rsrc -> $N/$user/$appId/filecache/idef
+   *   Copy $rsrc {@literal ->} $N/$user/$appId/filecache/idef
+   * </pre>
    * @param user user name of application owner
    * @param appId id of the application
    * @param nmPrivateContainerTokens path to localized credentials, rsrc by NM

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
index 431cf5d..77db1e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
@@ -65,11 +65,11 @@ public class NodeManagerHardwareUtils {
   }
 
   /**
-   * Gets the percentage of physical CPU that is configured for YARN containers
-   * This is percent > 0 and <= 100  based on
-   * YarnConfiguration.NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT
+   * Gets the percentage of physical CPU that is configured for YARN containers.
+   * This is percent {@literal >} 0 and {@literal <=} 100 based on
+   * {@link YarnConfiguration#NM_RESOURCE_PERCENTAGE_PHYSICAL_CPU_LIMIT}
    * @param conf Configuration object
-   * @return percent > 0 and <= 100
+   * @return percent {@literal >} 0 and {@literal <=} 100
    */
   public static int getNodeCpuPercentage(Configuration conf) {
     int nodeCpuPercentage =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
index cf8c2bb..b85174e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
@@ -216,11 +216,14 @@ public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
   /**
    * Return the flag which indicates whether the attempt failure should be
    * counted to attempt retry count.
-   * <ul>
+   * <p>
    * There failure types should not be counted to attempt retry count:
-   * <li>preempted by the scheduler.</li>
-   * <li>hardware failures, such as NM failing, lost NM and NM disk errors.</li>
-   * <li>killed by RM because of RM restart or failover.</li>
+   * <ul>
+   *   <li>preempted by the scheduler.</li>
+   *   <li>
+   *     hardware failures, such as NM failing, lost NM and NM disk errors.
+   *   </li>
+   *   <li>killed by RM because of RM restart or failover.</li>
    * </ul>
    */
   boolean shouldCountTowardsMaxAttemptRetry();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 957e8f6..2901134 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -114,7 +114,7 @@ public abstract class SchedulerNode {
 
   /**
    * Get the name of the node for scheduling matching decisions.
-   * <p/>
+   * <p>
    * Typically this is the 'hostname' reported by the node, but it could be
    * configured to be 'hostname:port' reported by the node via the
    * {@link YarnConfiguration#RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME} constant.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
index 65d6859..248cc08 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
@@ -194,8 +194,7 @@ public class SchedulerUtils {
    * Utility method to validate a resource request, by insuring that the
    * requested memory/vcore is non-negative and not greater than max
    * 
-   * @throws <code>InvalidResourceRequestException</code> when there is invalid
-   *         request
+   * @throws InvalidResourceRequestException when there is invalid request
    */
   public static void validateResourceRequest(ResourceRequest resReq,
       Resource maximumResource, String queueName, YarnScheduler scheduler)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
index 3bea985..f4fad32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
@@ -71,7 +71,7 @@ public class ComputeFairShares {
    * fair shares. The min and max shares and of the Schedulables are assumed to
    * be set beforehand. We compute the fairest possible allocation of shares to
    * the Schedulables that respects their min and max shares.
-   * 
+   * <p>
    * To understand what this method does, we must first define what weighted
    * fair sharing means in the presence of min and max shares. If there
    * were no minimum or maximum shares, then weighted fair sharing would be
@@ -79,30 +79,31 @@ public class ComputeFairShares {
    * Schedulable and all slots were assigned. Minimum and maximum shares add a
    * further twist - Some Schedulables may have a min share higher than their
    * assigned share or a max share lower than their assigned share.
-   * 
+   * <p>
    * To deal with these possibilities, we define an assignment of slots as being
    * fair if there exists a ratio R such that: Schedulables S where S.minShare
-   * > R * S.weight are given share S.minShare - Schedulables S where S.maxShare
-   * < R * S.weight are given S.maxShare - All other Schedulables S are
-   * assigned share R * S.weight - The sum of all the shares is totalSlots.
-   * 
+   * {@literal >} R * S.weight are given share S.minShare - Schedulables S
+   * where S.maxShare {@literal <} R * S.weight are given S.maxShare -
+   * All other Schedulables S are assigned share R * S.weight -
+   * The sum of all the shares is totalSlots.
+   * <p>
    * We call R the weight-to-slots ratio because it converts a Schedulable's
    * weight to the number of slots it is assigned.
-   * 
+   * <p>
    * We compute a fair allocation by finding a suitable weight-to-slot ratio R.
    * To do this, we use binary search. Given a ratio R, we compute the number of
    * slots that would be used in total with this ratio (the sum of the shares
    * computed using the conditions above). If this number of slots is less than
    * totalSlots, then R is too small and more slots could be assigned. If the
    * number of slots is more than totalSlots, then R is too large.
-   * 
+   * <p>
    * We begin the binary search with a lower bound on R of 0 (which means that
    * all Schedulables are only given their minShare) and an upper bound computed
    * to be large enough that too many slots are given (by doubling R until we
    * use more than totalResources resources). The helper method
    * resourceUsedWithWeightToResourceRatio computes the total resources used with a
    * given value of R.
-   * 
+   * <p>
    * The running time of this algorithm is linear in the number of Schedulables,
    * because resourceUsedWithWeightToResourceRatio is linear-time and the number of
    * iterations of binary search is a constant (dependent on desired precision).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
index dfcceb8..cb456d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
@@ -364,7 +364,6 @@ public class DelegationTokenRenewer extends AbstractService {
    * @param shouldCancelAtEnd true if tokens should be canceled when the app is
    * done else false. 
    * @param user user
-   * @throws IOException
    */
   public void addApplicationAsync(ApplicationId applicationId, Credentials ts,
       boolean shouldCancelAtEnd, String user) {
@@ -634,7 +633,6 @@ public class DelegationTokenRenewer extends AbstractService {
   
   /**
    * removing failed DT
-   * @param applicationId
    */
   private void removeFailedDelegationToken(DelegationTokenToRenew t) {
     ApplicationId applicationId = t.applicationId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3da9a97c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
index d678edf..e130225 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUriUtils.java
@@ -181,7 +181,7 @@ public class ProxyUriUtils {
   
   /**
    * Returns the scheme if present in the url
-   * eg. "https://issues.apache.org/jira/browse/YARN" > "https"
+   * eg. "https://issues.apache.org/jira/browse/YARN" {@literal ->} "https"
    */
   public static String getSchemeFromUrl(String url) {
     int index = 0;


[03/50] [abbrv] hadoop git commit: YARN-1884. Added nodeHttpAddress into ContainerReport and fixed the link to NM web page. Contributed by Xuan Gong.

Posted by zh...@apache.org.
YARN-1884. Added nodeHttpAddress into ContainerReport and fixed the link to NM web page. Contributed by Xuan Gong.


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

Branch: refs/heads/HDFS-7285
Commit: 85f6d67fa78511f255fcfa810afc9a156a7b483b
Parents: 7a346bc
Author: Zhijie Shen <zj...@apache.org>
Authored: Wed Mar 11 19:35:19 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Wed Mar 11 19:35:19 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../yarn/api/records/ContainerReport.java       | 17 ++++++++++++++-
 .../src/main/proto/yarn_protos.proto            |  1 +
 .../hadoop/yarn/client/cli/ApplicationCLI.java  | 11 +++++++---
 .../hadoop/yarn/client/ProtocolHATestBase.java  |  4 ++--
 .../yarn/client/api/impl/TestAHSClient.java     |  6 ++++--
 .../yarn/client/api/impl/TestYarnClient.java    | 16 ++++++++------
 .../hadoop/yarn/client/cli/TestYarnCLI.java     | 18 ++++++++++++----
 .../records/impl/pb/ContainerReportPBImpl.java  | 19 +++++++++++++++++
 .../ApplicationHistoryManagerImpl.java          |  2 +-
 ...pplicationHistoryManagerOnTimelineStore.java | 10 ++++++++-
 .../metrics/ContainerMetricsConstants.java      |  2 ++
 .../yarn/server/webapp/AppAttemptBlock.java     | 12 ++++++-----
 .../hadoop/yarn/server/webapp/AppBlock.java     | 11 ++++------
 .../yarn/server/webapp/ContainerBlock.java      |  7 ++++++-
 .../yarn/server/webapp/dao/ContainerInfo.java   |  5 +++++
 .../metrics/ContainerCreatedEvent.java          |  8 ++++++-
 .../metrics/SystemMetricsPublisher.java         |  5 ++++-
 .../rmcontainer/RMContainer.java                |  1 +
 .../rmcontainer/RMContainerImpl.java            | 22 ++++++++++++++++++--
 .../metrics/TestSystemMetricsPublisher.java     |  5 ++++-
 21 files changed, 147 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8930045..969c6a1 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -752,6 +752,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3295. Fix documentation nits found in markdown conversion.
     (Masatake Iwasaki via ozawa)
 
+    YARN-1884. Added nodeHttpAddress into ContainerReport and fixed the link to NM
+    web page. (Xuan Gong via zjshen)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
index 4cce77f..72b8edf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.util.Records;
  * <li>{@link ContainerState} of the container.</li>
  * <li>Diagnostic information in case of errors.</li>
  * <li>Log URL.</li>
+ * <li>nodeHttpAddress</li>
  * </ul>
  * </p>
  * 
@@ -54,7 +55,8 @@ public abstract class ContainerReport {
   public static ContainerReport newInstance(ContainerId containerId,
       Resource allocatedResource, NodeId assignedNode, Priority priority,
       long creationTime, long finishTime, String diagnosticInfo, String logUrl,
-      int containerExitStatus, ContainerState containerState) {
+      int containerExitStatus, ContainerState containerState,
+      String nodeHttpAddress) {
     ContainerReport report = Records.newRecord(ContainerReport.class);
     report.setContainerId(containerId);
     report.setAllocatedResource(allocatedResource);
@@ -66,6 +68,7 @@ public abstract class ContainerReport {
     report.setLogUrl(logUrl);
     report.setContainerExitStatus(containerExitStatus);
     report.setContainerState(containerState);
+    report.setNodeHttpAddress(nodeHttpAddress);
     return report;
   }
 
@@ -199,4 +202,16 @@ public abstract class ContainerReport {
   @Unstable
   public abstract void setContainerExitStatus(int containerExitStatus);
 
+  /**
+   * Get the Node Http address of the container
+   * 
+   * @return the node http address of the container
+   */
+  @Public
+  @Unstable
+  public abstract String getNodeHttpAddress();
+
+  @Private
+  @Unstable
+  public abstract void setNodeHttpAddress(String nodeHttpAddress);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 4e29d2f..90706ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -98,6 +98,7 @@ message ContainerReportProto {
   optional string log_url = 8;
   optional int32 container_exit_status = 9;
   optional ContainerStateProto container_state = 10;
+  optional string node_http_address = 11;
 }
 
 enum YarnApplicationStateProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index 108ad0b..dd4a949 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -63,7 +63,7 @@ public class ApplicationCLI extends YarnCLI {
     "%30s\t%20s\t%35s\t%35s"
       + System.getProperty("line.separator");
   private static final String CONTAINER_PATTERN = 
-    "%30s\t%20s\t%20s\t%20s\t%20s\t%35s"
+    "%30s\t%20s\t%20s\t%20s\t%20s\t%20s\t%35s"
       + System.getProperty("line.separator");
 
   private static final String APP_TYPE_CMD = "appTypes";
@@ -355,6 +355,9 @@ public class ApplicationCLI extends YarnCLI {
       containerReportStr.println(containerReport.getLogUrl());
       containerReportStr.print("\tHost : ");
       containerReportStr.println(containerReport.getAssignedNode());
+      containerReportStr.print("\tNodeHttpAddress : ");
+      containerReportStr.println(containerReport.getNodeHttpAddress() == null
+          ? "N/A" : containerReport.getNodeHttpAddress());
       containerReportStr.print("\tDiagnostics : ");
       containerReportStr.print(containerReport.getDiagnosticsInfo());
     } else {
@@ -595,7 +598,7 @@ public class ApplicationCLI extends YarnCLI {
         .getContainers(ConverterUtils.toApplicationAttemptId(appAttemptId));
     writer.println("Total number of containers " + ":" + appsReport.size());
     writer.printf(CONTAINER_PATTERN, "Container-Id", "Start Time",
-        "Finish Time", "State", "Host", "LOG-URL");
+        "Finish Time", "State", "Host", "Node Http Address", "LOG-URL");
     for (ContainerReport containerReport : appsReport) {
       writer.printf(
           CONTAINER_PATTERN,
@@ -603,7 +606,9 @@ public class ApplicationCLI extends YarnCLI {
           Times.format(containerReport.getCreationTime()),
           Times.format(containerReport.getFinishTime()),      
           containerReport.getContainerState(), containerReport
-              .getAssignedNode(), containerReport.getLogUrl());
+              .getAssignedNode(), containerReport.getNodeHttpAddress() == null
+                  ? "N/A" : containerReport.getNodeHttpAddress(),
+          containerReport.getLogUrl());
     }
     writer.flush();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
index 782bc43..f468bc1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
@@ -77,7 +77,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
-import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -715,7 +714,8 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
     public ContainerReport createFakeContainerReport() {
       return ContainerReport.newInstance(createFakeContainerId(), null,
           NodeId.newInstance("localhost", 0), null, 1000l, 1200l, "", "", 0,
-          ContainerState.COMPLETE);
+          ContainerState.COMPLETE,
+          "http://" + NodeId.newInstance("localhost", 0).toString());
     }
 
     public List<ContainerReport> createFakeContainerReports() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java
index a88189e..c3e3c41 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java
@@ -371,14 +371,16 @@ public class TestAHSClient {
           ContainerReport.newInstance(
             ContainerId.newContainerId(attempt.getApplicationAttemptId(), 1),
             null, NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234,
-            5678, "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
+            5678, "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE,
+            "http://" + NodeId.newInstance("host", 2345).toString());
       containerReports.add(container);
 
       ContainerReport container1 =
           ContainerReport.newInstance(
             ContainerId.newContainerId(attempt.getApplicationAttemptId(), 2),
             null, NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234,
-            5678, "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
+            5678, "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE,
+            "http://" + NodeId.newInstance("host", 2345).toString());
       containerReports.add(container1);
       containers.put(attempt.getApplicationAttemptId(), containerReports);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
index 9946506..de669f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
@@ -117,7 +117,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.UTCClock;
-import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -613,13 +612,15 @@ public class TestYarnClient {
       ContainerReport container = ContainerReport.newInstance(
           ContainerId.newContainerId(attempt.getApplicationAttemptId(), 1), null,
           NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
-          "diagnosticInfo", "logURL", 0, ContainerState.RUNNING);
+          "diagnosticInfo", "logURL", 0, ContainerState.RUNNING,
+          "http://" + NodeId.newInstance("host", 2345).toString());
       containerReports.add(container);
 
       ContainerReport container1 = ContainerReport.newInstance(
           ContainerId.newContainerId(attempt.getApplicationAttemptId(), 2), null,
           NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
-          "diagnosticInfo", "logURL", 0, ContainerState.RUNNING);
+          "diagnosticInfo", "logURL", 0, ContainerState.RUNNING,
+          "http://" + NodeId.newInstance("host", 2345).toString());
       containerReports.add(container1);
       containers.put(attempt.getApplicationAttemptId(), containerReports);
       
@@ -630,18 +631,21 @@ public class TestYarnClient {
       container = ContainerReport.newInstance(
           ContainerId.newContainerId(attempt.getApplicationAttemptId(), 1), null,
           NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
-          "diagnosticInfo", "logURL", 0, null);
+          "diagnosticInfo", "logURL", 0, null,
+          "http://" + NodeId.newInstance("host", 2345).toString());
       containerReportsForAHS.add(container);
 
       container1 = ContainerReport.newInstance(
           ContainerId.newContainerId(attempt.getApplicationAttemptId(), 2), null,
           NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
-          "diagnosticInfo", "HSlogURL", 0, null);
+          "diagnosticInfo", "HSlogURL", 0, null,
+          "http://" + NodeId.newInstance("host", 2345).toString());
       containerReportsForAHS.add(container1);
       ContainerReport container2 = ContainerReport.newInstance(
           ContainerId.newContainerId(attempt.getApplicationAttemptId(),3), null,
           NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
-          "diagnosticInfo", "HSlogURL", 0, ContainerState.COMPLETE);
+          "diagnosticInfo", "HSlogURL", 0, ContainerState.COMPLETE,
+          "http://" + NodeId.newInstance("host", 2345).toString());
       containerReportsForAHS.add(container2);
       containersFromAHS.put(attempt.getApplicationAttemptId(), containerReportsForAHS);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index 088969f..4b60c52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -224,7 +224,8 @@ public class TestYarnCLI {
     ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
     ContainerReport container = ContainerReport.newInstance(containerId, null,
         NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
-        "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
+        "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE,
+        "http://" + NodeId.newInstance("host", 2345).toString());
     when(client.getContainerReport(any(ContainerId.class))).thenReturn(
         container);
     int result = cli.run(new String[] { "container", "-status",
@@ -240,6 +241,7 @@ public class TestYarnCLI {
     pw.println("\tState : COMPLETE");
     pw.println("\tLOG-URL : logURL");
     pw.println("\tHost : host:1234");
+    pw.println("\tNodeHttpAddress : http://host:2345");
     pw.println("\tDiagnostics : diagnosticInfo");
     pw.close();
     String appReportStr = baos.toString("UTF-8");
@@ -259,13 +261,16 @@ public class TestYarnCLI {
     long time1=1234,time2=5678;
     ContainerReport container = ContainerReport.newInstance(containerId, null,
         NodeId.newInstance("host", 1234), Priority.UNDEFINED, time1, time2,
-        "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
+        "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE,
+        "http://" + NodeId.newInstance("host", 2345).toString());
     ContainerReport container1 = ContainerReport.newInstance(containerId1, null,
         NodeId.newInstance("host", 1234), Priority.UNDEFINED, time1, time2,
-        "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
+        "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE,
+        "http://" + NodeId.newInstance("host", 2345).toString());
     ContainerReport container2 = ContainerReport.newInstance(containerId2, null,
         NodeId.newInstance("host", 1234), Priority.UNDEFINED, time1,0,
-        "diagnosticInfo", "", 0, ContainerState.RUNNING);
+        "diagnosticInfo", "", 0, ContainerState.RUNNING,
+        "http://" + NodeId.newInstance("host", 2345).toString());
     List<ContainerReport> reports = new ArrayList<ContainerReport>();
     reports.add(container);
     reports.add(container1);
@@ -273,6 +278,7 @@ public class TestYarnCLI {
     DateFormat dateFormat=new SimpleDateFormat("EEE MMM dd HH:mm:ss Z yyyy");
     when(client.getContainers(any(ApplicationAttemptId.class))).thenReturn(
         reports);
+    sysOutStream.reset();
     int result = cli.run(new String[] { "container", "-list",
         attemptId.toString() });
     assertEquals(0, result);
@@ -285,24 +291,28 @@ public class TestYarnCLI {
     pw.print("\t         Finish Time");
     pw.print("\t               State");
     pw.print("\t                Host");
+    pw.print("\t   Node Http Address");
     pw.println("\t                            LOG-URL");
     pw.print(" container_1234_0005_01_000001");
     pw.print("\t"+dateFormat.format(new Date(time1)));
     pw.print("\t"+dateFormat.format(new Date(time2)));
     pw.print("\t            COMPLETE");
     pw.print("\t           host:1234");
+    pw.print("\t    http://host:2345");
     pw.println("\t                             logURL");
     pw.print(" container_1234_0005_01_000002");
     pw.print("\t"+dateFormat.format(new Date(time1)));
     pw.print("\t"+dateFormat.format(new Date(time2)));
     pw.print("\t            COMPLETE");
     pw.print("\t           host:1234");
+    pw.print("\t    http://host:2345");
     pw.println("\t                             logURL");
     pw.print(" container_1234_0005_01_000003");
     pw.print("\t"+dateFormat.format(new Date(time1)));
     pw.print("\t                 N/A");
     pw.print("\t             RUNNING");
     pw.print("\t           host:1234");
+    pw.print("\t    http://host:2345");
     pw.println("\t                                   ");
     pw.close();
     String appReportStr = baos.toString("UTF-8");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java
index 18c452f..1f0405f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java
@@ -338,4 +338,23 @@ public class ContainerReportPBImpl extends ContainerReport {
       ContainerStateProto containerState) {
     return ProtoUtils.convertFromProtoFormat(containerState);
   }
+
+  @Override
+  public String getNodeHttpAddress() {
+    ContainerReportProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasNodeHttpAddress()) {
+      return null;
+    }
+    return (p.getNodeHttpAddress());
+  }
+
+  @Override
+  public void setNodeHttpAddress(String nodeHttpAddress) {
+    maybeInitBuilder();
+    if (nodeHttpAddress == null) {
+      builder.clearNodeHttpAddress();
+      return;
+    }
+    builder.setNodeHttpAddress(nodeHttpAddress);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
index 803dc01..c7cf07b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
@@ -215,7 +215,7 @@ public class ApplicationHistoryManagerImpl extends AbstractService implements
       containerHistory.getStartTime(), containerHistory.getFinishTime(),
       containerHistory.getDiagnosticsInfo(), logUrl,
       containerHistory.getContainerExitStatus(),
-      containerHistory.getContainerState());
+      containerHistory.getContainerState(), null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
index 22418a8..1010f62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
@@ -415,6 +415,7 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
     String diagnosticsInfo = null;
     int exitStatus = ContainerExitStatus.INVALID;
     ContainerState state = null;
+    String nodeHttpAddress = null;
     Map<String, Object> entityInfo = entity.getOtherInfo();
     if (entityInfo != null) {
       if (entityInfo
@@ -444,6 +445,12 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
         allocatedPriority = (Integer) entityInfo.get(
                 ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO);
       }
+      if (entityInfo.containsKey(
+          ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO)) {
+        nodeHttpAddress =
+            (String) entityInfo
+              .get(ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO);
+      }
     }
     List<TimelineEvent> events = entity.getEvents();
     if (events != null) {
@@ -493,7 +500,8 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
         Resource.newInstance(allocatedMem, allocatedVcore),
         NodeId.newInstance(allocatedHost, allocatedPort),
         Priority.newInstance(allocatedPriority),
-        createdTime, finishedTime, diagnosticsInfo, logUrl, exitStatus, state);
+        createdTime, finishedTime, diagnosticsInfo, logUrl, exitStatus, state,
+        nodeHttpAddress);
   }
 
   private ApplicationReportExt generateApplicationReport(TimelineEntity entity,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java
index 8791da4..0d5540d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java
@@ -57,4 +57,6 @@ public class ContainerMetricsConstants {
   public static final String STATE_EVENT_INFO =
       "YARN_CONTAINER_STATE";
 
+  public static final String ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO =
+      "YARN_CONTAINER_ALLOCATED_HOST_HTTP_ADDRESS";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
index 1bba4d8..4a82c93 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
@@ -199,12 +199,14 @@ public class AppAttemptBlock extends HtmlBlock {
         .append(url("container", container.getContainerId()))
         .append("'>")
         .append(container.getContainerId())
-        .append("</a>\",\"<a href='")
-        .append("#") // TODO: replace with node http address (YARN-1884)
+        .append("</a>\",\"<a ")
+        .append(
+          container.getNodeHttpAddress() == null ? "#" : "href='"
+              + container.getNodeHttpAddress())
         .append("'>")
-        .append(container.getAssignedNodeId() == null ? "N/A" :
+        .append(container.getNodeHttpAddress() == null ? "N/A" :
             StringEscapeUtils.escapeJavaScript(StringEscapeUtils
-                .escapeHtml(container.getAssignedNodeId())))
+                .escapeHtml(container.getNodeHttpAddress())))
         .append("</a>\",\"")
         .append(container.getContainerExitStatus()).append("\",\"<a href='")
         .append(container.getLogUrl() == null ?
@@ -271,4 +273,4 @@ public class AppAttemptBlock extends HtmlBlock {
     }
     return false;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index d4c20d4..5fc5fa0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -285,15 +285,12 @@ public class AppBlock extends HtmlBlock {
       }
       long startTime = 0L;
       String logsLink = null;
+      String nodeLink = null;
       if (containerReport != null) {
         ContainerInfo container = new ContainerInfo(containerReport);
         startTime = container.getStartedTime();
         logsLink = containerReport.getLogUrl();
-      }
-      String nodeLink = null;
-      if (appAttempt.getHost() != null && appAttempt.getRpcPort() >= 0
-          && appAttempt.getRpcPort() < 65536) {
-        nodeLink = appAttempt.getHost() + ":" + appAttempt.getRpcPort();
+        nodeLink = containerReport.getNodeHttpAddress();
       }
       // AppAttemptID numerical value parsed by parseHadoopID in
       // yarn.dt.plugins.js
@@ -304,8 +301,8 @@ public class AppBlock extends HtmlBlock {
         .append(appAttempt.getAppAttemptId())
         .append("</a>\",\"")
         .append(startTime)
-        .append("\",\"<a href='")
-        .append("#") // TODO: replace with node http address (YARN-1884)
+        .append("\",\"<a ")
+        .append(nodeLink == null ? "#" : "href='" + nodeLink)
         .append("'>")
         .append(nodeLink == null ? "N/A" : StringEscapeUtils
             .escapeJavaScript(StringEscapeUtils.escapeHtml(nodeLink)))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
index ed50c7a..cae8d2e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
@@ -104,7 +104,12 @@ public class ContainerBlock extends HtmlBlock {
         container.getContainerState() == null ? UNAVAILABLE : container
           .getContainerState())
       ._("Exit Status:", container.getContainerExitStatus())
-      ._("Node:", container.getAssignedNodeId())
+      ._(
+        "Node:",
+        container.getNodeHttpAddress() == null ? "#" : container
+          .getNodeHttpAddress(),
+        container.getNodeHttpAddress() == null ? "N/A" : container
+          .getNodeHttpAddress())
       ._("Priority:", container.getPriority())
       ._("Started:", Times.format(container.getStartedTime()))
       ._(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java
index bdcc7b2..0d18e7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java
@@ -42,6 +42,7 @@ public class ContainerInfo {
   protected String logUrl;
   protected int containerExitStatus;
   protected ContainerState containerState;
+  protected String nodeHttpAddress;
 
   public ContainerInfo() {
     // JAXB needs this
@@ -64,6 +65,7 @@ public class ContainerInfo {
     logUrl = container.getLogUrl();
     containerExitStatus = container.getContainerExitStatus();
     containerState = container.getContainerState();
+    nodeHttpAddress = container.getNodeHttpAddress();
   }
 
   public String getContainerId() {
@@ -114,4 +116,7 @@ public class ContainerInfo {
     return containerState;
   }
 
+  public String getNodeHttpAddress() {
+    return nodeHttpAddress;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java
index eeda181..05b6781 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java
@@ -29,18 +29,21 @@ public class ContainerCreatedEvent extends SystemMetricsEvent {
   private Resource allocatedResource;
   private NodeId allocatedNode;
   private Priority allocatedPriority;
+  private String nodeHttpAddress;
 
   public ContainerCreatedEvent(
       ContainerId containerId,
       Resource allocatedResource,
       NodeId allocatedNode,
       Priority allocatedPriority,
-      long createdTime) {
+      long createdTime,
+      String nodeHttpAddress) {
     super(SystemMetricsEventType.CONTAINER_CREATED, createdTime);
     this.containerId = containerId;
     this.allocatedResource = allocatedResource;
     this.allocatedNode = allocatedNode;
     this.allocatedPriority = allocatedPriority;
+    this.nodeHttpAddress = nodeHttpAddress;
   }
 
   @Override
@@ -64,4 +67,7 @@ public class ContainerCreatedEvent extends SystemMetricsEvent {
     return allocatedPriority;
   }
 
+  public String getNodeHttpAddress() {
+    return nodeHttpAddress;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
index 3adf519..b849b00 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
@@ -181,7 +181,7 @@ public class SystemMetricsPublisher extends CompositeService {
               container.getAllocatedResource(),
               container.getAllocatedNode(),
               container.getAllocatedPriority(),
-              createdTime));
+              createdTime, container.getNodeHttpAddress()));
     }
   }
 
@@ -388,6 +388,9 @@ public class SystemMetricsPublisher extends CompositeService {
         event.getAllocatedNode().getPort());
     entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO,
         event.getAllocatedPriority().getPriority());
+    entityInfo.put(
+      ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO,
+      event.getNodeHttpAddress());
     entity.setOtherInfo(entityInfo);
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(ContainerMetricsConstants.CREATED_EVENT_TYPE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
index 9e9dcb9..20087f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
@@ -79,4 +79,5 @@ public interface RMContainer extends EventHandler<RMContainerEvent> {
   
   List<ResourceRequest> getResourceRequests();
 
+  String getNodeHttpAddress();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index e37d8fd..38a03ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
@@ -573,10 +572,29 @@ public class RMContainerImpl implements RMContainer {
           this.getAllocatedResource(), this.getAllocatedNode(),
           this.getAllocatedPriority(), this.getCreationTime(),
           this.getFinishTime(), this.getDiagnosticsInfo(), this.getLogURL(),
-          this.getContainerExitStatus(), this.getContainerState());
+          this.getContainerExitStatus(), this.getContainerState(),
+          this.getNodeHttpAddress());
     } finally {
       this.readLock.unlock();
     }
     return containerReport;
   }
+
+  @Override
+  public String getNodeHttpAddress() {
+    try {
+      readLock.lock();
+      if (container.getNodeHttpAddress() != null) {
+        StringBuilder httpAddress = new StringBuilder();
+        httpAddress.append(WebAppUtils.getHttpSchemePrefix(rmContext
+            .getYarnConfiguration()));
+        httpAddress.append(container.getNodeHttpAddress());
+        return httpAddress.toString();
+      } else {
+        return null;
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85f6d67f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
index 9f02721..7ed3835 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSWebApp;
 import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
@@ -386,6 +385,10 @@ public class TestSystemMetricsPublisher {
     when(container.getDiagnosticsInfo()).thenReturn("test diagnostics info");
     when(container.getContainerExitStatus()).thenReturn(-1);
     when(container.getContainerState()).thenReturn(ContainerState.COMPLETE);
+    Container mockContainer = mock(Container.class);
+    when(container.getContainer()).thenReturn(mockContainer);
+    when(mockContainer.getNodeHttpAddress())
+      .thenReturn("http://localhost:1234");
     return container;
   }
 


[04/50] [abbrv] hadoop git commit: HADOOP-10027. *Compressor_deflateBytesDirect passes instance instead of jclass to GetStaticObjectField. Contributed by Hui Zheng.

Posted by zh...@apache.org.
HADOOP-10027. *Compressor_deflateBytesDirect passes instance instead of jclass to GetStaticObjectField. 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/ff83ae72
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ff83ae72
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ff83ae72

Branch: refs/heads/HDFS-7285
Commit: ff83ae72318fe6c0f266a7bf08138bd2fdb51cbd
Parents: 85f6d67
Author: cnauroth <cn...@apache.org>
Authored: Wed Mar 11 23:27:49 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Wed Mar 11 23:27:49 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../io/compress/bzip2/Bzip2Compressor.java      |   3 -
 .../io/compress/bzip2/Bzip2Decompressor.java    |   3 -
 .../hadoop/io/compress/lz4/Lz4Compressor.java   |   4 -
 .../hadoop/io/compress/lz4/Lz4Decompressor.java |   4 -
 .../io/compress/snappy/SnappyCompressor.java    |   4 -
 .../io/compress/snappy/SnappyDecompressor.java  |   4 -
 .../hadoop/io/compress/zlib/ZlibCompressor.java |   3 -
 .../io/compress/zlib/ZlibDecompressor.java      |   5 +-
 .../hadoop/io/compress/bzip2/Bzip2Compressor.c  |   9 +-
 .../io/compress/bzip2/Bzip2Decompressor.c       |   7 --
 .../hadoop/io/compress/lz4/Lz4Compressor.c      |  13 ---
 .../hadoop/io/compress/lz4/Lz4Decompressor.c    |   8 --
 .../io/compress/snappy/SnappyCompressor.c       |   8 --
 .../io/compress/snappy/SnappyDecompressor.c     |   8 --
 .../hadoop/io/compress/zlib/ZlibCompressor.c    |  11 --
 .../hadoop/io/compress/zlib/ZlibDecompressor.c  |  10 --
 .../bzip2/TestBzip2CompressorDecompressor.java  | 104 +++++++++++++++++++
 .../lz4/TestLz4CompressorDecompressor.java      |  17 +++
 .../TestSnappyCompressorDecompressor.java       |  17 +++
 .../zlib/TestZlibCompressorDecompressor.java    |  17 +++
 21 files changed, 160 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index fb699bc..40b5cd0 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -450,6 +450,9 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
+    HADOOP-10027. *Compressor_deflateBytesDirect passes instance instead of
+    jclass to GetStaticObjectField. (Hui Zheng via cnauroth)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.java
index 0f333bb..a973dc9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.java
@@ -44,9 +44,6 @@ public class Bzip2Compressor implements Compressor {
 
   private static final Log LOG = LogFactory.getLog(Bzip2Compressor.class);
 
-  // HACK - Use this as a global lock in the JNI layer.
-  private static Class<Bzip2Compressor> clazz = Bzip2Compressor.class;
-
   private long stream;
   private int blockSize;
   private int workFactor;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.java
index 6720902..3135165 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.java
@@ -38,9 +38,6 @@ public class Bzip2Decompressor implements Decompressor {
   
   private static final Log LOG = LogFactory.getLog(Bzip2Decompressor.class);
 
-  // HACK - Use this as a global lock in the JNI layer.
-  private static Class<Bzip2Decompressor> clazz = Bzip2Decompressor.class;
-  
   private long stream;
   private boolean conserveMemory;
   private int directBufferSize;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Compressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Compressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Compressor.java
index b5db99f..ccfae8b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Compressor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Compressor.java
@@ -37,10 +37,6 @@ public class Lz4Compressor implements Compressor {
       LogFactory.getLog(Lz4Compressor.class.getName());
   private static final int DEFAULT_DIRECT_BUFFER_SIZE = 64 * 1024;
 
-  // HACK - Use this as a global lock in the JNI layer
-  @SuppressWarnings({"unchecked", "unused"})
-  private static Class clazz = Lz4Compressor.class;
-
   private int directBufferSize;
   private Buffer compressedDirectBuf = null;
   private int uncompressedDirectBufLen;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.java
index 22a3118..685956c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.java
@@ -36,10 +36,6 @@ public class Lz4Decompressor implements Decompressor {
       LogFactory.getLog(Lz4Compressor.class.getName());
   private static final int DEFAULT_DIRECT_BUFFER_SIZE = 64 * 1024;
 
-  // HACK - Use this as a global lock in the JNI layer
-  @SuppressWarnings({"unchecked", "unused"})
-  private static Class clazz = Lz4Decompressor.class;
-
   private int directBufferSize;
   private Buffer compressedDirectBuf = null;
   private int compressedDirectBufLen;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyCompressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyCompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyCompressor.java
index ab45f25..814718d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyCompressor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyCompressor.java
@@ -37,10 +37,6 @@ public class SnappyCompressor implements Compressor {
       LogFactory.getLog(SnappyCompressor.class.getName());
   private static final int DEFAULT_DIRECT_BUFFER_SIZE = 64 * 1024;
 
-  // HACK - Use this as a global lock in the JNI layer
-  @SuppressWarnings({"unchecked", "unused"})
-  private static Class clazz = SnappyCompressor.class;
-
   private int directBufferSize;
   private Buffer compressedDirectBuf = null;
   private int uncompressedDirectBufLen;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.java
index b5f5acf..dbffba8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.java
@@ -37,10 +37,6 @@ public class SnappyDecompressor implements Decompressor {
       LogFactory.getLog(SnappyCompressor.class.getName());
   private static final int DEFAULT_DIRECT_BUFFER_SIZE = 64 * 1024;
 
-  // HACK - Use this as a global lock in the JNI layer
-  @SuppressWarnings({"unchecked", "unused"})
-  private static Class clazz = SnappyDecompressor.class;
-
   private int directBufferSize;
   private Buffer compressedDirectBuf = null;
   private int compressedDirectBufLen;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java
index 6799403..b955044 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java
@@ -41,9 +41,6 @@ public class ZlibCompressor implements Compressor {
 
   private static final int DEFAULT_DIRECT_BUFFER_SIZE = 64*1024;
 
-  // HACK - Use this as a global lock in the JNI layer
-  private static Class clazz = ZlibCompressor.class;
-
   private long stream;
   private CompressionLevel level;
   private CompressionStrategy strategy;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java
index 89c879a..d728fad 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java
@@ -34,10 +34,7 @@ import org.apache.hadoop.util.NativeCodeLoader;
  */
 public class ZlibDecompressor implements Decompressor {
   private static final int DEFAULT_DIRECT_BUFFER_SIZE = 64*1024;
-  
-  // HACK - Use this as a global lock in the JNI layer
-  private static Class clazz = ZlibDecompressor.class;
-  
+
   private long stream;
   private CompressionHeader header;
   private int directBufferSize;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.c
index ef81bea..a92123e 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.c
@@ -25,7 +25,6 @@
 #include "org_apache_hadoop_io_compress_bzip2.h"
 #include "org_apache_hadoop_io_compress_bzip2_Bzip2Compressor.h"
 
-static jfieldID Bzip2Compressor_clazz;
 static jfieldID Bzip2Compressor_stream;
 static jfieldID Bzip2Compressor_uncompressedDirectBuf;
 static jfieldID Bzip2Compressor_uncompressedDirectBufOff;
@@ -74,8 +73,6 @@ Java_org_apache_hadoop_io_compress_bzip2_Bzip2Compressor_initIDs(
                         "BZ2_bzCompressEnd");
 
     // Initialize the requisite fieldIds.
-    Bzip2Compressor_clazz = (*env)->GetStaticFieldID(env, class, "clazz", 
-                                                     "Ljava/lang/Class;");
     Bzip2Compressor_stream = (*env)->GetFieldID(env, class, "stream", "J");
     Bzip2Compressor_finish = (*env)->GetFieldID(env, class, "finish", "Z");
     Bzip2Compressor_finished = (*env)->GetFieldID(env, class, "finished", "Z");
@@ -155,9 +152,7 @@ Java_org_apache_hadoop_io_compress_bzip2_Bzip2Compressor_deflateBytesDirect(
         return (jint)0;
     } 
 
-    jobject clazz = (*env)->GetStaticObjectField(env, this, 
-                                                 Bzip2Compressor_clazz);
-    jobject uncompressed_direct_buf = (*env)->GetObjectField(env, this, 
+    jobject uncompressed_direct_buf = (*env)->GetObjectField(env, this,
                                      Bzip2Compressor_uncompressedDirectBuf);
     jint uncompressed_direct_buf_off = (*env)->GetIntField(env, this, 
                                    Bzip2Compressor_uncompressedDirectBufOff);
@@ -173,12 +168,10 @@ Java_org_apache_hadoop_io_compress_bzip2_Bzip2Compressor_deflateBytesDirect(
                                               Bzip2Compressor_finish);
 
     // Get the input and output direct buffers.
-    LOCK_CLASS(env, clazz, "Bzip2Compressor");
     char* uncompressed_bytes = (*env)->GetDirectBufferAddress(env, 
                                                 uncompressed_direct_buf);
     char* compressed_bytes = (*env)->GetDirectBufferAddress(env, 
                                                 compressed_direct_buf);
-    UNLOCK_CLASS(env, clazz, "Bzip2Compressor");
 
     if (!uncompressed_bytes || !compressed_bytes) {
         return (jint)0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.c
index ad9bcb7..3d9fc08 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.c
@@ -25,7 +25,6 @@
 #include "org_apache_hadoop_io_compress_bzip2.h"
 #include "org_apache_hadoop_io_compress_bzip2_Bzip2Decompressor.h"
 
-static jfieldID Bzip2Decompressor_clazz;
 static jfieldID Bzip2Decompressor_stream;
 static jfieldID Bzip2Decompressor_compressedDirectBuf;
 static jfieldID Bzip2Decompressor_compressedDirectBufOff;
@@ -73,8 +72,6 @@ Java_org_apache_hadoop_io_compress_bzip2_Bzip2Decompressor_initIDs(
                         "BZ2_bzDecompressEnd");
 
     // Initialize the requisite fieldIds.
-    Bzip2Decompressor_clazz = (*env)->GetStaticFieldID(env, class, "clazz", 
-                                                       "Ljava/lang/Class;");
     Bzip2Decompressor_stream = (*env)->GetFieldID(env, class, "stream", "J");
     Bzip2Decompressor_finished = (*env)->GetFieldID(env, class,
                                                     "finished", "Z");
@@ -144,8 +141,6 @@ Java_org_apache_hadoop_io_compress_bzip2_Bzip2Decompressor_inflateBytesDirect(
         return (jint)0;
     } 
 
-    jobject clazz = (*env)->GetStaticObjectField(env, this, 
-                                                 Bzip2Decompressor_clazz);
     jarray compressed_direct_buf = (jarray)(*env)->GetObjectField(env,
                                 this, Bzip2Decompressor_compressedDirectBuf);
     jint compressed_direct_buf_off = (*env)->GetIntField(env, this, 
@@ -159,12 +154,10 @@ Java_org_apache_hadoop_io_compress_bzip2_Bzip2Decompressor_inflateBytesDirect(
                                 Bzip2Decompressor_directBufferSize);
 
     // Get the input and output direct buffers.
-    LOCK_CLASS(env, clazz, "Bzip2Decompressor");
     char* compressed_bytes = (*env)->GetDirectBufferAddress(env, 
                                                 compressed_direct_buf);
     char* uncompressed_bytes = (*env)->GetDirectBufferAddress(env, 
                                                 uncompressed_direct_buf);
-    UNLOCK_CLASS(env, clazz, "Bzip2Decompressor");
 
     if (!compressed_bytes || !uncompressed_bytes) {
         return (jint)0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Compressor.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Compressor.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Compressor.c
index 9f14312..f742384 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Compressor.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Compressor.c
@@ -27,7 +27,6 @@
 #include "lz4hc.h"
 
 
-static jfieldID Lz4Compressor_clazz;
 static jfieldID Lz4Compressor_uncompressedDirectBuf;
 static jfieldID Lz4Compressor_uncompressedDirectBufLen;
 static jfieldID Lz4Compressor_compressedDirectBuf;
@@ -37,8 +36,6 @@ static jfieldID Lz4Compressor_directBufferSize;
 JNIEXPORT void JNICALL Java_org_apache_hadoop_io_compress_lz4_Lz4Compressor_initIDs
 (JNIEnv *env, jclass clazz){
 
-  Lz4Compressor_clazz = (*env)->GetStaticFieldID(env, clazz, "clazz",
-                                                 "Ljava/lang/Class;");
   Lz4Compressor_uncompressedDirectBuf = (*env)->GetFieldID(env, clazz,
                                                            "uncompressedDirectBuf",
                                                            "Ljava/nio/Buffer;");
@@ -57,25 +54,20 @@ JNIEXPORT jint JNICALL Java_org_apache_hadoop_io_compress_lz4_Lz4Compressor_comp
   char *compressed_bytes;
 
   // Get members of Lz4Compressor
-  jobject clazz = (*env)->GetStaticObjectField(env, thisj, Lz4Compressor_clazz);
   jobject uncompressed_direct_buf = (*env)->GetObjectField(env, thisj, Lz4Compressor_uncompressedDirectBuf);
   jint uncompressed_direct_buf_len = (*env)->GetIntField(env, thisj, Lz4Compressor_uncompressedDirectBufLen);
   jobject compressed_direct_buf = (*env)->GetObjectField(env, thisj, Lz4Compressor_compressedDirectBuf);
   jint compressed_direct_buf_len = (*env)->GetIntField(env, thisj, Lz4Compressor_directBufferSize);
 
   // Get the input direct buffer
-  LOCK_CLASS(env, clazz, "Lz4Compressor");
   uncompressed_bytes = (const char*)(*env)->GetDirectBufferAddress(env, uncompressed_direct_buf);
-  UNLOCK_CLASS(env, clazz, "Lz4Compressor");
 
   if (uncompressed_bytes == 0) {
     return (jint)0;
   }
 
   // Get the output direct buffer
-  LOCK_CLASS(env, clazz, "Lz4Compressor");
   compressed_bytes = (char *)(*env)->GetDirectBufferAddress(env, compressed_direct_buf);
-  UNLOCK_CLASS(env, clazz, "Lz4Compressor");
 
   if (compressed_bytes == 0) {
     return (jint)0;
@@ -104,25 +96,20 @@ JNIEXPORT jint JNICALL Java_org_apache_hadoop_io_compress_lz4_Lz4Compressor_comp
   char* compressed_bytes = NULL;
 
   // Get members of Lz4Compressor
-  jobject clazz = (*env)->GetStaticObjectField(env, thisj, Lz4Compressor_clazz);
   jobject uncompressed_direct_buf = (*env)->GetObjectField(env, thisj, Lz4Compressor_uncompressedDirectBuf);
   jint uncompressed_direct_buf_len = (*env)->GetIntField(env, thisj, Lz4Compressor_uncompressedDirectBufLen);
   jobject compressed_direct_buf = (*env)->GetObjectField(env, thisj, Lz4Compressor_compressedDirectBuf);
   jint compressed_direct_buf_len = (*env)->GetIntField(env, thisj, Lz4Compressor_directBufferSize);
 
   // Get the input direct buffer
-  LOCK_CLASS(env, clazz, "Lz4Compressor");
   uncompressed_bytes = (const char*)(*env)->GetDirectBufferAddress(env, uncompressed_direct_buf);
-  UNLOCK_CLASS(env, clazz, "Lz4Compressor");
 
   if (uncompressed_bytes == 0) {
     return (jint)0;
   }
 
   // Get the output direct buffer
-  LOCK_CLASS(env, clazz, "Lz4Compressor");
   compressed_bytes = (char *)(*env)->GetDirectBufferAddress(env, compressed_direct_buf);
-  UNLOCK_CLASS(env, clazz, "Lz4Compressor");
 
   if (compressed_bytes == 0) {
     return (jint)0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.c
index 2b8c91c..cdeaa31 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.c
@@ -25,7 +25,6 @@
 #include "lz4.h"
 
 
-static jfieldID Lz4Decompressor_clazz;
 static jfieldID Lz4Decompressor_compressedDirectBuf;
 static jfieldID Lz4Decompressor_compressedDirectBufLen;
 static jfieldID Lz4Decompressor_uncompressedDirectBuf;
@@ -34,8 +33,6 @@ static jfieldID Lz4Decompressor_directBufferSize;
 JNIEXPORT void JNICALL Java_org_apache_hadoop_io_compress_lz4_Lz4Decompressor_initIDs
 (JNIEnv *env, jclass clazz){
 
-  Lz4Decompressor_clazz = (*env)->GetStaticFieldID(env, clazz, "clazz",
-                                                   "Ljava/lang/Class;");
   Lz4Decompressor_compressedDirectBuf = (*env)->GetFieldID(env,clazz,
                                                            "compressedDirectBuf",
                                                            "Ljava/nio/Buffer;");
@@ -54,25 +51,20 @@ JNIEXPORT jint JNICALL Java_org_apache_hadoop_io_compress_lz4_Lz4Decompressor_de
   char *uncompressed_bytes;
 
   // Get members of Lz4Decompressor
-  jobject clazz = (*env)->GetStaticObjectField(env,thisj, Lz4Decompressor_clazz);
   jobject compressed_direct_buf = (*env)->GetObjectField(env,thisj, Lz4Decompressor_compressedDirectBuf);
   jint compressed_direct_buf_len = (*env)->GetIntField(env,thisj, Lz4Decompressor_compressedDirectBufLen);
   jobject uncompressed_direct_buf = (*env)->GetObjectField(env,thisj, Lz4Decompressor_uncompressedDirectBuf);
   size_t uncompressed_direct_buf_len = (*env)->GetIntField(env, thisj, Lz4Decompressor_directBufferSize);
 
   // Get the input direct buffer
-  LOCK_CLASS(env, clazz, "Lz4Decompressor");
   compressed_bytes = (const char*)(*env)->GetDirectBufferAddress(env, compressed_direct_buf);
-  UNLOCK_CLASS(env, clazz, "Lz4Decompressor");
 
   if (compressed_bytes == 0) {
     return (jint)0;
   }
 
   // Get the output direct buffer
-  LOCK_CLASS(env, clazz, "Lz4Decompressor");
   uncompressed_bytes = (char *)(*env)->GetDirectBufferAddress(env, uncompressed_direct_buf);
-  UNLOCK_CLASS(env, clazz, "Lz4Decompressor");
 
   if (uncompressed_bytes == 0) {
     return (jint)0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyCompressor.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyCompressor.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyCompressor.c
index fe827f0..9a09f07 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyCompressor.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyCompressor.c
@@ -38,7 +38,6 @@
 
 #define JINT_MAX 0x7fffffff
 
-static jfieldID SnappyCompressor_clazz;
 static jfieldID SnappyCompressor_uncompressedDirectBuf;
 static jfieldID SnappyCompressor_uncompressedDirectBufLen;
 static jfieldID SnappyCompressor_compressedDirectBuf;
@@ -84,8 +83,6 @@ JNIEXPORT void JNICALL Java_org_apache_hadoop_io_compress_snappy_SnappyCompresso
   LOAD_DYNAMIC_SYMBOL(__dlsym_snappy_compress, dlsym_snappy_compress, env, libsnappy, "snappy_compress");
 #endif
 
-  SnappyCompressor_clazz = (*env)->GetStaticFieldID(env, clazz, "clazz",
-                                                 "Ljava/lang/Class;");
   SnappyCompressor_uncompressedDirectBuf = (*env)->GetFieldID(env, clazz,
                                                            "uncompressedDirectBuf",
                                                            "Ljava/nio/Buffer;");
@@ -104,7 +101,6 @@ JNIEXPORT jint JNICALL Java_org_apache_hadoop_io_compress_snappy_SnappyCompresso
   char* compressed_bytes;
   snappy_status ret;
   // Get members of SnappyCompressor
-  jobject clazz = (*env)->GetStaticObjectField(env, thisj, SnappyCompressor_clazz);
   jobject uncompressed_direct_buf = (*env)->GetObjectField(env, thisj, SnappyCompressor_uncompressedDirectBuf);
   jint uncompressed_direct_buf_len = (*env)->GetIntField(env, thisj, SnappyCompressor_uncompressedDirectBufLen);
   jobject compressed_direct_buf = (*env)->GetObjectField(env, thisj, SnappyCompressor_compressedDirectBuf);
@@ -112,18 +108,14 @@ JNIEXPORT jint JNICALL Java_org_apache_hadoop_io_compress_snappy_SnappyCompresso
   size_t buf_len;
 
   // Get the input direct buffer
-  LOCK_CLASS(env, clazz, "SnappyCompressor");
   uncompressed_bytes = (const char*)(*env)->GetDirectBufferAddress(env, uncompressed_direct_buf);
-  UNLOCK_CLASS(env, clazz, "SnappyCompressor");
 
   if (uncompressed_bytes == 0) {
     return (jint)0;
   }
 
   // Get the output direct buffer
-  LOCK_CLASS(env, clazz, "SnappyCompressor");
   compressed_bytes = (char *)(*env)->GetDirectBufferAddress(env, compressed_direct_buf);
-  UNLOCK_CLASS(env, clazz, "SnappyCompressor");
 
   if (compressed_bytes == 0) {
     return (jint)0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.c
index d1fd13c..69ec101 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.c
@@ -31,7 +31,6 @@
 
 #include "org_apache_hadoop_io_compress_snappy_SnappyDecompressor.h"
 
-static jfieldID SnappyDecompressor_clazz;
 static jfieldID SnappyDecompressor_compressedDirectBuf;
 static jfieldID SnappyDecompressor_compressedDirectBufLen;
 static jfieldID SnappyDecompressor_uncompressedDirectBuf;
@@ -79,8 +78,6 @@ JNIEXPORT void JNICALL Java_org_apache_hadoop_io_compress_snappy_SnappyDecompres
   LOAD_DYNAMIC_SYMBOL(__dlsym_snappy_uncompress, dlsym_snappy_uncompress, env, libsnappy, "snappy_uncompress");
 #endif
 
-  SnappyDecompressor_clazz = (*env)->GetStaticFieldID(env, clazz, "clazz",
-                                                   "Ljava/lang/Class;");
   SnappyDecompressor_compressedDirectBuf = (*env)->GetFieldID(env,clazz,
                                                            "compressedDirectBuf",
                                                            "Ljava/nio/Buffer;");
@@ -99,25 +96,20 @@ JNIEXPORT jint JNICALL Java_org_apache_hadoop_io_compress_snappy_SnappyDecompres
   char* uncompressed_bytes = NULL;
   snappy_status ret;
   // Get members of SnappyDecompressor
-  jobject clazz = (*env)->GetStaticObjectField(env,thisj, SnappyDecompressor_clazz);
   jobject compressed_direct_buf = (*env)->GetObjectField(env,thisj, SnappyDecompressor_compressedDirectBuf);
   jint compressed_direct_buf_len = (*env)->GetIntField(env,thisj, SnappyDecompressor_compressedDirectBufLen);
   jobject uncompressed_direct_buf = (*env)->GetObjectField(env,thisj, SnappyDecompressor_uncompressedDirectBuf);
   size_t uncompressed_direct_buf_len = (*env)->GetIntField(env, thisj, SnappyDecompressor_directBufferSize);
 
   // Get the input direct buffer
-  LOCK_CLASS(env, clazz, "SnappyDecompressor");
   compressed_bytes = (const char*)(*env)->GetDirectBufferAddress(env, compressed_direct_buf);
-  UNLOCK_CLASS(env, clazz, "SnappyDecompressor");
 
   if (compressed_bytes == 0) {
     return (jint)0;
   }
 
   // Get the output direct buffer
-  LOCK_CLASS(env, clazz, "SnappyDecompressor");
   uncompressed_bytes = (char *)(*env)->GetDirectBufferAddress(env, uncompressed_direct_buf);
-  UNLOCK_CLASS(env, clazz, "SnappyDecompressor");
 
   if (uncompressed_bytes == 0) {
     return (jint)0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/zlib/ZlibCompressor.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/zlib/ZlibCompressor.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/zlib/ZlibCompressor.c
index f7c0cb9..51f7bed 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/zlib/ZlibCompressor.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/zlib/ZlibCompressor.c
@@ -28,7 +28,6 @@
 #include "org_apache_hadoop_io_compress_zlib.h"
 #include "org_apache_hadoop_io_compress_zlib_ZlibCompressor.h"
 
-static jfieldID ZlibCompressor_clazz;
 static jfieldID ZlibCompressor_stream;
 static jfieldID ZlibCompressor_uncompressedDirectBuf;
 static jfieldID ZlibCompressor_uncompressedDirectBufOff;
@@ -141,8 +140,6 @@ Java_org_apache_hadoop_io_compress_zlib_ZlibCompressor_initIDs(
 #endif
 
 	// Initialize the requisite fieldIds
-    ZlibCompressor_clazz = (*env)->GetStaticFieldID(env, class, "clazz",
-                                                      "Ljava/lang/Class;");
     ZlibCompressor_stream = (*env)->GetFieldID(env, class, "stream", "J");
     ZlibCompressor_finish = (*env)->GetFieldID(env, class, "finish", "Z");
     ZlibCompressor_finished = (*env)->GetFieldID(env, class, "finished", "Z");
@@ -239,7 +236,6 @@ JNIEXPORT jint JNICALL
 Java_org_apache_hadoop_io_compress_zlib_ZlibCompressor_deflateBytesDirect(
 	JNIEnv *env, jobject this
 	) {
-    jobject clazz = NULL;
     jobject uncompressed_direct_buf = NULL;
     jint uncompressed_direct_buf_off = 0;
     jint uncompressed_direct_buf_len = 0;
@@ -260,9 +256,6 @@ Java_org_apache_hadoop_io_compress_zlib_ZlibCompressor_deflateBytesDirect(
 		return (jint)0;
     }
 
-    // Get members of ZlibCompressor
-    clazz = (*env)->GetStaticObjectField(env, this,
-                                                 ZlibCompressor_clazz);
 	uncompressed_direct_buf = (*env)->GetObjectField(env, this,
 									ZlibCompressor_uncompressedDirectBuf);
 	uncompressed_direct_buf_off = (*env)->GetIntField(env, this,
@@ -278,20 +271,16 @@ Java_org_apache_hadoop_io_compress_zlib_ZlibCompressor_deflateBytesDirect(
 	finish = (*env)->GetBooleanField(env, this, ZlibCompressor_finish);
 
     // Get the input direct buffer
-    LOCK_CLASS(env, clazz, "ZlibCompressor");
     uncompressed_bytes = (*env)->GetDirectBufferAddress(env,
 											uncompressed_direct_buf);
-    UNLOCK_CLASS(env, clazz, "ZlibCompressor");
 
   	if (uncompressed_bytes == 0) {
     	return (jint)0;
 	}
 
     // Get the output direct buffer
-    LOCK_CLASS(env, clazz, "ZlibCompressor");
     compressed_bytes = (*env)->GetDirectBufferAddress(env,
 										compressed_direct_buf);
-    UNLOCK_CLASS(env, clazz, "ZlibCompressor");
 
   	if (compressed_bytes == 0) {
 		return (jint)0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c
index 8b78f41..b9f23b1 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c
@@ -28,7 +28,6 @@
 #include "org_apache_hadoop_io_compress_zlib.h"
 #include "org_apache_hadoop_io_compress_zlib_ZlibDecompressor.h"
 
-static jfieldID ZlibDecompressor_clazz;
 static jfieldID ZlibDecompressor_stream;
 static jfieldID ZlibDecompressor_compressedDirectBuf;
 static jfieldID ZlibDecompressor_compressedDirectBufOff;
@@ -104,8 +103,6 @@ JNIEnv *env, jclass class
 
 
   // Initialize the requisite fieldIds
-    ZlibDecompressor_clazz = (*env)->GetStaticFieldID(env, class, "clazz",
-                                                      "Ljava/lang/Class;");
     ZlibDecompressor_stream = (*env)->GetFieldID(env, class, "stream", "J");
     ZlibDecompressor_needDict = (*env)->GetFieldID(env, class, "needDict", "Z");
     ZlibDecompressor_finished = (*env)->GetFieldID(env, class, "finished", "Z");
@@ -197,7 +194,6 @@ JNIEXPORT jint JNICALL
 Java_org_apache_hadoop_io_compress_zlib_ZlibDecompressor_inflateBytesDirect(
 	JNIEnv *env, jobject this
 	) {
-    jobject clazz = NULL;
     jarray compressed_direct_buf = NULL;
     jint compressed_direct_buf_off = 0;
     jint compressed_direct_buf_len = 0;
@@ -218,8 +214,6 @@ Java_org_apache_hadoop_io_compress_zlib_ZlibDecompressor_inflateBytesDirect(
     }
 
     // Get members of ZlibDecompressor
-    clazz = (*env)->GetStaticObjectField(env, this,
-                                                 ZlibDecompressor_clazz);
 	compressed_direct_buf = (jarray)(*env)->GetObjectField(env, this,
 											ZlibDecompressor_compressedDirectBuf);
 	compressed_direct_buf_off = (*env)->GetIntField(env, this,
@@ -233,20 +227,16 @@ Java_org_apache_hadoop_io_compress_zlib_ZlibDecompressor_inflateBytesDirect(
 										ZlibDecompressor_directBufferSize);
 
     // Get the input direct buffer
-    LOCK_CLASS(env, clazz, "ZlibDecompressor");
 	compressed_bytes = (*env)->GetDirectBufferAddress(env,
 										compressed_direct_buf);
-    UNLOCK_CLASS(env, clazz, "ZlibDecompressor");
 
 	if (!compressed_bytes) {
 	    return (jint)0;
 	}
 
     // Get the output direct buffer
-    LOCK_CLASS(env, clazz, "ZlibDecompressor");
 	uncompressed_bytes = (*env)->GetDirectBufferAddress(env,
 											uncompressed_direct_buf);
-    UNLOCK_CLASS(env, clazz, "ZlibDecompressor");
 
 	if (!uncompressed_bytes) {
 	    return (jint)0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/bzip2/TestBzip2CompressorDecompressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/bzip2/TestBzip2CompressorDecompressor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/bzip2/TestBzip2CompressorDecompressor.java
new file mode 100644
index 0000000..c585a46
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/bzip2/TestBzip2CompressorDecompressor.java
@@ -0,0 +1,104 @@
+/**
+ * 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.compress.bzip2;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.compress.*;
+import org.apache.hadoop.io.compress.bzip2.Bzip2Compressor;
+import org.apache.hadoop.io.compress.bzip2.Bzip2Decompressor;
+import org.apache.hadoop.test.MultithreadedTestUtil;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.*;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+import static org.junit.Assume.*;
+import static org.junit.Assume.assumeTrue;
+
+public class TestBzip2CompressorDecompressor {
+  
+  private static final Random rnd = new Random(12345l);
+
+  @Before
+  public void before() {
+    assumeTrue(Bzip2Factory.isNativeBzip2Loaded(new Configuration()));
+  }
+
+  // test compress/decompress process 
+  @Test
+  public void testCompressDecompress() {
+    byte[] rawData = null;
+    int rawDataSize = 0;
+    rawDataSize = 1024 * 64;
+    rawData = generate(rawDataSize);
+    try {
+      Bzip2Compressor compressor = new Bzip2Compressor();
+      Bzip2Decompressor decompressor = new Bzip2Decompressor();
+      assertFalse("testBzip2CompressDecompress finished error",
+          compressor.finished());
+      compressor.setInput(rawData, 0, rawData.length);
+      assertTrue("testBzip2CompressDecompress getBytesRead before error",
+          compressor.getBytesRead() == 0);
+      compressor.finish();
+
+      byte[] compressedResult = new byte[rawDataSize];
+      int cSize = compressor.compress(compressedResult, 0, rawDataSize);
+      assertTrue("testBzip2CompressDecompress getBytesRead after error",
+          compressor.getBytesRead() == rawDataSize);
+      assertTrue(
+          "testBzip2CompressDecompress compressed size no less than original size",
+          cSize < rawDataSize);
+      decompressor.setInput(compressedResult, 0, cSize);
+      byte[] decompressedBytes = new byte[rawDataSize];
+      decompressor.decompress(decompressedBytes, 0, decompressedBytes.length);
+      assertArrayEquals("testBzip2CompressDecompress arrays not equals ",
+          rawData, decompressedBytes);
+      compressor.reset();
+      decompressor.reset();
+    } catch (IOException ex) {
+      fail("testBzip2CompressDecompress ex !!!" + ex);
+    }
+  }
+
+  public static byte[] generate(int size) {
+    byte[] array = new byte[size];
+    for (int i = 0; i < size; i++)
+      array[i] = (byte)rnd.nextInt(16);
+    return array;
+  }
+
+  @Test
+  public void testBzip2CompressDecompressInMultiThreads() throws Exception {
+    MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext();
+    for(int i=0;i<10;i++) {
+      ctx.addThread( new MultithreadedTestUtil.TestingThread(ctx) {
+        @Override
+        public void doWork() throws Exception {
+          testCompressDecompress();
+        }
+      });
+    }
+    ctx.startThreads();
+
+    ctx.waitFor(60000);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/lz4/TestLz4CompressorDecompressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/lz4/TestLz4CompressorDecompressor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/lz4/TestLz4CompressorDecompressor.java
index e8555b2..6f3b076 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/lz4/TestLz4CompressorDecompressor.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/lz4/TestLz4CompressorDecompressor.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.io.compress.CompressionOutputStream;
 import org.apache.hadoop.io.compress.Lz4Codec;
 import org.apache.hadoop.io.compress.lz4.Lz4Compressor;
 import org.apache.hadoop.io.compress.lz4.Lz4Decompressor;
+import org.apache.hadoop.test.MultithreadedTestUtil;
 import org.junit.Before;
 import org.junit.Test;
 import static org.junit.Assume.*;
@@ -313,4 +314,20 @@ public class TestLz4CompressorDecompressor {
       array[i] = (byte)rnd.nextInt(16);
     return array;
   }
+
+  @Test
+  public void testLz4CompressDecompressInMultiThreads() throws Exception {
+    MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext();
+    for(int i=0;i<10;i++) {
+      ctx.addThread( new MultithreadedTestUtil.TestingThread(ctx) {
+        @Override
+        public void doWork() throws Exception {
+          testCompressDecompress();
+        }
+      });
+    }
+    ctx.startThreads();
+
+    ctx.waitFor(60000);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/snappy/TestSnappyCompressorDecompressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/snappy/TestSnappyCompressorDecompressor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/snappy/TestSnappyCompressorDecompressor.java
index 77fbcc0..cc986c7 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/snappy/TestSnappyCompressorDecompressor.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/snappy/TestSnappyCompressorDecompressor.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.io.compress.CompressionInputStream;
 import org.apache.hadoop.io.compress.CompressionOutputStream;
 import org.apache.hadoop.io.compress.SnappyCodec;
 import org.apache.hadoop.io.compress.snappy.SnappyDecompressor.SnappyDirectDecompressor;
+import org.apache.hadoop.test.MultithreadedTestUtil;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -391,4 +392,20 @@ public class TestSnappyCompressorDecompressor {
       return array;
     }
   }
+
+  @Test
+  public void testSnappyCompressDecompressInMultiThreads() throws Exception {
+    MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext();
+    for(int i=0;i<10;i++) {
+      ctx.addThread( new MultithreadedTestUtil.TestingThread(ctx) {
+        @Override
+        public void doWork() throws Exception {
+          testSnappyCompressDecompress();
+        }
+      });
+    }
+    ctx.startThreads();
+
+    ctx.waitFor(60000);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff83ae72/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java
index db5784c..e751125 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.io.compress.CompressDecompressTester.CompressionTestStr
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionStrategy;
 import org.apache.hadoop.io.compress.zlib.ZlibDecompressor.ZlibDirectDecompressor;
+import org.apache.hadoop.test.MultithreadedTestUtil;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Before;
 import org.junit.Test;
@@ -419,4 +420,20 @@ public class TestZlibCompressorDecompressor {
       data[i] = (byte)random.nextInt(16);
     return data;
   }
+
+  @Test
+  public void testZlibCompressDecompressInMultiThreads() throws Exception {
+    MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext();
+    for(int i=0;i<10;i++) {
+      ctx.addThread( new MultithreadedTestUtil.TestingThread(ctx) {
+        @Override
+        public void doWork() throws Exception {
+          testZlibCompressDecompress();
+        }
+      });
+    }
+    ctx.startThreads();
+
+    ctx.waitFor(60000);
+  }
 }


[39/50] [abbrv] hadoop git commit: HDFS-7716. Erasure Coding: extend BlockInfo to handle EC info. Contributed by Jing Zhao.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index c4612a3..3a5e66e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@ -24,6 +24,7 @@ import java.util.List;
 import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
@@ -80,10 +81,10 @@ public class DatanodeStorageInfo {
   /**
    * Iterates over the list of blocks belonging to the data-node.
    */
-  class BlockIterator implements Iterator<BlockInfoContiguous> {
-    private BlockInfoContiguous current;
+  class BlockIterator implements Iterator<BlockInfo> {
+    private BlockInfo current;
 
-    BlockIterator(BlockInfoContiguous head) {
+    BlockIterator(BlockInfo head) {
       this.current = head;
     }
 
@@ -91,8 +92,8 @@ public class DatanodeStorageInfo {
       return current != null;
     }
 
-    public BlockInfoContiguous next() {
-      BlockInfoContiguous res = current;
+    public BlockInfo next() {
+      BlockInfo res = current;
       current = current.getNext(current.findStorageInfo(DatanodeStorageInfo.this));
       return res;
     }
@@ -112,7 +113,7 @@ public class DatanodeStorageInfo {
   private volatile long remaining;
   private long blockPoolUsed;
 
-  private volatile BlockInfoContiguous blockList = null;
+  private volatile BlockInfo blockList = null;
   private int numBlocks = 0;
 
   /** The number of block reports received */
@@ -215,7 +216,7 @@ public class DatanodeStorageInfo {
     return blockPoolUsed;
   }
 
-  public AddBlockResult addBlock(BlockInfoContiguous b) {
+  public AddBlockResult addBlock(BlockInfo b, Block reportedBlock) {
     // First check whether the block belongs to a different storage
     // on the same DN.
     AddBlockResult result = AddBlockResult.ADDED;
@@ -234,13 +235,21 @@ public class DatanodeStorageInfo {
     }
 
     // add to the head of the data-node list
-    b.addStorage(this);
+    b.addStorage(this, reportedBlock);
+    insertToList(b);
+    return result;
+  }
+
+  AddBlockResult addBlock(BlockInfoContiguous b) {
+    return addBlock(b, b);
+  }
+
+  public void insertToList(BlockInfo b) {
     blockList = b.listInsert(blockList, this);
     numBlocks++;
-    return result;
   }
 
-  public boolean removeBlock(BlockInfoContiguous b) {
+  public boolean removeBlock(BlockInfo b) {
     blockList = b.listRemove(blockList, this);
     if (b.removeStorage(this)) {
       numBlocks--;
@@ -254,16 +263,15 @@ public class DatanodeStorageInfo {
     return numBlocks;
   }
   
-  Iterator<BlockInfoContiguous> getBlockIterator() {
+  Iterator<BlockInfo> getBlockIterator() {
     return new BlockIterator(blockList);
-
   }
 
   /**
    * Move block to the head of the list of blocks belonging to the data-node.
    * @return the index of the head of the blockList
    */
-  int moveBlockToHead(BlockInfoContiguous b, int curIndex, int headIndex) {
+  int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) {
     blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex);
     return curIndex;
   }
@@ -273,7 +281,7 @@ public class DatanodeStorageInfo {
    * @return the head of the blockList
    */
   @VisibleForTesting
-  BlockInfoContiguous getBlockListHeadForTesting(){
+  BlockInfo getBlockListHeadForTesting(){
     return blockList;
   }
 
@@ -360,6 +368,6 @@ public class DatanodeStorageInfo {
   }
 
   static enum AddBlockResult {
-    ADDED, REPLACED, ALREADY_EXIST;
+    ADDED, REPLACED, ALREADY_EXIST
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicaUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicaUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicaUnderConstruction.java
new file mode 100644
index 0000000..f4600cb7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicaUnderConstruction.java
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+
+/**
+ * ReplicaUnderConstruction contains information about replicas (or blocks
+ * belonging to a block group) while they are under construction.
+ *
+ * The GS, the length and the state of the replica is as reported by the
+ * datanode.
+ *
+ * It is not guaranteed, but expected, that datanodes actually have
+ * corresponding replicas.
+ */
+class ReplicaUnderConstruction extends Block {
+  private final DatanodeStorageInfo expectedLocation;
+  private HdfsServerConstants.ReplicaState state;
+  private boolean chosenAsPrimary;
+
+  ReplicaUnderConstruction(Block block,
+      DatanodeStorageInfo target,
+      HdfsServerConstants.ReplicaState state) {
+    super(block);
+    this.expectedLocation = target;
+    this.state = state;
+    this.chosenAsPrimary = false;
+  }
+
+  /**
+   * Expected block replica location as assigned when the block was allocated.
+   * This defines the pipeline order.
+   * It is not guaranteed, but expected, that the data-node actually has
+   * the replica.
+   */
+  DatanodeStorageInfo getExpectedStorageLocation() {
+    return expectedLocation;
+  }
+
+  /**
+   * Get replica state as reported by the data-node.
+   */
+  HdfsServerConstants.ReplicaState getState() {
+    return state;
+  }
+
+  /**
+   * Whether the replica was chosen for recovery.
+   */
+  boolean getChosenAsPrimary() {
+    return chosenAsPrimary;
+  }
+
+  /**
+   * Set replica state.
+   */
+  void setState(HdfsServerConstants.ReplicaState s) {
+    state = s;
+  }
+
+  /**
+   * Set whether this replica was chosen for recovery.
+   */
+  void setChosenAsPrimary(boolean chosenAsPrimary) {
+    this.chosenAsPrimary = chosenAsPrimary;
+  }
+
+  /**
+   * Is data-node the replica belongs to alive.
+   */
+  boolean isAlive() {
+    return expectedLocation.getDatanodeDescriptor().isAlive;
+  }
+
+  @Override // Block
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override // Block
+  public boolean equals(Object obj) {
+    // Sufficient to rely on super's implementation
+    return (this == obj) || super.equals(obj);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder b = new StringBuilder(50);
+    appendStringTo(b);
+    return b.toString();
+  }
+
+  @Override
+  public void appendStringTo(StringBuilder sb) {
+    sb.append("ReplicaUC[")
+        .append(expectedLocation)
+        .append("|")
+        .append(state)
+        .append("]");
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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 627abfa..4e1f63a 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
@@ -466,8 +466,8 @@ public class FSDirectory implements Closeable {
    * Add a block to the file. Returns a reference to the added block.
    */
   BlockInfoContiguous addBlock(String path, INodesInPath inodesInPath,
-      Block block, DatanodeStorageInfo[] targets,
-      boolean isStriped) throws IOException {
+      Block block, DatanodeStorageInfo[] targets, boolean isStriped)
+      throws IOException {
     writeLock();
     try {
       final INodeFile fileINode = inodesInPath.getLastINode().asFile();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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 016a400..f9939b9 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
@@ -3160,8 +3160,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             src + ". Returning previously allocated block " + lastBlockInFile);
         long offset = pendingFile.computeFileSize();
         onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
-            ((BlockInfoContiguousUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
-            offset);
+            ((BlockInfoContiguousUnderConstruction)lastBlockInFile)
+                .getExpectedStorageLocations(), offset);
         return new FileState(pendingFile, src, iip);
       } else {
         // Case 3
@@ -3504,9 +3504,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  private static boolean isCompleteBlock(String src, BlockInfoContiguous b, int minRepl) {
+  private static boolean isCompleteBlock(String src, BlockInfoContiguous b,
+      int minRepl) {
     if (!b.isComplete()) {
-      final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)b;
+      final BlockInfoContiguousUnderConstruction uc =
+          (BlockInfoContiguousUnderConstruction) b;
       final int numNodes = b.numNodes();
       LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
           + uc.getBlockUCState() + ", replication# = " + numNodes
@@ -3993,7 +3995,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new AlreadyBeingCreatedException(message);
     case UNDER_CONSTRUCTION:
     case UNDER_RECOVERY:
-      final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock;
+      final BlockInfoContiguousUnderConstruction uc =
+          (BlockInfoContiguousUnderConstruction)lastBlock;
       // determine if last block was intended to be truncated
       Block recoveryBlock = uc.getTruncateBlock();
       boolean truncateRecovery = recoveryBlock != null;
@@ -4103,9 +4106,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     blockManager.checkReplication(pendingFile);
   }
 
-  @VisibleForTesting
-  BlockInfoContiguous getStoredBlock(Block block) {
-    return blockManager.getStoredBlock(block);
+  public BlockInfoContiguous getStoredBlock(Block block) {
+    return (BlockInfoContiguous) blockManager.getStoredBlock(block);
   }
   
   @Override
@@ -4264,9 +4266,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i));
             if (storageInfo != null) {
               if(copyTruncate) {
-                storageInfo.addBlock(truncatedBlock);
+                storageInfo.addBlock(truncatedBlock, truncatedBlock);
               } else {
-                storageInfo.addBlock(storedBlock);
+                storageInfo.addBlock(storedBlock, storedBlock);
               }
             }
           }
@@ -4630,7 +4632,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
         while (it.hasNext()) {
           Block b = it.next();
-          BlockInfoContiguous blockInfo = blockManager.getStoredBlock(b);
+          BlockInfoContiguous blockInfo = getStoredBlock(b);
           if (blockInfo.getBlockCollection().getStoragePolicyID() == lpPolicy.getId()) {
             filesToDelete.add(blockInfo.getBlockCollection());
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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 1d2439c..1db74b7 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
@@ -235,7 +235,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       //get blockInfo
       Block block = new Block(Block.getBlockId(blockId));
       //find which file this block belongs to
-      BlockInfoContiguous blockInfo = bm.getStoredBlock(block);
+      BlockInfoContiguous blockInfo = namenode.getNamesystem()
+          .getStoredBlock(block);
       if(blockInfo == null) {
         out.println("Block "+ blockId +" " + NONEXISTENT_STATUS);
         LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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 c4cbbc1..87b370a 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
@@ -239,10 +239,12 @@ public class FSImageFormatPBSnapshot {
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
             pbf.getFileSize());
         List<BlockProto> bpl = pbf.getBlocksList();
+        // TODO: also persist striped blocks
         BlockInfoContiguous[] blocks = new BlockInfoContiguous[bpl.size()];
         for(int j = 0, e = bpl.size(); j < e; ++j) {
           Block blk = PBHelper.convert(bpl.get(j));
-          BlockInfoContiguous storedBlock =  fsn.getBlockManager().getStoredBlock(blk);
+          BlockInfoContiguous storedBlock =
+              (BlockInfoContiguous) fsn.getBlockManager().getStoredBlock(blk);
           if(storedBlock == null) {
             storedBlock = fsn.getBlockManager().addBlockCollection(
                 new BlockInfoContiguous(blk, copy.getFileReplication()), file);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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 7e7ff39..bb4f7aa 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
@@ -1560,8 +1560,8 @@ public class DFSTestUtil {
    */
   public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn,
       ExtendedBlock blk) {
-    BlockManager bm0 = nn.getNamesystem().getBlockManager();
-    BlockInfoContiguous storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
+    FSNamesystem fsn = nn.getNamesystem();
+    BlockInfoContiguous storedBlock = fsn.getStoredBlock(blk.getLocalBlock());
     assertTrue("Block " + blk + " should be under construction, " +
         "got: " + storedBlock,
         storedBlock instanceof BlockInfoContiguousUnderConstruction);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
index 7425c6a..89fd6db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
@@ -53,7 +53,7 @@ public class TestBlockInfo {
 
     final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1");
 
-    boolean added = blockInfo.addStorage(storage);
+    boolean added = blockInfo.addStorage(storage, blockInfo);
 
     Assert.assertTrue(added);
     Assert.assertEquals(storage, blockInfo.getStorageInfo(0));
@@ -108,7 +108,7 @@ public class TestBlockInfo {
     // list length should be equal to the number of blocks we inserted
     LOG.info("Checking list length...");
     assertEquals("Length should be MAX_BLOCK", MAX_BLOCKS, dd.numBlocks());
-    Iterator<BlockInfoContiguous> it = dd.getBlockIterator();
+    Iterator<BlockInfo> it = dd.getBlockIterator();
     int len = 0;
     while (it.hasNext()) {
       it.next();
@@ -130,7 +130,7 @@ public class TestBlockInfo {
     // move head of the list to the head - this should not change the list
     LOG.info("Moving head to the head...");
 
-    BlockInfoContiguous temp = dd.getBlockListHeadForTesting();
+    BlockInfo temp = dd.getBlockListHeadForTesting();
     curIndex = 0;
     headIndex = 0;
     dd.moveBlockToHead(temp, curIndex, headIndex);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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
new file mode 100644
index 0000000..74ddac0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
@@ -0,0 +1,219 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
+
+/**
+ * Test {@link BlockInfoStriped}
+ */
+public class TestBlockInfoStriped {
+  private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
+  private static final long BASE_ID = -1600;
+  private static final Block baseBlock = new Block(BASE_ID);
+  private BlockInfoStriped info;
+
+  @Before
+  public void setup() {
+    info = new BlockInfoStriped(baseBlock, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+  }
+
+  private Block[] createReportedBlocks(int num) {
+    Block[] blocks = new Block[num];
+    for (int i = 0; i < num; i++) {
+      blocks[i] = new Block(BASE_ID + i);
+    }
+    return blocks;
+  }
+
+  /**
+   * Test adding storage and reported block
+   */
+  @Test
+  public void testAddStorage() {
+    // first add NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS storages, i.e., a complete
+    // group of blocks/storages
+    DatanodeStorageInfo[] storageInfos = DFSTestUtil.createDatanodeStorageInfos(
+        TOTAL_NUM_BLOCKS);
+    Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS);
+    int i = 0;
+    for (; i < storageInfos.length; i += 2) {
+      info.addStorage(storageInfos[i], blocks[i]);
+      Assert.assertEquals(i/2 + 1, info.numNodes());
+    }
+    i /= 2;
+    for (int j = 1; j < storageInfos.length; j += 2) {
+      Assert.assertTrue(info.addStorage(storageInfos[j], blocks[j]));
+      Assert.assertEquals(i + (j+1)/2, info.numNodes());
+    }
+
+    // check
+    byte[] indices = (byte[]) Whitebox.getInternalState(info, "indices");
+    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity());
+    Assert.assertEquals(TOTAL_NUM_BLOCKS, indices.length);
+    i = 0;
+    for (DatanodeStorageInfo storage : storageInfos) {
+      int index = info.findStorageInfo(storage);
+      Assert.assertEquals(i++, index);
+      Assert.assertEquals(index, indices[index]);
+    }
+
+    // the same block is reported from the same storage twice
+    i = 0;
+    for (DatanodeStorageInfo storage : storageInfos) {
+      Assert.assertTrue(info.addStorage(storage, blocks[i++]));
+    }
+    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity());
+    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.numNodes());
+    Assert.assertEquals(TOTAL_NUM_BLOCKS, indices.length);
+    i = 0;
+    for (DatanodeStorageInfo storage : storageInfos) {
+      int index = info.findStorageInfo(storage);
+      Assert.assertEquals(i++, index);
+      Assert.assertEquals(index, indices[index]);
+    }
+
+    // the same block is reported from another storage
+    DatanodeStorageInfo[] storageInfos2 = DFSTestUtil.createDatanodeStorageInfos(
+        TOTAL_NUM_BLOCKS * 2);
+    // only add the second half of info2
+    for (i = TOTAL_NUM_BLOCKS; i < storageInfos2.length; i++) {
+      info.addStorage(storageInfos2[i], blocks[i % TOTAL_NUM_BLOCKS]);
+      Assert.assertEquals(i + 1, info.getCapacity());
+      Assert.assertEquals(i + 1, info.numNodes());
+      indices = (byte[]) Whitebox.getInternalState(info, "indices");
+      Assert.assertEquals(i + 1, indices.length);
+    }
+    for (i = TOTAL_NUM_BLOCKS; i < storageInfos2.length; i++) {
+      int index = info.findStorageInfo(storageInfos2[i]);
+      Assert.assertEquals(i++, index);
+      Assert.assertEquals(index - TOTAL_NUM_BLOCKS, indices[index]);
+    }
+  }
+
+  @Test
+  public void testRemoveStorage() {
+    // first add TOTAL_NUM_BLOCKS into the BlockInfoStriped
+    DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(
+        TOTAL_NUM_BLOCKS);
+    Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS);
+    for (int i = 0; i < storages.length; i++) {
+      info.addStorage(storages[i], blocks[i]);
+    }
+
+    // remove two storages
+    info.removeStorage(storages[0]);
+    info.removeStorage(storages[2]);
+
+    // check
+    Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity());
+    Assert.assertEquals(TOTAL_NUM_BLOCKS - 2, info.numNodes());
+    byte[] indices = (byte[]) Whitebox.getInternalState(info, "indices");
+    for (int i = 0; i < storages.length; i++) {
+      int index = info.findStorageInfo(storages[i]);
+      if (i != 0 && i != 2) {
+        Assert.assertEquals(i, index);
+        Assert.assertEquals(index, indices[index]);
+      } else {
+        Assert.assertEquals(-1, index);
+        Assert.assertEquals(-1, indices[i]);
+      }
+    }
+
+    // the same block is reported from another storage
+    DatanodeStorageInfo[] storages2 = DFSTestUtil.createDatanodeStorageInfos(
+        TOTAL_NUM_BLOCKS * 2);
+    for (int i = TOTAL_NUM_BLOCKS; i < storages2.length; i++) {
+      info.addStorage(storages2[i], blocks[i % TOTAL_NUM_BLOCKS]);
+    }
+    // now we should have 8 storages
+    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.numNodes());
+    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.getCapacity());
+    indices = (byte[]) Whitebox.getInternalState(info, "indices");
+    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, indices.length);
+    int j = TOTAL_NUM_BLOCKS;
+    for (int i = TOTAL_NUM_BLOCKS; i < storages2.length; i++) {
+      int index = info.findStorageInfo(storages2[i]);
+      if (i == TOTAL_NUM_BLOCKS || i == TOTAL_NUM_BLOCKS + 2) {
+        Assert.assertEquals(i - TOTAL_NUM_BLOCKS, index);
+      } else {
+        Assert.assertEquals(j++, index);
+      }
+    }
+
+    // remove the storages from storages2
+    for (int i = 0; i < TOTAL_NUM_BLOCKS; i++) {
+      info.removeStorage(storages2[i + TOTAL_NUM_BLOCKS]);
+    }
+    // now we should have 3 storages
+    Assert.assertEquals(TOTAL_NUM_BLOCKS - 2, info.numNodes());
+    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.getCapacity());
+    indices = (byte[]) Whitebox.getInternalState(info, "indices");
+    Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, indices.length);
+    for (int i = 0; i < TOTAL_NUM_BLOCKS; i++) {
+      if (i == 0 || i == 2) {
+        int index = info.findStorageInfo(storages2[i + TOTAL_NUM_BLOCKS]);
+        Assert.assertEquals(-1, index);
+      } else {
+        int index = info.findStorageInfo(storages[i]);
+        Assert.assertEquals(i, index);
+      }
+    }
+    for (int i = TOTAL_NUM_BLOCKS; i < TOTAL_NUM_BLOCKS * 2 - 2; i++) {
+      Assert.assertEquals(-1, indices[i]);
+      Assert.assertNull(info.getDatanode(i));
+    }
+  }
+
+  @Test
+  public void testReplaceBlock() {
+    DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(
+        TOTAL_NUM_BLOCKS);
+    Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS);
+    // add block/storage 0, 2, 4 into the BlockInfoStriped
+    for (int i = 0; i < storages.length; i += 2) {
+      Assert.assertEquals(AddBlockResult.ADDED,
+          storages[i].addBlock(info, blocks[i]));
+    }
+
+    BlockInfoStriped newBlockInfo = new BlockInfoStriped(info);
+    info.replaceBlock(newBlockInfo);
+
+    // make sure the newBlockInfo is correct
+    byte[] indices = (byte[]) Whitebox.getInternalState(newBlockInfo, "indices");
+    for (int i = 0; i < storages.length; i += 2) {
+      int index = newBlockInfo.findStorageInfo(storages[i]);
+      Assert.assertEquals(i, index);
+      Assert.assertEquals(index, indices[i]);
+
+      // make sure the newBlockInfo is added to the linked list of the storage
+      Assert.assertSame(newBlockInfo, storages[i].getBlockListHeadForTesting());
+      Assert.assertEquals(1, storages[i].numBlocks());
+      Assert.assertNull(newBlockInfo.getNext());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index d9ac9e5..42d2107 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -378,7 +378,7 @@ public class TestBlockManager {
     for (int i = 1; i < pipeline.length; i++) {
       DatanodeStorageInfo storage = pipeline[i];
       bm.addBlock(storage, blockInfo, null);
-      blockInfo.addStorage(storage);
+      blockInfo.addStorage(storage, blockInfo);
     }
   }
 
@@ -388,7 +388,7 @@ public class TestBlockManager {
 
     for (DatanodeDescriptor dn : nodes) {
       for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
-        blockInfo.addStorage(storage);
+        blockInfo.addStorage(storage, blockInfo);
       }
     }
     return blockInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 24fd81d..1a2a9f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -1234,7 +1234,7 @@ public class TestReplicationPolicy {
     when(storage.removeBlock(any(BlockInfoContiguous.class))).thenReturn(true);
     when(storage.addBlock(any(BlockInfoContiguous.class))).thenReturn
         (DatanodeStorageInfo.AddBlockResult.ADDED);
-    ucBlock.addStorage(storage);
+    ucBlock.addStorage(storage, ucBlock);
 
     when(mbc.setLastBlock((BlockInfoContiguous) any(), (DatanodeStorageInfo[]) any()))
     .thenReturn(ucBlock);


[09/50] [abbrv] hadoop git commit: YARN-3154. Added additional APIs in LogAggregationContext to avoid aggregating running logs of application when rolling is enabled. Contributed by Xuan Gong.

Posted by zh...@apache.org.
YARN-3154. Added additional APIs in LogAggregationContext to avoid aggregating running logs of application when rolling is enabled. Contributed by Xuan Gong.


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

Branch: refs/heads/HDFS-7285
Commit: 863079bb874ba77918ca1c0741eae10e245995c8
Parents: b49c3a1
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Thu Mar 12 13:32:29 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Thu Mar 12 13:32:29 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../yarn/api/records/LogAggregationContext.java | 79 ++++++++++++++++++--
 .../src/main/proto/yarn_protos.proto            |  2 +
 .../impl/pb/LogAggregationContextPBImpl.java    | 39 ++++++++++
 .../logaggregation/AggregatedLogFormat.java     | 39 +++++-----
 .../logaggregation/AppLogAggregatorImpl.java    | 22 ++++--
 .../TestContainerManagerRecovery.java           | 10 ++-
 .../TestLogAggregationService.java              | 73 +++++++++++++++---
 .../capacity/TestContainerAllocation.java       |  8 +-
 9 files changed, 227 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/863079bb/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 11d1cc9..80d2697 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -757,6 +757,9 @@ Release 2.7.0 - UNRELEASED
 
     YARN-3338. Exclude jline dependency from YARN. (Zhijie Shen via xgong)
 
+    YARN-3154. Added additional APIs in LogAggregationContext to avoid aggregating
+    running logs of application when rolling is enabled. (Xuan Gong via vinodkv)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/863079bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
index 46c1809..e582d2c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationContext.java
@@ -32,11 +32,20 @@ import org.apache.hadoop.yarn.util.Records;
  *   <ul>
  *     <li>includePattern. It uses Java Regex to filter the log files
  *     which match the defined include pattern and those log files
- *     will be uploaded. </li>
+ *     will be uploaded when the application finishes. </li>
  *     <li>excludePattern. It uses Java Regex to filter the log files
  *     which match the defined exclude pattern and those log files
- *     will not be uploaded. If the log file name matches both the
- *     include and the exclude pattern, this file will be excluded eventually</li>
+ *     will not be uploaded when application finishes. If the log file
+ *     name matches both the include and the exclude pattern, this file
+ *     will be excluded eventually</li>
+ *     <li>rolledLogsIncludePattern. It uses Java Regex to filter the log files
+ *     which match the defined include pattern and those log files
+ *     will be aggregated in a rolling fashion.</li>
+ *     <li>rolledLogsExcludePattern. It uses Java Regex to filter the log files
+ *     which match the defined exclude pattern and those log files
+ *     will not be aggregated in a rolling fashion. If the log file
+ *     name matches both the include and the exclude pattern, this file
+ *     will be excluded eventually</li>
  *   </ul>
  * </p>
  *
@@ -57,8 +66,23 @@ public abstract class LogAggregationContext {
     return context;
   }
 
+  @Public
+  @Unstable
+  public static LogAggregationContext newInstance(String includePattern,
+      String excludePattern, String rolledLogsIncludePattern,
+      String rolledLogsExcludePattern) {
+    LogAggregationContext context =
+        Records.newRecord(LogAggregationContext.class);
+    context.setIncludePattern(includePattern);
+    context.setExcludePattern(excludePattern);
+    context.setRolledLogsIncludePattern(rolledLogsIncludePattern);
+    context.setRolledLogsExcludePattern(rolledLogsExcludePattern);
+    return context;
+  }
+
   /**
-   * Get include pattern
+   * Get include pattern. This includePattern only takes affect
+   * on logs that exist at the time of application finish.
    *
    * @return include pattern
    */
@@ -67,7 +91,8 @@ public abstract class LogAggregationContext {
   public abstract String getIncludePattern();
 
   /**
-   * Set include pattern
+   * Set include pattern. This includePattern only takes affect
+   * on logs that exist at the time of application finish.
    *
    * @param includePattern
    */
@@ -76,7 +101,8 @@ public abstract class LogAggregationContext {
   public abstract void setIncludePattern(String includePattern);
 
   /**
-   * Get exclude pattern
+   * Get exclude pattern. This excludePattern only takes affect
+   * on logs that exist at the time of application finish.
    *
    * @return exclude pattern
    */
@@ -85,11 +111,50 @@ public abstract class LogAggregationContext {
   public abstract String getExcludePattern();
 
   /**
-   * Set exclude pattern
+   * Set exclude pattern. This excludePattern only takes affect
+   * on logs that exist at the time of application finish.
    *
    * @param excludePattern
    */
   @Public
   @Unstable
   public abstract void setExcludePattern(String excludePattern);
+
+  /**
+   * Get include pattern in a rolling fashion.
+   * 
+   * @return include pattern
+   */
+  @Public
+  @Unstable
+  public abstract String getRolledLogsIncludePattern();
+
+  /**
+   * Set include pattern in a rolling fashion.
+   * 
+   * @param rolledLogsIncludePattern
+   */
+  @Public
+  @Unstable
+  public abstract void setRolledLogsIncludePattern(
+      String rolledLogsIncludePattern);
+
+  /**
+   * Get exclude pattern for aggregation in a rolling fashion.
+   * 
+   * @return exclude pattern
+   */
+  @Public
+  @Unstable
+  public abstract String getRolledLogsExcludePattern();
+
+  /**
+   * Set exclude pattern for in a rolling fashion.
+   * 
+   * @param rolledLogsExcludePattern
+   */
+  @Public
+  @Unstable
+  public abstract void setRolledLogsExcludePattern(
+      String rolledLogsExcludePattern);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/863079bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 90706ed..2edff99 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -314,6 +314,8 @@ message ApplicationSubmissionContextProto {
 message LogAggregationContextProto {
  optional string include_pattern = 1 [default = ".*"];
  optional string exclude_pattern = 2 [default = ""];
+ optional string rolled_logs_include_pattern = 3 [default = ""];
+ optional string rolled_logs_exclude_pattern = 4 [default = ".*"];
 }
 
 enum ApplicationAccessTypeProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/863079bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LogAggregationContextPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LogAggregationContextPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LogAggregationContextPBImpl.java
index dc7a21d..f6409bb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LogAggregationContextPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LogAggregationContextPBImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationContextProtoOrBuilder;
+
 import com.google.protobuf.TextFormat;
 
 public class LogAggregationContextPBImpl extends LogAggregationContext{
@@ -116,4 +117,42 @@ public class LogAggregationContextPBImpl extends LogAggregationContext{
     }
     builder.setExcludePattern(excludePattern);
   }
+
+  @Override
+  public String getRolledLogsIncludePattern() {
+    LogAggregationContextProtoOrBuilder p = viaProto ? proto : builder;
+    if (! p.hasRolledLogsIncludePattern()) {
+      return null;
+    }
+    return p.getRolledLogsIncludePattern();
+  }
+
+  @Override
+  public void setRolledLogsIncludePattern(String rolledLogsIncludePattern) {
+    maybeInitBuilder();
+    if (rolledLogsIncludePattern == null) {
+      builder.clearRolledLogsIncludePattern();
+      return;
+    }
+    builder.setRolledLogsIncludePattern(rolledLogsIncludePattern);
+  }
+
+  @Override
+  public String getRolledLogsExcludePattern() {
+    LogAggregationContextProtoOrBuilder p = viaProto ? proto : builder;
+    if (! p.hasRolledLogsExcludePattern()) {
+      return null;
+    }
+    return p.getRolledLogsExcludePattern();
+  }
+
+  @Override
+  public void setRolledLogsExcludePattern(String rolledLogsExcludePattern) {
+    maybeInitBuilder();
+    if (rolledLogsExcludePattern == null) {
+      builder.clearRolledLogsExcludePattern();
+      return;
+    }
+    builder.setRolledLogsExcludePattern(rolledLogsExcludePattern);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/863079bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index b669332..29122dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -167,17 +167,18 @@ public class AggregatedLogFormat {
     private Set<File> uploadedFiles = new HashSet<File>();
     private final Set<String> alreadyUploadedLogFiles;
     private Set<String> allExistingFileMeta = new HashSet<String>();
+    private final boolean appFinished;
     // TODO Maybe add a version string here. Instead of changing the version of
     // the entire k-v format
 
     public LogValue(List<String> rootLogDirs, ContainerId containerId,
         String user) {
-      this(rootLogDirs, containerId, user, null, new HashSet<String>());
+      this(rootLogDirs, containerId, user, null, new HashSet<String>(), true);
     }
 
     public LogValue(List<String> rootLogDirs, ContainerId containerId,
         String user, LogAggregationContext logAggregationContext,
-        Set<String> alreadyUploadedLogFiles) {
+        Set<String> alreadyUploadedLogFiles, boolean appFinished) {
       this.rootLogDirs = new ArrayList<String>(rootLogDirs);
       this.containerId = containerId;
       this.user = user;
@@ -186,6 +187,7 @@ public class AggregatedLogFormat {
       Collections.sort(this.rootLogDirs);
       this.logAggregationContext = logAggregationContext;
       this.alreadyUploadedLogFiles = alreadyUploadedLogFiles;
+      this.appFinished = appFinished;
     }
 
     private Set<File> getPendingLogFilesToUploadForThisContainer() {
@@ -296,17 +298,15 @@ public class AggregatedLogFormat {
       }
 
       if (this.logAggregationContext != null && candidates.size() > 0) {
-        if (this.logAggregationContext.getIncludePattern() != null
-            && !this.logAggregationContext.getIncludePattern().isEmpty()) {
-          filterFiles(this.logAggregationContext.getIncludePattern(),
-              candidates, false);
-        }
+        filterFiles(
+          this.appFinished ? this.logAggregationContext.getIncludePattern()
+              : this.logAggregationContext.getRolledLogsIncludePattern(),
+          candidates, false);
 
-        if (this.logAggregationContext.getExcludePattern() != null
-            && !this.logAggregationContext.getExcludePattern().isEmpty()) {
-          filterFiles(this.logAggregationContext.getExcludePattern(),
-              candidates, true);
-        }
+        filterFiles(
+          this.appFinished ? this.logAggregationContext.getExcludePattern()
+              : this.logAggregationContext.getRolledLogsExcludePattern(),
+          candidates, true);
 
         Iterable<File> mask =
             Iterables.filter(candidates, new Predicate<File>() {
@@ -323,14 +323,15 @@ public class AggregatedLogFormat {
 
     private void filterFiles(String pattern, Set<File> candidates,
         boolean exclusion) {
-      Pattern filterPattern =
-          Pattern.compile(pattern);
-      for (Iterator<File> candidatesItr = candidates.iterator(); candidatesItr
+      if (pattern != null && !pattern.isEmpty()) {
+        Pattern filterPattern = Pattern.compile(pattern);
+        for (Iterator<File> candidatesItr = candidates.iterator(); candidatesItr
           .hasNext();) {
-        File candidate = candidatesItr.next();
-        boolean match = filterPattern.matcher(candidate.getName()).find();
-        if ((!match && !exclusion) || (match && exclusion)) {
-          candidatesItr.remove();
+          File candidate = candidatesItr.next();
+          boolean match = filterPattern.matcher(candidate.getName()).find();
+          if ((!match && !exclusion) || (match && exclusion)) {
+            candidatesItr.remove();
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/863079bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
index 787422b..ff70a68 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -116,6 +115,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
   private final Context context;
   private final int retentionSize;
   private final long rollingMonitorInterval;
+  private final boolean logAggregationInRolling;
   private final NodeId nodeId;
   // This variable is only for testing
   private final AtomicBoolean waiting = new AtomicBoolean(false);
@@ -193,9 +193,14 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
       }
       this.rollingMonitorInterval = configuredRollingMonitorInterval;
     }
+    this.logAggregationInRolling =
+        this.rollingMonitorInterval <= 0 || this.logAggregationContext == null
+            || this.logAggregationContext.getRolledLogsIncludePattern() == null
+            || this.logAggregationContext.getRolledLogsIncludePattern()
+              .isEmpty() ? false : true;
   }
 
-  private void uploadLogsForContainers() {
+  private void uploadLogsForContainers(boolean appFinished) {
     if (this.logAggregationDisabled) {
       return;
     }
@@ -262,7 +267,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
           containerLogAggregators.put(container, aggregator);
         }
         Set<Path> uploadedFilePathsInThisCycle =
-            aggregator.doContainerLogAggregation(writer);
+            aggregator.doContainerLogAggregation(writer, appFinished);
         if (uploadedFilePathsInThisCycle.size() > 0) {
           uploadedLogsInThisCycle = true;
         }
@@ -394,12 +399,12 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
       synchronized(this) {
         try {
           waiting.set(true);
-          if (this.rollingMonitorInterval > 0) {
+          if (logAggregationInRolling) {
             wait(this.rollingMonitorInterval * 1000);
             if (this.appFinishing.get() || this.aborted.get()) {
               break;
             }
-            uploadLogsForContainers();
+            uploadLogsForContainers(false);
           } else {
             wait(THREAD_SLEEP_TIME);
           }
@@ -415,7 +420,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     }
 
     // App is finished, upload the container logs.
-    uploadLogsForContainers();
+    uploadLogsForContainers(true);
 
     // Remove the local app-log-dirs
     List<Path> localAppLogDirs = new ArrayList<Path>();
@@ -536,7 +541,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
       this.containerId = containerId;
     }
 
-    public Set<Path> doContainerLogAggregation(LogWriter writer) {
+    public Set<Path> doContainerLogAggregation(LogWriter writer,
+        boolean appFinished) {
       LOG.info("Uploading logs for container " + containerId
           + ". Current good log dirs are "
           + StringUtils.join(",", dirsHandler.getLogDirs()));
@@ -544,7 +550,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
       final LogValue logValue =
           new LogValue(dirsHandler.getLogDirs(), containerId,
             userUgi.getShortUserName(), logAggregationContext,
-            this.uploadedFileMeta);
+            this.uploadedFileMeta, appFinished);
       try {
         writer.append(logKey, logValue);
       } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/863079bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index a73d583..c45ffbb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -130,8 +130,10 @@ public class TestContainerManagerRecovery {
         containerTokens, acls);
     // create the logAggregationContext
     LogAggregationContext logAggregationContext =
-        LogAggregationContext.newInstance("includePattern", "excludePattern");
-    StartContainersResponse startResponse = startContainer(context, cm, cid,
+        LogAggregationContext.newInstance("includePattern", "excludePattern",
+          "includePatternInRollingAggregation",
+          "excludePatternInRollingAggregation");
+   StartContainersResponse startResponse = startContainer(context, cm, cid,
         clc, logAggregationContext);
     assertTrue(startResponse.getFailedRequests().isEmpty());
     assertEquals(1, context.getApplications().size());
@@ -171,6 +173,10 @@ public class TestContainerManagerRecovery {
       recovered.getIncludePattern());
     assertEquals(logAggregationContext.getExcludePattern(),
       recovered.getExcludePattern());
+    assertEquals(logAggregationContext.getRolledLogsIncludePattern(),
+      recovered.getRolledLogsIncludePattern());
+    assertEquals(logAggregationContext.getRolledLogsExcludePattern(),
+      recovered.getRolledLogsExcludePattern());
 
     waitForAppState(app, ApplicationState.INITING);
     assertTrue(context.getApplicationACLsManager().checkAccess(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/863079bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index 901e45a..df51a0d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -698,7 +698,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     }
   }
 
-  private String verifyContainerLogs(LogAggregationService logAggregationService,
+  private LogFileStatusInLastCycle verifyContainerLogs(LogAggregationService logAggregationService,
       ApplicationId appId, ContainerId[] expectedContainerIds,
       String[] logFiles, int numOfContainerLogs, boolean multiLogs)
       throws IOException {
@@ -743,7 +743,9 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         new AggregatedLogFormat.LogReader(this.conf, targetNodeFile.getPath());
     Assert.assertEquals(this.user, reader.getApplicationOwner());
     verifyAcls(reader.getApplicationAcls());
-    
+
+    List<String> fileTypes = new ArrayList<String>();
+
     try {
       Map<String, Map<String, String>> logMap =
           new HashMap<String, Map<String, String>>();
@@ -769,6 +771,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
 
             Assert.assertEquals("LogType:", writtenLines[0].substring(0, 8));
             String fileType = writtenLines[0].substring(8);
+            fileTypes.add(fileType);
 
             Assert.assertEquals("LogLength:", writtenLines[1].substring(0, 10));
             String fileLengthStr = writtenLines[1].substring(10);
@@ -811,7 +814,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         Assert.assertEquals(0, thisContainerMap.size());
       }
       Assert.assertEquals(0, logMap.size());
-      return targetNodeFile.getPath().getName();
+      return new LogFileStatusInLastCycle(targetNodeFile.getPath().getName(), fileTypes);
     } finally {
       reader.close();
     }
@@ -1289,6 +1292,12 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
       throws Exception {
     LogAggregationContext logAggregationContextWithInterval =
         Records.newRecord(LogAggregationContext.class);
+    // set IncludePattern/excludePattern in rolling fashion
+    // we expect all the logs except std_final will be uploaded
+    // when app is running. The std_final will be uploaded when
+    // the app finishes.
+    logAggregationContextWithInterval.setRolledLogsIncludePattern(".*");
+    logAggregationContextWithInterval.setRolledLogsExcludePattern("std_final");
     this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
     this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
       this.remoteRootLogDir.getAbsolutePath());
@@ -1338,9 +1347,14 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
       this.user, null, ContainerLogsRetentionPolicy.ALL_CONTAINERS, this.acls,
       logAggregationContextWithInterval));
 
+    LogFileStatusInLastCycle logFileStatusInLastCycle = null;
     // Simulate log-file creation
-    String[] logFiles1 = new String[] { "stdout", "stderr", "syslog" };
-    writeContainerLogs(appLogDir, container, logFiles1);
+    // create std_final in log directory which will not be aggregated
+    // until the app finishes.
+    String[] logFiles1WithFinalLog =
+        new String[] { "stdout", "stderr", "syslog", "std_final" };
+    String[] logFiles1 = new String[] { "stdout", "stderr", "syslog"};
+    writeContainerLogs(appLogDir, container, logFiles1WithFinalLog);
 
     // Do log aggregation
     AppLogAggregatorImpl aggregator =
@@ -1355,10 +1369,16 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
       Assert.assertTrue(waitAndCheckLogNum(logAggregationService, application,
         50, 1, false, null));
     }
-    String logFileInLastCycle = null;
     // Container logs should be uploaded
-    logFileInLastCycle = verifyContainerLogs(logAggregationService, application,
+    logFileStatusInLastCycle = verifyContainerLogs(logAggregationService, application,
         new ContainerId[] { container }, logFiles1, 3, true);
+    for(String logFile : logFiles1) {
+      Assert.assertTrue(logFileStatusInLastCycle.getLogFileTypesInLastCycle()
+        .contains(logFile));
+    }
+    // Make sure the std_final is not uploaded.
+    Assert.assertFalse(logFileStatusInLastCycle.getLogFileTypesInLastCycle()
+      .contains("std_final"));
 
     Thread.sleep(2000);
 
@@ -1380,15 +1400,23 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
 
     if (retentionSizeLimitation) {
       Assert.assertTrue(waitAndCheckLogNum(logAggregationService, application,
-        50, 1, true, logFileInLastCycle));
+        50, 1, true, logFileStatusInLastCycle.getLogFilePathInLastCycle()));
     } else {
       Assert.assertTrue(waitAndCheckLogNum(logAggregationService, application,
         50, 2, false, null));
     }
     // Container logs should be uploaded
-    logFileInLastCycle = verifyContainerLogs(logAggregationService, application,
+    logFileStatusInLastCycle = verifyContainerLogs(logAggregationService, application,
         new ContainerId[] { container }, logFiles2, 3, true);
 
+    for(String logFile : logFiles2) {
+      Assert.assertTrue(logFileStatusInLastCycle.getLogFileTypesInLastCycle()
+        .contains(logFile));
+    }
+    // Make sure the std_final is not uploaded.
+    Assert.assertFalse(logFileStatusInLastCycle.getLogFileTypesInLastCycle()
+      .contains("std_final"));
+
     Thread.sleep(2000);
 
     // create another logs
@@ -1402,13 +1430,17 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     logAggregationService.handle(new LogHandlerAppFinishedEvent(application));
     if (retentionSizeLimitation) {
       Assert.assertTrue(waitAndCheckLogNum(logAggregationService, application,
-        50, 1, true, logFileInLastCycle));
+        50, 1, true, logFileStatusInLastCycle.getLogFilePathInLastCycle()));
     } else {
       Assert.assertTrue(waitAndCheckLogNum(logAggregationService, application,
         50, 3, false, null));
     }
+
+    // the app is finished. The log "std_final" should be aggregated this time.
+    String[] logFiles3WithFinalLog =
+        new String[] { "stdout_2", "stderr_2", "syslog_2", "std_final" };
     verifyContainerLogs(logAggregationService, application,
-      new ContainerId[] { container }, logFiles3, 3, true);
+      new ContainerId[] { container }, logFiles3WithFinalLog, 4, true);
     logAggregationService.stop();
     assertEquals(0, logAggregationService.getNumAggregators());
   }
@@ -1512,4 +1544,23 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     return numOfLogsAvailable(logAggregationService, application, sizeLimited,
       lastLogFile) == expectNum;
   }
+
+  private static class LogFileStatusInLastCycle {
+    private String logFilePathInLastCycle;
+    private List<String> logFileTypesInLastCycle;
+
+    public LogFileStatusInLastCycle(String logFilePathInLastCycle,
+        List<String> logFileTypesInLastCycle) {
+      this.logFilePathInLastCycle = logFilePathInLastCycle;
+      this.logFileTypesInLastCycle = logFileTypesInLastCycle;
+    }
+
+    public String getLogFilePathInLastCycle() {
+      return this.logFilePathInLastCycle;
+    }
+
+    public List<String> getLogFileTypesInLastCycle() {
+      return this.logFileTypesInLastCycle;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/863079bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
index 169517d..0ad2957 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
@@ -230,12 +230,18 @@ public class TestContainerAllocation {
     // create a not-null LogAggregationContext
     LogAggregationContext logAggregationContext =
         LogAggregationContext.newInstance(
-          "includePattern", "excludePattern");
+          "includePattern", "excludePattern",
+          "rolledLogsIncludePattern",
+          "rolledLogsExcludePattern");
     LogAggregationContext returned =
         getLogAggregationContextFromContainerToken(rm1, nm2,
           logAggregationContext);
     Assert.assertEquals("includePattern", returned.getIncludePattern());
     Assert.assertEquals("excludePattern", returned.getExcludePattern());
+    Assert.assertEquals("rolledLogsIncludePattern",
+      returned.getRolledLogsIncludePattern());
+    Assert.assertEquals("rolledLogsExcludePattern",
+      returned.getRolledLogsExcludePattern());
     rm1.stop();
   }
 


[35/50] [abbrv] hadoop git commit: HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding (Kai Zheng via umamahesh)

Posted by zh...@apache.org.
HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding (Kai Zheng via umamahesh)


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

Branch: refs/heads/HDFS-7285
Commit: ea10f9b0f6f3afc109e25f0752ef0b9ca731fb8c
Parents: 68ecc4e
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Thu Jan 29 14:15:13 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:08:49 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  4 +
 .../apache/hadoop/io/erasurecode/ECChunk.java   | 82 +++++++++++++++++
 .../rawcoder/AbstractRawErasureCoder.java       | 63 +++++++++++++
 .../rawcoder/AbstractRawErasureDecoder.java     | 93 ++++++++++++++++++++
 .../rawcoder/AbstractRawErasureEncoder.java     | 93 ++++++++++++++++++++
 .../erasurecode/rawcoder/RawErasureCoder.java   | 78 ++++++++++++++++
 .../erasurecode/rawcoder/RawErasureDecoder.java | 55 ++++++++++++
 .../erasurecode/rawcoder/RawErasureEncoder.java | 54 ++++++++++++
 8 files changed, 522 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea10f9b0/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
new file mode 100644
index 0000000..8ce5a89
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -0,0 +1,4 @@
+  BREAKDOWN OF HADOOP-11264 SUBTASKS AND RELATED JIRAS (Common part of HDFS-7285)
+
+    HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding
+    (Kai Zheng via umamahesh)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea10f9b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
new file mode 100644
index 0000000..f84eb11
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A wrapper for ByteBuffer or bytes array for an erasure code chunk.
+ */
+public class ECChunk {
+
+  private ByteBuffer chunkBuffer;
+
+  /**
+   * Wrapping a ByteBuffer
+   * @param buffer
+   */
+  public ECChunk(ByteBuffer buffer) {
+    this.chunkBuffer = buffer;
+  }
+
+  /**
+   * Wrapping a bytes array
+   * @param buffer
+   */
+  public ECChunk(byte[] buffer) {
+    this.chunkBuffer = ByteBuffer.wrap(buffer);
+  }
+
+  /**
+   * Convert to ByteBuffer
+   * @return ByteBuffer
+   */
+  public ByteBuffer getBuffer() {
+    return chunkBuffer;
+  }
+
+  /**
+   * Convert an array of this chunks to an array of ByteBuffers
+   * @param chunks
+   * @return an array of ByteBuffers
+   */
+  public static ByteBuffer[] toBuffers(ECChunk[] chunks) {
+    ByteBuffer[] buffers = new ByteBuffer[chunks.length];
+
+    for (int i = 0; i < chunks.length; i++) {
+      buffers[i] = chunks[i].getBuffer();
+    }
+
+    return buffers;
+  }
+
+  /**
+   * Convert an array of this chunks to an array of byte array
+   * @param chunks
+   * @return an array of byte array
+   */
+  public static byte[][] toArray(ECChunk[] chunks) {
+    byte[][] bytesArr = new byte[chunks.length][];
+
+    for (int i = 0; i < chunks.length; i++) {
+      bytesArr[i] = chunks[i].getBuffer().array();
+    }
+
+    return bytesArr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea10f9b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
new file mode 100644
index 0000000..474542b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
@@ -0,0 +1,63 @@
+/**
+ * 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 common class of basic facilities to be shared by encoder and decoder
+ *
+ * It implements the {@link RawErasureCoder} interface.
+ */
+public abstract class AbstractRawErasureCoder implements RawErasureCoder {
+
+  private int dataSize;
+  private int paritySize;
+  private int chunkSize;
+
+  @Override
+  public void initialize(int numDataUnits, int numParityUnits,
+                         int chunkSize) {
+    this.dataSize = numDataUnits;
+    this.paritySize = numParityUnits;
+    this.chunkSize = chunkSize;
+  }
+
+  @Override
+  public int getNumDataUnits() {
+    return dataSize;
+  }
+
+  @Override
+  public int getNumParityUnits() {
+    return paritySize;
+  }
+
+  @Override
+  public int getChunkSize() {
+    return chunkSize;
+  }
+
+  @Override
+  public boolean preferNativeBuffer() {
+    return false;
+  }
+
+  @Override
+  public void release() {
+    // Nothing to do by default
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea10f9b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
new file mode 100644
index 0000000..4613b25
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.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 java.nio.ByteBuffer;
+
+/**
+ * An abstract raw erasure decoder that's to be inherited by new decoders.
+ *
+ * It implements the {@link RawErasureDecoder} interface.
+ */
+public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
+    implements RawErasureDecoder {
+
+  @Override
+  public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
+                     ByteBuffer[] outputs) {
+    if (erasedIndexes.length == 0) {
+      return;
+    }
+
+    doDecode(inputs, erasedIndexes, outputs);
+  }
+
+  /**
+   * Perform the real decoding using ByteBuffer
+   * @param inputs
+   * @param erasedIndexes
+   * @param outputs
+   */
+  protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                                   ByteBuffer[] outputs);
+
+  @Override
+  public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) {
+    if (erasedIndexes.length == 0) {
+      return;
+    }
+
+    doDecode(inputs, erasedIndexes, outputs);
+  }
+
+  /**
+   * Perform the real decoding using bytes array
+   * @param inputs
+   * @param erasedIndexes
+   * @param outputs
+   */
+  protected abstract void doDecode(byte[][] inputs, int[] erasedIndexes,
+                                   byte[][] outputs);
+
+  @Override
+  public void decode(ECChunk[] inputs, int[] erasedIndexes,
+                     ECChunk[] outputs) {
+    doDecode(inputs, erasedIndexes, outputs);
+  }
+
+  /**
+   * Perform the real decoding using chunks
+   * @param inputs
+   * @param erasedIndexes
+   * @param outputs
+   */
+  protected void doDecode(ECChunk[] inputs, int[] erasedIndexes,
+                          ECChunk[] outputs) {
+    if (inputs[0].getBuffer().hasArray()) {
+      byte[][] inputBytesArr = ECChunk.toArray(inputs);
+      byte[][] outputBytesArr = ECChunk.toArray(outputs);
+      doDecode(inputBytesArr, erasedIndexes, outputBytesArr);
+    } else {
+      ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
+      ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
+      doDecode(inputBuffers, erasedIndexes, outputBuffers);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea10f9b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
new file mode 100644
index 0000000..4feaf39
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.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 java.nio.ByteBuffer;
+
+/**
+ * An abstract raw erasure encoder that's to be inherited by new encoders.
+ *
+ * It implements the {@link RawErasureEncoder} interface.
+ */
+public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
+    implements RawErasureEncoder {
+
+  @Override
+  public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+    assert (inputs.length == getNumDataUnits());
+    assert (outputs.length == getNumParityUnits());
+
+    doEncode(inputs, outputs);
+  }
+
+  /**
+   * Perform the real encoding work using ByteBuffer
+   * @param inputs
+   * @param outputs
+   */
+  protected abstract void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs);
+
+  @Override
+  public void encode(byte[][] inputs, byte[][] outputs) {
+    assert (inputs.length == getNumDataUnits());
+    assert (outputs.length == getNumParityUnits());
+
+    doEncode(inputs, outputs);
+  }
+
+  /**
+   * Perform the real encoding work using bytes array
+   * @param inputs
+   * @param outputs
+   */
+  protected abstract void doEncode(byte[][] inputs, byte[][] outputs);
+
+  @Override
+  public void encode(ECChunk[] inputs, ECChunk[] outputs) {
+    assert (inputs.length == getNumDataUnits());
+    assert (outputs.length == getNumParityUnits());
+
+    doEncode(inputs, outputs);
+  }
+
+  /**
+   * Perform the real encoding work using chunks.
+   * @param inputs
+   * @param outputs
+   */
+  protected void doEncode(ECChunk[] inputs, ECChunk[] outputs) {
+    /**
+     * Note callers may pass byte array, or ByteBuffer via ECChunk according
+     * to how ECChunk is created. Some implementations of coder use byte array
+     * (ex: pure Java), some use native ByteBuffer (ex: ISA-L), all for the
+     * better performance.
+     */
+    if (inputs[0].getBuffer().hasArray()) {
+      byte[][] inputBytesArr = ECChunk.toArray(inputs);
+      byte[][] outputBytesArr = ECChunk.toArray(outputs);
+      doEncode(inputBytesArr, outputBytesArr);
+    } else {
+      ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
+      ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
+      doEncode(inputBuffers, outputBuffers);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea10f9b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
new file mode 100644
index 0000000..91a9abf
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.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;
+
+/**
+ * RawErasureCoder is a common interface for {@link RawErasureEncoder} and
+ * {@link RawErasureDecoder} as both encoder and decoder share some properties.
+ *
+ * RawErasureCoder is part of ErasureCodec framework, where ErasureCoder is
+ * used to encode/decode a group of blocks (BlockGroup) according to the codec
+ * specific BlockGroup layout and logic. An ErasureCoder extracts chunks of
+ * data from the blocks and can employ various low level RawErasureCoders to
+ * perform encoding/decoding against the chunks.
+ *
+ * To distinguish from ErasureCoder, here RawErasureCoder is used to mean the
+ * low level constructs, since it only takes care of the math calculation with
+ * a group of byte buffers.
+ */
+public interface RawErasureCoder {
+
+  /**
+   * Initialize with the important parameters for the code.
+   * @param numDataUnits how many data inputs for the coding
+   * @param numParityUnits how many parity outputs the coding generates
+   * @param chunkSize the size of the input/output buffer
+   */
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize);
+
+  /**
+   * The number of data input units for the coding. A unit can be a byte,
+   * chunk or buffer or even a block.
+   * @return count of data input units
+   */
+  public int getNumDataUnits();
+
+  /**
+   * The number of parity output units for the coding. A unit can be a byte,
+   * chunk, buffer or even a block.
+   * @return count of parity output units
+   */
+  public int getNumParityUnits();
+
+  /**
+   * Chunk buffer size for the input/output
+   * @return chunk buffer size
+   */
+  public int getChunkSize();
+
+  /**
+   * Tell if native or off-heap buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, either on heap or off heap.
+   * It will return false by default.
+   * @return true if native buffer is preferred for performance consideration,
+   * otherwise false.
+   */
+  public boolean preferNativeBuffer();
+
+  /**
+   * Should be called when release this coder. Good chance to release encoding
+   * or decoding buffers
+   */
+  public void release();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea10f9b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
new file mode 100644
index 0000000..1358b7d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
@@ -0,0 +1,55 @@
+/**
+ * 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 java.nio.ByteBuffer;
+
+/**
+ * RawErasureDecoder performs decoding given chunks of input data and generates
+ * missing data that corresponds to an erasure code scheme, like XOR and
+ * Reed-Solomon.
+ *
+ * It extends the {@link RawErasureCoder} interface.
+ */
+public interface RawErasureDecoder extends RawErasureCoder {
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
+                     ByteBuffer[] outputs);
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs);
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea10f9b0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java
new file mode 100644
index 0000000..974f86c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.ECChunk;
+
+import java.nio.ByteBuffer;
+
+/**
+ * RawErasureEncoder performs encoding given chunks of input data and generates
+ * parity outputs that corresponds to an erasure code scheme, like XOR and
+ * Reed-Solomon.
+ *
+ * It extends the {@link RawErasureCoder} interface.
+ */
+public interface RawErasureEncoder extends RawErasureCoder {
+
+  /**
+   * Encode with inputs and generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs);
+
+  /**
+   * Encode with inputs and generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void encode(byte[][] inputs, byte[][] outputs);
+
+  /**
+   * Encode with inputs and generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void encode(ECChunk[] inputs, ECChunk[] outputs);
+
+}


[34/50] [abbrv] hadoop git commit: HDFS-7652. Process block reports for erasure coded blocks. Contributed by Zhe Zhang

Posted by zh...@apache.org.
HDFS-7652. Process block reports for erasure coded blocks. 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/86bc54a5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/86bc54a5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/86bc54a5

Branch: refs/heads/HDFS-7285
Commit: 86bc54a52b0035a7cb766a3950a93876c74bf426
Parents: 10c794c
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Feb 9 10:27:14 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:08:48 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockIdManager.java    |  8 ++++++++
 .../hdfs/server/blockmanagement/BlockManager.java | 18 +++++++++++++-----
 2 files changed, 21 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/86bc54a5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index c8b9d20..e7f8a05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -211,4 +211,12 @@ public class BlockIdManager {
       .LAST_RESERVED_BLOCK_ID);
     generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
   }
+
+  public static boolean isStripedBlockID(long id) {
+    return id < 0;
+  }
+
+  public static long convertToGroupID(long id) {
+    return id & (~(HdfsConstants.MAX_BLOCKS_IN_GROUP - 1));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86bc54a5/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 f155375..ec07a5f 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
@@ -1876,7 +1876,7 @@ public class BlockManager {
           break;
         }
 
-        BlockInfoContiguous bi = blocksMap.getStoredBlock(b);
+        BlockInfoContiguous bi = getStoredBlock(b);
         if (bi == null) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
@@ -1979,7 +1979,7 @@ public class BlockManager {
         continue;
       }
       
-      BlockInfoContiguous storedBlock = blocksMap.getStoredBlock(iblk);
+      BlockInfoContiguous storedBlock = getStoredBlock(iblk);
       // If block does not belong to any file, we are done.
       if (storedBlock == null) continue;
       
@@ -2119,7 +2119,7 @@ public class BlockManager {
     }
     
     // find block by blockId
-    BlockInfoContiguous storedBlock = blocksMap.getStoredBlock(block);
+    BlockInfoContiguous storedBlock = getStoredBlock(block);
     if(storedBlock == null) {
       // If blocksMap does not contain reported block id,
       // the replica should be removed from the data-node.
@@ -2410,7 +2410,7 @@ public class BlockManager {
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     if (block instanceof BlockInfoContiguousUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
-      storedBlock = blocksMap.getStoredBlock(block);
+      storedBlock = getStoredBlock(block);
     } else {
       storedBlock = block;
     }
@@ -3275,7 +3275,15 @@ public class BlockManager {
   }
 
   public BlockInfoContiguous getStoredBlock(Block block) {
-    return blocksMap.getStoredBlock(block);
+    BlockInfoContiguous info = null;
+    if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
+      info = blocksMap.getStoredBlock(
+          new Block(BlockIdManager.convertToGroupID(block.getBlockId())));
+    }
+    if (info == null) {
+      info = blocksMap.getStoredBlock(block);
+    }
+    return info;
   }
 
   /** updates a block in under replication queue */


[19/50] [abbrv] hadoop git commit: Revert "HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)" (jenkins didn't run yet)

Posted by zh...@apache.org.
Revert "HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)" (jenkins didn't run yet)

This reverts commit 5aa892ed486d42ae6b94c4866b92cd2b382ea640.


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

Branch: refs/heads/HDFS-7285
Commit: 32741cf3d25d85a92e3deb11c302cc2a718d71dd
Parents: 5aa892e
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Mar 13 18:40:20 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Mar 13 18:40:20 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 -
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 23 +----
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  2 -
 .../datatransfer/DataTransferProtocol.java      |  5 +-
 .../hdfs/protocol/datatransfer/Receiver.java    |  2 +-
 .../hdfs/protocol/datatransfer/Sender.java      |  4 +-
 .../hdfs/server/datanode/DataXceiver.java       | 95 ++++++++------------
 .../server/datanode/ShortCircuitRegistry.java   | 13 +--
 .../src/main/proto/datatransfer.proto           | 11 ---
 .../shortcircuit/TestShortCircuitCache.java     | 63 -------------
 10 files changed, 43 insertions(+), 178 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ff00b0c..c3f9367 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1177,9 +1177,6 @@ Release 2.7.0 - UNRELEASED
       HDFS-7722. DataNode#checkDiskError should also remove Storage when error
       is found. (Lei Xu via Colin P. McCabe)
 
-      HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and
-      fail to tell the DFSClient about it because of a network error (cmccabe)
-
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
index 1e915b2..ba48c79 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
-
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -71,12 +69,6 @@ import com.google.common.base.Preconditions;
 public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
 
-  public static class FailureInjector {
-    public void injectRequestFileDescriptorsFailure() throws IOException {
-      // do nothing
-    }
-  }
-
   @VisibleForTesting
   static ShortCircuitReplicaCreator
       createShortCircuitReplicaInfoCallback = null;
@@ -84,11 +76,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   private final DFSClient.Conf conf;
 
   /**
-   * Injects failures into specific operations during unit tests.
-   */
-  private final FailureInjector failureInjector;
-
-  /**
    * The file name, for logging and debugging purposes.
    */
   private String fileName;
@@ -182,7 +169,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
 
   public BlockReaderFactory(DFSClient.Conf conf) {
     this.conf = conf;
-    this.failureInjector = conf.brfFailureInjector;
     this.remainingCacheTries = conf.nCachedConnRetry;
   }
 
@@ -532,12 +518,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
     SlotId slotId = slot == null ? null : slot.getSlotId();
-    new Sender(out).requestShortCircuitFds(block, token, slotId, 1, true);
+    new Sender(out).requestShortCircuitFds(block, token, slotId, 1);
     DataInputStream in = new DataInputStream(peer.getInputStream());
     BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
         PBHelper.vintPrefixed(in));
     DomainSocket sock = peer.getDomainSocket();
-    failureInjector.injectRequestFileDescriptorsFailure();
     switch (resp.getStatus()) {
     case SUCCESS:
       byte buf[] = new byte[1];
@@ -547,13 +532,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       try {
         ExtendedBlockId key =
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
-        if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
-          LOG.trace("Sending receipt verification byte for slot " + slot);
-          sock.getOutputStream().write(0);
-        }
         replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
             Time.monotonicNow(), slot);
-        return new ShortCircuitReplicaInfo(replica);
       } catch (IOException e) {
         // This indicates an error reading from disk, or a format error.  Since
         // it's not a socket communication problem, we return null rather than
@@ -565,6 +545,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
           IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
         }
       }
+      return new ShortCircuitReplicaInfo(replica);
     case ERROR_UNSUPPORTED:
       if (!resp.hasShortCircuitAccessVersion()) {
         LOG.warn("short-circuit read access is disabled for " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/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 f970fef..aac7b51 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
@@ -337,8 +337,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     final long shortCircuitCacheStaleThresholdMs;
 
     final long keyProviderCacheExpiryMs;
-    public BlockReaderFactory.FailureInjector brfFailureInjector =
-      new BlockReaderFactory.FailureInjector();
 
     public Conf(Configuration conf) {
       // The hdfsTimeout is currently the same as the ipc timeout 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
index 48e931d..4be42a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
@@ -138,13 +138,10 @@ public interface DataTransferProtocol {
    *                          to use no slot id.
    * @param maxVersion      Maximum version of the block data the client 
    *                          can understand.
-   * @param supportsReceiptVerification  True if the client supports
-   *                          receipt verification.
    */
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
-        throws IOException;
+      SlotId slotId, int maxVersion) throws IOException;
 
   /**
    * Release a pair of short-circuit FDs requested earlier.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
index 31bdc5e..7994027 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
@@ -186,7 +186,7 @@ public abstract class Receiver implements DataTransferProtocol {
     try {
       requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
           PBHelper.convert(proto.getHeader().getToken()),
-          slotId, proto.getMaxVersion(), true);
+          slotId, proto.getMaxVersion());
     } finally {
       if (traceScope != null) traceScope.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index df69125..7fea33e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
@@ -181,8 +181,7 @@ public class Sender implements DataTransferProtocol {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
-        throws IOException {
+      SlotId slotId, int maxVersion) throws IOException {
     OpRequestShortCircuitAccessProto.Builder builder =
         OpRequestShortCircuitAccessProto.newBuilder()
           .setHeader(DataTransferProtoUtil.buildBaseHeader(
@@ -190,7 +189,6 @@ public class Sender implements DataTransferProtocol {
     if (slotId != null) {
       builder.setSlotId(PBHelper.convert(slotId));
     }
-    builder.setSupportsReceiptVerification(supportsReceiptVerification);
     OpRequestShortCircuitAccessProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 84504fb..e9547a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -22,8 +22,6 @@ import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ER
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_INVALID;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.DO_NOT_USE_RECEIPT_VERIFICATION;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 import static org.apache.hadoop.util.Time.now;
 
@@ -293,83 +291,64 @@ class DataXceiver extends Receiver implements Runnable {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> token,
-      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
-        throws IOException {
+      SlotId slotId, int maxVersion) throws IOException {
     updateCurrentThreadName("Passing file descriptors for block " + blk);
     BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
     FileInputStream fis[] = null;
-    SlotId registeredSlotId = null;
-    boolean success = false;
     try {
+      if (peer.getDomainSocket() == null) {
+        throw new IOException("You cannot pass file descriptors over " +
+            "anything but a UNIX domain socket.");
+      }
+      if (slotId != null) {
+        boolean isCached = datanode.data.
+            isCached(blk.getBlockPoolId(), blk.getBlockId());
+        datanode.shortCircuitRegistry.registerSlot(
+            ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
+      }
       try {
-        if (peer.getDomainSocket() == null) {
-          throw new IOException("You cannot pass file descriptors over " +
-              "anything but a UNIX domain socket.");
-        }
-        if (slotId != null) {
-          boolean isCached = datanode.data.
-              isCached(blk.getBlockPoolId(), blk.getBlockId());
-          datanode.shortCircuitRegistry.registerSlot(
-              ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
-          registeredSlotId = slotId;
-        }
         fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
-        Preconditions.checkState(fis != null);
-        bld.setStatus(SUCCESS);
-        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-      } catch (ShortCircuitFdsVersionException e) {
-        bld.setStatus(ERROR_UNSUPPORTED);
-        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-        bld.setMessage(e.getMessage());
-      } catch (ShortCircuitFdsUnsupportedException e) {
-        bld.setStatus(ERROR_UNSUPPORTED);
-        bld.setMessage(e.getMessage());
-      } catch (InvalidToken e) {
-        bld.setStatus(ERROR_ACCESS_TOKEN);
-        bld.setMessage(e.getMessage());
-      } catch (IOException e) {
-        bld.setStatus(ERROR);
-        bld.setMessage(e.getMessage());
+      } finally {
+        if ((fis == null) && (slotId != null)) {
+          datanode.shortCircuitRegistry.unregisterSlot(slotId);
+        }
       }
+      bld.setStatus(SUCCESS);
+      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+    } catch (ShortCircuitFdsVersionException e) {
+      bld.setStatus(ERROR_UNSUPPORTED);
+      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+      bld.setMessage(e.getMessage());
+    } catch (ShortCircuitFdsUnsupportedException e) {
+      bld.setStatus(ERROR_UNSUPPORTED);
+      bld.setMessage(e.getMessage());
+    } catch (InvalidToken e) {
+      bld.setStatus(ERROR_ACCESS_TOKEN);
+      bld.setMessage(e.getMessage());
+    } catch (IOException e) {
+      bld.setStatus(ERROR);
+      bld.setMessage(e.getMessage());
+    }
+    try {
       bld.build().writeDelimitedTo(socketOut);
       if (fis != null) {
         FileDescriptor fds[] = new FileDescriptor[fis.length];
         for (int i = 0; i < fds.length; i++) {
           fds[i] = fis[i].getFD();
         }
-        byte buf[] = new byte[1];
-        if (supportsReceiptVerification) {
-          buf[0] = (byte)USE_RECEIPT_VERIFICATION.getNumber();
-        } else {
-          buf[0] = (byte)DO_NOT_USE_RECEIPT_VERIFICATION.getNumber();
-        }
-        DomainSocket sock = peer.getDomainSocket();
-        sock.sendFileDescriptors(fds, buf, 0, buf.length);
-        if (supportsReceiptVerification) {
-          LOG.trace("Reading receipt verification byte for " + slotId);
-          int val = sock.getInputStream().read();
-          if (val < 0) {
-            throw new EOFException();
-          }
-        } else {
-          LOG.trace("Receipt verification is not enabled on the DataNode.  " +
-                    "Not verifying " + slotId);
-        }
-        success = true;
+        byte buf[] = new byte[] { (byte)0 };
+        peer.getDomainSocket().
+          sendFileDescriptors(fds, buf, 0, buf.length);
       }
     } finally {
-      if ((!success) && (registeredSlotId != null)) {
-        LOG.info("Unregistering " + registeredSlotId + " because the " +
-            "requestShortCircuitFdsForRead operation failed.");
-        datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId);
-      }
       if (ClientTraceLog.isInfoEnabled()) {
         DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk
             .getBlockPoolId());
         BlockSender.ClientTraceLog.info(String.format(
             "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
             " blockid: %s, srvID: %s, success: %b",
-            blk.getBlockId(), dnR.getDatanodeUuid(), success));
+            blk.getBlockId(), dnR.getDatanodeUuid(), (fis != null)
+          ));
       }
       if (fis != null) {
         IOUtils.cleanup(LOG, fis);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
index b32c0d1..32906f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
@@ -30,7 +30,6 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -84,7 +83,7 @@ public class ShortCircuitRegistry {
 
   private static final int SHM_LENGTH = 8192;
 
-  public static class RegisteredShm extends ShortCircuitShm
+  private static class RegisteredShm extends ShortCircuitShm
       implements DomainSocketWatcher.Handler {
     private final String clientName;
     private final ShortCircuitRegistry registry;
@@ -384,14 +383,4 @@ public class ShortCircuitRegistry {
     }
     IOUtils.closeQuietly(watcher);
   }
-
-  public static interface Visitor {
-    void accept(HashMap<ShmId, RegisteredShm> segments,
-                HashMultimap<ExtendedBlockId, Slot> slots);
-  }
-
-  @VisibleForTesting
-  public synchronized void visit(Visitor visitor) {
-    visitor.accept(segments, slots);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
index 8426198..d72bb5e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
@@ -179,12 +179,6 @@ message OpRequestShortCircuitAccessProto {
    * The shared memory slot to use, if we are using one.
    */
   optional ShortCircuitShmSlotProto slotId = 3;
-
-  /**
-   * True if the client supports verifying that the file descriptor has been
-   * sent successfully.
-   */
-  optional bool supportsReceiptVerification = 4 [default = false];
 }
 
 message ReleaseShortCircuitAccessRequestProto {
@@ -236,11 +230,6 @@ enum Status {
   IN_PROGRESS = 12;
 }
 
-enum ShortCircuitFdResponse {
-  DO_NOT_USE_RECEIPT_VERIFICATION = 0;
-  USE_RECEIPT_VERIFICATION = 1;
-}
-
 message PipelineAckProto {
   required sint64 seqno = 1;
   repeated uint32 reply = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/32741cf3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
index 7daabd0..bfa871c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
@@ -36,16 +36,13 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
-import com.google.common.collect.HashMultimap;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.BlockReaderTestUtil;
-import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -55,14 +52,11 @@ import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
-import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
-import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.RegisteredShm;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.PerDatanodeVisitorInfo;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.Visitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.CacheVisitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -621,61 +615,4 @@ public class TestShortCircuitCache {
     cluster.shutdown();
     sockDir.close();
   }
-
-  public static class TestCleanupFailureInjector
-        extends BlockReaderFactory.FailureInjector {
-    @Override
-    public void injectRequestFileDescriptorsFailure() throws IOException {
-      throw new IOException("injected I/O error");
-    }
-  }
-
-  // Regression test for HDFS-7915
-  @Test(timeout=60000)
-  public void testDataXceiverCleansUpSlotsOnFailure() throws Exception {
-    BlockReaderTestUtil.enableShortCircuitShmTracing();
-    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
-    Configuration conf = createShortCircuitConf(
-        "testDataXceiverCleansUpSlotsOnFailure", sockDir);
-    conf.setLong(DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
-        1000000000L);
-    MiniDFSCluster cluster =
-        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-    cluster.waitActive();
-    DistributedFileSystem fs = cluster.getFileSystem();
-    final Path TEST_PATH1 = new Path("/test_file1");
-    final Path TEST_PATH2 = new Path("/test_file2");
-    final int TEST_FILE_LEN = 4096;
-    final int SEED = 0xFADE1;
-    DFSTestUtil.createFile(fs, TEST_PATH1, TEST_FILE_LEN,
-        (short)1, SEED);
-    DFSTestUtil.createFile(fs, TEST_PATH2, TEST_FILE_LEN,
-        (short)1, SEED);
-
-    // The first read should allocate one shared memory segment and slot.
-    DFSTestUtil.readFileBuffer(fs, TEST_PATH1);
-
-    // The second read should fail, and we should only have 1 segment and 1 slot
-    // left.
-    fs.getClient().getConf().brfFailureInjector =
-        new TestCleanupFailureInjector();
-    try {
-      DFSTestUtil.readFileBuffer(fs, TEST_PATH2);
-    } catch (Throwable t) {
-      GenericTestUtils.assertExceptionContains("TCP reads were disabled for " +
-          "testing, but we failed to do a non-TCP read.", t);
-    }
-    ShortCircuitRegistry registry =
-      cluster.getDataNodes().get(0).getShortCircuitRegistry();
-    registry.visit(new ShortCircuitRegistry.Visitor() {
-      @Override
-      public void accept(HashMap<ShmId, RegisteredShm> segments,
-                         HashMultimap<ExtendedBlockId, Slot> slots) {
-        Assert.assertEquals(1, segments.size());
-        Assert.assertEquals(1, slots.size());
-      }
-    });
-    cluster.shutdown();
-    sockDir.close();
-  }
 }


[42/50] [abbrv] hadoop git commit: HDFS-7749. Erasure Coding: Add striped block support in INodeFile. Contributed by Jing Zhao.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java
new file mode 100644
index 0000000..47445be
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.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 org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+
+/**
+ * Feature for file with striped blocks
+ */
+class FileWithStripedBlocksFeature implements INode.Feature {
+  private BlockInfoStriped[] blocks;
+
+  FileWithStripedBlocksFeature() {
+    blocks = new BlockInfoStriped[0];
+  }
+
+  FileWithStripedBlocksFeature(BlockInfoStriped[] blocks) {
+    Preconditions.checkArgument(blocks != null);
+    this.blocks = blocks;
+  }
+
+  BlockInfoStriped[] getBlocks() {
+    return this.blocks;
+  }
+
+  void setBlock(int index, BlockInfoStriped blk) {
+    blocks[index] = blk;
+  }
+
+  BlockInfoStriped getLastBlock() {
+    return blocks == null || blocks.length == 0 ?
+        null : blocks[blocks.length - 1];
+  }
+
+  int numBlocks() {
+    return blocks == null ? 0 : blocks.length;
+  }
+
+  void updateBlockCollection(INodeFile file) {
+    if (blocks != null) {
+      for (BlockInfoStriped blk : blocks) {
+        blk.setBlockCollection(file);
+      }
+    }
+  }
+
+  private void setBlocks(BlockInfoStriped[] blocks) {
+    this.blocks = blocks;
+  }
+
+  void addBlock(BlockInfoStriped newBlock) {
+    if (this.blocks == null) {
+      this.setBlocks(new BlockInfoStriped[]{newBlock});
+    } else {
+      int size = this.blocks.length;
+      BlockInfoStriped[] newlist = new BlockInfoStriped[size + 1];
+      System.arraycopy(this.blocks, 0, newlist, 0, size);
+      newlist[size] = newBlock;
+      this.setBlocks(newlist);
+    }
+  }
+
+  boolean removeLastBlock(Block oldblock) {
+    if (blocks == null || blocks.length == 0) {
+      return false;
+    }
+    int newSize = blocks.length - 1;
+    if (!blocks[newSize].equals(oldblock)) {
+      return false;
+    }
+
+    //copy to a new list
+    BlockInfoStriped[] newlist = new BlockInfoStriped[newSize];
+    System.arraycopy(blocks, 0, newlist, 0, newSize);
+    setBlocks(newlist);
+    return true;
+  }
+
+  void truncateStripedBlocks(int n) {
+    final BlockInfoStriped[] newBlocks;
+    if (n == 0) {
+      newBlocks = new BlockInfoStriped[0];
+    } else {
+      newBlocks = new BlockInfoStriped[n];
+      System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
+    }
+    // set new blocks
+    setBlocks(newBlocks);
+  }
+
+  void clear() {
+    this.blocks = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 9259d2c..5fbcf92 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 
@@ -36,12 +37,12 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
@@ -173,6 +174,31 @@ public class INodeFile extends INodeWithAdditionalFields
         && getXAttrFeature() == other.getXAttrFeature();
   }
 
+  /* Start of StripedBlock Feature */
+
+  public final FileWithStripedBlocksFeature getStripedBlocksFeature() {
+    return getFeature(FileWithStripedBlocksFeature.class);
+  }
+
+  public FileWithStripedBlocksFeature addStripedBlocksFeature() {
+    assert blocks == null || blocks.length == 0:
+        "The file contains contiguous blocks";
+    assert !isWithStripedBlocks();
+    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;
+  }
+
   /* Start of Under-Construction Feature */
 
   /**
@@ -207,7 +233,7 @@ public class INodeFile extends INodeWithAdditionalFields
         "file is no longer under construction");
     FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
     if (uc != null) {
-      assertAllBlocksComplete();
+      assertAllBlocksComplete(getBlocks());
       removeFeature(uc);
       this.setModificationTime(mtime);
     }
@@ -215,37 +241,56 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** Assert all blocks are complete. */
-  private void assertAllBlocksComplete() {
-    if (blocks == null) {
+  private void assertAllBlocksComplete(BlockInfo[] blks) {
+    if (blks == null) {
       return;
     }
-    for (int i = 0; i < blocks.length; i++) {
-      Preconditions.checkState(blocks[i].isComplete(), "Failed to finalize"
+    for (int i = 0; i < blks.length; i++) {
+      Preconditions.checkState(blks[i].isComplete(), "Failed to finalize"
           + " %s %s since blocks[%s] is non-complete, where blocks=%s.",
-          getClass().getSimpleName(), this, i, Arrays.asList(blocks));
+          getClass().getSimpleName(), this, i, Arrays.asList(blks));
     }
   }
 
+  /**
+   * Instead of adding a new block, this function is usually used while loading
+   * fsimage or converting the last block to UC/Complete.
+   */
   @Override // BlockCollection
-  public void setBlock(int index, BlockInfoContiguous blk) {
-    this.blocks[index] = blk;
+  public void setBlock(int index, BlockInfo blk) {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      assert blk instanceof BlockInfoContiguous;
+      this.blocks[index] = (BlockInfoContiguous) blk;
+    } else {
+      assert blk instanceof BlockInfoStriped;
+      assert hasNoContiguousBlock();
+      sb.setBlock(index, (BlockInfoStriped) blk);
+    }
   }
 
   @Override // BlockCollection, the file should be under construction
-  public BlockInfoContiguousUnderConstruction setLastBlock(
-      BlockInfoContiguous lastBlock, DatanodeStorageInfo[] locations)
-      throws IOException {
+  public void convertLastBlockToUC(BlockInfo lastBlock,
+      DatanodeStorageInfo[] locations) throws IOException {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
-
     if (numBlocks() == 0) {
       throw new IOException("Failed to set last block: File is empty.");
     }
-    BlockInfoContiguousUnderConstruction ucBlock =
-      lastBlock.convertToBlockUnderConstruction(
-          BlockUCState.UNDER_CONSTRUCTION, locations);
+
+    final BlockInfo ucBlock;
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      assert lastBlock instanceof BlockInfoContiguous;
+      ucBlock = ((BlockInfoContiguous) lastBlock)
+          .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
+    } else {
+      assert hasNoContiguousBlock();
+      assert lastBlock instanceof BlockInfoStriped;
+      ucBlock = ((BlockInfoStriped) lastBlock)
+          .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
+    }
     setBlock(numBlocks() - 1, ucBlock);
-    return ucBlock;
   }
 
   /**
@@ -255,19 +300,25 @@ public class INodeFile extends INodeWithAdditionalFields
   boolean removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
-    if (blocks == null || blocks.length == 0) {
-      return false;
-    }
-    int size_1 = blocks.length - 1;
-    if (!blocks[size_1].equals(oldblock)) {
-      return false;
-    }
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      if (blocks == null || blocks.length == 0) {
+        return false;
+      }
+      int size_1 = blocks.length - 1;
+      if (!blocks[size_1].equals(oldblock)) {
+        return false;
+      }
 
-    //copy to a new list
-    BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1];
-    System.arraycopy(blocks, 0, newlist, 0, size_1);
-    setBlocks(newlist);
-    return true;
+      //copy to a new list
+      BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1];
+      System.arraycopy(blocks, 0, newlist, 0, size_1);
+      setContiguousBlocks(newlist);
+      return true;
+    } else {
+      assert hasNoContiguousBlock();
+      return sb.removeLastBlock(oldblock);
+    }
   }
 
   /* End of Under-Construction Feature */
@@ -368,13 +419,15 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** Set the replication factor of this file. */
-  public final void setFileReplication(short replication) {
+  private void setFileReplication(short replication) {
     header = HeaderFormat.REPLICATION.BITS.combine(replication, header);
   }
 
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication,
       int latestSnapshotId) throws QuotaExceededException {
+    Preconditions.checkState(!isWithStripedBlocks(),
+        "Cannot set replication to a file with striped blocks");
     recordModification(latestSnapshotId);
     setFileReplication(replication);
     return this;
@@ -412,42 +465,57 @@ public class INodeFile extends INodeWithAdditionalFields
     setStoragePolicyID(storagePolicyId);
   }
 
-  @Override
+  @Override // INodeFileAttributes
   public long getHeaderLong() {
     return header;
   }
 
-  /** @return the storagespace required for a full block. */
-  final long getPreferredBlockStoragespace() {
-    return getPreferredBlockSize() * getBlockReplication();
+  /** @return the blocks of the file. */
+  @Override // BlockCollection
+  public BlockInfo[] getBlocks() {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb != null) {
+      assert hasNoContiguousBlock();
+      return sb.getBlocks();
+    } else {
+      return this.blocks;
+    }
   }
 
-  /** @return the blocks of the file. */
-  @Override
-  public BlockInfoContiguous[] getBlocks() {
+  /** Used by snapshot diff */
+  public BlockInfoContiguous[] getContiguousBlocks() {
     return this.blocks;
   }
 
   /** @return blocks of the file corresponding to the snapshot. */
-  public BlockInfoContiguous[] getBlocks(int snapshot) {
-    if(snapshot == CURRENT_STATE_ID || getDiffs() == null)
+  public BlockInfo[] getBlocks(int snapshot) {
+    if (snapshot == CURRENT_STATE_ID || getDiffs() == null) {
       return getBlocks();
+    }
+    // find blocks stored in snapshot diffs (for truncate)
     FileDiff diff = getDiffs().getDiffById(snapshot);
-    BlockInfoContiguous[] snapshotBlocks =
-        diff == null ? getBlocks() : diff.getBlocks();
-    if(snapshotBlocks != null)
+    // note that currently FileDiff can only store contiguous blocks
+    BlockInfo[] snapshotBlocks = diff == null ? getBlocks() : diff.getBlocks();
+    if (snapshotBlocks != null) {
       return snapshotBlocks;
+    }
     // Blocks are not in the current snapshot
     // Find next snapshot with blocks present or return current file blocks
     snapshotBlocks = getDiffs().findLaterSnapshotBlocks(snapshot);
     return (snapshotBlocks == null) ? getBlocks() : snapshotBlocks;
   }
 
-  void updateBlockCollection() {
-    if (blocks != null) {
+  /** Used during concat to update the BlockCollection for each block */
+  private void updateBlockCollection() {
+    if (blocks != null && blocks.length > 0) {
       for(BlockInfoContiguous b : blocks) {
         b.setBlockCollection(this);
       }
+    } else {
+      FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+      if (sb != null) {
+        sb.updateBlockCollection(this);
+      }
     }
   }
 
@@ -470,33 +538,33 @@ public class INodeFile extends INodeWithAdditionalFields
       size += in.blocks.length;
     }
 
-    setBlocks(newlist);
+    setContiguousBlocks(newlist);
     updateBlockCollection();
   }
   
   /**
-   * add a block to the block list
+   * add a contiguous block to the block list
    */
   void addBlock(BlockInfoContiguous newblock) {
     if (this.blocks == null) {
-      this.setBlocks(new BlockInfoContiguous[]{newblock});
+      this.setContiguousBlocks(new BlockInfoContiguous[]{newblock});
     } else {
       int size = this.blocks.length;
       BlockInfoContiguous[] newlist = new BlockInfoContiguous[size + 1];
       System.arraycopy(this.blocks, 0, newlist, 0, size);
       newlist[size] = newblock;
-      this.setBlocks(newlist);
+      this.setContiguousBlocks(newlist);
     }
   }
 
   /** Set the blocks. */
-  public void setBlocks(BlockInfoContiguous[] blocks) {
+  public void setContiguousBlocks(BlockInfoContiguous[] blocks) {
     this.blocks = blocks;
   }
 
   @Override
-  public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, final int snapshot,
-                                  int priorSnapshotId,
+  public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps,
+      final int snapshot, int priorSnapshotId,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
@@ -526,13 +594,19 @@ public class INodeFile extends INodeWithAdditionalFields
   @Override
   public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
       BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
-    if (blocks != null && collectedBlocks != null) {
-      for (BlockInfoContiguous blk : blocks) {
+    BlockInfo[] blks = getBlocks();
+    if (blks != null && collectedBlocks != null) {
+      for (BlockInfo blk : blks) {
         collectedBlocks.addDeleteBlock(blk);
         blk.setBlockCollection(null);
       }
     }
-    setBlocks(null);
+    setContiguousBlocks(null);
+
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb != null) {
+      sb.clear();
+    }
     if (getAclFeature() != null) {
       AclStorage.removeAclFeature(getAclFeature());
     }
@@ -710,7 +784,7 @@ public class INodeFile extends INodeWithAdditionalFields
       size += block.getNumBytes();
     }
     // check if the last block is under construction
-    BlockInfoContiguous lastBlock = getLastBlock();
+    BlockInfo lastBlock = getLastBlock();
     if(lastBlock != null &&
         lastBlock instanceof BlockInfoContiguousUnderConstruction) {
       size += getPreferredBlockSize() - lastBlock.getNumBytes();
@@ -718,15 +792,6 @@ public class INodeFile extends INodeWithAdditionalFields
     return size;
   }
 
-  public final long storagespaceConsumed(int lastSnapshotId) {
-    if (lastSnapshotId != CURRENT_STATE_ID) {
-      return computeFileSize(lastSnapshotId)
-        * getFileReplication(lastSnapshotId);
-    } else {
-      return storagespaceConsumed();
-    }
-  }
-
   public final short getReplication(int lastSnapshotId) {
     if (lastSnapshotId != CURRENT_STATE_ID) {
       return getFileReplication(lastSnapshotId);
@@ -746,21 +811,33 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * Return the penultimate allocated block for this file.
    */
-  BlockInfoContiguous getPenultimateBlock() {
-    if (blocks == null || blocks.length <= 1) {
-      return null;
-    }
-    return blocks[blocks.length - 2];
+  BlockInfo getPenultimateBlock() {
+    BlockInfo[] blks = getBlocks();
+    return (blks == null || blks.length <= 1) ?
+        null : blks[blks.length - 2];
   }
 
   @Override
-  public BlockInfoContiguous getLastBlock() {
-    return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
+  public BlockInfo getLastBlock() {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      return blocks == null || blocks.length == 0 ?
+          null : blocks[blocks.length - 1];
+    } else {
+      assert hasNoContiguousBlock();
+      return sb.getLastBlock();
+    }
   }
 
   @Override
   public int numBlocks() {
-    return blocks == null ? 0 : blocks.length;
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      return blocks == null ? 0 : blocks.length;
+    } else {
+      assert hasNoContiguousBlock();
+      return sb.numBlocks();
+    }
   }
 
   @VisibleForTesting
@@ -772,6 +849,7 @@ public class INodeFile extends INodeWithAdditionalFields
     // only compare the first block
     out.print(", blocks=");
     out.print(blocks == null || blocks.length == 0? null: blocks[0]);
+    // TODO print striped blocks
     out.println();
   }
 
@@ -781,9 +859,10 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   public long collectBlocksBeyondMax(final long max,
       final BlocksMapUpdateInfo collectedBlocks) {
-    final BlockInfoContiguous[] oldBlocks = getBlocks();
-    if (oldBlocks == null)
+    final BlockInfo[] oldBlocks = getBlocks();
+    if (oldBlocks == null) {
       return 0;
+    }
     // find the minimum n such that the size of the first n blocks > max
     int n = 0;
     long size = 0;
@@ -807,21 +886,36 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   void truncateBlocksTo(int n) {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      truncateContiguousBlocks(n);
+    } else {
+      sb.truncateStripedBlocks(n);
+    }
+  }
+
+  private void truncateContiguousBlocks(int n) {
     final BlockInfoContiguous[] newBlocks;
     if (n == 0) {
       newBlocks = BlockInfoContiguous.EMPTY_ARRAY;
     } else {
       newBlocks = new BlockInfoContiguous[n];
-      System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
+      System.arraycopy(blocks, 0, newBlocks, 0, n);
     }
     // set new blocks
-    setBlocks(newBlocks);
+    setContiguousBlocks(newBlocks);
   }
 
+  /**
+   * This function is only called when block list is stored in snapshot
+   * diffs. Note that this can only happen when truncation happens with
+   * snapshots. Since we do not support truncation with striped blocks,
+   * we only need to handle contiguous blocks here.
+   */
   public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks,
                                           BlocksMapUpdateInfo collectedBlocks) {
-    BlockInfoContiguous[] oldBlocks = getBlocks();
-    if(snapshotBlocks == null || oldBlocks == null)
+    BlockInfoContiguous[] oldBlocks = this.blocks;
+    if (snapshotBlocks == null || oldBlocks == null)
       return;
     // Skip blocks in common between the file and the snapshot
     int n = 0;
@@ -829,7 +923,7 @@ public class INodeFile extends INodeWithAdditionalFields
           oldBlocks[n] == snapshotBlocks[n]) {
       n++;
     }
-    truncateBlocksTo(n);
+    truncateContiguousBlocks(n);
     // Collect the remaining blocks of the file
     while(n < oldBlocks.length) {
       collectedBlocks.addDeleteBlock(oldBlocks[n++]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index 0dafaae..f076215 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.Daemon;
 
@@ -120,10 +120,10 @@ public class LeaseManager {
         } catch (UnresolvedLinkException e) {
           throw new AssertionError("Lease files should reside on this FS");
         }
-        BlockInfoContiguous[] blocks = cons.getBlocks();
+        BlockInfo[] blocks = cons.getBlocks();
         if(blocks == null)
           continue;
-        for(BlockInfoContiguous b : blocks) {
+        for(BlockInfo b : blocks) {
           if(!b.isComplete())
             numUCBlocks++;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 1db74b7..3d5c251 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
@@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactor
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
@@ -235,8 +236,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       //get blockInfo
       Block block = new Block(Block.getBlockId(blockId));
       //find which file this block belongs to
-      BlockInfoContiguous blockInfo = namenode.getNamesystem()
-          .getStoredBlock(block);
+      BlockInfo blockInfo = namenode.getNamesystem().getStoredBlock(block);
       if(blockInfo == null) {
         out.println("Block "+ blockId +" " + NONEXISTENT_STATUS);
         LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 3442e7b..4695c3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
@@ -45,5 +46,5 @@ public interface Namesystem extends RwLock, SafeMode {
 
   public void checkOperation(OperationCategory read) throws StandbyException;
 
-  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC);
+  public boolean isInSnapshot(BlockCollection bc);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 87b370a..74baec5 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
@@ -239,15 +239,16 @@ public class FSImageFormatPBSnapshot {
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
             pbf.getFileSize());
         List<BlockProto> bpl = pbf.getBlocksList();
-        // TODO: also persist striped blocks
+        // in file diff there can only be contiguous blocks
         BlockInfoContiguous[] blocks = new BlockInfoContiguous[bpl.size()];
         for(int j = 0, e = bpl.size(); j < e; ++j) {
           Block blk = PBHelper.convert(bpl.get(j));
           BlockInfoContiguous storedBlock =
               (BlockInfoContiguous) fsn.getBlockManager().getStoredBlock(blk);
           if(storedBlock == null) {
-            storedBlock = fsn.getBlockManager().addBlockCollection(
-                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
+            storedBlock = (BlockInfoContiguous) fsn.getBlockManager()
+                .addBlockCollection(new BlockInfoContiguous(blk,
+                    copy.getFileReplication()), file);
           }
           blocks[j] = storedBlock;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index 5c9e121..a1263c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -54,8 +54,11 @@ public class FileDiffList extends
       INodeFileAttributes snapshotCopy, boolean withBlocks) {
     final FileDiff diff =
         super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
-    if(withBlocks)  // Store blocks if this is the first update
-      diff.setBlocks(iNodeFile.getBlocks());
+    if (withBlocks) {  // Store blocks if this is the first update
+      BlockInfoContiguous[] blks = iNodeFile.getContiguousBlocks();
+      assert blks != null;
+      diff.setBlocks(blks);
+    }
   }
 
   public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) {
@@ -118,7 +121,7 @@ public class FileDiffList extends
         (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
     // Find later snapshot (or file itself) with blocks
     BlockInfoContiguous[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
-    laterBlocks = (laterBlocks==null) ? file.getBlocks() : laterBlocks;
+    laterBlocks = (laterBlocks == null) ? file.getContiguousBlocks() : laterBlocks;
     // Skip blocks, which belong to either the earlier or the later lists
     int i = 0;
     for(; i < removedBlocks.length; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index 3bd1d91..b6fd033 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -91,6 +91,10 @@ message INodeSection {
     optional string clientMachine = 2;
   }
 
+  message StripedBlocksFeature {
+    repeated StripedBlockProto blocks = 1;
+  }
+
   message AclFeatureProto {
     /**
      * An ACL entry is represented by a 32-bit integer in Big Endian
@@ -139,6 +143,7 @@ message INodeSection {
     optional AclFeatureProto acl = 8;
     optional XAttrFeatureProto xAttrs = 9;
     optional uint32 storagePolicyID = 10;
+    optional StripedBlocksFeature stripedBlocks = 11;
   }
 
   message QuotaByStorageTypeEntryProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 31e5585..1fa3743 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -476,6 +476,16 @@ message BlockProto {
 }
 
 /**
+ * Striped block information. Besides the basic information for a block,
+ * it also contains the number of data/parity blocks.
+ */
+message StripedBlockProto {
+  required BlockProto block = 1;
+  optional uint32 dataBlockNum = 2;
+  optional uint32 parityBlockNum = 3;
+}
+
+/**
  * Block and datanodes where is it located
  */
 message BlockWithLocationsProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 bb4f7aa..c3dac35 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
@@ -56,6 +56,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 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;
@@ -1561,7 +1562,7 @@ public class DFSTestUtil {
   public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn,
       ExtendedBlock blk) {
     FSNamesystem fsn = nn.getNamesystem();
-    BlockInfoContiguous storedBlock = fsn.getStoredBlock(blk.getLocalBlock());
+    BlockInfo storedBlock = fsn.getStoredBlock(blk.getLocalBlock());
     assertTrue("Block " + blk + " should be under construction, " +
         "got: " + storedBlock,
         storedBlock instanceof BlockInfoContiguousUnderConstruction);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 1a2a9f0..6a7c924 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -1236,8 +1236,8 @@ public class TestReplicationPolicy {
         (DatanodeStorageInfo.AddBlockResult.ADDED);
     ucBlock.addStorage(storage, ucBlock);
 
-    when(mbc.setLastBlock((BlockInfoContiguous) any(), (DatanodeStorageInfo[]) any()))
-    .thenReturn(ucBlock);
+    BlockInfo lastBlk = mbc.getLastBlock();
+    when(mbc.getLastBlock()).thenReturn(lastBlk, ucBlock);
 
     bm.convertLastBlockToUnderConstruction(mbc, 0L);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
index a417c3d..301ee25 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.junit.After;
 import org.junit.Before;
@@ -87,21 +87,21 @@ public class TestAddBlock {
     
     // check file1
     INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
-    BlockInfoContiguous[] file1Blocks = file1Node.getBlocks();
+    BlockInfo[] file1Blocks = file1Node.getBlocks();
     assertEquals(1, file1Blocks.length);
     assertEquals(BLOCKSIZE - 1, file1Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file1Blocks[0].getBlockUCState());
     
     // check file2
     INodeFile file2Node = fsdir.getINode4Write(file2.toString()).asFile();
-    BlockInfoContiguous[] file2Blocks = file2Node.getBlocks();
+    BlockInfo[] file2Blocks = file2Node.getBlocks();
     assertEquals(1, file2Blocks.length);
     assertEquals(BLOCKSIZE, file2Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file2Blocks[0].getBlockUCState());
     
     // check file3
     INodeFile file3Node = fsdir.getINode4Write(file3.toString()).asFile();
-    BlockInfoContiguous[] file3Blocks = file3Node.getBlocks();
+    BlockInfo[] file3Blocks = file3Node.getBlocks();
     assertEquals(2, file3Blocks.length);
     assertEquals(BLOCKSIZE, file3Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file3Blocks[0].getBlockUCState());
@@ -110,7 +110,7 @@ public class TestAddBlock {
     
     // check file4
     INodeFile file4Node = fsdir.getINode4Write(file4.toString()).asFile();
-    BlockInfoContiguous[] file4Blocks = file4Node.getBlocks();
+    BlockInfo[] file4Blocks = file4Node.getBlocks();
     assertEquals(2, file4Blocks.length);
     assertEquals(BLOCKSIZE, file4Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file4Blocks[0].getBlockUCState());
@@ -141,7 +141,7 @@ public class TestAddBlock {
       FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
       
       INodeFile fileNode = fsdir.getINode4Write(file1.toString()).asFile();
-      BlockInfoContiguous[] fileBlocks = fileNode.getBlocks();
+      BlockInfo[] fileBlocks = fileNode.getBlocks();
       assertEquals(2, fileBlocks.length);
       assertEquals(BLOCKSIZE, fileBlocks[0].getNumBytes());
       assertEquals(BlockUCState.COMPLETE, fileBlocks[0].getBlockUCState());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
index 06dfade..a2ef7b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.junit.After;
 import org.junit.Before;
@@ -75,7 +76,7 @@ public class TestAddBlockgroup {
     final Path file1 = new Path("/file1");
     DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L);
     INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
-    BlockInfoContiguous[] file1Blocks = file1Node.getBlocks();
+    BlockInfo[] file1Blocks = file1Node.getBlocks();
     assertEquals(2, file1Blocks.length);
     assertEquals(GROUP_SIZE, file1Blocks[0].numNodes());
     assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
index 1fbe160..f372bec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.TestFileCreation;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -91,12 +91,12 @@ public class TestBlockUnderConstruction {
         " isUnderConstruction = " + inode.isUnderConstruction() +
         " expected to be " + isFileOpen,
         inode.isUnderConstruction() == isFileOpen);
-    BlockInfoContiguous[] blocks = inode.getBlocks();
+    BlockInfo[] blocks = inode.getBlocks();
     assertTrue("File does not have blocks: " + inode.toString(),
         blocks != null && blocks.length > 0);
     
     int idx = 0;
-    BlockInfoContiguous curBlock;
+    BlockInfo curBlock;
     // all blocks but the last two should be regular blocks
     for(; idx < blocks.length - 2; idx++) {
       curBlock = blocks[idx];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/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 7b9ea93..913e0a7 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
@@ -24,6 +24,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.EnumSet;
 
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
@@ -39,7 +40,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.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
@@ -105,7 +105,7 @@ public class TestFSImage {
       INodeFile file2Node = fsn.dir.getINode4Write(file2.toString()).asFile();
       assertEquals("hello".length(), file2Node.computeFileSize());
       assertTrue(file2Node.isUnderConstruction());
-      BlockInfoContiguous[] blks = file2Node.getBlocks();
+      BlockInfo[] blks = file2Node.getBlocks();
       assertEquals(1, blks.length);
       assertEquals(BlockUCState.UNDER_CONSTRUCTION, blks[0].getBlockUCState());
       // check lease manager

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 3b6e107..81c5eb7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -1042,7 +1043,8 @@ public class TestFileTruncate {
     iip = fsn.getFSDirectory().getINodesInPath(src, true);
     file = iip.getLastINode().asFile();
     file.recordModification(iip.getLatestSnapshotId(), true);
-    assertThat(file.isBlockInLatestSnapshot(file.getLastBlock()), is(true));
+    assertThat(file.isBlockInLatestSnapshot(
+        (BlockInfoContiguous) file.getLastBlock()), is(true));
     initialGenStamp = file.getLastBlock().getGenerationStamp();
     // Test that prepareFileForTruncate sets up copy-on-write truncate
     fsn.writeLock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 70deb1b..cddc457 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -77,7 +77,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -801,7 +801,7 @@ public class TestFsck {
       // intentionally corrupt NN data structure
       INodeFile node = (INodeFile) cluster.getNamesystem().dir.getINode
           (fileName, true);
-      final BlockInfoContiguous[] blocks = node.getBlocks();
+      final BlockInfo[] blocks = node.getBlocks();
       assertEquals(blocks.length, 1);
       blocks[0].setNumBytes(-1L);  // set the block length to be negative
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
index c6c8dad..43f43f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -108,14 +108,14 @@ public class TestSnapshotBlocksMap {
      final FSDirectory dir, final BlockManager blkManager) throws Exception {
     final INodeFile file = INodeFile.valueOf(dir.getINode(path), path);
     assertEquals(numBlocks, file.getBlocks().length);
-    for(BlockInfoContiguous b : file.getBlocks()) {
+    for(BlockInfo b : file.getBlocks()) {
       assertBlockCollection(blkManager, file, b);
     }
     return file;
   }
 
   static void assertBlockCollection(final BlockManager blkManager,
-      final INodeFile file, final BlockInfoContiguous b) {
+      final INodeFile file, final BlockInfo b) {
     Assert.assertSame(b, blkManager.getStoredBlock(b));
     Assert.assertSame(file, blkManager.getBlockCollection(b));
     Assert.assertSame(file, b.getBlockCollection());
@@ -146,10 +146,10 @@ public class TestSnapshotBlocksMap {
     {
       final INodeFile f2 = assertBlockCollection(file2.toString(), 3, fsdir,
           blockmanager);
-      BlockInfoContiguous[] blocks = f2.getBlocks();
+      BlockInfo[] blocks = f2.getBlocks();
       hdfs.delete(sub2, true);
       // The INode should have been removed from the blocksMap
-      for(BlockInfoContiguous b : blocks) {
+      for(BlockInfo b : blocks) {
         assertNull(blockmanager.getBlockCollection(b));
       }
     }
@@ -177,7 +177,7 @@ public class TestSnapshotBlocksMap {
     // Check the block information for file0
     final INodeFile f0 = assertBlockCollection(file0.toString(), 4, fsdir,
         blockmanager);
-    BlockInfoContiguous[] blocks0 = f0.getBlocks();
+    BlockInfo[] blocks0 = f0.getBlocks();
     
     // Also check the block information for snapshot of file0
     Path snapshotFile0 = SnapshotTestHelper.getSnapshotPath(sub1, "s0",
@@ -187,7 +187,7 @@ public class TestSnapshotBlocksMap {
     // Delete file0
     hdfs.delete(file0, true);
     // Make sure the blocks of file0 is still in blocksMap
-    for(BlockInfoContiguous b : blocks0) {
+    for(BlockInfo b : blocks0) {
       assertNotNull(blockmanager.getBlockCollection(b));
     }
     assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
@@ -201,7 +201,7 @@ public class TestSnapshotBlocksMap {
     hdfs.deleteSnapshot(sub1, "s1");
 
     // Make sure the first block of file0 is still in blocksMap
-    for(BlockInfoContiguous b : blocks0) {
+    for(BlockInfo b : blocks0) {
       assertNotNull(blockmanager.getBlockCollection(b));
     }
     assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
@@ -293,7 +293,7 @@ public class TestSnapshotBlocksMap {
     hdfs.append(bar);
 
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     assertEquals(BLOCKSIZE, blks[0].getNumBytes());
     ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
@@ -331,7 +331,7 @@ public class TestSnapshotBlocksMap {
     hdfs.append(bar);
 
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
     cluster.getNameNodeRpc()
@@ -370,7 +370,7 @@ public class TestSnapshotBlocksMap {
     hdfs.append(bar);
 
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
     cluster.getNameNodeRpc()
@@ -421,7 +421,7 @@ public class TestSnapshotBlocksMap {
     out.write(testData);
     out.close();
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     assertEquals(testData.length, blks[0].getNumBytes());
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e90e36b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
index a679183..452ff3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -260,12 +260,12 @@ public class TestSnapshotDeletion {
     DFSTestUtil.createFile(hdfs, tempFile, BLOCKSIZE, REPLICATION, seed);
     final INodeFile temp = TestSnapshotBlocksMap.assertBlockCollection(
         tempFile.toString(), 1, fsdir, blockmanager);
-    BlockInfoContiguous[] blocks = temp.getBlocks();
+    BlockInfo[] blocks = temp.getBlocks();
     hdfs.delete(tempDir, true);
     // check dir's quota usage
     checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3);
     // check blocks of tempFile
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -342,7 +342,7 @@ public class TestSnapshotDeletion {
     // while deletion, we add diff for subsub and metaChangeFile1, and remove
     // newFile
     checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -479,7 +479,7 @@ public class TestSnapshotDeletion {
     
     final INodeFile toDeleteFileNode = TestSnapshotBlocksMap
         .assertBlockCollection(toDeleteFile.toString(), 1, fsdir, blockmanager);
-    BlockInfoContiguous[] blocks = toDeleteFileNode.getBlocks();
+    BlockInfo[] blocks = toDeleteFileNode.getBlocks();
     
     // create snapshot s0 on dir
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
@@ -505,7 +505,7 @@ public class TestSnapshotDeletion {
     // metaChangeDir's diff, dir's diff. diskspace: remove toDeleteFile, and 
     // metaChangeFile's replication factor decreases
     checkQuotaUsageComputation(dir, 6, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -799,7 +799,7 @@ public class TestSnapshotDeletion {
     FileStatus statusBeforeDeletion13 = hdfs.getFileStatus(file13_s1);
     INodeFile file14Node = TestSnapshotBlocksMap.assertBlockCollection(
         file14_s2.toString(), 1, fsdir, blockmanager);
-    BlockInfoContiguous[] blocks_14 = file14Node.getBlocks();
+    BlockInfo[] blocks_14 = file14Node.getBlocks();
     TestSnapshotBlocksMap.assertBlockCollection(file15_s2.toString(), 1, fsdir,
         blockmanager);
     
@@ -836,7 +836,7 @@ public class TestSnapshotDeletion {
         modDirStr + "file15");
     assertFalse(hdfs.exists(file14_s1));
     assertFalse(hdfs.exists(file15_s1));
-    for (BlockInfoContiguous b : blocks_14) {
+    for (BlockInfo b : blocks_14) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     


[33/50] [abbrv] hadoop git commit: HDFS-7339. Allocating and persisting block groups in NameNode. Contributed by Zhe Zhang

Posted by zh...@apache.org.
HDFS-7339. Allocating and persisting block groups in NameNode. Contributed by Zhe Zhang

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java


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

Branch: refs/heads/HDFS-7285
Commit: 10c794ce8c54f530e1c82e9888918bb5532ac37b
Parents: 2e80653
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Jan 30 16:16:26 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:08:48 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  2 +
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  4 +
 .../server/blockmanagement/BlockIdManager.java  |  8 +-
 .../SequentialBlockGroupIdGenerator.java        | 82 +++++++++++++++++++
 .../SequentialBlockIdGenerator.java             |  6 +-
 .../hdfs/server/namenode/FSDirectory.java       |  8 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 34 +++++---
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 11 +++
 .../hdfs/server/namenode/TestAddBlockgroup.java | 84 ++++++++++++++++++++
 9 files changed, 223 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10c794ce/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 1e864bd..2dded68 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
@@ -219,6 +219,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT = 3;
   public static final String  DFS_NAMENODE_REPLICATION_MIN_KEY = "dfs.namenode.replication.min";
   public static final int     DFS_NAMENODE_REPLICATION_MIN_DEFAULT = 1;
+  public static final String  DFS_NAMENODE_STRIPE_MIN_KEY = "dfs.namenode.stripe.min";
+  public static final int     DFS_NAMENODE_STRIPE_MIN_DEFAULT = 1;
   public static final String  DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY = "dfs.namenode.replication.pending.timeout-sec";
   public static final int     DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT = -1;
   public static final String  DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY = "dfs.namenode.replication.max-streams";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10c794ce/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 54c650b..de60b6e 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
@@ -181,4 +181,8 @@ public class HdfsConstants {
   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;
+  public static final byte NUM_PARITY_BLOCKS = 2;
+  public static final byte MAX_BLOCKS_IN_GROUP = 16;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10c794ce/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 1c69203..c8b9d20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -53,10 +53,12 @@ public class BlockIdManager {
    * The global block ID space for this file system.
    */
   private final SequentialBlockIdGenerator blockIdGenerator;
+  private final SequentialBlockGroupIdGenerator blockGroupIdGenerator;
 
   public BlockIdManager(BlockManager blockManager) {
     this.generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
+    this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager);
   }
 
   /**
@@ -190,6 +192,10 @@ public class BlockIdManager {
     return blockIdGenerator.nextValue();
   }
 
+  public long nextBlockGroupId() {
+    return blockGroupIdGenerator.nextValue();
+  }
+
   public boolean isGenStampInFuture(Block block) {
     if (isLegacyBlock(block)) {
       return block.getGenerationStamp() > getGenerationStampV1();
@@ -205,4 +211,4 @@ public class BlockIdManager {
       .LAST_RESERVED_BLOCK_ID);
     generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10c794ce/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
new file mode 100644
index 0000000..e9e22ee
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.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;
+
+/**
+ * Generate the next valid block group ID by incrementing the maximum block
+ * group ID allocated so far, with the first 2^10 block group IDs reserved.
+ * HDFS-EC introduces a hierarchical protocol to name blocks and groups:
+ * Contiguous: {reserved block IDs | flag | block ID}
+ * Striped: {reserved block IDs | flag | block group ID | index in group}
+ *
+ * Following n bits of reserved block IDs, The (n+1)th bit in an ID
+ * distinguishes contiguous (0) and striped (1) blocks. For a striped block,
+ * bits (n+2) to (64-m) represent the ID of its block group, while the last m
+ * bits represent its index of the group. The value m is determined by the
+ * maximum number of blocks in a group (MAX_BLOCKS_IN_GROUP).
+ */
+@InterfaceAudience.Private
+public class SequentialBlockGroupIdGenerator extends SequentialNumber {
+
+  private final BlockManager blockManager;
+
+  SequentialBlockGroupIdGenerator(BlockManager blockManagerRef) {
+    super(Long.MIN_VALUE);
+    this.blockManager = blockManagerRef;
+  }
+
+  @Override // NumberGenerator
+  public long nextValue() {
+    // Skip to next legitimate block group ID based on the naming protocol
+    while (super.getCurrentValue() % HdfsConstants.MAX_BLOCKS_IN_GROUP > 0) {
+      super.nextValue();
+    }
+    // Make sure there's no conflict with existing random block IDs
+    while (hasValidBlockInRange(super.getCurrentValue())) {
+      super.skipTo(super.getCurrentValue() +
+          HdfsConstants.MAX_BLOCKS_IN_GROUP);
+    }
+    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.");
+    }
+    return super.getCurrentValue();
+  }
+
+  /**
+   *
+   * @param id The starting ID of the range
+   * @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);
+      if (blockManager.getBlockCollection(b) != null) {
+        return true;
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10c794ce/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 eef8857..c97de4b 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
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.util.SequentialNumber;
 
 /**
@@ -54,6 +53,11 @@ public class SequentialBlockIdGenerator extends SequentialNumber {
     while(isValidBlock(b)) {
       b.setBlockId(super.nextValue());
     }
+    if (b.getBlockId() < 0) {
+      BlockManager.LOG.warn("All positive block IDs are used, " +
+          "wrapping to negative IDs, " +
+          "which might conflict with erasure coded block groups.");
+    }
     return b.getBlockId();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10c794ce/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 f6ab077..627abfa 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
@@ -466,10 +466,14 @@ public class FSDirectory implements Closeable {
    * Add a block to the file. Returns a reference to the added block.
    */
   BlockInfoContiguous addBlock(String path, INodesInPath inodesInPath,
-      Block block, DatanodeStorageInfo[] targets) throws IOException {
+      Block block, DatanodeStorageInfo[] targets,
+      boolean isStriped) throws IOException {
     writeLock();
     try {
       final INodeFile fileINode = inodesInPath.getLastINode().asFile();
+      short numLocations = isStriped ?
+          HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS :
+          fileINode.getFileReplication();
       Preconditions.checkState(fileINode.isUnderConstruction());
 
       // check quota limits and updated space consumed
@@ -480,7 +484,7 @@ public class FSDirectory implements Closeable {
       BlockInfoContiguousUnderConstruction blockInfo =
         new BlockInfoContiguousUnderConstruction(
             block,
-            fileINode.getFileReplication(),
+            numLocations,
             BlockUCState.UNDER_CONSTRUCTION,
             targets);
       getBlockManager().addBlockCollection(blockInfo, fileINode);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10c794ce/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 b384ce6..016a400 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
@@ -2036,7 +2036,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     BlockInfoContiguous oldBlock = file.getLastBlock();
     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock);
     if(newBlock == null) {
-      newBlock = (shouldCopyOnTruncate) ? createNewBlock() :
+      newBlock = (shouldCopyOnTruncate) ? createNewBlock(file.isStriped()) :
           new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
               nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock)));
     }
@@ -2931,8 +2931,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       ExtendedBlock previous, Set<Node> excludedNodes, 
       List<String> favoredNodes) throws IOException {
     final long blockSize;
-    final int replication;
+    final short numTargets;
     final byte storagePolicyID;
+    final boolean isStriped;
     Node clientNode = null;
     String clientMachine = null;
 
@@ -2970,7 +2971,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           .getClientMachine();
       clientNode = blockManager.getDatanodeManager().getDatanodeByHost(
           clientMachine);
-      replication = pendingFile.getFileReplication();
+      // TODO: make block group size configurable (HDFS-7337)
+      isStriped = pendingFile.isStriped();
+      numTargets = isStriped ?
+          HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS :
+          pendingFile.getFileReplication();
       storagePolicyID = pendingFile.getStoragePolicyID();
     } finally {
       readUnlock();
@@ -2982,7 +2987,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // choose targets for the new block to be allocated.
     final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget4NewBlock( 
-        src, replication, clientNode, excludedNodes, blockSize, favoredNodes,
+        src, numTargets, clientNode, excludedNodes, blockSize, favoredNodes,
         storagePolicyID);
 
     // Part II.
@@ -3021,9 +3026,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                                 ExtendedBlock.getLocalBlock(previous));
 
       // allocate new block, record block locations in INode.
-      newBlock = createNewBlock();
+      newBlock = createNewBlock(isStriped);
       INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
-      saveAllocatedBlock(src, inodesInPath, newBlock, targets);
+      saveAllocatedBlock(src, inodesInPath, newBlock, targets, isStriped);
 
       persistNewBlock(src, pendingFile);
       offset = pendingFile.computeFileSize();
@@ -3444,13 +3449,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *                     The last INode is the INode for {@code src} file.
    * @param newBlock newly allocated block to be save
    * @param targets target datanodes where replicas of the new block is placed
+   * @param isStriped is the file under striping or contigunous layout?
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
   BlockInfoContiguous saveAllocatedBlock(String src, INodesInPath inodesInPath,
-      Block newBlock, DatanodeStorageInfo[] targets)
+      Block newBlock, DatanodeStorageInfo[] targets, boolean isStriped)
           throws IOException {
     assert hasWriteLock();
-    BlockInfoContiguous b = dir.addBlock(src, inodesInPath, newBlock, targets);
+    BlockInfoContiguous b = dir.addBlock(src, inodesInPath, newBlock, targets,
+        isStriped);
     NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
     return b;
@@ -3458,10 +3465,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Create new block with a unique block id and a new generation stamp.
+   * @param isStriped is the file under striping or contiguous layout?
    */
-  Block createNewBlock() throws IOException {
+  Block createNewBlock(boolean isStriped) throws IOException {
     assert hasWriteLock();
-    Block b = new Block(nextBlockId(), 0, 0);
+    Block b = new Block(nextBlockId(isStriped), 0, 0);
     // Increment the generation stamp for every new block.
     b.setGenerationStamp(nextGenerationStamp(false));
     return b;
@@ -6057,11 +6065,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Increments, logs and then returns the block ID
+   * @param isStriped is the file under striping or contiguous layout?
    */
-  private long nextBlockId() throws IOException {
+  private long nextBlockId(boolean isStriped) throws IOException {
     assert hasWriteLock();
     checkNameNodeSafeMode("Cannot get next block ID");
-    final long blockId = blockIdManager.nextBlockId();
+    final long blockId = isStriped ?
+        blockIdManager.nextBlockGroupId() : blockIdManager.nextBlockId();
     getEditLog().logAllocateBlockId(blockId);
     // NB: callers sync the log
     return blockId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10c794ce/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 887a259..9259d2c 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
@@ -33,12 +33,14 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
@@ -866,4 +868,13 @@ public class INodeFile extends INodeWithAdditionalFields
     return snapshotBlocks != null &&
         Arrays.asList(snapshotBlocks).contains(block);
   }
+
+  @VisibleForTesting
+  /**
+   * @return true if the file is in the striping layout.
+   */
+  // TODO: move erasure coding policy to file XAttr (HDFS-7337)
+  public boolean isStriped() {
+    return getStoragePolicyID() == HdfsConstants.EC_STORAGE_POLICY_ID;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10c794ce/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
new file mode 100644
index 0000000..95133ce
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+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.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestAddBlockgroup {
+
+  public static final Log LOG = LogFactory.getLog(TestAddBlockgroup.class);
+
+  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
+      HdfsConstants.NUM_PARITY_BLOCKS;
+  private final short NUM_DATANODES = GROUP_SIZE;
+
+  private static final int BLOCKSIZE = 1024;
+  private static final short REPLICATION = 3;
+
+  private MiniDFSCluster cluster;
+  private Configuration conf;
+
+  @Before
+  public void setup() throws IOException {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES)
+        .build();
+    cluster.waitActive();
+    cluster.getFileSystem().setStoragePolicy(new Path("/"),
+        HdfsConstants.EC_STORAGE_POLICY_NAME);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testAddBlockGroup() throws Exception {
+    DistributedFileSystem fs = cluster.getFileSystem();
+    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+
+    final Path file1 = new Path("/file1");
+    DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L);
+    INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
+    BlockInfo[] file1Blocks = file1Node.getBlocks();
+    assertEquals(2, file1Blocks.length);
+    assertEquals(GROUP_SIZE, file1Blocks[0].numNodes());
+    assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP,
+        file1Blocks[1].getBlockId() - file1Blocks[0].getBlockId());
+  }
+}


[44/50] [abbrv] hadoop git commit: HDFS-7837. Erasure Coding: allocate and persist striped blocks in NameNode. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-7837. Erasure Coding: allocate and persist striped blocks in NameNode. Contributed by Jing Zhao.

 Conflicts:
     hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java


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

Branch: refs/heads/HDFS-7285
Commit: bc962947a19ed0a9ee130f62e819b65a64a8fb18
Parents: 5e90e36
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 2 13:44:33 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:12:57 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockIdManager.java  |  31 +++-
 .../hdfs/server/blockmanagement/BlockInfo.java  |   4 +-
 .../blockmanagement/BlockInfoContiguous.java    |   5 +
 .../blockmanagement/BlockInfoStriped.java       |   8 +-
 .../server/blockmanagement/BlockManager.java    |  44 ++++--
 .../hdfs/server/blockmanagement/BlocksMap.java  |  20 ++-
 .../blockmanagement/DecommissionManager.java    |   9 +-
 .../hdfs/server/namenode/FSDirectory.java       |  27 +++-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  69 ++++++---
 .../hdfs/server/namenode/FSImageFormat.java     |  12 +-
 .../server/namenode/FSImageFormatPBINode.java   |   5 +-
 .../server/namenode/FSImageFormatProtobuf.java  |   9 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  39 ++---
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  25 +++-
 .../server/namenode/NameNodeLayoutVersion.java  |   3 +-
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |   1 +
 .../hdfs/server/namenode/TestAddBlockgroup.java |  85 -----------
 .../server/namenode/TestAddStripedBlocks.java   | 146 +++++++++++++++++++
 18 files changed, 354 insertions(+), 188 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 3ae54ce..1d69d74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -103,21 +103,38 @@ public class BlockIdManager {
   }
 
   /**
-   * Sets the maximum allocated block ID for this filesystem. This is
+   * Sets the maximum allocated contiguous block ID for this filesystem. This is
    * the basis for allocating new block IDs.
    */
-  public void setLastAllocatedBlockId(long blockId) {
+  public void setLastAllocatedContiguousBlockId(long blockId) {
     blockIdGenerator.skipTo(blockId);
   }
 
   /**
-   * Gets the maximum sequentially allocated block ID for this filesystem
+   * Gets the maximum sequentially allocated contiguous block ID for this
+   * filesystem
    */
-  public long getLastAllocatedBlockId() {
+  public long getLastAllocatedContiguousBlockId() {
     return blockIdGenerator.getCurrentValue();
   }
 
   /**
+   * Sets the maximum allocated striped block ID for this filesystem. This is
+   * the basis for allocating new block IDs.
+   */
+  public void setLastAllocatedStripedBlockId(long blockId) {
+    blockGroupIdGenerator.skipTo(blockId);
+  }
+
+  /**
+   * Gets the maximum sequentially allocated striped block ID for this
+   * filesystem
+   */
+  public long getLastAllocatedStripedBlockId() {
+    return blockGroupIdGenerator.getCurrentValue();
+  }
+
+  /**
    * Sets the current generation stamp for legacy blocks
    */
   public void setGenerationStampV1(long stamp) {
@@ -188,11 +205,11 @@ public class BlockIdManager {
   /**
    * Increments, logs and then returns the block ID
    */
-  public long nextBlockId() {
+  public long nextContiguousBlockId() {
     return blockIdGenerator.nextValue();
   }
 
-  public long nextBlockGroupId() {
+  public long nextStripedBlockId() {
     return blockGroupIdGenerator.nextValue();
   }
 
@@ -216,7 +233,7 @@ public class BlockIdManager {
     return id < 0;
   }
 
-  public static long convertToGroupID(long id) {
+  public static long convertToStripedID(long id) {
     return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index d15cbec..1d8afbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -169,6 +169,8 @@ public abstract class BlockInfo extends Block
    */
   abstract void replaceBlock(BlockInfo newBlock);
 
+  public abstract boolean isStriped();
+
   /**
    * Find specified DatanodeDescriptor.
    * @return index or -1 if not found.
@@ -332,7 +334,7 @@ public abstract class BlockInfo extends Block
   }
 
   static BlockInfo copyOf(BlockInfo b) {
-    if (b instanceof BlockInfoContiguous) {
+    if (!b.isStriped()) {
       return new BlockInfoContiguous((BlockInfoContiguous) b);
     } else {
       return new BlockInfoStriped((BlockInfoStriped) b);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index e54cba3..b5d3d59 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -144,4 +144,9 @@ public class BlockInfoContiguous extends BlockInfo {
     ucBlock.setBlockCollection(getBlockCollection());
     return ucBlock;
   }
+
+  @Override
+  public final boolean isStriped() {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/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 57de772..8b458df 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
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
 /**
@@ -57,7 +56,7 @@ public class BlockInfoStriped extends BlockInfo {
     this.setBlockCollection(b.getBlockCollection());
   }
 
-  private short getTotalBlockNum() {
+  short getTotalBlockNum() {
     return (short) (dataBlockNum + parityBlockNum);
   }
 
@@ -175,6 +174,11 @@ public class BlockInfoStriped extends BlockInfo {
   }
 
   @Override
+  public final boolean isStriped() {
+    return true;
+  }
+
+  @Override
   public int numNodes() {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert triplets.length % 3 == 0 : "Malformed BlockInfo";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/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 a898696..7f6a2ff 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
@@ -578,11 +578,22 @@ public class BlockManager {
     return maxReplicationStreams;
   }
 
-  /**
-   * @return true if the block has minimum replicas
-   */
-  public boolean checkMinReplication(Block block) {
-    return (countNodes(block).liveReplicas() >= minReplication);
+  public int getDefaultStorageNum(BlockInfo block) {
+    return block.isStriped() ?
+        ((BlockInfoStriped) block).getTotalBlockNum() : defaultReplication;
+  }
+
+  public short getMinStorageNum(BlockInfo block) {
+    return block.isStriped() ?
+        ((BlockInfoStriped) block).getDataBlockNum() : minReplication;
+  }
+
+  public boolean checkMinStorage(BlockInfo block) {
+    return countNodes(block).liveReplicas() >= getMinStorageNum(block);
+  }
+
+  public boolean checkMinStorage(BlockInfo block, int liveNum) {
+    return liveNum >= getMinStorageNum(block);
   }
 
   /**
@@ -626,7 +637,7 @@ public class BlockManager {
       return false; // already completed (e.g. by syncBlock)
     
     final boolean b = commitBlock(lastBlock, commitBlock);
-    if (countNodes(lastBlock).liveReplicas() >= minReplication) {
+    if (checkMinStorage(lastBlock)) {
       completeBlock(bc, bc.numBlocks() - 1, false);
     }
     return b;
@@ -650,7 +661,7 @@ public class BlockManager {
     }
 
     int numNodes = curBlock.numNodes();
-    if (!force && numNodes < minReplication) {
+    if (!force && !checkMinStorage(curBlock, numNodes)) {
       throw new IOException("Cannot complete block: " +
           "block does not satisfy minimal replication requirement.");
     }
@@ -694,9 +705,8 @@ public class BlockManager {
    * when tailing edit logs as a Standby.
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
-      final BlockInfoContiguousUnderConstruction block) throws IOException {
-    // TODO: support BlockInfoStripedUC for editlog
-    block.commitBlock(block);
+      final BlockInfo block) throws IOException {
+    BlockInfo.commitBlock(block, block);
     return completeBlock(bc, block, true);
   }
 
@@ -747,7 +757,7 @@ public class BlockManager {
     // count in safe-mode.
     namesystem.adjustSafeModeBlockTotals(
         // decrement safe if we had enough
-        targets.length >= minReplication ? -1 : 0,
+        checkMinStorage(oldBlock, targets.length) ? -1 : 0,
         // always decrement total blocks
         -1);
 
@@ -1191,8 +1201,8 @@ public class BlockManager {
     NumberReplicas numberOfReplicas = countNodes(b.stored);
     boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >= bc
         .getBlockReplication();
-    boolean minReplicationSatisfied =
-        numberOfReplicas.liveReplicas() >= minReplication;
+    boolean minReplicationSatisfied = checkMinStorage(b.stored,
+        numberOfReplicas.liveReplicas());
     boolean hasMoreCorruptReplicas = minReplicationSatisfied &&
         (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) >
         bc.getBlockReplication();
@@ -2408,7 +2418,7 @@ public class BlockManager {
     // Now check for completion of blocks and safe block count
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
-        && numCurrentReplica >= minReplication) {
+        && checkMinStorage(storedBlock, numCurrentReplica)) {
       completeBlock(storedBlock.getBlockCollection(), storedBlock, false);
     } else if (storedBlock.isComplete()) {
       // check whether safe replication is reached for the block
@@ -2486,7 +2496,7 @@ public class BlockManager {
       + pendingReplications.getNumReplicas(storedBlock);
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
-        numLiveReplicas >= minReplication) {
+        checkMinStorage(storedBlock, numLiveReplicas)) {
       storedBlock = completeBlock(bc, storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
@@ -3156,6 +3166,8 @@ public class BlockManager {
   /**
    * Return the number of nodes hosting a given block, grouped
    * by the state of those replicas.
+   * For a striped block, this includes nodes storing blocks belonging to the
+   * striped block group.
    */
   public NumberReplicas countNodes(Block b) {
     int decommissioned = 0;
@@ -3304,7 +3316,7 @@ public class BlockManager {
     BlockInfo info = null;
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
       info = blocksMap.getStoredBlock(
-          new Block(BlockIdManager.convertToGroupID(block.getBlockId())));
+          new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
     }
     if (info == null) {
       info = blocksMap.getStoredBlock(block);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index d383de8..0cf5fe6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@ -43,8 +43,15 @@ class BlocksMap {
 
     @Override
     public boolean hasNext() {
-      return blockInfo != null && nextIdx < blockInfo.getCapacity()
-              && blockInfo.getDatanode(nextIdx) != null;
+      if (blockInfo == null) {
+        return false;
+      }
+      while (nextIdx < blockInfo.getCapacity() &&
+          blockInfo.getDatanode(nextIdx) == null) {
+        // note that for striped blocks there may be null in the triplets
+        nextIdx++;
+      }
+      return nextIdx < blockInfo.getCapacity();
     }
 
     @Override
@@ -123,10 +130,13 @@ class BlocksMap {
       return;
 
     blockInfo.setBlockCollection(null);
-    // TODO: fix this logic for block group
-    for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
+    final int size = blockInfo instanceof BlockInfoContiguous ?
+        blockInfo.numNodes() : blockInfo.getCapacity();
+    for(int idx = size - 1; idx >= 0; idx--) {
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
-      dn.removeBlock(blockInfo); // remove from the list and wipe the location
+      if (dn != null) {
+        dn.removeBlock(blockInfo); // remove from the list and wipe the location
+      }
     }
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 71c88f1..a60fe77 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -545,7 +545,7 @@ public class DecommissionManager {
       int underReplicatedInOpenFiles = 0;
       while (it.hasNext()) {
         numBlocksChecked++;
-        final BlockInfoContiguous block = it.next();
+        final BlockInfo block = it.next();
         // Remove the block from the list if it's no longer in the block map,
         // e.g. the containing file has been deleted
         if (blockManager.blocksMap.getStoredBlock(block) == null) {
@@ -579,8 +579,9 @@ public class DecommissionManager {
         }
 
         // Even if the block is under-replicated, 
-        // it doesn't block decommission if it's sufficiently replicated 
-        if (isSufficientlyReplicated(block, bc, num)) {
+        // it doesn't block decommission if it's sufficiently replicated
+        BlockInfoContiguous blk = (BlockInfoContiguous) block;
+        if (isSufficientlyReplicated(blk, bc, num)) {
           if (pruneSufficientlyReplicated) {
             it.remove();
           }
@@ -589,7 +590,7 @@ public class DecommissionManager {
 
         // We've found an insufficiently replicated block.
         if (insufficientlyReplicated != null) {
-          insufficientlyReplicated.add(block);
+          insufficientlyReplicated.add(blk);
         }
         // Log if this is our first time through
         if (firstReplicationLog) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/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 0c1032e..16b59ec 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
@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -410,6 +411,10 @@ 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()) {
+        newNode.addStripedBlocksFeature();
+      }
     } finally {
       writeUnlock();
     }
@@ -445,6 +450,10 @@ 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()) {
+          newNode.addStripedBlocksFeature();
+        }
         if (aclEntries != null) {
           AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID);
         }
@@ -466,7 +475,7 @@ public class FSDirectory implements Closeable {
   /**
    * Add a block to the file. Returns a reference to the added block.
    */
-  BlockInfoContiguous addBlock(String path, INodesInPath inodesInPath,
+  BlockInfo addBlock(String path, INodesInPath inodesInPath,
       Block block, DatanodeStorageInfo[] targets, boolean isStriped)
       throws IOException {
     writeLock();
@@ -478,16 +487,20 @@ public class FSDirectory implements Closeable {
       Preconditions.checkState(fileINode.isUnderConstruction());
 
       // check quota limits and updated space consumed
+      // TODO add quota usage for EC files
       updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
           fileINode.getBlockReplication(), true);
 
       // associate new last block for the file
-      BlockInfoContiguousUnderConstruction blockInfo =
-        new BlockInfoContiguousUnderConstruction(
-            block,
-            numLocations,
-            BlockUCState.UNDER_CONSTRUCTION,
-            targets);
+      final BlockInfo blockInfo;
+      if (isStriped) {
+        blockInfo = new BlockInfoStripedUnderConstruction(block,
+            HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS,
+            BlockUCState.UNDER_CONSTRUCTION, targets);
+      } else {
+        blockInfo = new BlockInfoContiguousUnderConstruction(block,
+            numLocations, BlockUCState.UNDER_CONSTRUCTION, targets);
+      }
       getBlockManager().addBlockCollection(blockInfo, fileINode);
       fileINode.addBlock(blockInfo);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/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 869f3d4..19a41f4 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
@@ -35,7 +35,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.XAttrSetFlag;
 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.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -414,7 +416,8 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      updateBlocks(fsDir, addCloseOp, iip, newFile);
+      // TODO whether the file is striped should later be retrieved from iip
+      updateBlocks(fsDir, addCloseOp, iip, newFile, newFile.isStriped());
       break;
     }
     case OP_CLOSE: {
@@ -434,7 +437,8 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      updateBlocks(fsDir, addCloseOp, iip, file);
+      // TODO whether the file is striped should later be retrieved from iip
+      updateBlocks(fsDir, addCloseOp, iip, file, file.isStriped());
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -492,7 +496,8 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
-      updateBlocks(fsDir, updateOp, iip, oldFile);
+      // TODO whether the file is striped should later be retrieved from iip
+      updateBlocks(fsDir, updateOp, iip, oldFile, oldFile.isStriped());
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
@@ -508,7 +513,8 @@ public class FSEditLogLoader {
       }
       INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
       // add the new block to the INodeFile
-      addNewBlock(addBlockOp, oldFile);
+      // TODO whether the file is striped should later be retrieved from iip
+      addNewBlock(addBlockOp, oldFile, oldFile.isStriped());
       break;
     }
     case OP_SET_REPLICATION: {
@@ -784,8 +790,15 @@ public class FSEditLogLoader {
     }
     case OP_ALLOCATE_BLOCK_ID: {
       AllocateBlockIdOp allocateBlockIdOp = (AllocateBlockIdOp) op;
-      fsNamesys.getBlockIdManager().setLastAllocatedBlockId(
-          allocateBlockIdOp.blockId);
+      if (BlockIdManager.isStripedBlockID(allocateBlockIdOp.blockId)) {
+        // ALLOCATE_BLOCK_ID is added for sequential block id, thus if the id
+        // is negative, it must belong to striped blocks
+        fsNamesys.getBlockIdManager().setLastAllocatedStripedBlockId(
+            allocateBlockIdOp.blockId);
+      } else {
+        fsNamesys.getBlockIdManager().setLastAllocatedContiguousBlockId(
+            allocateBlockIdOp.blockId);
+      }
       break;
     }
     case OP_ROLLING_UPGRADE_START: {
@@ -937,9 +950,9 @@ public class FSEditLogLoader {
 
   /**
    * Add a new block into the given INodeFile
-   * TODO support adding striped block
    */
-  private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException {
+  private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped)
+      throws IOException {
     BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
     Block newBlock= op.getLastBlock();
@@ -947,7 +960,7 @@ public class FSEditLogLoader {
     if (pBlock != null) { // the penultimate block is not null
       assert oldBlocks != null && oldBlocks.length > 0;
       // compare pBlock with the last block of oldBlocks
-      Block oldLastBlock = oldBlocks[oldBlocks.length - 1];
+      BlockInfo oldLastBlock = oldBlocks[oldBlocks.length - 1];
       if (oldLastBlock.getBlockId() != pBlock.getBlockId()
           || oldLastBlock.getGenerationStamp() != pBlock.getGenerationStamp()) {
         throw new IOException(
@@ -957,29 +970,33 @@ public class FSEditLogLoader {
       }
       
       oldLastBlock.setNumBytes(pBlock.getNumBytes());
-      if (oldLastBlock instanceof BlockInfoContiguousUnderConstruction) {
-        fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoContiguousUnderConstruction) oldLastBlock);
+      if (!oldLastBlock.isComplete()) {
+        fsNamesys.getBlockManager().forceCompleteBlock(file, oldLastBlock);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
       }
     } else { // the penultimate block is null
       Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
     }
     // add the new block
-    BlockInfoContiguous newBI = new BlockInfoContiguousUnderConstruction(
-          newBlock, file.getBlockReplication());
-    fsNamesys.getBlockManager().addBlockCollection(newBI, file);
-    file.addBlock(newBI);
+    final BlockInfo newBlockInfo;
+    if (isStriped) {
+      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
+          HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+    } else {
+      newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
+          file.getBlockReplication());
+    }
+    fsNamesys.getBlockManager().addBlockCollection(newBlockInfo, file);
+    file.addBlock(newBlockInfo);
     fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
   }
   
   /**
    * Update in-memory data structures with new block information.
-   * TODO support adding striped block
    * @throws IOException
    */
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
-      INodesInPath iip, INodeFile file) throws IOException {
+      INodesInPath iip, INodeFile file, boolean isStriped) throws IOException {
     // Update its block list
     BlockInfo[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
@@ -1008,11 +1025,10 @@ public class FSEditLogLoader {
         oldBlock.getGenerationStamp() != newBlock.getGenerationStamp();
       oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
       
-      if (oldBlock instanceof BlockInfoContiguousUnderConstruction &&
+      if (!oldBlock.isComplete() &&
           (!isLastBlock || op.shouldCompleteLastBlock())) {
         changeMade = true;
-        fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoContiguousUnderConstruction) oldBlock);
+        fsNamesys.getBlockManager().forceCompleteBlock(file, oldBlock);
       }
       if (changeMade) {
         // The state or gen-stamp of the block has changed. So, we may be
@@ -1041,13 +1057,18 @@ public class FSEditLogLoader {
       // We're adding blocks
       for (int i = oldBlocks.length; i < newBlocks.length; i++) {
         Block newBlock = newBlocks[i];
-        BlockInfoContiguous newBI;
+        final BlockInfo newBI;
         if (!op.shouldCompleteLastBlock()) {
           // TODO: shouldn't this only be true for the last block?
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
-          newBI = new BlockInfoContiguousUnderConstruction(
-              newBlock, file.getBlockReplication());
+          if (isStriped) {
+            newBI = new BlockInfoStripedUnderConstruction(newBlock,
+                HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+          } else {
+            newBI = new BlockInfoContiguousUnderConstruction(newBlock,
+                file.getBlockReplication());
+          }
         } else {
           // OP_CLOSE should add finalized blocks. This code path
           // is only executed when loading edits written by prior

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/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 7601ffa..0601cad 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
@@ -357,7 +357,14 @@ public class FSImageFormat {
 
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
-          namesystem.getBlockIdManager().setLastAllocatedBlockId(maxSequentialBlockId);
+          namesystem.getBlockIdManager().setLastAllocatedContiguousBlockId(
+              maxSequentialBlockId);
+          if (NameNodeLayoutVersion.supports(
+              NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)) {
+            final long maxStripedBlockId = in.readLong();
+            namesystem.getBlockIdManager().setLastAllocatedStripedBlockId(
+                maxStripedBlockId);
+          }
         } else {
 
           long startingGenStamp = namesystem.getBlockIdManager()
@@ -1263,7 +1270,8 @@ public class FSImageFormat {
         out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV1());
         out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV2());
         out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampAtblockIdSwitch());
-        out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedBlockId());
+        out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedContiguousBlockId());
+        out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedStripedBlockId());
         out.writeLong(context.getTxId());
         out.writeLong(sourceNamesystem.dir.getLastInodeId());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/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 a025bb0..5627788 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
@@ -630,8 +630,9 @@ public final class FSImageFormatPBINode {
       INodeSection.INodeFile.Builder b = buildINodeFile(n,
           parent.getSaverContext());
 
-      if (n.getBlocks() != null) {
-        for (Block block : n.getBlocks()) {
+      BlockInfoContiguous[] cBlks = n.getContiguousBlocks();
+      if (cBlks != null) {
+        for (Block block : cBlks) {
           b.addBlocks(PBHelper.convert(block));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index 3ee848a..62dc068 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -296,7 +296,11 @@ public final class FSImageFormatProtobuf {
       blockIdManager.setGenerationStampV1(s.getGenstampV1());
       blockIdManager.setGenerationStampV2(s.getGenstampV2());
       blockIdManager.setGenerationStampV1Limit(s.getGenstampV1Limit());
-      blockIdManager.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
+      blockIdManager.setLastAllocatedContiguousBlockId(s.getLastAllocatedBlockId());
+      if (s.hasLastAllocatedStripedBlockId()) {
+        blockIdManager.setLastAllocatedStripedBlockId(
+            s.getLastAllocatedStripedBlockId());
+      }
       imgTxId = s.getTransactionId();
       if (s.hasRollingUpgradeStartTime()
           && fsn.getFSImage().hasRollbackFSImage()) {
@@ -536,7 +540,8 @@ public final class FSImageFormatProtobuf {
           .setGenstampV1(blockIdManager.getGenerationStampV1())
           .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit())
           .setGenstampV2(blockIdManager.getGenerationStampV2())
-          .setLastAllocatedBlockId(blockIdManager.getLastAllocatedBlockId())
+          .setLastAllocatedBlockId(blockIdManager.getLastAllocatedContiguousBlockId())
+          .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId())
           .setTransactionId(context.getTxId());
 
       // We use the non-locked version of getNamespaceInfo here since

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/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 fba8784..fa18e1c 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
@@ -205,7 +205,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -2042,7 +2041,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean shouldRecoverNow = (newBlock == null);
 
     BlockInfo oldBlock = file.getLastBlock();
-    assert oldBlock instanceof BlockInfoContiguous;
+    assert !oldBlock.isStriped();
 
     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file,
         (BlockInfoContiguous) oldBlock);
@@ -3006,6 +3005,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // choose targets for the new block to be allocated.
+    // TODO we need block placement policy for striped block groups (HDFS-7613)
     final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget4NewBlock( 
         src, numTargets, clientNode, excludedNodes, blockSize, favoredNodes,
         storagePolicyID);
@@ -3046,8 +3046,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       // allocate new block, record block locations in INode.
       newBlock = createNewBlock(isStriped);
-      INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
-      saveAllocatedBlock(src, inodesInPath, newBlock, targets, isStriped);
+      saveAllocatedBlock(src, fileState.iip, newBlock, targets, isStriped);
 
       persistNewBlock(src, pendingFile);
       offset = pendingFile.computeFileSize();
@@ -3470,13 +3469,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param isStriped is the file under striping or contigunous layout?
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
-  // TODO: support striped block
-  BlockInfoContiguous saveAllocatedBlock(String src, INodesInPath inodesInPath,
+  BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath,
       Block newBlock, DatanodeStorageInfo[] targets, boolean isStriped)
-          throws IOException {
+      throws IOException {
     assert hasWriteLock();
-    BlockInfoContiguous b = dir.addBlock(src, inodesInPath, newBlock, targets,
-        isStriped);
+    BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets, isStriped);
     NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
     return b;
@@ -3526,13 +3523,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private boolean isCompleteBlock(String src, BlockInfo b) {
     if (!b.isComplete()) {
       final int numNodes = b.numNodes();
-      final int min;
+      final int min = blockManager.getMinStorageNum(b);
       final BlockUCState state = b.getBlockUCState();
-      if (b instanceof BlockInfoStripedUnderConstruction) {
-        min = ((BlockInfoStripedUnderConstruction) b).getDataBlockNum();
-      } else {
-        min = blockManager.minReplication;
-      }
       LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " + state
           + ", replication# = " + numNodes + (numNodes < min ? " < " : " >= ")
           + " minimum = " + min + ") in file " + src);
@@ -3723,7 +3715,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         BlockInfo bi = getStoredBlock(b);
         if (bi.isComplete()) {
           numRemovedComplete++;
-          if (bi.numNodes() >= blockManager.minReplication) {
+          if (blockManager.checkMinStorage(bi, bi.numNodes())) {
             numRemovedSafe++;
           }
         }
@@ -3952,7 +3944,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
         break;
-      assert blockManager.checkMinReplication(curBlock) :
+      assert blockManager.checkMinStorage(curBlock) :
               "A COMPLETE block is not minimally replicated in " + src;
     }
 
@@ -3987,8 +3979,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
 
     // If penultimate block doesn't exist then its minReplication is met
-    boolean penultimateBlockMinReplication = penultimateBlock == null ||
-        blockManager.checkMinReplication(penultimateBlock);
+    boolean penultimateBlockMinStorage = penultimateBlock == null ||
+        blockManager.checkMinStorage(penultimateBlock);
 
     switch(lastBlockState) {
     case COMPLETE:
@@ -3996,8 +3988,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       break;
     case COMMITTED:
       // Close file if committed blocks are minimally replicated
-      if(penultimateBlockMinReplication &&
-          blockManager.checkMinReplication(lastBlock)) {
+      if(penultimateBlockMinStorage &&
+          blockManager.checkMinStorage(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId());
         NameNode.stateChangeLog.warn("BLOCK*"
@@ -4097,6 +4089,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // Adjust disk space consumption if required
+    // TODO: support EC files
     final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();    
     if (diff > 0) {
       try {
@@ -4979,8 +4972,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Persist the new block (the last block of the given file).
-   * @param path
-   * @param file
    */
   private void persistNewBlock(String path, INodeFile file) {
     Preconditions.checkArgument(file.isUnderConstruction());
@@ -6092,7 +6083,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     assert hasWriteLock();
     checkNameNodeSafeMode("Cannot get next block ID");
     final long blockId = isStriped ?
-        blockIdManager.nextBlockGroupId() : blockIdManager.nextBlockId();
+        blockIdManager.nextStripedBlockId() : blockIdManager.nextContiguousBlockId();
     getEditLog().logAllocateBlockId(blockId);
     // NB: callers sync the log
     return blockId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/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 5fbcf92..da12e68 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
@@ -85,7 +85,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   static enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
-    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 1),
+    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 0),
     STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
         0);
 
@@ -260,10 +260,10 @@ public class INodeFile extends INodeWithAdditionalFields
   public void setBlock(int index, BlockInfo blk) {
     FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
     if (sb == null) {
-      assert blk instanceof BlockInfoContiguous;
+      assert !blk.isStriped();
       this.blocks[index] = (BlockInfoContiguous) blk;
     } else {
-      assert blk instanceof BlockInfoStriped;
+      assert blk.isStriped();
       assert hasNoContiguousBlock();
       sb.setBlock(index, (BlockInfoStriped) blk);
     }
@@ -281,12 +281,12 @@ public class INodeFile extends INodeWithAdditionalFields
     final BlockInfo ucBlock;
     FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
     if (sb == null) {
-      assert lastBlock instanceof BlockInfoContiguous;
+      assert !lastBlock.isStriped();
       ucBlock = ((BlockInfoContiguous) lastBlock)
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
     } else {
       assert hasNoContiguousBlock();
-      assert lastBlock instanceof BlockInfoStriped;
+      assert lastBlock.isStriped();
       ucBlock = ((BlockInfoStriped) lastBlock)
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
     }
@@ -545,7 +545,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * add a contiguous block to the block list
    */
-  void addBlock(BlockInfoContiguous newblock) {
+  private void addContiguousBlock(BlockInfoContiguous newblock) {
     if (this.blocks == null) {
       this.setContiguousBlocks(new BlockInfoContiguous[]{newblock});
     } else {
@@ -557,6 +557,19 @@ public class INodeFile extends INodeWithAdditionalFields
     }
   }
 
+  /** add a striped or contiguous block */
+  void addBlock(BlockInfo newblock) {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      assert !newblock.isStriped();
+      addContiguousBlock((BlockInfoContiguous) newblock);
+    } else {
+      assert newblock.isStriped();
+      assert hasNoContiguousBlock();
+      sb.addBlock((BlockInfoStriped) newblock);
+    }
+  }
+
   /** Set the blocks. */
   public void setContiguousBlocks(BlockInfoContiguous[] blocks) {
     this.blocks = blocks;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
index d235e2b..f93218f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
@@ -72,7 +72,8 @@ public class NameNodeLayoutVersion {
     BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
     TRUNCATE(-61, "Truncate"),
     APPEND_NEW_BLOCK(-62, "Support appending to new block"),
-    QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types");
+    QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types"),
+    ERASURE_CODING(-64, "Support erasure coding");
 
     private final FeatureInfo info;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index b6fd033..3f3a71e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -73,6 +73,7 @@ message NameSystemSection {
   optional uint64 lastAllocatedBlockId = 5;
   optional uint64 transactionId = 6;
   optional uint64 rollingUpgradeStartTime = 7;
+  optional uint64 lastAllocatedStripedBlockId = 8;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
deleted file mode 100644
index a2ef7b2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import org.apache.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.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestAddBlockgroup {
-
-  public static final Log LOG = LogFactory.getLog(TestAddBlockgroup.class);
-
-  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
-      HdfsConstants.NUM_PARITY_BLOCKS;
-  private final short NUM_DATANODES = GROUP_SIZE;
-
-  private static final int BLOCKSIZE = 1024;
-  private static final short REPLICATION = 3;
-
-  private MiniDFSCluster cluster;
-  private Configuration conf;
-
-  @Before
-  public void setup() throws IOException {
-    conf = new Configuration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES)
-        .build();
-    cluster.waitActive();
-    cluster.getFileSystem().setStoragePolicy(new Path("/"),
-        HdfsConstants.EC_STORAGE_POLICY_NAME);
-  }
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testAddBlockGroup() throws Exception {
-    DistributedFileSystem fs = cluster.getFileSystem();
-    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
-
-    final Path file1 = new Path("/file1");
-    DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L);
-    INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
-    BlockInfo[] file1Blocks = file1Node.getBlocks();
-    assertEquals(2, file1Blocks.length);
-    assertEquals(GROUP_SIZE, file1Blocks[0].numNodes());
-    assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP,
-        file1Blocks[1].getBlockId() - file1Blocks[0].getBlockId());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc962947/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
new file mode 100644
index 0000000..7226f51
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+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.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+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 static org.junit.Assert.assertEquals;
+
+public class TestAddStripedBlocks {
+  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
+      HdfsConstants.NUM_PARITY_BLOCKS;
+
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem dfs;
+
+  @Before
+  public void setup() throws IOException {
+    cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
+        .numDataNodes(GROUP_SIZE).build();
+    cluster.waitActive();
+    dfs = cluster.getFileSystem();
+    dfs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testAddStripedBlock() throws Exception {
+    final Path file = new Path("/file1");
+    // create an empty file
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1);
+
+      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+      LocatedBlock newBlock = cluster.getNamesystem().getAdditionalBlock(
+          file.toString(), fileNode.getId(), dfs.getClient().getClientName(),
+          null, null, null);
+      assertEquals(GROUP_SIZE, newBlock.getLocations().length);
+      assertEquals(GROUP_SIZE, newBlock.getStorageIDs().length);
+
+      BlockInfo[] blocks = fileNode.getBlocks();
+      assertEquals(1, blocks.length);
+      Assert.assertTrue(blocks[0].isStriped());
+
+      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), true);
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+
+    // restart NameNode to check editlog
+    cluster.restartNameNode(true);
+    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+    INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+    BlockInfo[] blocks = fileNode.getBlocks();
+    assertEquals(1, blocks.length);
+    Assert.assertTrue(blocks[0].isStriped());
+    checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
+
+    // save namespace, restart namenode, and check
+    dfs = cluster.getFileSystem();
+    dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    dfs.saveNamespace();
+    dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    cluster.restartNameNode(true);
+    fsdir = cluster.getNamesystem().getFSDirectory();
+    fileNode = fsdir.getINode4Write(file.toString()).asFile();
+    blocks = fileNode.getBlocks();
+    assertEquals(1, blocks.length);
+    Assert.assertTrue(blocks[0].isStriped());
+    checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
+  }
+
+  private void checkStripedBlockUC(BlockInfoStriped block,
+      boolean checkReplica) {
+    assertEquals(0, block.numNodes());
+    Assert.assertFalse(block.isComplete());
+    Assert.assertEquals(HdfsConstants.NUM_DATA_BLOCKS, block.getDataBlockNum());
+    Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS,
+        block.getParityBlockNum());
+    Assert.assertEquals(0,
+        block.getBlockId() & HdfsConstants.BLOCK_GROUP_INDEX_MASK);
+
+    final BlockInfoStripedUnderConstruction blockUC =
+        (BlockInfoStripedUnderConstruction) block;
+    Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+        blockUC.getBlockUCState());
+    if (checkReplica) {
+      Assert.assertEquals(GROUP_SIZE, blockUC.getNumExpectedLocations());
+      DatanodeStorageInfo[] storages = blockUC.getExpectedStorageLocations();
+      for (DataNode dn : cluster.getDataNodes()) {
+        Assert.assertTrue(includeDataNode(dn.getDatanodeId(), storages));
+      }
+    }
+  }
+
+  private boolean includeDataNode(DatanodeID dn, DatanodeStorageInfo[] storages) {
+    for (DatanodeStorageInfo storage : storages) {
+      if (storage.getDatanodeDescriptor().equals(dn)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}


[49/50] [abbrv] hadoop git commit: HADOOP-11705. Make erasure coder configurable. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11705. Make erasure coder configurable. 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/7018997b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7018997b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7018997b

Branch: refs/heads/HDFS-7285
Commit: 7018997bbbda7f333d995db94571fbe7d9bd1366
Parents: fc774e7
Author: drankye <ka...@intel.com>
Authored: Thu Mar 12 23:35:22 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:14:01 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  4 +++
 .../erasurecode/coder/AbstractErasureCoder.java |  5 ++-
 .../rawcoder/AbstractRawErasureCoder.java       |  5 ++-
 .../hadoop/io/erasurecode/TestCoderBase.java    |  6 ++++
 .../erasurecode/coder/TestErasureCoderBase.java | 36 +++++++++++++++++---
 .../erasurecode/rawcoder/TestRawCoderBase.java  | 13 +++++--
 6 files changed, 60 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7018997b/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 c17a1bd..a97dc34 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -18,3 +18,7 @@
     HADOOP-11646. Erasure Coder API for encoding and decoding of block group
     ( Kai Zheng via vinayakumarb )
 
+    HADOOP-11705. Make erasure coder configurable. Contributed by Kai Zheng
+    ( Kai Zheng )    
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7018997b/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 f2cc041..8d3bc34 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
@@ -17,12 +17,15 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
+import org.apache.hadoop.conf.Configured;
+
 /**
  * A common class of basic facilities to be shared by encoder and decoder
  *
  * It implements the {@link ErasureCoder} interface.
  */
-public abstract class AbstractErasureCoder implements ErasureCoder {
+public abstract class AbstractErasureCoder
+    extends Configured implements ErasureCoder {
 
   private int numDataUnits;
   private int numParityUnits;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7018997b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
index 74d2ab6..e6f3d92 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
@@ -17,12 +17,15 @@
  */
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
+import org.apache.hadoop.conf.Configured;
+
 /**
  * A common class of basic facilities to be shared by encoder and decoder
  *
  * It implements the {@link RawErasureCoder} interface.
  */
-public abstract class AbstractRawErasureCoder implements RawErasureCoder {
+public abstract class AbstractRawErasureCoder
+    extends Configured implements RawErasureCoder {
 
   private int numDataUnits;
   private int numParityUnits;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7018997b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
index 3c4288c..194413a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -43,6 +43,12 @@ public abstract class TestCoderBase {
   // may go to different coding implementations.
   protected boolean usingDirectBuffer = true;
 
+  /**
+   * Prepare before running the case.
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedIndexes
+   */
   protected void prepare(int numDataUnits, int numParityUnits,
                          int[] erasedIndexes) {
     this.numDataUnits = numDataUnits;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7018997b/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 ca5c1c9..36e061a 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECChunk;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
@@ -29,6 +30,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   protected Class<? extends ErasureEncoder> encoderClass;
   protected Class<? extends ErasureDecoder> decoderClass;
 
+  private Configuration conf;
   protected int numChunksInBlock = 16;
 
   /**
@@ -46,6 +48,19 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   }
 
   /**
+   * Prepare before running the case.
+   * @param conf
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedIndexes
+   */
+  protected void prepare(Configuration conf, int numDataUnits,
+                         int numParityUnits, int[] erasedIndexes) {
+    this.conf = conf;
+    super.prepare(numDataUnits, numParityUnits, erasedIndexes);
+  }
+
+  /**
    * Generating source data, encoding, recovering and then verifying.
    * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
    * it supports two kinds of ByteBuffers, one is array backed, the other is
@@ -56,6 +71,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     this.usingDirectBuffer = usingDirectBuffer;
 
     ErasureEncoder encoder = createEncoder();
+
     // Generate data and encode
     ECBlockGroup blockGroup = prepareBlockGroupForEncoding();
     // Backup all the source chunks for later recovering because some coders
@@ -65,17 +81,25 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     // Make a copy of a strip for later comparing
     TestBlock[] toEraseBlocks = copyDataBlocksToErase(clonedDataBlocks);
 
-    ErasureCodingStep codingStep = encoder.encode(blockGroup);
-    performCodingStep(codingStep);
+    ErasureCodingStep codingStep;
+    try {
+      codingStep = encoder.encode(blockGroup);
+      performCodingStep(codingStep);
+    } finally {
+      encoder.release();
+    }
     // Erase the copied sources
     eraseSomeDataBlocks(clonedDataBlocks);
 
     //Decode
     blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
     ErasureDecoder decoder = createDecoder();
-    codingStep = decoder.decode(blockGroup);
-    performCodingStep(codingStep);
-
+    try {
+      codingStep = decoder.decode(blockGroup);
+      performCodingStep(codingStep);
+    } finally {
+      decoder.release();
+    }
     //Compare
     compareAndVerify(toEraseBlocks, codingStep.getOutputBlocks());
   }
@@ -138,6 +162,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     encoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    encoder.setConf(conf);
     return encoder;
   }
 
@@ -154,6 +179,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     decoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    decoder.setConf(conf);
     return decoder;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7018997b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
index 5f6ccda..890f632 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
@@ -49,7 +49,11 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     // Make a copy of a strip for later comparing
     ECChunk[] toEraseDataChunks = copyDataChunksToErase(clonedDataChunks);
 
-    encoder.encode(dataChunks, parityChunks);
+    try {
+      encoder.encode(dataChunks, parityChunks);
+    } finally {
+      encoder.release();
+    }
     // Erase the copied sources
     eraseSomeDataBlocks(clonedDataChunks);
 
@@ -58,7 +62,12 @@ public abstract class TestRawCoderBase extends TestCoderBase {
         parityChunks);
     ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
     RawErasureDecoder decoder = createDecoder();
-    decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks);
+    try {
+      decoder.decode(inputChunks,
+          getErasedIndexesForDecoding(), recoveredChunks);
+    } finally {
+      decoder.release();
+    }
 
     //Compare
     compareAndVerify(toEraseDataChunks, recoveredChunks);


[07/50] [abbrv] hadoop git commit: HDFS-6833. DirectoryScanner should not register a deleting block with memory of DataNode. Contributed by Shinichi Yamashita

Posted by zh...@apache.org.
HDFS-6833.  DirectoryScanner should not register a deleting block with memory of DataNode.  Contributed by Shinichi Yamashita


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

Branch: refs/heads/HDFS-7285
Commit: 6dae6d12ec5abb716e1501cd4e18b10ae7809b94
Parents: 06ce1d9
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Mar 13 02:25:32 2015 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Fri Mar 13 02:25:32 2015 +0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/datanode/DirectoryScanner.java  | 20 ++++---
 .../server/datanode/fsdataset/FsDatasetSpi.java |  5 ++
 .../impl/FsDatasetAsyncDiskService.java         | 31 +++++++++-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 41 +++++++++++++-
 .../server/datanode/SimulatedFSDataset.java     |  5 ++
 .../extdataset/ExternalDatasetImpl.java         |  5 ++
 .../fsdataset/impl/TestFsDatasetImpl.java       | 59 ++++++++++++++++++++
 8 files changed, 158 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6dae6d12/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 07213dd..e52b849 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1139,6 +1139,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7830. DataNode does not release the volume lock when adding a volume
     fails. (Lei Xu via Colin P. Mccabe)
 
+    HDFS-6833.  DirectoryScanner should not register a deleting block with
+    memory of DataNode.  (Shinichi Yamashita via szetszwo)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6dae6d12/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 01f967f..61dfb14 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -443,13 +443,14 @@ public class DirectoryScanner implements Runnable {
         int d = 0; // index for blockpoolReport
         int m = 0; // index for memReprot
         while (m < memReport.length && d < blockpoolReport.length) {
-          FinalizedReplica memBlock = memReport[Math.min(m, memReport.length - 1)];
-          ScanInfo info = blockpoolReport[Math.min(
-              d, blockpoolReport.length - 1)];
+          FinalizedReplica memBlock = memReport[m];
+          ScanInfo info = blockpoolReport[d];
           if (info.getBlockId() < memBlock.getBlockId()) {
-            // Block is missing in memory
-            statsRecord.missingMemoryBlocks++;
-            addDifference(diffRecord, statsRecord, info);
+            if (!dataset.isDeletingBlock(bpid, info.getBlockId())) {
+              // Block is missing in memory
+              statsRecord.missingMemoryBlocks++;
+              addDifference(diffRecord, statsRecord, info);
+            }
             d++;
             continue;
           }
@@ -495,8 +496,11 @@ public class DirectoryScanner implements Runnable {
                         current.getBlockId(), current.getVolume());
         }
         while (d < blockpoolReport.length) {
-          statsRecord.missingMemoryBlocks++;
-          addDifference(diffRecord, statsRecord, blockpoolReport[d++]);
+          if (!dataset.isDeletingBlock(bpid, blockpoolReport[d].getBlockId())) {
+            statsRecord.missingMemoryBlocks++;
+            addDifference(diffRecord, statsRecord, blockpoolReport[d]);
+          }
+          d++;
         }
         LOG.info(statsRecord.toString());
       } //end for

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6dae6d12/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 10c8369..5b183e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -543,4 +543,9 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * Check whether the block was pinned
    */
   public boolean getPinning(ExtendedBlock block) throws IOException;
+  
+  /**
+   * Confirm whether the block is deleting
+   */
+  public boolean isDeletingBlock(String bpid, long blockId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6dae6d12/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
index 13e854f..c1d3990 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
@@ -22,7 +22,10 @@ import java.io.File;
 import java.io.FileDescriptor;
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -64,9 +67,14 @@ class FsDatasetAsyncDiskService {
   private static final long THREADS_KEEP_ALIVE_SECONDS = 60; 
   
   private final DataNode datanode;
+  private final FsDatasetImpl fsdatasetImpl;
   private final ThreadGroup threadGroup;
   private Map<File, ThreadPoolExecutor> executors
       = new HashMap<File, ThreadPoolExecutor>();
+  private Map<String, Set<Long>> deletedBlockIds 
+      = new HashMap<String, Set<Long>>();
+  private static final int MAX_DELETED_BLOCKS = 64;
+  private int numDeletedBlocks = 0;
   
   /**
    * Create a AsyncDiskServices with a set of volumes (specified by their
@@ -75,8 +83,9 @@ class FsDatasetAsyncDiskService {
    * The AsyncDiskServices uses one ThreadPool per volume to do the async
    * disk operations.
    */
-  FsDatasetAsyncDiskService(DataNode datanode) {
+  FsDatasetAsyncDiskService(DataNode datanode, FsDatasetImpl fsdatasetImpl) {
     this.datanode = datanode;
+    this.fsdatasetImpl = fsdatasetImpl;
     this.threadGroup = new ThreadGroup(getClass().getSimpleName());
   }
 
@@ -286,7 +295,27 @@ class FsDatasetAsyncDiskService {
         LOG.info("Deleted " + block.getBlockPoolId() + " "
             + block.getLocalBlock() + " file " + blockFile);
       }
+      updateDeletedBlockId(block);
       IOUtils.cleanup(null, volumeRef);
     }
   }
+  
+  private synchronized void updateDeletedBlockId(ExtendedBlock block) {
+    Set<Long> blockIds = deletedBlockIds.get(block.getBlockPoolId());
+    if (blockIds == null) {
+      blockIds = new HashSet<Long>();
+      deletedBlockIds.put(block.getBlockPoolId(), blockIds);
+    }
+    blockIds.add(block.getBlockId());
+    numDeletedBlocks++;
+    if (numDeletedBlocks == MAX_DELETED_BLOCKS) {
+      for (Entry<String, Set<Long>> e : deletedBlockIds.entrySet()) {
+        String bpid = e.getKey();
+        Set<Long> bs = e.getValue();
+        fsdatasetImpl.removeDeletedBlocks(bpid, bs);
+        bs.clear();
+      }
+      numDeletedBlocks = 0;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6dae6d12/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 0f28aa4..48ac6ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -237,6 +237,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   private volatile boolean fsRunning;
 
   final ReplicaMap volumeMap;
+  final Map<String, Set<Long>> deletingBlock;
   final RamDiskReplicaTracker ramDiskReplicaTracker;
   final RamDiskAsyncLazyPersistService asyncLazyPersistService;
 
@@ -298,8 +299,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
             VolumeChoosingPolicy.class), conf);
     volumes = new FsVolumeList(volumeFailureInfos, datanode.getBlockScanner(),
         blockChooserImpl);
-    asyncDiskService = new FsDatasetAsyncDiskService(datanode);
+    asyncDiskService = new FsDatasetAsyncDiskService(datanode, this);
     asyncLazyPersistService = new RamDiskAsyncLazyPersistService(datanode);
+    deletingBlock = new HashMap<String, Set<Long>>();
 
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
       addVolume(dataLocations, storage.getStorageDir(idx));
@@ -1795,7 +1797,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
               +  ". Parent not found for file " + f);
           continue;
         }
-        volumeMap.remove(bpid, invalidBlks[i]);
+        ReplicaInfo removing = volumeMap.remove(bpid, invalidBlks[i]);
+        addDeletingBlock(bpid, removing.getBlockId());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Block file " + removing.getBlockFile().getName()
+              + " is to be deleted");
+        }
       }
 
       if (v.isTransientStorage()) {
@@ -3005,5 +3012,35 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     FileStatus fss = localFS.getFileStatus(new Path(f.getAbsolutePath()));
     return fss.getPermission().getStickyBit();
   }
+  
+  @Override
+  public boolean isDeletingBlock(String bpid, long blockId) {
+    synchronized(deletingBlock) {
+      Set<Long> s = deletingBlock.get(bpid);
+      return s != null ? s.contains(blockId) : false;
+    }
+  }
+  
+  public void removeDeletedBlocks(String bpid, Set<Long> blockIds) {
+    synchronized (deletingBlock) {
+      Set<Long> s = deletingBlock.get(bpid);
+      if (s != null) {
+        for (Long id : blockIds) {
+          s.remove(id);
+        }
+      }
+    }
+  }
+  
+  private void addDeletingBlock(String bpid, Long blockId) {
+    synchronized(deletingBlock) {
+      Set<Long> s = deletingBlock.get(bpid);
+      if (s == null) {
+        s = new HashSet<Long>();
+        deletingBlock.put(bpid, s);
+      }
+      s.add(blockId);
+    }
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6dae6d12/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 8ae5415..f0dbd0f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -1318,5 +1318,10 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   public boolean getPinning(ExtendedBlock b) throws IOException {
     return blockMap.get(b.getBlockPoolId()).get(b.getLocalBlock()).pinned;
   }
+  
+  @Override
+  public boolean isDeletingBlock(String bpid, long blockId) {
+    throw new UnsupportedOperationException();
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6dae6d12/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index a3c9935..6653cca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -429,4 +429,9 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   public boolean getPinning(ExtendedBlock block) throws IOException {
     return false;
   }
+  
+  @Override
+  public boolean isDeletingBlock(String bpid, long blockId) {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6dae6d12/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
index 3b47dd0..403cb2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
@@ -18,10 +18,14 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import com.google.common.collect.Lists;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+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.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -29,8 +33,11 @@ import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DNConf;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaHandler;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy;
@@ -88,6 +95,8 @@ public class TestFsDatasetImpl {
   private DataNode datanode;
   private DataStorage storage;
   private FsDatasetImpl dataset;
+  
+  private final static String BLOCKPOOL = "BP-TEST";
 
   private static Storage.StorageDirectory createStorageDirectory(File root) {
     Storage.StorageDirectory sd = new Storage.StorageDirectory(root);
@@ -334,4 +343,54 @@ public class TestFsDatasetImpl {
 
     FsDatasetTestUtil.assertFileLockReleased(badDir.toString());
   }
+  
+  @Test
+  public void testDeletingBlocks() throws IOException {
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()).build();
+    try {
+      cluster.waitActive();
+      DataNode dn = cluster.getDataNodes().get(0);
+      
+      FsDatasetImpl ds = (FsDatasetImpl) DataNodeTestUtils.getFSDataset(dn);
+      FsVolumeImpl vol = ds.getVolumes().get(0);
+
+      ExtendedBlock eb;
+      ReplicaInfo info;
+      List<Block> blockList = new ArrayList<Block>();
+      for (int i = 1; i <= 63; i++) {
+        eb = new ExtendedBlock(BLOCKPOOL, i, 1, 1000 + i);
+        info = new FinalizedReplica(
+            eb.getLocalBlock(), vol, vol.getCurrentDir().getParentFile());
+        ds.volumeMap.add(BLOCKPOOL, info);
+        info.getBlockFile().createNewFile();
+        info.getMetaFile().createNewFile();
+        blockList.add(info);
+      }
+      ds.invalidate(BLOCKPOOL, blockList.toArray(new Block[0]));
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        // Nothing to do
+      }
+      assertTrue(ds.isDeletingBlock(BLOCKPOOL, blockList.get(0).getBlockId()));
+
+      blockList.clear();
+      eb = new ExtendedBlock(BLOCKPOOL, 64, 1, 1064);
+      info = new FinalizedReplica(
+          eb.getLocalBlock(), vol, vol.getCurrentDir().getParentFile());
+      ds.volumeMap.add(BLOCKPOOL, info);
+      info.getBlockFile().createNewFile();
+      info.getMetaFile().createNewFile();
+      blockList.add(info);
+      ds.invalidate(BLOCKPOOL, blockList.toArray(new Block[0]));
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        // Nothing to do
+      }
+      assertFalse(ds.isDeletingBlock(BLOCKPOOL, blockList.get(0).getBlockId()));
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }


[02/50] [abbrv] hadoop git commit: HADOOP-11693. Azure Storage FileSystem rename operations are throttled too aggressively to complete HBase WAL archiving. Contributed by Duo Xu.

Posted by zh...@apache.org.
HADOOP-11693. Azure Storage FileSystem rename operations are throttled too aggressively to complete HBase WAL archiving. Contributed by Duo Xu.


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

Branch: refs/heads/HDFS-7285
Commit: 7a346bcb4fa5b56191ed00a39e72e51c9bdf1b56
Parents: fb34f45
Author: cnauroth <cn...@apache.org>
Authored: Wed Mar 11 14:36:51 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Wed Mar 11 14:36:51 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../fs/azure/AzureNativeFileSystemStore.java    | 55 ++++++++++++++++++--
 .../hadoop/fs/azure/StorageInterface.java       |  6 ++-
 .../hadoop/fs/azure/StorageInterfaceImpl.java   |  4 +-
 .../hadoop/fs/azure/MockStorageInterface.java   |  2 +-
 5 files changed, 63 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a346bcb/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 5bca61f..fb699bc 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1088,6 +1088,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11618. DelegateToFileSystem erroneously uses default FS's port in
     constructor. (Brahma Reddy Battula via gera)
 
+    HADOOP-11693. Azure Storage FileSystem rename operations are throttled too
+    aggressively to complete HBase WAL archiving. (Duo Xu via cnauroth)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a346bcb/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index b664fe7..c6ba84f 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -134,6 +134,15 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   private static final String KEY_MAX_BACKOFF_INTERVAL = "fs.azure.io.retry.max.backoff.interval";
   private static final String KEY_BACKOFF_INTERVAL = "fs.azure.io.retry.backoff.interval";
   private static final String KEY_MAX_IO_RETRIES = "fs.azure.io.retry.max.retries";
+  
+  private static final String KEY_COPYBLOB_MIN_BACKOFF_INTERVAL = 
+    "fs.azure.io.copyblob.retry.min.backoff.interval";
+  private static final String KEY_COPYBLOB_MAX_BACKOFF_INTERVAL = 
+    "fs.azure.io.copyblob.retry.max.backoff.interval";
+  private static final String KEY_COPYBLOB_BACKOFF_INTERVAL = 
+    "fs.azure.io.copyblob.retry.backoff.interval";
+  private static final String KEY_COPYBLOB_MAX_IO_RETRIES = 
+    "fs.azure.io.copyblob.retry.max.retries";  
 
   private static final String KEY_SELF_THROTTLE_ENABLE = "fs.azure.selfthrottling.enable";
   private static final String KEY_SELF_THROTTLE_READ_FACTOR = "fs.azure.selfthrottling.read.factor";
@@ -199,6 +208,11 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   private static final int DEFAULT_MAX_BACKOFF_INTERVAL = 30 * 1000; // 30s
   private static final int DEFAULT_BACKOFF_INTERVAL = 1 * 1000; // 1s
   private static final int DEFAULT_MAX_RETRY_ATTEMPTS = 15;
+  
+  private static final int DEFAULT_COPYBLOB_MIN_BACKOFF_INTERVAL = 3  * 1000;
+  private static final int DEFAULT_COPYBLOB_MAX_BACKOFF_INTERVAL = 90 * 1000;
+  private static final int DEFAULT_COPYBLOB_BACKOFF_INTERVAL = 30 * 1000;
+  private static final int DEFAULT_COPYBLOB_MAX_RETRY_ATTEMPTS = 15;  
 
   // Self-throttling defaults. Allowed range = (0,1.0]
   // Value of 1.0 means no self-throttling.
@@ -2435,11 +2449,46 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       // Rename the source blob to the destination blob by copying it to
       // the destination blob then deleting it.
       //
-      dstBlob.startCopyFromBlob(srcUri, getInstrumentedContext());
+      // Copy blob operation in Azure storage is very costly. It will be highly
+      // likely throttled during Azure storage gc. Short term fix will be using
+      // a more intensive exponential retry policy when the cluster is getting 
+      // throttled.
+      try {
+        dstBlob.startCopyFromBlob(srcUri, null, getInstrumentedContext());
+      } catch (StorageException se) {
+        if (se.getErrorCode().equals(
+		  StorageErrorCode.SERVER_BUSY.toString())) {
+          int copyBlobMinBackoff = sessionConfiguration.getInt(
+            KEY_COPYBLOB_MIN_BACKOFF_INTERVAL,
+			DEFAULT_COPYBLOB_MIN_BACKOFF_INTERVAL);
+
+          int copyBlobMaxBackoff = sessionConfiguration.getInt(
+            KEY_COPYBLOB_MAX_BACKOFF_INTERVAL,
+			DEFAULT_COPYBLOB_MAX_BACKOFF_INTERVAL);
+
+          int copyBlobDeltaBackoff = sessionConfiguration.getInt(
+            KEY_COPYBLOB_BACKOFF_INTERVAL,
+			DEFAULT_COPYBLOB_BACKOFF_INTERVAL);
+
+          int copyBlobMaxRetries = sessionConfiguration.getInt(
+            KEY_COPYBLOB_MAX_IO_RETRIES,
+			DEFAULT_COPYBLOB_MAX_RETRY_ATTEMPTS);
+	        
+          BlobRequestOptions options = new BlobRequestOptions();
+          options.setRetryPolicyFactory(new RetryExponentialRetry(
+            copyBlobMinBackoff, copyBlobDeltaBackoff, copyBlobMaxBackoff, 
+			copyBlobMaxRetries));
+          dstBlob.startCopyFromBlob(srcUri, options, getInstrumentedContext());
+        } else {
+          throw se;
+        }
+      }
       waitForCopyToComplete(dstBlob, getInstrumentedContext());
-
       safeDelete(srcBlob, lease);
-    } catch (Exception e) {
+    } catch (StorageException e) {
+      // Re-throw exception as an Azure storage exception.
+      throw new AzureException(e);
+    } catch (URISyntaxException e) {
       // Re-throw exception as an Azure storage exception.
       throw new AzureException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a346bcb/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
index 91928a2..e89151d 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
@@ -383,6 +383,10 @@ abstract class StorageInterface {
      *
      * @param source
      *            A <code>java.net.URI</code> The URI of a source blob.
+     * @param options
+     *            A {@link BlobRequestOptions} object that specifies any additional options for the request. Specifying
+     *            <code>null</code> will use the default request options from the associated service client (
+     *            {@link CloudBlobClient}).
      * @param opContext
      *            An {@link OperationContext} object that represents the context for the current operation. This object
      *            is used to track requests to the storage service, and to provide additional runtime information about
@@ -394,7 +398,7 @@ abstract class StorageInterface {
      *
      */
     public abstract void startCopyFromBlob(URI source,
-        OperationContext opContext)
+        BlobRequestOptions options, OperationContext opContext)
         throws StorageException, URISyntaxException;
     
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a346bcb/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
index 2120536..90d4d88 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
@@ -393,11 +393,11 @@ class StorageInterfaceImpl extends StorageInterface {
     }
 
     @Override
-    public void startCopyFromBlob(URI source,
+    public void startCopyFromBlob(URI source, BlobRequestOptions options,
         OperationContext opContext)
             throws StorageException, URISyntaxException {
       getBlob().startCopyFromBlob(source,
-          null, null, null, opContext);
+          null, null, options, opContext);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a346bcb/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
index c51c05b..cde0e38 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
@@ -429,7 +429,7 @@ public class MockStorageInterface extends StorageInterface {
     }
 
     @Override
-    public void startCopyFromBlob(URI source,
+    public void startCopyFromBlob(URI source, BlobRequestOptions options,
         OperationContext opContext) throws StorageException, URISyntaxException {
       backingStore.copy(convertUriToDecodedString(source), convertUriToDecodedString(uri));
       //TODO: set the backingStore.properties.CopyState and


[12/50] [abbrv] hadoop git commit: YARN-3267. Timelineserver applies the ACL rules after applying the limit on the number of records (Chang Li via jeagles)

Posted by zh...@apache.org.
YARN-3267. Timelineserver applies the ACL rules after applying the limit on the number of records (Chang Li via jeagles)


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

Branch: refs/heads/HDFS-7285
Commit: 8180e676abb2bb500a48b3a0c0809d2a807ab235
Parents: 387f271
Author: Jonathan Eagles <je...@gmail.com>
Authored: Fri Mar 13 12:04:30 2015 -0500
Committer: Jonathan Eagles <je...@gmail.com>
Committed: Fri Mar 13 12:04:30 2015 -0500

----------------------------------------------------------------------
 .../jobhistory/TestJobHistoryEventHandler.java  | 14 +++---
 .../mapred/TestMRTimelineEventHandling.java     | 12 ++---
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../distributedshell/TestDistributedShell.java  |  4 +-
 .../server/timeline/LeveldbTimelineStore.java   | 18 +++++--
 .../server/timeline/MemoryTimelineStore.java    | 12 ++++-
 .../server/timeline/TimelineDataManager.java    | 50 +++++++++++---------
 .../yarn/server/timeline/TimelineReader.java    |  3 +-
 .../timeline/TestLeveldbTimelineStore.java      | 16 +++----
 .../timeline/TestTimelineDataManager.java       | 26 +++++++++-
 .../server/timeline/TimelineStoreTestUtils.java | 33 +++++++++----
 11 files changed, 126 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
index de35d84..43e3dbe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
@@ -464,7 +464,7 @@ public class TestJobHistoryEventHandler {
               t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000),
               currentTime - 10));
       TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
-              null, null, null, null, null, null);
+              null, null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       TimelineEntity tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
@@ -480,7 +480,7 @@ public class TestJobHistoryEventHandler {
               new HashMap<JobACL, AccessControlList>(), "default"),
               currentTime + 10));
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
@@ -498,7 +498,7 @@ public class TestJobHistoryEventHandler {
               new JobQueueChangeEvent(TypeConverter.fromYarn(t.jobId), "q2"),
               currentTime - 20));
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
@@ -520,7 +520,7 @@ public class TestJobHistoryEventHandler {
               new JobFinishedEvent(TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0,
               0, new Counters(), new Counters(), new Counters()), currentTime));
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
@@ -546,7 +546,7 @@ public class TestJobHistoryEventHandler {
             new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId),
             0, 0, 0, JobStateInternal.KILLED.toString()), currentTime + 20));
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
@@ -575,7 +575,7 @@ public class TestJobHistoryEventHandler {
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
             new TaskStartedEvent(t.taskID, 0, TaskType.MAP, "")));
       entities = ts.getEntities("MAPREDUCE_TASK", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId());
@@ -588,7 +588,7 @@ public class TestJobHistoryEventHandler {
       handleEvent(jheh, new JobHistoryEvent(t.jobId,
             new TaskStartedEvent(t.taskID, 0, TaskType.REDUCE, "")));
       entities = ts.getEntities("MAPREDUCE_TASK", null, null, null,
-              null, null, null, null, null);
+              null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
index 346953f..c2ef128 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
@@ -55,7 +55,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.SUCCEEDED,
               job.getJobStatus().getState().getValue());
       TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
-              null, null, null, null, null, null);
+              null, null, null, null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       TimelineEntity tEntity = entities.getEntities().get(0);
       Assert.assertEquals(job.getID().toString(), tEntity.getEntityId());
@@ -70,7 +70,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.FAILED,
               job.getJobStatus().getState().getValue());
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null,
-              null, null, null);
+              null, null, null, null);
       Assert.assertEquals(2, entities.getEntities().size());
       tEntity = entities.getEntities().get(0);
       Assert.assertEquals(job.getID().toString(), tEntity.getEntityId());
@@ -109,7 +109,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.SUCCEEDED,
           job.getJobStatus().getState().getValue());
       TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
-          null, null, null, null, null, null);
+          null, null, null, null, null, null, null);
       Assert.assertEquals(0, entities.getEntities().size());
 
       conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true);
@@ -117,7 +117,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.SUCCEEDED,
           job.getJobStatus().getState().getValue());
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null,
-          null, null, null);
+          null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       TimelineEntity tEntity = entities.getEntities().get(0);
       Assert.assertEquals(job.getID().toString(), tEntity.getEntityId());
@@ -148,7 +148,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.SUCCEEDED,
           job.getJobStatus().getState().getValue());
       TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
-          null, null, null, null, null, null);
+          null, null, null, null, null, null, null);
       Assert.assertEquals(0, entities.getEntities().size());
 
       conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true);
@@ -156,7 +156,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(JobStatus.SUCCEEDED,
           job.getJobStatus().getState().getValue());
       entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null,
-          null, null, null);
+          null, null, null, null);
       Assert.assertEquals(1, entities.getEntities().size());
       TimelineEntity tEntity = entities.getEntities().get(0);
       Assert.assertEquals(job.getID().toString(), tEntity.getEntityId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 80d2697..94f992d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -760,6 +760,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3154. Added additional APIs in LogAggregationContext to avoid aggregating
     running logs of application when rolling is enabled. (Xuan Gong via vinodkv)
 
+    YARN-3267. Timelineserver applies the ACL rules after applying the limit on
+    the number of records (Chang Li via jeagles)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 46b5850..5e6fa46 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -243,7 +243,7 @@ public class TestDistributedShell {
         .getApplicationHistoryServer()
         .getTimelineStore()
         .getEntities(ApplicationMaster.DSEntity.DS_APP_ATTEMPT.toString(),
-            null, null, null, null, null, null, null, null);
+            null, null, null, null, null, null, null, null, null);
     Assert.assertNotNull(entitiesAttempts);
     Assert.assertEquals(1, entitiesAttempts.getEntities().size());
     Assert.assertEquals(2, entitiesAttempts.getEntities().get(0).getEvents()
@@ -261,7 +261,7 @@ public class TestDistributedShell {
         .getApplicationHistoryServer()
         .getTimelineStore()
         .getEntities(ApplicationMaster.DSEntity.DS_CONTAINER.toString(), null,
-            null, null, null, null, null, null, null);
+            null, null, null, null, null, null, null, null);
     Assert.assertNotNull(entities);
     Assert.assertEquals(2, entities.getEntities().size());
     Assert.assertEquals(entities.getEntities().get(0).getEntityType()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
index 9fd2cfc..d521f70 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager.CheckAcl;
 import org.apache.hadoop.yarn.server.timeline.util.LeveldbUtils.KeyBuilder;
 import org.apache.hadoop.yarn.server.timeline.util.LeveldbUtils.KeyParser;
 import org.apache.hadoop.yarn.server.utils.LeveldbIterator;
@@ -56,6 +57,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import static org.apache.hadoop.yarn.server.timeline.GenericObjectMapper.readReverseOrderedLong;
 import static org.apache.hadoop.yarn.server.timeline.GenericObjectMapper.writeReverseOrderedLong;
+import static org.apache.hadoop.yarn.server.timeline.TimelineDataManager.DEFAULT_DOMAIN_ID;
 import static org.apache.hadoop.yarn.server.timeline.util.LeveldbUtils.prefixMatches;
 import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 
@@ -549,12 +551,13 @@ public class LeveldbTimelineStore extends AbstractService
   public TimelineEntities getEntities(String entityType,
       Long limit, Long windowStart, Long windowEnd, String fromId, Long fromTs,
       NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
-      EnumSet<Field> fields) throws IOException {
+      EnumSet<Field> fields, CheckAcl checkAcl) throws IOException {
     if (primaryFilter == null) {
       // if no primary filter is specified, prefix the lookup with
       // ENTITY_ENTRY_PREFIX
       return getEntityByTime(ENTITY_ENTRY_PREFIX, entityType, limit,
-          windowStart, windowEnd, fromId, fromTs, secondaryFilters, fields);
+          windowStart, windowEnd, fromId, fromTs, secondaryFilters, 
+          fields, checkAcl);
     } else {
       // if a primary filter is specified, prefix the lookup with
       // INDEXED_ENTRY_PREFIX + primaryFilterName + primaryFilterValue +
@@ -564,7 +567,7 @@ public class LeveldbTimelineStore extends AbstractService
           .add(GenericObjectMapper.write(primaryFilter.getValue()), true)
           .add(ENTITY_ENTRY_PREFIX).getBytesForLookup();
       return getEntityByTime(base, entityType, limit, windowStart, windowEnd,
-          fromId, fromTs, secondaryFilters, fields);
+          fromId, fromTs, secondaryFilters, fields, checkAcl);
     }
   }
 
@@ -586,7 +589,7 @@ public class LeveldbTimelineStore extends AbstractService
   private TimelineEntities getEntityByTime(byte[] base,
       String entityType, Long limit, Long starttime, Long endtime,
       String fromId, Long fromTs, Collection<NameValuePair> secondaryFilters,
-      EnumSet<Field> fields) throws IOException {
+      EnumSet<Field> fields, CheckAcl checkAcl) throws IOException {
     LeveldbIterator iterator = null;
     try {
       KeyBuilder kb = KeyBuilder.newInstance().add(base).add(entityType);
@@ -683,7 +686,12 @@ public class LeveldbTimelineStore extends AbstractService
           }
         }
         if (filterPassed) {
-          entities.addEntity(entity);
+          if (entity.getDomainId() == null) {
+            entity.setDomainId(DEFAULT_DOMAIN_ID);
+          }
+          if (checkAcl == null || checkAcl.check(entity)) {
+            entities.addEntity(entity);
+          }
         }
       }
       return entities;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
index 9c5419e..3489114 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/MemoryTimelineStore.java
@@ -47,6 +47,9 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager.CheckAcl;
+
+import static org.apache.hadoop.yarn.server.timeline.TimelineDataManager.DEFAULT_DOMAIN_ID;
 
 /**
  * In-memory implementation of {@link TimelineStore}. This
@@ -79,7 +82,7 @@ public class MemoryTimelineStore
   public synchronized TimelineEntities getEntities(String entityType, Long limit,
       Long windowStart, Long windowEnd, String fromId, Long fromTs,
       NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
-      EnumSet<Field> fields) {
+      EnumSet<Field> fields, CheckAcl checkAcl) throws IOException {
     if (limit == null) {
       limit = DEFAULT_LIMIT;
     }
@@ -146,7 +149,12 @@ public class MemoryTimelineStore
           continue;
         }
       }
-      entitiesSelected.add(entity);
+      if (entity.getDomainId() == null) {
+        entity.setDomainId(DEFAULT_DOMAIN_ID);
+      }
+      if (checkAcl == null || checkAcl.check(entity)) {
+        entitiesSelected.add(entity);
+      }
     }
     List<TimelineEntity> entitiesToReturn = new ArrayList<TimelineEntity>();
     for (TimelineEntity entitySelected : entitiesSelected) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
index 888c283..8c6b83a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
@@ -90,6 +90,31 @@ public class TimelineDataManager extends AbstractService {
     super.serviceInit(conf);
   }
 
+  public interface CheckAcl {
+    boolean check(TimelineEntity entity) throws IOException;
+  }
+
+  class CheckAclImpl implements CheckAcl {
+    final UserGroupInformation ugi;
+
+    public CheckAclImpl(UserGroupInformation callerUGI) {
+      ugi = callerUGI;
+    }
+
+    public boolean check(TimelineEntity entity) throws IOException {
+      try{
+        return timelineACLsManager.checkAccess(
+          ugi, ApplicationAccessType.VIEW_APP, entity);
+      } catch (YarnException e) {
+        LOG.info("Error when verifying access for user " + ugi
+          + " on the events of the timeline entity "
+          + new EntityIdentifier(entity.getEntityId(),
+          entity.getEntityType()), e);
+        return false;
+      }
+    }
+  }
+
   /**
    * Get the timeline entities that the given user have access to. The meaning
    * of each argument has been documented with
@@ -118,28 +143,9 @@ public class TimelineDataManager extends AbstractService {
         fromTs,
         primaryFilter,
         secondaryFilter,
-        fields);
-    if (entities != null) {
-      Iterator<TimelineEntity> entitiesItr =
-          entities.getEntities().iterator();
-      while (entitiesItr.hasNext()) {
-        TimelineEntity entity = entitiesItr.next();
-        addDefaultDomainIdIfAbsent(entity);
-        try {
-          // check ACLs
-          if (!timelineACLsManager.checkAccess(
-              callerUGI, ApplicationAccessType.VIEW_APP, entity)) {
-            entitiesItr.remove();
-          }
-        } catch (YarnException e) {
-          LOG.error("Error when verifying access for user " + callerUGI
-              + " on the events of the timeline entity "
-              + new EntityIdentifier(entity.getEntityId(),
-                  entity.getEntityType()), e);
-          entitiesItr.remove();
-        }
-      }
-    }
+        fields,
+        new CheckAclImpl(callerUGI));
+
     if (entities == null) {
       return new TimelineEntities();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java
index aba1ba2..012f4fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineReader.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager.CheckAcl;
 
 /**
  * This interface is for retrieving timeline information.
@@ -106,7 +107,7 @@ public interface TimelineReader {
   TimelineEntities getEntities(String entityType,
       Long limit, Long windowStart, Long windowEnd, String fromId, Long fromTs,
       NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
-      EnumSet<Field> fieldsToRetrieve) throws IOException;
+      EnumSet<Field> fieldsToRetrieve, CheckAcl checkAcl) throws IOException;
 
   /**
    * This method retrieves the entity information for a given entity.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
index 15edecd..c5c0f93 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
@@ -164,13 +164,13 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
   @Test
   public void testGetEntityTypes() throws IOException {
     List<String> entityTypes = ((LeveldbTimelineStore)store).getEntityTypes();
-    assertEquals(6, entityTypes.size());
-    assertEquals("OLD_ENTITY_TYPE_1", entityTypes.get(0));
-    assertEquals(entityType1, entityTypes.get(1));
-    assertEquals(entityType2, entityTypes.get(2));
-    assertEquals(entityType4, entityTypes.get(3));
-    assertEquals(entityType5, entityTypes.get(4));
-    assertEquals(entityType7, entityTypes.get(5));
+    assertEquals(7, entityTypes.size());
+    assertEquals("ACL_ENTITY_TYPE_1", entityTypes.get(0));
+    assertEquals("OLD_ENTITY_TYPE_1", entityTypes.get(1));
+    assertEquals(entityType1, entityTypes.get(2));
+    assertEquals(entityType2, entityTypes.get(3));
+    assertEquals(entityType4, entityTypes.get(4));
+    assertEquals(entityType5, entityTypes.get(5));
   }
 
   @Test
@@ -201,7 +201,7 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     ((LeveldbTimelineStore)store).discardOldEntities(-123l);
     assertEquals(2, getEntities("type_1").size());
     assertEquals(0, getEntities("type_2").size());
-    assertEquals(5, ((LeveldbTimelineStore)store).getEntityTypes().size());
+    assertEquals(6, ((LeveldbTimelineStore)store).getEntityTypes().size());
 
     ((LeveldbTimelineStore)store).discardOldEntities(123l);
     assertEquals(0, getEntities("type_1").size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
index f749567..87c3b24 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.AdminACLsManager;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
 import org.junit.After;
 import org.junit.Assert;
@@ -40,7 +41,8 @@ public class TestTimelineDataManager extends TimelineStoreTestUtils {
   private FileContext fsContext;
   private File fsPath;
   private TimelineDataManager dataManaer;
-
+  private static TimelineACLsManager aclsManager;
+  private static AdminACLsManager adminACLsManager;
   @Before
   public void setup() throws Exception {
     fsPath = new File("target", this.getClass().getSimpleName() +
@@ -58,8 +60,12 @@ public class TestTimelineDataManager extends TimelineStoreTestUtils {
     loadVerificationEntityData();
     loadTestDomainData();
 
-    TimelineACLsManager aclsManager = new TimelineACLsManager(conf);
+    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, false);
+    aclsManager = new TimelineACLsManager(conf);
     dataManaer = new TimelineDataManager(store, aclsManager);
+    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    conf.set(YarnConfiguration.YARN_ADMIN_ACL, "admin");
+    adminACLsManager = new AdminACLsManager(conf);
   }
 
   @After
@@ -85,6 +91,22 @@ public class TestTimelineDataManager extends TimelineStoreTestUtils {
   }
 
   @Test
+  public void testGetEntitiesAclEnabled() throws Exception {
+    AdminACLsManager oldAdminACLsManager =
+      aclsManager.setAdminACLsManager(adminACLsManager);
+    try {
+      TimelineEntities entities = dataManaer.getEntities(
+        "ACL_ENTITY_TYPE_1", null, null, null, null, null, null, 1l, null,
+        UserGroupInformation.createUserForTesting("owner_1", new String[] {"group1"}));
+      Assert.assertEquals(1, entities.getEntities().size());
+      Assert.assertEquals("ACL_ENTITY_ID_11",
+        entities.getEntities().get(0).getEntityId());
+    } finally {
+      aclsManager.setAdminACLsManager(oldAdminACLsManager);
+    }
+  }
+
+  @Test
   public void testGetOldEntitiesWithOutDomainId() throws Exception {
     TimelineEntities entities = dataManaer.getEntities(
         "OLD_ENTITY_TYPE_1", null, null, null, null, null, null, null, null,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8180e676/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
index c99786d..da71f46 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java
@@ -353,6 +353,19 @@ public class TimelineStoreTestUtils {
     domain3.setReaders("reader_user_4 reader_group_4");
     domain3.setWriters("writer_user_4 writer_group_4");
     store.put(domain3);
+
+    TimelineEntities entities = new TimelineEntities();
+    if (store instanceof LeveldbTimelineStore) {
+      LeveldbTimelineStore leveldb = (LeveldbTimelineStore) store;
+      entities.setEntities(Collections.singletonList(createEntity(
+              "ACL_ENTITY_ID_11", "ACL_ENTITY_TYPE_1", 63l, null, null, null, null,
+              "domain_id_4")));
+      leveldb.put(entities);
+      entities.setEntities(Collections.singletonList(createEntity(
+              "ACL_ENTITY_ID_22", "ACL_ENTITY_TYPE_1", 64l, null, null, null, null,
+              "domain_id_2")));
+      leveldb.put(entities);
+    }
   }
 
   public void testGetSingleEntity() throws IOException {
@@ -419,66 +432,66 @@ public class TimelineStoreTestUtils {
   protected List<TimelineEntity> getEntities(String entityType)
       throws IOException {
     return store.getEntities(entityType, null, null, null, null, null,
-        null, null, null).getEntities();
+        null, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesWithPrimaryFilter(
       String entityType, NameValuePair primaryFilter) throws IOException {
     return store.getEntities(entityType, null, null, null, null, null,
-        primaryFilter, null, null).getEntities();
+        primaryFilter, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromId(String entityType,
       String fromId) throws IOException {
     return store.getEntities(entityType, null, null, null, fromId, null,
-        null, null, null).getEntities();
+        null, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromTs(String entityType,
       long fromTs) throws IOException {
     return store.getEntities(entityType, null, null, null, null, fromTs,
-        null, null, null).getEntities();
+        null, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromIdWithPrimaryFilter(
       String entityType, NameValuePair primaryFilter, String fromId)
       throws IOException {
     return store.getEntities(entityType, null, null, null, fromId, null,
-        primaryFilter, null, null).getEntities();
+        primaryFilter, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromTsWithPrimaryFilter(
       String entityType, NameValuePair primaryFilter, long fromTs)
       throws IOException {
     return store.getEntities(entityType, null, null, null, null, fromTs,
-        primaryFilter, null, null).getEntities();
+        primaryFilter, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromIdWithWindow(String entityType,
       Long windowEnd, String fromId) throws IOException {
     return store.getEntities(entityType, null, null, windowEnd, fromId, null,
-        null, null, null).getEntities();
+        null, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesFromIdWithPrimaryFilterAndWindow(
       String entityType, Long windowEnd, String fromId,
       NameValuePair primaryFilter) throws IOException {
     return store.getEntities(entityType, null, null, windowEnd, fromId, null,
-        primaryFilter, null, null).getEntities();
+        primaryFilter, null, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntitiesWithFilters(String entityType,
       NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters)
       throws IOException {
     return store.getEntities(entityType, null, null, null, null, null,
-        primaryFilter, secondaryFilters, null).getEntities();
+        primaryFilter, secondaryFilters, null, null).getEntities();
   }
 
   protected List<TimelineEntity> getEntities(String entityType, Long limit,
       Long windowStart, Long windowEnd, NameValuePair primaryFilter,
       EnumSet<Field> fields) throws IOException {
     return store.getEntities(entityType, limit, windowStart, windowEnd, null,
-        null, primaryFilter, null, fields).getEntities();
+        null, primaryFilter, null, fields, null).getEntities();
   }
 
   public void testGetEntities() throws IOException {


[41/50] [abbrv] hadoop git commit: HADOOP-11542. Raw Reed-Solomon coder in pure Java. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11542. Raw Reed-Solomon coder in pure Java. 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/03778d47
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/03778d47
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/03778d47

Branch: refs/heads/HDFS-7285
Commit: 03778d476bffa426a98435d8ab53a82d0c2e867d
Parents: 10488c9
Author: drankye <dr...@gmail.com>
Authored: Thu Feb 12 21:12:44 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:10:17 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |   4 +
 .../io/erasurecode/rawcoder/JRSRawDecoder.java  |  69 +++
 .../io/erasurecode/rawcoder/JRSRawEncoder.java  |  78 +++
 .../erasurecode/rawcoder/RawErasureCoder.java   |   2 +-
 .../erasurecode/rawcoder/util/GaloisField.java  | 497 +++++++++++++++++++
 .../io/erasurecode/rawcoder/util/RSUtil.java    |  22 +
 .../hadoop/io/erasurecode/TestCoderBase.java    |  28 +-
 .../erasurecode/rawcoder/TestJRSRawCoder.java   |  93 ++++
 .../erasurecode/rawcoder/TestRawCoderBase.java  |   5 +-
 .../erasurecode/rawcoder/TestXorRawCoder.java   |   1 -
 10 files changed, 786 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/03778d47/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 9728f97..7bbacf7 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -8,3 +8,7 @@
 
     HADOOP-11541. Raw XOR coder
     ( Kai Zheng )
+
+    HADOOP-11542. Raw Reed-Solomon coder in pure Java. Contributed by Kai Zheng
+    ( Kai Zheng )
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03778d47/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
new file mode 100644
index 0000000..dbb689e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.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 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/03778d47/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
new file mode 100644
index 0000000..6ea7551
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.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 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/03778d47/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
index 91a9abf..6e07cf1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -71,7 +71,7 @@ public interface RawErasureCoder {
   public boolean preferNativeBuffer();
 
   /**
-   * Should be called when release this coder. Good chance to release encoding
+   * Should be called when release this blockcoder. Good chance to release encoding
    * or decoding buffers
    */
   public void release();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03778d47/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
new file mode 100644
index 0000000..77544c6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
@@ -0,0 +1,497 @@
+/**
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Implementation of Galois field arithmetic with 2^p elements. The input must
+ * be unsigned integers. It's ported from HDFS-RAID, slightly adapted.
+ */
+public class GaloisField {
+
+  // Field size 256 is good for byte based system
+  private static final int DEFAULT_FIELD_SIZE = 256;
+  // primitive polynomial 1 + X^2 + X^3 + X^4 + X^8 (substitute 2)
+  private static final int DEFAULT_PRIMITIVE_POLYNOMIAL = 285;
+  static private final Map<Integer, GaloisField> instances =
+      new HashMap<Integer, GaloisField>();
+  private final int[] logTable;
+  private final int[] powTable;
+  private final int[][] mulTable;
+  private final int[][] divTable;
+  private final int fieldSize;
+  private final int primitivePeriod;
+  private final int primitivePolynomial;
+
+  private GaloisField(int fieldSize, int primitivePolynomial) {
+    assert fieldSize > 0;
+    assert primitivePolynomial > 0;
+
+    this.fieldSize = fieldSize;
+    this.primitivePeriod = fieldSize - 1;
+    this.primitivePolynomial = primitivePolynomial;
+    logTable = new int[fieldSize];
+    powTable = new int[fieldSize];
+    mulTable = new int[fieldSize][fieldSize];
+    divTable = new int[fieldSize][fieldSize];
+    int value = 1;
+    for (int pow = 0; pow < fieldSize - 1; pow++) {
+      powTable[pow] = value;
+      logTable[value] = pow;
+      value = value * 2;
+      if (value >= fieldSize) {
+        value = value ^ primitivePolynomial;
+      }
+    }
+    // building multiplication table
+    for (int i = 0; i < fieldSize; i++) {
+      for (int j = 0; j < fieldSize; j++) {
+        if (i == 0 || j == 0) {
+          mulTable[i][j] = 0;
+          continue;
+        }
+        int z = logTable[i] + logTable[j];
+        z = z >= primitivePeriod ? z - primitivePeriod : z;
+        z = powTable[z];
+        mulTable[i][j] = z;
+      }
+    }
+    // building division table
+    for (int i = 0; i < fieldSize; i++) {
+      for (int j = 1; j < fieldSize; j++) {
+        if (i == 0) {
+          divTable[i][j] = 0;
+          continue;
+        }
+        int z = logTable[i] - logTable[j];
+        z = z < 0 ? z + primitivePeriod : z;
+        z = powTable[z];
+        divTable[i][j] = z;
+      }
+    }
+  }
+
+  /**
+   * Get the object performs Galois field arithmetics
+   *
+   * @param fieldSize           size of the field
+   * @param primitivePolynomial a primitive polynomial corresponds to the size
+   */
+  public static GaloisField getInstance(int fieldSize,
+                                        int primitivePolynomial) {
+    int key = ((fieldSize << 16) & 0xFFFF0000)
+        + (primitivePolynomial & 0x0000FFFF);
+    GaloisField gf;
+    synchronized (instances) {
+      gf = instances.get(key);
+      if (gf == null) {
+        gf = new GaloisField(fieldSize, primitivePolynomial);
+        instances.put(key, gf);
+      }
+    }
+    return gf;
+  }
+
+  /**
+   * Get the object performs Galois field arithmetic with default setting
+   */
+  public static GaloisField getInstance() {
+    return getInstance(DEFAULT_FIELD_SIZE, DEFAULT_PRIMITIVE_POLYNOMIAL);
+  }
+
+  /**
+   * Return number of elements in the field
+   *
+   * @return number of elements in the field
+   */
+  public int getFieldSize() {
+    return fieldSize;
+  }
+
+  /**
+   * Return the primitive polynomial in GF(2)
+   *
+   * @return primitive polynomial as a integer
+   */
+  public int getPrimitivePolynomial() {
+    return primitivePolynomial;
+  }
+
+  /**
+   * Compute the sum of two fields
+   *
+   * @param x input field
+   * @param y input field
+   * @return result of addition
+   */
+  public int add(int x, int y) {
+    assert (x >= 0 && x < getFieldSize() && y >= 0 && y < getFieldSize());
+    return x ^ y;
+  }
+
+  /**
+   * Compute the multiplication of two fields
+   *
+   * @param x input field
+   * @param y input field
+   * @return result of multiplication
+   */
+  public int multiply(int x, int y) {
+    assert (x >= 0 && x < getFieldSize() && y >= 0 && y < getFieldSize());
+    return mulTable[x][y];
+  }
+
+  /**
+   * Compute the division of two fields
+   *
+   * @param x input field
+   * @param y input field
+   * @return x/y
+   */
+  public int divide(int x, int y) {
+    assert (x >= 0 && x < getFieldSize() && y > 0 && y < getFieldSize());
+    return divTable[x][y];
+  }
+
+  /**
+   * Compute power n of a field
+   *
+   * @param x input field
+   * @param n power
+   * @return x^n
+   */
+  public int power(int x, int n) {
+    assert (x >= 0 && x < getFieldSize());
+    if (n == 0) {
+      return 1;
+    }
+    if (x == 0) {
+      return 0;
+    }
+    x = logTable[x] * n;
+    if (x < primitivePeriod) {
+      return powTable[x];
+    }
+    x = x % primitivePeriod;
+    return powTable[x];
+  }
+
+  /**
+   * Given a Vandermonde matrix V[i][j]=x[j]^i and vector y, solve for z such
+   * that Vz=y. The output z will be placed in y.
+   *
+   * @param x the vector which describe the Vandermonde matrix
+   * @param y right-hand side of the Vandermonde system equation. will be
+   *          replaced the output in this vector
+   */
+  public void solveVandermondeSystem(int[] x, int[] y) {
+    solveVandermondeSystem(x, y, x.length);
+  }
+
+  /**
+   * Given a Vandermonde matrix V[i][j]=x[j]^i and vector y, solve for z such
+   * that Vz=y. The output z will be placed in y.
+   *
+   * @param x   the vector which describe the Vandermonde matrix
+   * @param y   right-hand side of the Vandermonde system equation. will be
+   *            replaced the output in this vector
+   * @param len consider x and y only from 0...len-1
+   */
+  public void solveVandermondeSystem(int[] x, int[] y, int len) {
+    assert (x.length <= len && y.length <= len);
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        y[j] = y[j] ^ mulTable[x[i]][y[j - 1]];
+      }
+    }
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        y[j] = divTable[y[j]][x[j] ^ x[j - i - 1]];
+      }
+      for (int j = i; j < len - 1; j++) {
+        y[j] = y[j] ^ y[j + 1];
+      }
+    }
+  }
+
+  /**
+   * A "bulk" version to the solving of Vandermonde System
+   */
+  public void solveVandermondeSystem(int[] x, byte[][] y,
+                                     int len, int dataLen) {
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j][k] = (byte) (y[j][k] ^ mulTable[x[i]][y[j - 1][k] &
+              0x000000FF]);
+        }
+      }
+    }
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j][k] = (byte) (divTable[y[j][k] & 0x000000FF][x[j] ^
+              x[j - i - 1]]);
+        }
+      }
+      for (int j = i; j < len - 1; j++) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j][k] = (byte) (y[j][k] ^ y[j + 1][k]);
+        }
+      }
+    }
+  }
+
+  /**
+   * A "bulk" version of the solveVandermondeSystem, using ByteBuffer.
+   */
+  public void solveVandermondeSystem(int[] x, ByteBuffer[] y,
+                                     int len, int dataLen) {
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j].put(k, (byte) (y[j].get(k) ^ mulTable[x[i]][y[j - 1].get(k) &
+              0x000000FF]));
+        }
+      }
+    }
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j].put(k, (byte) (divTable[y[j].get(k) & 0x000000FF][x[j] ^
+              x[j - i - 1]]));
+        }
+      }
+      for (int j = i; j < len - 1; j++) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j].put(k, (byte) (y[j].get(k) ^ y[j + 1].get(k)));
+        }
+      }
+    }
+  }
+
+  /**
+   * Compute the multiplication of two polynomials. The index in the array
+   * corresponds to the power of the entry. For example p[0] is the constant
+   * term of the polynomial p.
+   *
+   * @param p input polynomial
+   * @param q input polynomial
+   * @return polynomial represents p*q
+   */
+  public int[] multiply(int[] p, int[] q) {
+    int len = p.length + q.length - 1;
+    int[] result = new int[len];
+    for (int i = 0; i < len; i++) {
+      result[i] = 0;
+    }
+    for (int i = 0; i < p.length; i++) {
+
+      for (int j = 0; j < q.length; j++) {
+        result[i + j] = add(result[i + j], multiply(p[i], q[j]));
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Compute the remainder of a dividend and divisor pair. The index in the
+   * array corresponds to the power of the entry. For example p[0] is the
+   * constant term of the polynomial p.
+   *
+   * @param dividend dividend polynomial, the remainder will be placed
+   *                 here when return
+   * @param divisor  divisor polynomial
+   */
+  public void remainder(int[] dividend, int[] divisor) {
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      int ratio = divTable[dividend[i +
+          divisor.length - 1]][divisor[divisor.length - 1]];
+      for (int j = 0; j < divisor.length; j++) {
+        int k = j + i;
+        dividend[k] = dividend[k] ^ mulTable[ratio][divisor[j]];
+      }
+    }
+  }
+
+  /**
+   * Compute the sum of two polynomials. The index in the array corresponds to
+   * the power of the entry. For example p[0] is the constant term of the
+   * polynomial p.
+   *
+   * @param p input polynomial
+   * @param q input polynomial
+   * @return polynomial represents p+q
+   */
+  public int[] add(int[] p, int[] q) {
+    int len = Math.max(p.length, q.length);
+    int[] result = new int[len];
+    for (int i = 0; i < len; i++) {
+      if (i < p.length && i < q.length) {
+        result[i] = add(p[i], q[i]);
+      } else if (i < p.length) {
+        result[i] = p[i];
+      } else {
+        result[i] = q[i];
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Substitute x into polynomial p(x).
+   *
+   * @param p input polynomial
+   * @param x input field
+   * @return p(x)
+   */
+  public int substitute(int[] p, int x) {
+    int result = 0;
+    int y = 1;
+    for (int i = 0; i < p.length; i++) {
+      result = result ^ mulTable[p[i]][y];
+      y = mulTable[x][y];
+    }
+    return result;
+  }
+
+  /**
+   * A "bulk" version of the substitute.
+   * Tends to be 2X faster than the "int" substitute in a loop.
+   *
+   * @param p input polynomial
+   * @param q store the return result
+   * @param x input field
+   */
+  public void substitute(byte[][] p, byte[] q, int x) {
+    int y = 1;
+    for (int i = 0; i < p.length; i++) {
+      byte[] pi = p[i];
+      for (int j = 0; j < pi.length; j++) {
+        int pij = pi[j] & 0x000000FF;
+        q[j] = (byte) (q[j] ^ mulTable[pij][y]);
+      }
+      y = mulTable[x][y];
+    }
+  }
+
+  /**
+   * A "bulk" version of the substitute, using ByteBuffer.
+   * Tends to be 2X faster than the "int" substitute in a loop.
+   *
+   * @param p input polynomial
+   * @param q store the return result
+   * @param x input field
+   */
+  public void substitute(ByteBuffer[] p, ByteBuffer q, int x) {
+    int y = 1;
+    for (int i = 0; i < p.length; i++) {
+      ByteBuffer pi = p[i];
+      int len = pi.remaining();
+      for (int j = 0; j < len; j++) {
+        int pij = pi.get(j) & 0x000000FF;
+        q.put(j, (byte) (q.get(j) ^ mulTable[pij][y]));
+      }
+      y = mulTable[x][y];
+    }
+  }
+
+  /**
+   * The "bulk" version of the remainder.
+   * Warning: This function will modify the "dividend" inputs.
+   */
+  public void remainder(byte[][] dividend, int[] divisor) {
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      for (int j = 0; j < divisor.length; j++) {
+        for (int k = 0; k < dividend[i].length; k++) {
+          int ratio = divTable[dividend[i + divisor.length - 1][k] &
+              0x00FF][divisor[divisor.length - 1]];
+          dividend[j + i][k] = (byte) ((dividend[j + i][k] & 0x00FF) ^
+              mulTable[ratio][divisor[j]]);
+        }
+      }
+    }
+  }
+
+  /**
+   * The "bulk" version of the remainder, using ByteBuffer.
+   * Warning: This function will modify the "dividend" inputs.
+   */
+  public void remainder(ByteBuffer[] dividend, int[] divisor) {
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      int width = dividend[i].remaining();
+      for (int j = 0; j < divisor.length; j++) {
+        for (int k = 0; k < width; k++) {
+          int ratio = divTable[dividend[i + divisor.length - 1].get(k) &
+              0x00FF][divisor[divisor.length - 1]];
+          dividend[j + i].put(k, (byte) ((dividend[j + i].get(k) & 0x00FF) ^
+              mulTable[ratio][divisor[j]]));
+        }
+      }
+    }
+  }
+
+  /**
+   * Perform Gaussian elimination on the given matrix. This matrix has to be a
+   * fat matrix (number of rows > number of columns).
+   */
+  public void gaussianElimination(int[][] matrix) {
+    assert(matrix != null && matrix.length > 0 && matrix[0].length > 0
+        && matrix.length < matrix[0].length);
+    int height = matrix.length;
+    int width = matrix[0].length;
+    for (int i = 0; i < height; i++) {
+      boolean pivotFound = false;
+      // scan the column for a nonzero pivot and swap it to the diagonal
+      for (int j = i; j < height; j++) {
+        if (matrix[i][j] != 0) {
+          int[] tmp = matrix[i];
+          matrix[i] = matrix[j];
+          matrix[j] = tmp;
+          pivotFound = true;
+          break;
+        }
+      }
+      if (!pivotFound) {
+        continue;
+      }
+      int pivot = matrix[i][i];
+      for (int j = i; j < width; j++) {
+        matrix[i][j] = divide(matrix[i][j], pivot);
+      }
+      for (int j = i + 1; j < height; j++) {
+        int lead = matrix[j][i];
+        for (int k = i; k < width; k++) {
+          matrix[j][k] = add(matrix[j][k], multiply(lead, matrix[i][k]));
+        }
+      }
+    }
+    for (int i = height - 1; i >=0; i--) {
+      for (int j = 0; j < i; j++) {
+        int lead = matrix[j][i];
+        for (int k = i; k < width; k++) {
+          matrix[j][k] = add(matrix[j][k], multiply(lead, matrix[i][k]));
+        }
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03778d47/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
new file mode 100644
index 0000000..33ba561
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
@@ -0,0 +1,22 @@
+package org.apache.hadoop.io.erasurecode.rawcoder.util;
+
+/**
+ * Some utilities for Reed-Solomon coding.
+ */
+public class RSUtil {
+
+  // We always use the byte system (with symbol size 8, field size 256,
+  // primitive polynomial 285, and primitive root 2).
+  public static GaloisField GF = GaloisField.getInstance();
+  public static final int PRIMITIVE_ROOT = 2;
+
+  public static int[] getPrimitivePower(int numDataUnits, int numParityUnits) {
+    int[] primitivePower = new int[numDataUnits + numParityUnits];
+    // compute powers of the primitive root
+    for (int i = 0; i < numDataUnits + numParityUnits; i++) {
+      primitivePower[i] = GF.power(PRIMITIVE_ROOT, i);
+    }
+    return primitivePower;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03778d47/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
index 9482b43..3c4288c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -18,9 +18,11 @@
 package org.apache.hadoop.io.erasurecode;
 
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Random;
 
 import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Test base of common utilities for tests not only raw coders but also block
@@ -41,6 +43,14 @@ public abstract class TestCoderBase {
   // may go to different coding implementations.
   protected boolean usingDirectBuffer = true;
 
+  protected void prepare(int numDataUnits, int numParityUnits,
+                         int[] erasedIndexes) {
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+    this.erasedDataIndexes = erasedIndexes != null ?
+        erasedIndexes : new int[] {0};
+  }
+
   /**
    * Compare and verify if erased chunks are equal to recovered chunks
    * @param erasedChunks
@@ -50,10 +60,8 @@ public abstract class TestCoderBase {
                                   ECChunk[] recoveredChunks) {
     byte[][] erased = ECChunk.toArray(erasedChunks);
     byte[][] recovered = ECChunk.toArray(recoveredChunks);
-    for (int i = 0; i < erasedChunks.length; ++i) {
-      assertArrayEquals("Decoding and comparing failed.", erased[i],
-          recovered[i]);
-    }
+    boolean result = Arrays.deepEquals(erased, recovered);
+    assertTrue("Decoding and comparing failed.", result);
   }
 
   /**
@@ -63,7 +71,7 @@ public abstract class TestCoderBase {
    */
   protected int[] getErasedIndexesForDecoding() {
     int[] erasedIndexesForDecoding = new int[erasedDataIndexes.length];
-    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
       erasedIndexesForDecoding[i] = erasedDataIndexes[i] + numParityUnits;
     }
     return erasedIndexesForDecoding;
@@ -100,7 +108,7 @@ public abstract class TestCoderBase {
     ECChunk[] copiedChunks = new ECChunk[erasedDataIndexes.length];
 
     int j = 0;
-    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
       copiedChunks[j ++] = cloneChunkWithData(dataChunks[erasedDataIndexes[i]]);
     }
 
@@ -112,7 +120,7 @@ public abstract class TestCoderBase {
    * @param dataChunks
    */
   protected void eraseSomeDataBlocks(ECChunk[] dataChunks) {
-    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
       eraseDataFromChunk(dataChunks[erasedDataIndexes[i]]);
     }
   }
@@ -122,7 +130,7 @@ public abstract class TestCoderBase {
    * @param chunks
    */
   protected void eraseDataFromChunks(ECChunk[] chunks) {
-    for (int i = 0; i < chunks.length; ++i) {
+    for (int i = 0; i < chunks.length; i++) {
       eraseDataFromChunk(chunks[i]);
     }
   }
@@ -135,7 +143,7 @@ public abstract class TestCoderBase {
     ByteBuffer chunkBuffer = chunk.getBuffer();
     // erase the data
     chunkBuffer.position(0);
-    for (int i = 0; i < chunkSize; ++i) {
+    for (int i = 0; i < chunkSize; i++) {
       chunkBuffer.put((byte) 0);
     }
     chunkBuffer.flip();
@@ -150,7 +158,7 @@ public abstract class TestCoderBase {
    */
   protected static ECChunk[] cloneChunksWithData(ECChunk[] chunks) {
     ECChunk[] results = new ECChunk[chunks.length];
-    for (int i = 0; i < chunks.length; ++i) {
+    for (int i = 0; i < chunks.length; i++) {
       results[i] = cloneChunkWithData(chunks[i]);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03778d47/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
new file mode 100644
index 0000000..e54f647
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.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 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(10, 4, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4() {
+    prepare(10, 4, null);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
+    prepare(10, 4, new int[] {2, 4});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_all() {
+    prepare(10, 4, new int[] {0, 1, 2, 3});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_3x3() {
+    prepare(3, 3, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_3x3() {
+    prepare(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/03778d47/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
index 9119211..5f6ccda 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
@@ -31,10 +31,13 @@ public abstract class TestRawCoderBase extends TestCoderBase {
    * Generating source data, encoding, recovering and then verifying.
    * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
    * it supports two kinds of ByteBuffers, one is array backed, the other is
-   * direct ByteBuffer. Have usingDirectBuffer to indicate which case to test.
+   * direct ByteBuffer. Use usingDirectBuffer indicate which case to test.
+   *
    * @param usingDirectBuffer
    */
   protected void testCoding(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+
     // Generate data and encode
     ECChunk[] dataChunks = prepareDataChunksForEncoding();
     ECChunk[] parityChunks = prepareParityChunksForEncoding();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03778d47/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
index 8e59b8a..ff48586 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
@@ -26,7 +26,6 @@ import java.util.Random;
  * Test XOR encoding and decoding.
  */
 public class TestXorRawCoder extends TestRawCoderBase {
-  private static Random RAND = new Random();
 
   @Before
   public void setup() {


[47/50] [abbrv] hadoop git commit: HDFS-7872. Erasure Coding: INodeFile.dumpTreeRecursively() supports to print striped blocks. Contributed by Takuya Fukudome.

Posted by zh...@apache.org.
HDFS-7872. Erasure Coding: INodeFile.dumpTreeRecursively() supports to print striped blocks. Contributed by Takuya Fukudome.


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

Branch: refs/heads/HDFS-7285
Commit: a91afb2db40fd4f74f44db4476a3ec14675c34d6
Parents: fc71e8b
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Mar 5 16:44:38 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:13:16 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a91afb2d/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 da12e68..a876f99 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
@@ -861,8 +861,8 @@ public class INodeFile extends INodeWithAdditionalFields
     out.print(", fileSize=" + computeFileSize(snapshotId));
     // only compare the first block
     out.print(", blocks=");
-    out.print(blocks == null || blocks.length == 0? null: blocks[0]);
-    // TODO print striped blocks
+    BlockInfo[] blks = getBlocks();
+    out.print(blks == null || blks.length == 0? null: blks[0]);
     out.println();
   }
 


[46/50] [abbrv] hadoop git commit: HADOOP-11646. Erasure Coder API for encoding and decoding of block group ( Contributed by Kai Zheng )

Posted by zh...@apache.org.
HADOOP-11646. Erasure Coder API for encoding and decoding of block group ( 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/ca332bf9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ca332bf9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ca332bf9

Branch: refs/heads/HDFS-7285
Commit: ca332bf9e3405b5c0c72fb569ad8a443e63ae731
Parents: a91afb2
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Mar 9 12:32:26 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:13:16 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |   2 +
 .../apache/hadoop/io/erasurecode/ECBlock.java   |  80 ++++++
 .../hadoop/io/erasurecode/ECBlockGroup.java     |  82 ++++++
 .../erasurecode/coder/AbstractErasureCoder.java |  63 +++++
 .../coder/AbstractErasureCodingStep.java        |  59 ++++
 .../coder/AbstractErasureDecoder.java           | 152 +++++++++++
 .../coder/AbstractErasureEncoder.java           |  50 ++++
 .../io/erasurecode/coder/ErasureCoder.java      |  77 ++++++
 .../io/erasurecode/coder/ErasureCodingStep.java |  55 ++++
 .../io/erasurecode/coder/ErasureDecoder.java    |  41 +++
 .../erasurecode/coder/ErasureDecodingStep.java  |  52 ++++
 .../io/erasurecode/coder/ErasureEncoder.java    |  39 +++
 .../erasurecode/coder/ErasureEncodingStep.java  |  49 ++++
 .../io/erasurecode/coder/XorErasureDecoder.java |  78 ++++++
 .../io/erasurecode/coder/XorErasureEncoder.java |  45 ++++
 .../erasurecode/rawcoder/RawErasureCoder.java   |   2 +-
 .../erasurecode/coder/TestErasureCoderBase.java | 266 +++++++++++++++++++
 .../io/erasurecode/coder/TestXorCoder.java      |  50 ++++
 18 files changed, 1241 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/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 ee42c84..c17a1bd 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -15,4 +15,6 @@
     HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng
     ( Kai Zheng )
 
+    HADOOP-11646. Erasure Coder API for encoding and decoding of block group
+    ( Kai Zheng via vinayakumarb )
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java
new file mode 100644
index 0000000..956954a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java
@@ -0,0 +1,80 @@
+/**
+ * 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;
+
+/**
+ * A wrapper of block level data source/output that {@link ECChunk}s can be
+ * extracted from. For HDFS, it can be an HDFS block (250MB). Note it only cares
+ * about erasure coding specific logic thus avoids coupling with any HDFS block
+ * details. We can have something like HdfsBlock extend it.
+ */
+public class ECBlock {
+
+  private boolean isParity;
+  private boolean isErased;
+
+  /**
+   * A default constructor. isParity and isErased are false by default.
+   */
+  public ECBlock() {
+    this(false, false);
+  }
+
+  /**
+   * A constructor specifying isParity and isErased.
+   * @param isParity
+   * @param isErased
+   */
+  public ECBlock(boolean isParity, boolean isErased) {
+    this.isParity = isParity;
+    this.isErased = isErased;
+  }
+
+  /**
+   * Set true if it's for a parity block.
+   * @param isParity
+   */
+  public void setParity(boolean isParity) {
+    this.isParity = isParity;
+  }
+
+  /**
+   * Set true if the block is missing.
+   * @param isMissing
+   */
+  public void setErased(boolean isMissing) {
+    this.isErased = isMissing;
+  }
+
+  /**
+   *
+   * @return true if it's parity block, otherwise false
+   */
+  public boolean isParity() {
+    return isParity;
+  }
+
+  /**
+   *
+   * @return true if it's missing or corrupt due to erasure, otherwise false
+   */
+  public boolean isErased() {
+    return isErased;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/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
new file mode 100644
index 0000000..2c851a5
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode;
+
+/**
+ * A group of blocks or {@link ECBlock} incurred in an erasure coding task.
+ */
+public class ECBlockGroup {
+
+  private ECBlock[] dataBlocks;
+  private ECBlock[] parityBlocks;
+
+  /**
+   * A constructor specifying data blocks and parity blocks.
+   * @param dataBlocks
+   * @param parityBlocks
+   */
+  public ECBlockGroup(ECBlock[] dataBlocks, ECBlock[] parityBlocks) {
+    this.dataBlocks = dataBlocks;
+    this.parityBlocks = parityBlocks;
+  }
+
+  /**
+   * Get data blocks
+   * @return data blocks
+   */
+  public ECBlock[] getDataBlocks() {
+    return dataBlocks;
+  }
+
+  /**
+   * Get parity blocks
+   * @return parity blocks
+   */
+  public ECBlock[] getParityBlocks() {
+    return parityBlocks;
+  }
+
+  /**
+   * Any erased data block?
+   * @return true if any erased data block, false otherwise
+   */
+  public boolean anyErasedDataBlock() {
+    for (int i = 0; i < dataBlocks.length; ++i) {
+      if (dataBlocks[i].isErased()) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Any erased parity block?
+   * @return true if any erased parity block, false otherwise
+   */
+  public boolean anyErasedParityBlock() {
+    for (int i = 0; i < parityBlocks.length; ++i) {
+      if (parityBlocks[i].isErased()) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/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
new file mode 100644
index 0000000..f2cc041
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -0,0 +1,63 @@
+/**
+ * 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;
+
+/**
+ * A common class of basic facilities to be shared by encoder and decoder
+ *
+ * It implements the {@link ErasureCoder} interface.
+ */
+public abstract class AbstractErasureCoder implements ErasureCoder {
+
+  private int numDataUnits;
+  private int numParityUnits;
+  private int chunkSize;
+
+  @Override
+  public void initialize(int numDataUnits, int numParityUnits,
+                         int chunkSize) {
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+    this.chunkSize = chunkSize;
+  }
+
+  @Override
+  public int getNumDataUnits() {
+    return numDataUnits;
+  }
+
+  @Override
+  public int getNumParityUnits() {
+    return numParityUnits;
+  }
+
+  @Override
+  public int getChunkSize() {
+    return chunkSize;
+  }
+
+  @Override
+  public boolean preferNativeBuffer() {
+    return false;
+  }
+
+  @Override
+  public void release() {
+    // Nothing to do by default
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
new file mode 100644
index 0000000..c429d49
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
@@ -0,0 +1,59 @@
+/**
+ * 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;
+
+/**
+ * Abstract class for common facilities shared by {@link ErasureEncodingStep}
+ * and {@link ErasureDecodingStep}.
+ *
+ * It implements {@link ErasureEncodingStep}.
+ */
+public abstract class AbstractErasureCodingStep implements ErasureCodingStep {
+
+  private ECBlock[] inputBlocks;
+  private ECBlock[] outputBlocks;
+
+  /**
+   * Constructor given input blocks and output blocks.
+   * @param inputBlocks
+   * @param outputBlocks
+   */
+  public AbstractErasureCodingStep(ECBlock[] inputBlocks,
+                                   ECBlock[] outputBlocks) {
+    this.inputBlocks = inputBlocks;
+    this.outputBlocks = outputBlocks;
+  }
+
+  @Override
+  public ECBlock[] getInputBlocks() {
+    return inputBlocks;
+  }
+
+  @Override
+  public ECBlock[] getOutputBlocks() {
+    return outputBlocks;
+  }
+
+  @Override
+  public void finish() {
+    // NOOP by default
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/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
new file mode 100644
index 0000000..df71528
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
@@ -0,0 +1,152 @@
+/**
+ * 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;
+
+/**
+ * An abstract erasure decoder that's to be inherited by new decoders.
+ *
+ * It implements the {@link ErasureDecoder} interface.
+ */
+public abstract class AbstractErasureDecoder extends AbstractErasureCoder
+    implements ErasureDecoder {
+
+  @Override
+  public ErasureCodingStep decode(ECBlockGroup blockGroup) {
+    return performDecoding(blockGroup);
+  }
+
+  /**
+   * Perform decoding against a block blockGroup.
+   * @param blockGroup
+   * @return decoding step for caller to do the real work
+   */
+  protected abstract ErasureCodingStep performDecoding(ECBlockGroup blockGroup);
+
+  /**
+   * We have all the data blocks and parity blocks as input blocks for
+   * recovering by default. It's codec specific
+   * @param blockGroup
+   * @return
+   */
+  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
+    ECBlock[] inputBlocks = new ECBlock[getNumParityUnits()
+        + getNumDataUnits()];
+
+    System.arraycopy(blockGroup.getParityBlocks(), 0, inputBlocks, 0,
+        getNumParityUnits());
+    System.arraycopy(blockGroup.getDataBlocks(), 0, inputBlocks,
+        getNumParityUnits(), getNumDataUnits());
+
+    return inputBlocks;
+  }
+
+  /**
+   * Which blocks were erased ? We only care data blocks here. Sub-classes can
+   * override this behavior.
+   * @param blockGroup
+   * @return output blocks to recover
+   */
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    ECBlock[] outputBlocks = new ECBlock[
+        getNumErasedBlocks(blockGroup.getDataBlocks())];
+
+    int idx = 0;
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      if (blockGroup.getDataBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
+      }
+    }
+
+    return outputBlocks;
+  }
+
+  /**
+   * Get the number of erased blocks in the block group.
+   * @param blockGroup
+   * @return number of erased blocks
+   */
+  protected int getNumErasedBlocks(ECBlockGroup blockGroup) {
+    int num = getNumErasedBlocks(blockGroup.getParityBlocks());
+    num += getNumErasedBlocks(blockGroup.getDataBlocks());
+    return num;
+  }
+
+  /**
+   * Find out how many blocks are erased.
+   * @param inputBlocks all the input blocks
+   * @return number of erased blocks
+   */
+  protected static int getNumErasedBlocks(ECBlock[] inputBlocks) {
+    int numErased = 0;
+    for (int i = 0; i < inputBlocks.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        numErased ++;
+      }
+    }
+
+    return numErased;
+  }
+
+  /**
+   * Get indexes of erased blocks from inputBlocks
+   * @param inputBlocks
+   * @return indexes of erased blocks from inputBlocks
+   */
+  protected int[] getErasedIndexes(ECBlock[] inputBlocks) {
+    int numErased = getNumErasedBlocks(inputBlocks);
+    if (numErased == 0) {
+      return new int[0];
+    }
+
+    int[] erasedIndexes = new int[numErased];
+    int i = 0, j = 0;
+    for (; i < inputBlocks.length && j < erasedIndexes.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        erasedIndexes[j++] = i;
+      }
+    }
+
+    return erasedIndexes;
+  }
+
+  /**
+   * Get erased input blocks from inputBlocks
+   * @param inputBlocks
+   * @return an array of erased blocks from inputBlocks
+   */
+  protected ECBlock[] getErasedBlocks(ECBlock[] inputBlocks) {
+    int numErased = getNumErasedBlocks(inputBlocks);
+    if (numErased == 0) {
+      return new ECBlock[0];
+    }
+
+    ECBlock[] erasedBlocks = new ECBlock[numErased];
+    int i = 0, j = 0;
+    for (; i < inputBlocks.length && j < erasedBlocks.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        erasedBlocks[j++] = inputBlocks[i];
+      }
+    }
+
+    return erasedBlocks;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/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
new file mode 100644
index 0000000..80b9875
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.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.apache.hadoop.io.erasurecode.ECBlock;
+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.
+ */
+public abstract class AbstractErasureEncoder extends AbstractErasureCoder
+    implements ErasureEncoder {
+
+  @Override
+  public ErasureCodingStep encode(ECBlockGroup blockGroup) {
+    return performEncoding(blockGroup);
+  }
+
+  /**
+   * Perform encoding against a block group.
+   * @param blockGroup
+   * @return encoding step for caller to do the real work
+   */
+  protected abstract ErasureCodingStep performEncoding(ECBlockGroup blockGroup);
+
+  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
+    return blockGroup.getDataBlocks();
+  }
+
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    return blockGroup.getParityBlocks();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/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
new file mode 100644
index 0000000..68875c0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
@@ -0,0 +1,77 @@
+/**
+ * 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;
+
+/**
+ * An erasure coder to perform encoding or decoding given a group. Generally it
+ * involves calculating necessary internal steps according to codec logic. For
+ * each step,it calculates necessary input blocks to read chunks from and output
+ * parity blocks to write parity chunks into from the group. It also takes care
+ * of appropriate raw coder to use for the step. And encapsulates all the
+ * necessary info (input blocks, output blocks and raw coder) into a step
+ * represented by {@link ErasureCodingStep}. ErasureCoder callers can use the
+ * step to do the real work with retrieved input and output chunks.
+ *
+ * Note, currently only one coding step is supported. Will support complex cases
+ * of multiple coding steps.
+ *
+ */
+public interface ErasureCoder {
+
+  /**
+   * Initialize with the important parameters for the code.
+   * @param numDataUnits how many data inputs for the coding
+   * @param numParityUnits how many parity outputs the coding generates
+   * @param chunkSize the size of the input/output buffer
+   */
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize);
+
+  /**
+   * The number of data input units for the coding. A unit can be a byte,
+   * chunk or buffer or even a block.
+   * @return count of data input units
+   */
+  public int getNumDataUnits();
+
+  /**
+   * The number of parity output units for the coding. A unit can be a byte,
+   * chunk, buffer or even a block.
+   * @return count of parity output units
+   */
+  public int getNumParityUnits();
+
+  /**
+   * Chunk buffer size for the input/output
+   * @return chunk buffer size
+   */
+  public int getChunkSize();
+
+  /**
+   * Tell if native or off-heap buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, either on heap or off heap.
+   * It will return false by default.
+   * @return true if native buffer is preferred for performance consideration,
+   * otherwise false.
+   */
+  public boolean preferNativeBuffer();
+
+  /**
+   * Release the resources if any. Good chance to invoke RawErasureCoder#release.
+   */
+  public void release();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
new file mode 100644
index 0000000..a3b177f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
@@ -0,0 +1,55 @@
+/**
+ * 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.ECChunk;
+
+/**
+ * Erasure coding step that's involved in encoding/decoding of a block group.
+ */
+public interface ErasureCodingStep {
+
+  /**
+   * Input blocks of readable data involved in this step, may be data blocks
+   * or parity blocks.
+   * @return input blocks
+   */
+  public ECBlock[] getInputBlocks();
+
+  /**
+   * Output blocks of writable buffers involved in this step, may be data
+   * blocks or parity blocks.
+   * @return output blocks
+   */
+  public ECBlock[] getOutputBlocks();
+
+  /**
+   * Perform encoding or decoding given the input chunks, and generated results
+   * will be written to the output chunks.
+   * @param inputChunks
+   * @param outputChunks
+   */
+  public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks);
+
+  /**
+   * Notify erasure coder that all the chunks of input blocks are processed so
+   * the coder can be able to update internal states, considering next step.
+   */
+  public void finish();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/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
new file mode 100644
index 0000000..dfd9e54
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.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.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/ca332bf9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
new file mode 100644
index 0000000..980c580
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
@@ -0,0 +1,52 @@
+/**
+ * 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.ECChunk;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+
+/**
+ * Erasure decoding step, a wrapper of all the necessary information to perform
+ * a decoding step involved in the whole process of decoding a block group.
+ */
+public class ErasureDecodingStep extends AbstractErasureCodingStep {
+  private int[] erasedIndexes;
+  private RawErasureDecoder rawDecoder;
+
+  /**
+   * The constructor with all the necessary info.
+   * @param inputBlocks
+   * @param erasedIndexes the indexes of erased blocks in inputBlocks array
+   * @param outputBlocks
+   * @param rawDecoder
+   */
+  public ErasureDecodingStep(ECBlock[] inputBlocks, int[] erasedIndexes,
+                             ECBlock[] outputBlocks,
+                             RawErasureDecoder rawDecoder) {
+    super(inputBlocks, outputBlocks);
+    this.erasedIndexes = erasedIndexes;
+    this.rawDecoder = rawDecoder;
+  }
+
+  @Override
+  public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) {
+    rawDecoder.decode(inputChunks, erasedIndexes, outputChunks);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/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
new file mode 100644
index 0000000..e837d22
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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/ca332bf9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
new file mode 100644
index 0000000..bd7587f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.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.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+
+/**
+ * Erasure encoding step, a wrapper of all the necessary information to perform
+ * an encoding step involved in the whole process of encoding a block group.
+ */
+public class ErasureEncodingStep extends AbstractErasureCodingStep {
+
+  private RawErasureEncoder rawEncoder;
+
+  /**
+   * The constructor with all the necessary info.
+   * @param inputBlocks
+   * @param outputBlocks
+   * @param rawEncoder
+   */
+  public ErasureEncodingStep(ECBlock[] inputBlocks, ECBlock[] outputBlocks,
+                             RawErasureEncoder rawEncoder) {
+    super(inputBlocks, outputBlocks);
+    this.rawEncoder = rawEncoder;
+  }
+
+  @Override
+  public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) {
+    rawEncoder.encode(inputChunks, outputChunks);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/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..9a6f6e2
--- /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 performDecoding(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/ca332bf9/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..db6e3db
--- /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 performEncoding(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/ca332bf9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
index 6e07cf1..91a9abf 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -71,7 +71,7 @@ public interface RawErasureCoder {
   public boolean preferNativeBuffer();
 
   /**
-   * Should be called when release this blockcoder. Good chance to release encoding
+   * Should be called when release this coder. Good chance to release encoding
    * or decoding buffers
    */
   public void release();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca332bf9/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
new file mode 100644
index 0000000..ca5c1c9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
@@ -0,0 +1,266 @@
+/**
+ * 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.ECChunk;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+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 int numChunksInBlock = 16;
+
+  /**
+   * It's just a block for this test purpose. We don't use HDFS block here
+   * at all for simple.
+   */
+  protected static class TestBlock extends ECBlock {
+    private ECChunk[] chunks;
+
+    // For simple, just assume the block have the chunks already ready.
+    // In practice we need to read/write chunks from/to the block via file IO.
+    public TestBlock(ECChunk[] chunks) {
+      this.chunks = chunks;
+    }
+  }
+
+  /**
+   * Generating source data, encoding, recovering and then verifying.
+   * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
+   * it supports two kinds of ByteBuffers, one is array backed, the other is
+   * direct ByteBuffer. Have usingDirectBuffer to indicate which case to test.
+   * @param usingDirectBuffer
+   */
+  protected void testCoding(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+
+    ErasureEncoder encoder = createEncoder();
+    // Generate data and encode
+    ECBlockGroup blockGroup = prepareBlockGroupForEncoding();
+    // Backup all the source chunks for later recovering because some coders
+    // may affect the source data.
+    TestBlock[] clonedDataBlocks = cloneBlocksWithData((TestBlock[])
+        blockGroup.getDataBlocks());
+    // Make a copy of a strip for later comparing
+    TestBlock[] toEraseBlocks = copyDataBlocksToErase(clonedDataBlocks);
+
+    ErasureCodingStep codingStep = encoder.encode(blockGroup);
+    performCodingStep(codingStep);
+    // Erase the copied sources
+    eraseSomeDataBlocks(clonedDataBlocks);
+
+    //Decode
+    blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
+    ErasureDecoder decoder = createDecoder();
+    codingStep = decoder.decode(blockGroup);
+    performCodingStep(codingStep);
+
+    //Compare
+    compareAndVerify(toEraseBlocks, codingStep.getOutputBlocks());
+  }
+
+  /**
+   * This is typically how a coding step should be performed.
+   * @param codingStep
+   */
+  private void performCodingStep(ErasureCodingStep codingStep) {
+    // Pretend that we're opening these input blocks and output blocks.
+    ECBlock[] inputBlocks = codingStep.getInputBlocks();
+    ECBlock[] outputBlocks = codingStep.getOutputBlocks();
+    // We allocate input and output chunks accordingly.
+    ECChunk[] inputChunks = new ECChunk[inputBlocks.length];
+    ECChunk[] outputChunks = new ECChunk[outputBlocks.length];
+
+    for (int i = 0; i < numChunksInBlock; ++i) {
+      // Pretend that we're reading input chunks from input blocks.
+      for (int j = 0; j < inputBlocks.length; ++j) {
+        inputChunks[j] = ((TestBlock) inputBlocks[j]).chunks[i];
+      }
+
+      // Pretend that we allocate and will write output results to the blocks.
+      for (int j = 0; j < outputBlocks.length; ++j) {
+        outputChunks[j] = allocateOutputChunk();
+        ((TestBlock) outputBlocks[j]).chunks[i] = outputChunks[j];
+      }
+
+      // Given the input chunks and output chunk buffers, just call it !
+      codingStep.performCoding(inputChunks, outputChunks);
+    }
+
+    codingStep.finish();
+  }
+
+  /**
+   * Compare and verify if recovered blocks data are the same with the erased
+   * blocks data.
+   * @param erasedBlocks
+   * @param recoveredBlocks
+   */
+  protected void compareAndVerify(ECBlock[] erasedBlocks,
+                                  ECBlock[] recoveredBlocks) {
+    for (int i = 0; i < erasedBlocks.length; ++i) {
+      compareAndVerify(((TestBlock) erasedBlocks[i]).chunks,
+          ((TestBlock) recoveredBlocks[i]).chunks);
+    }
+  }
+
+  /**
+   * Create erasure encoder for test.
+   * @return
+   */
+  private ErasureEncoder createEncoder() {
+    ErasureEncoder encoder;
+    try {
+      encoder = encoderClass.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create encoder", e);
+    }
+
+    encoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    return encoder;
+  }
+
+  /**
+   * Create the erasure decoder for the test.
+   * @return
+   */
+  private ErasureDecoder createDecoder() {
+    ErasureDecoder decoder;
+    try {
+      decoder = decoderClass.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create decoder", e);
+    }
+
+    decoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    return decoder;
+  }
+
+  /**
+   * Prepare a block group for encoding.
+   * @return
+   */
+  protected ECBlockGroup prepareBlockGroupForEncoding() {
+    ECBlock[] dataBlocks = new TestBlock[numDataUnits];
+    ECBlock[] parityBlocks = new TestBlock[numParityUnits];
+
+    for (int i = 0; i < numDataUnits; i++) {
+      dataBlocks[i] = generateDataBlock();
+    }
+
+    for (int i = 0; i < numParityUnits; i++) {
+      parityBlocks[i] = allocateOutputBlock();
+    }
+
+    return new ECBlockGroup(dataBlocks, parityBlocks);
+  }
+
+  /**
+   * Generate random data and return a data block.
+   * @return
+   */
+  protected ECBlock generateDataBlock() {
+    ECChunk[] chunks = new ECChunk[numChunksInBlock];
+
+    for (int i = 0; i < numChunksInBlock; ++i) {
+      chunks[i] = generateDataChunk();
+    }
+
+    return new TestBlock(chunks);
+  }
+
+  /**
+   * Copy those data blocks that's to be erased for later comparing and
+   * verifying.
+   * @param dataBlocks
+   * @return
+   */
+  protected TestBlock[] copyDataBlocksToErase(TestBlock[] dataBlocks) {
+    TestBlock[] copiedBlocks = new TestBlock[erasedDataIndexes.length];
+
+    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+      copiedBlocks[i] = cloneBlockWithData(dataBlocks[erasedDataIndexes[i]]);
+    }
+
+    return copiedBlocks;
+  }
+
+  /**
+   * Allocate an output block. Note the chunk buffer will be allocated by the
+   * up caller when performing the coding step.
+   * @return
+   */
+  protected TestBlock allocateOutputBlock() {
+    ECChunk[] chunks = new ECChunk[numChunksInBlock];
+
+    return new TestBlock(chunks);
+  }
+
+  /**
+   * Clone blocks with data copied along with, avoiding affecting the original
+   * blocks.
+   * @param blocks
+   * @return
+   */
+  protected static TestBlock[] cloneBlocksWithData(TestBlock[] blocks) {
+    TestBlock[] results = new TestBlock[blocks.length];
+    for (int i = 0; i < blocks.length; ++i) {
+      results[i] = cloneBlockWithData(blocks[i]);
+    }
+
+    return results;
+  }
+
+  /**
+   * Clone exactly a block, avoiding affecting the original block.
+   * @param block
+   * @return a new block
+   */
+  protected static TestBlock cloneBlockWithData(TestBlock block) {
+    ECChunk[] newChunks = cloneChunksWithData(block.chunks);
+
+    return new TestBlock(newChunks);
+  }
+
+  /**
+   * Erase some data blocks specified by the indexes from the data blocks.
+   * @param dataBlocks
+   */
+  protected void eraseSomeDataBlocks(TestBlock[] dataBlocks) {
+    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+      eraseDataFromBlock(dataBlocks, erasedDataIndexes[i]);
+    }
+  }
+
+  /**
+   * Erase data from a block specified by erased index.
+   * @param blocks
+   * @param erasedIndex
+   */
+  protected void eraseDataFromBlock(TestBlock[] blocks, int erasedIndex) {
+    TestBlock theBlock = blocks[erasedIndex];
+    eraseDataFromChunks(theBlock.chunks);
+    theBlock.setErased(true);
+  }
+}

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


[16/50] [abbrv] hadoop git commit: HDFS-2605. Remove redundant "Release 0.21.1" section from CHANGES.txt. Contributed by Allen Wittenauer.

Posted by zh...@apache.org.
HDFS-2605. Remove redundant "Release 0.21.1" section from CHANGES.txt. Contributed by Allen Wittenauer.

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

Branch: refs/heads/HDFS-7285
Commit: dfd32017001e6902829671dc8cc68afbca61e940
Parents: 6acb7f2
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Fri Mar 13 13:32:45 2015 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Fri Mar 13 13:32:45 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfd32017/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a149f18..c3f9367 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -746,6 +746,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7435. PB encoding of block reports is very inefficient.
     (Daryn Sharp via kihwal)
 
+    HDFS-2605. Remove redundant "Release 0.21.1" section from CHANGES.txt.
+    (Allen Wittenauer via shv)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
@@ -10299,8 +10302,6 @@ Release 0.22.0 - 2011-11-29
 
     HDFS-2287. TestParallelRead has a small off-by-one bug. (todd)
 
-Release 0.21.1 - Unreleased
-
     HDFS-1466. TestFcHdfsSymlink relies on /tmp/test not existing. (eli)
 
     HDFS-874. TestHDFSFileContextMainOperations fails on weirdly 


[48/50] [abbrv] hadoop git commit: HDFS-7853. Erasure coding: extend LocatedBlocks to support reading from striped files. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-7853. Erasure coding: extend LocatedBlocks to support reading from striped files. 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/fc774e7e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fc774e7e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fc774e7e

Branch: refs/heads/HDFS-7285
Commit: fc774e7e7410dc9a687f87f5053e03c0c3aeee44
Parents: ca332bf
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 9 14:59:58 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:13:45 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/LocatedBlock.java      |   5 +-
 .../hdfs/protocol/LocatedStripedBlock.java      |  68 +++++++++
 ...tNamenodeProtocolServerSideTranslatorPB.java |  14 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  13 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |   2 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   2 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  80 +++++++----
 .../blockmanagement/BlockInfoStriped.java       |   5 +
 .../BlockInfoStripedUnderConstruction.java      |  99 +++++++------
 .../server/blockmanagement/BlockManager.java    |  51 ++++---
 .../blockmanagement/DatanodeDescriptor.java     |   4 +-
 .../blockmanagement/DatanodeStorageInfo.java    |   3 +-
 .../server/namenode/FSImageFormatPBINode.java   |  21 +--
 .../hdfs/server/namenode/FSNamesystem.java      |  34 +++--
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |   1 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  13 ++
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |  16 +--
 .../datanode/TestIncrementalBrVariations.java   |  14 +-
 .../server/namenode/TestAddStripedBlocks.java   | 141 +++++++++++++++++++
 .../hdfs/server/namenode/TestFSImage.java       |   5 +-
 20 files changed, 445 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
index 0d52191..97dc534 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.common.collect.Lists;
@@ -51,14 +50,14 @@ public class LocatedBlock {
   // else false. If block has few corrupt replicas, they are filtered and 
   // their locations are not part of this object
   private boolean corrupt;
-  private Token<BlockTokenIdentifier> blockToken = new Token<BlockTokenIdentifier>();
+  private Token<BlockTokenIdentifier> blockToken = new Token<>();
   /**
    * List of cached datanode locations
    */
   private DatanodeInfo[] cachedLocs;
 
   // Used when there are no locations
-  private static final DatanodeInfoWithStorage[] EMPTY_LOCS =
+  static final DatanodeInfoWithStorage[] EMPTY_LOCS =
       new DatanodeInfoWithStorage[0];
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 ce8c392..16da085 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
@@ -422,7 +422,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           req.getClientName(), flags);
       AppendResponseProto.Builder builder = AppendResponseProto.newBuilder();
       if (result.getLastBlock() != null) {
-        builder.setBlock(PBHelper.convert(result.getLastBlock()));
+        builder.setBlock(PBHelper.convertLocatedBlock(result.getLastBlock()));
       }
       if (result.getFileStatus() != null) {
         builder.setStat(PBHelper.convert(result.getFileStatus()));
@@ -498,7 +498,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           (favor == null || favor.size() == 0) ? null : favor
               .toArray(new String[favor.size()]));
       return AddBlockResponseProto.newBuilder()
-          .setBlock(PBHelper.convert(result)).build();
+          .setBlock(PBHelper.convertLocatedBlock(result)).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -522,7 +522,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
               new DatanodeInfoProto[excludesList.size()])), 
           req.getNumAdditionalNodes(), req.getClientName());
       return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
-          PBHelper.convert(result))
+          PBHelper.convertLocatedBlock(result))
           .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -548,8 +548,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       ReportBadBlocksRequestProto req) throws ServiceException {
     try {
       List<LocatedBlockProto> bl = req.getBlocksList();
-      server.reportBadBlocks(PBHelper.convertLocatedBlock(
-              bl.toArray(new LocatedBlockProto[bl.size()])));
+      server.reportBadBlocks(PBHelper.convertLocatedBlocks(
+          bl.toArray(new LocatedBlockProto[bl.size()])));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -952,8 +952,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, UpdateBlockForPipelineRequestProto req)
       throws ServiceException {
     try {
-      LocatedBlockProto result = PBHelper.convert(server
-          .updateBlockForPipeline(PBHelper.convert(req.getBlock()),
+      LocatedBlockProto result = PBHelper.convertLocatedBlock(
+          server.updateBlockForPipeline(PBHelper.convert(req.getBlock()),
               req.getClientName()));
       return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result)
           .build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 e970293..b10a6ab 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
@@ -327,7 +327,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       AppendResponseProto res = rpcProxy.append(null, req);
       LocatedBlock lastBlock = res.hasBlock() ? PBHelper
-          .convert(res.getBlock()) : null;
+          .convertLocatedBlockProto(res.getBlock()) : null;
       HdfsFileStatus stat = (res.hasStat()) ? PBHelper.convert(res.getStat())
           : null;
       return new LastBlockWithStatus(lastBlock, stat);
@@ -415,7 +415,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
       req.addAllFavoredNodes(Arrays.asList(favoredNodes));
     }
     try {
-      return PBHelper.convert(rpcProxy.addBlock(null, req.build()).getBlock());
+      return PBHelper.convertLocatedBlockProto(
+          rpcProxy.addBlock(null, req.build()).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -440,8 +441,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setClientName(clientName)
         .build();
     try {
-      return PBHelper.convert(rpcProxy.getAdditionalDatanode(null, req)
-          .getBlock());
+      return PBHelper.convertLocatedBlockProto(
+          rpcProxy.getAdditionalDatanode(null, req).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -468,7 +469,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
     ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
-        .addAllBlocks(Arrays.asList(PBHelper.convertLocatedBlock(blocks)))
+        .addAllBlocks(Arrays.asList(PBHelper.convertLocatedBlocks(blocks)))
         .build();
     try {
       rpcProxy.reportBadBlocks(null, req);
@@ -898,7 +899,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setClientName(clientName)
         .build();
     try {
-      return PBHelper.convert(
+      return PBHelper.convertLocatedBlockProto(
           rpcProxy.updateBlockForPipeline(null, req).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index c4003f1..19a8f22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -276,7 +276,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     ReportBadBlocksRequestProto.Builder builder = ReportBadBlocksRequestProto
         .newBuilder();
     for (int i = 0; i < blocks.length; i++) {
-      builder.addBlocks(i, PBHelper.convert(blocks[i]));
+      builder.addBlocks(i, PBHelper.convertLocatedBlock(blocks[i]));
     }
     ReportBadBlocksRequestProto req = builder.build();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index e18081f..dea476c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -256,7 +256,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     List<LocatedBlockProto> lbps = request.getBlocksList();
     LocatedBlock [] blocks = new LocatedBlock [lbps.size()];
     for(int i=0; i<lbps.size(); i++) {
-      blocks[i] = PBHelper.convert(lbps.get(i));
+      blocks[i] = PBHelper.convertLocatedBlockProto(lbps.get(i));
     }
     try {
       impl.reportBadBlocks(blocks);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 eaa26bc..ad82ccd 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
@@ -80,6 +80,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -625,7 +626,7 @@ public class PBHelper {
     if (b == null) {
       return null;
     }
-    LocatedBlockProto lb = PBHelper.convert((LocatedBlock)b);
+    LocatedBlockProto lb = PBHelper.convertLocatedBlock(b);
     RecoveringBlockProto.Builder builder = RecoveringBlockProto.newBuilder();
     builder.setBlock(lb).setNewGenStamp(b.getNewGenerationStamp());
     if(b.getNewBlock() != null)
@@ -774,7 +775,7 @@ public class PBHelper {
     }
   }
   
-  public static LocatedBlockProto convert(LocatedBlock b) {
+  public static LocatedBlockProto convertLocatedBlock(LocatedBlock b) {
     if (b == null) return null;
     Builder builder = LocatedBlockProto.newBuilder();
     DatanodeInfo[] locs = b.getLocations();
@@ -795,21 +796,27 @@ public class PBHelper {
 
     StorageType[] storageTypes = b.getStorageTypes();
     if (storageTypes != null) {
-      for (int i = 0; i < storageTypes.length; ++i) {
-        builder.addStorageTypes(PBHelper.convertStorageType(storageTypes[i]));
+      for (StorageType storageType : storageTypes) {
+        builder.addStorageTypes(PBHelper.convertStorageType(storageType));
       }
     }
     final String[] storageIDs = b.getStorageIDs();
     if (storageIDs != null) {
       builder.addAllStorageIDs(Arrays.asList(storageIDs));
     }
+    if (b instanceof LocatedStripedBlock) {
+      int[] indices = ((LocatedStripedBlock) b).getBlockIndices();
+      for (int index : indices) {
+        builder.addBlockIndex(index);
+      }
+    }
 
     return builder.setB(PBHelper.convert(b.getBlock()))
         .setBlockToken(PBHelper.convert(b.getBlockToken()))
         .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
   }
   
-  public static LocatedBlock convert(LocatedBlockProto proto) {
+  public static LocatedBlock convertLocatedBlockProto(LocatedBlockProto proto) {
     if (proto == null) return null;
     List<DatanodeInfoProto> locs = proto.getLocsList();
     DatanodeInfo[] targets = new DatanodeInfo[locs.size()];
@@ -829,6 +836,15 @@ public class PBHelper {
       storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
     }
 
+    int[] indices = null;
+    final int indexCount = proto.getBlockIndexCount();
+    if (indexCount > 0) {
+      indices = new int[indexCount];
+      for (int i = 0; i < indexCount; i++) {
+        indices[i] = proto.getBlockIndex(i);
+      }
+    }
+
     // Set values from the isCached list, re-using references from loc
     List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
     List<Boolean> isCachedList = proto.getIsCachedList();
@@ -838,9 +854,17 @@ public class PBHelper {
       }
     }
 
-    LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
-        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
-        cachedLocs.toArray(new DatanodeInfo[0]));
+    final LocatedBlock lb;
+    if (indices == null) {
+      lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets, storageIDs,
+          storageTypes, proto.getOffset(), proto.getCorrupt(),
+          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
+    } else {
+      lb = new LocatedStripedBlock(PBHelper.convert(proto.getB()), targets,
+          storageIDs, storageTypes, indices, proto.getOffset(),
+          proto.getCorrupt(),
+          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
+    }
     lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
 
     return lb;
@@ -1256,36 +1280,36 @@ public class PBHelper {
   }
   
   // Located Block Arrays and Lists
-  public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) {
+  public static LocatedBlockProto[] convertLocatedBlocks(LocatedBlock[] lb) {
     if (lb == null) return null;
-    return convertLocatedBlock2(Arrays.asList(lb)).toArray(
-        new LocatedBlockProto[lb.length]);
+    return convertLocatedBlocks2(Arrays.asList(lb))
+        .toArray(new LocatedBlockProto[lb.length]);
   }
   
-  public static LocatedBlock[] convertLocatedBlock(LocatedBlockProto[] lb) {
+  public static LocatedBlock[] convertLocatedBlocks(LocatedBlockProto[] lb) {
     if (lb == null) return null;
-    return convertLocatedBlock(Arrays.asList(lb)).toArray(
-        new LocatedBlock[lb.length]);
+    return convertLocatedBlocks(Arrays.asList(lb))
+        .toArray(new LocatedBlock[lb.length]);
   }
   
-  public static List<LocatedBlock> convertLocatedBlock(
+  public static List<LocatedBlock> convertLocatedBlocks(
       List<LocatedBlockProto> lb) {
     if (lb == null) return null;
     final int len = lb.size();
-    List<LocatedBlock> result = 
-        new ArrayList<LocatedBlock>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(PBHelper.convert(lb.get(i)));
+    List<LocatedBlock> result = new ArrayList<>(len);
+    for (LocatedBlockProto aLb : lb) {
+      result.add(PBHelper.convertLocatedBlockProto(aLb));
     }
     return result;
   }
   
-  public static List<LocatedBlockProto> convertLocatedBlock2(List<LocatedBlock> lb) {
+  public static List<LocatedBlockProto> convertLocatedBlocks2(
+      List<LocatedBlock> lb) {
     if (lb == null) return null;
     final int len = lb.size();
-    List<LocatedBlockProto> result = new ArrayList<LocatedBlockProto>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(PBHelper.convert(lb.get(i)));
+    List<LocatedBlockProto> result = new ArrayList<>(len);
+    for (LocatedBlock aLb : lb) {
+      result.add(PBHelper.convertLocatedBlock(aLb));
     }
     return result;
   }
@@ -1295,8 +1319,9 @@ public class PBHelper {
   public static LocatedBlocks convert(LocatedBlocksProto lb) {
     return new LocatedBlocks(
         lb.getFileLength(), lb.getUnderConstruction(),
-        PBHelper.convertLocatedBlock(lb.getBlocksList()),
-        lb.hasLastBlock() ? PBHelper.convert(lb.getLastBlock()) : null,
+        PBHelper.convertLocatedBlocks(lb.getBlocksList()),
+        lb.hasLastBlock() ?
+            PBHelper.convertLocatedBlockProto(lb.getLastBlock()) : null,
         lb.getIsLastBlockComplete(),
         lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) :
             null);
@@ -1309,14 +1334,15 @@ public class PBHelper {
     LocatedBlocksProto.Builder builder = 
         LocatedBlocksProto.newBuilder();
     if (lb.getLastLocatedBlock() != null) {
-      builder.setLastBlock(PBHelper.convert(lb.getLastLocatedBlock()));
+      builder.setLastBlock(
+          PBHelper.convertLocatedBlock(lb.getLastLocatedBlock()));
     }
     if (lb.getFileEncryptionInfo() != null) {
       builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
     }
     return builder.setFileLength(lb.getFileLength())
         .setUnderConstruction(lb.isUnderConstruction())
-        .addAllBlocks(PBHelper.convertLocatedBlock2(lb.getLocatedBlocks()))
+        .addAllBlocks(PBHelper.convertLocatedBlocks2(lb.getLocatedBlocks()))
         .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 8b458df..84c3be6 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
@@ -123,6 +123,11 @@ public class BlockInfoStriped extends BlockInfo {
     return -1;
   }
 
+  int getStorageBlockIndex(DatanodeStorageInfo storage) {
+    int i = this.findStorageInfo(storage);
+    return i == -1 ? -1 : indices[i];
+  }
+
   @Override
   boolean removeStorage(DatanodeStorageInfo storage) {
     int dnIndex = findStorageInfoFromEnd(storage);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 151241b2..b1857bb 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
@@ -23,9 +23,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
@@ -39,12 +36,8 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
 
   /**
    * Block replicas as assigned when the block was allocated.
-   *
-   * TODO: we need to update this attribute, along with the return type of
-   * getExpectedStorageLocations and LocatedBlock. For striped blocks, clients
-   * need to understand the index of each striped block in the block group.
    */
-  private List<ReplicaUnderConstruction> replicas;
+  private ReplicaUnderConstruction[] replicas;
 
   /**
    * The new generation stamp, which this block will have
@@ -75,12 +68,12 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
 
   /**
    * Convert an under construction striped block to a complete striped block.
-   * 
+   *
    * @return BlockInfoStriped - a complete block.
-   * @throws IOException if the state of the block 
-   * (the generation stamp and the length) has not been committed by 
-   * the client or it does not have at least a minimal number of replicas 
-   * reported from data-nodes. 
+   * @throws IOException if the state of the block
+   * (the generation stamp and the length) has not been committed by
+   * the client or it does not have at least a minimal number of replicas
+   * reported from data-nodes.
    */
   BlockInfoStriped convertToCompleteBlock() throws IOException {
     assert getBlockUCState() != COMPLETE :
@@ -91,10 +84,13 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   /** Set expected locations */
   public void setExpectedLocations(DatanodeStorageInfo[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
-    this.replicas = new ArrayList<>(numLocations);
+    this.replicas = new ReplicaUnderConstruction[numLocations];
     for(int i = 0; i < numLocations; i++) {
-      replicas.add(new ReplicaUnderConstruction(this, targets[i],
-          ReplicaState.RBW));
+      // when creating a new block we simply sequentially assign block index to
+      // each storage
+      Block blk = new Block(this.getBlockId() + i, this.getGenerationStamp(), 0);
+      replicas[i] = new ReplicaUnderConstruction(blk, targets[i],
+          ReplicaState.RBW);
     }
   }
 
@@ -106,14 +102,24 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
     int numLocations = getNumExpectedLocations();
     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
     for (int i = 0; i < numLocations; i++) {
-      storages[i] = replicas.get(i).getExpectedStorageLocation();
+      storages[i] = replicas[i].getExpectedStorageLocation();
     }
     return storages;
   }
 
+  /** @return the index array indicating the block index in each storage */
+  public int[] getBlockIndices() {
+    int numLocations = getNumExpectedLocations();
+    int[] indices = new int[numLocations];
+    for (int i = 0; i < numLocations; i++) {
+      indices[i] = BlockIdManager.getBlockIndex(replicas[i]);
+    }
+    return indices;
+  }
+
   /** Get the number of expected locations */
   public int getNumExpectedLocations() {
-    return replicas == null ? 0 : replicas.size();
+    return replicas == null ? 0 : replicas.length;
   }
 
   /**
@@ -178,7 +184,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   public void initializeBlockRecovery(long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
-    if (replicas == null || replicas.size() == 0) {
+    if (replicas == null || replicas.length == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK*" +
           " BlockInfoUnderConstruction.initLeaseRecovery:" +
           " No blocks found, lease removed.");
@@ -186,28 +192,36 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
     // TODO we need to implement different recovery logic here
   }
 
-  void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block block,
+  void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block reportedBlock,
       ReplicaState rState) {
-    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
-    while (it.hasNext()) {
-      ReplicaUnderConstruction r = it.next();
-      DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
-      if (expectedLocation == storage) {
-        // Record the gen stamp from the report
-        r.setGenerationStamp(block.getGenerationStamp());
-        return;
-      } else if (expectedLocation != null &&
-          expectedLocation.getDatanodeDescriptor() ==
-              storage.getDatanodeDescriptor()) {
-        // The Datanode reported that the block is on a different storage
-        // than the one chosen by BlockPlacementPolicy. This can occur as
-        // we allow Datanodes to choose the target storage. Update our
-        // state by removing the stale entry and adding a new one.
-        it.remove();
-        break;
+    if (replicas == null) {
+      replicas = new ReplicaUnderConstruction[1];
+      replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage, rState);
+    } else {
+      for (int i = 0; i < replicas.length; i++) {
+        DatanodeStorageInfo expected = replicas[i].getExpectedStorageLocation();
+        if (expected == storage) {
+          replicas[i].setBlockId(reportedBlock.getBlockId());
+          replicas[i].setGenerationStamp(reportedBlock.getGenerationStamp());
+          return;
+        } else if (expected != null && expected.getDatanodeDescriptor() ==
+            storage.getDatanodeDescriptor()) {
+          // The Datanode reported that the block is on a different storage
+          // than the one chosen by BlockPlacementPolicy. This can occur as
+          // we allow Datanodes to choose the target storage. Update our
+          // state by removing the stale entry and adding a new one.
+          replicas[i] = new ReplicaUnderConstruction(reportedBlock, storage,
+              rState);
+          return;
+        }
       }
+      ReplicaUnderConstruction[] newReplicas =
+          new ReplicaUnderConstruction[replicas.length + 1];
+      System.arraycopy(replicas, 0, newReplicas, 0, replicas.length);
+      newReplicas[newReplicas.length - 1] = new ReplicaUnderConstruction(
+          reportedBlock, storage, rState);
+      replicas = newReplicas;
     }
-    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
   }
 
   @Override
@@ -226,12 +240,11 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   private void appendUCParts(StringBuilder sb) {
     sb.append("{UCState=").append(blockUCState).append(", replicas=[");
     if (replicas != null) {
-      Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
-      if (iter.hasNext()) {
-        iter.next().appendStringTo(sb);
-        while (iter.hasNext()) {
+      int i = 0;
+      for (ReplicaUnderConstruction r : replicas) {
+        r.appendStringTo(sb);
+        if (++i < replicas.length) {
           sb.append(", ");
-          iter.next().appendStringTo(sb);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 7f6a2ff..f22e9f4 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
@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
@@ -838,21 +839,26 @@ public class BlockManager {
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfo blk,
-      final long pos) throws IOException {
-    if (blk instanceof BlockInfoContiguousUnderConstruction) {
-      if (blk.isComplete()) {
-        throw new IOException(
-            "blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
-            + ", blk=" + blk);
+  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) {
+    if (!blk.isComplete()) {
+      if (blk.isStriped()) {
+        final BlockInfoStripedUnderConstruction uc =
+            (BlockInfoStripedUnderConstruction) blk;
+        final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
+        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
+            blk);
+        return new LocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
+            false);
+      } else {
+        assert blk instanceof BlockInfoContiguousUnderConstruction;
+        final BlockInfoContiguousUnderConstruction uc =
+            (BlockInfoContiguousUnderConstruction) blk;
+        final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
+        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
+            blk);
+        return new LocatedBlock(eb, storages, pos, false);
       }
-      final BlockInfoContiguousUnderConstruction uc =
-          (BlockInfoContiguousUnderConstruction) blk;
-      final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-      final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
-      return new LocatedBlock(eb, storages, pos, false);
     }
-    // TODO support BlockInfoStripedUC
 
     // get block locations
     final int numCorruptNodes = countNodes(blk).corruptReplicas();
@@ -867,13 +873,21 @@ public class BlockManager {
     final boolean isCorrupt = numCorruptNodes == numNodes;
     final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes;
     final DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines];
-    int j = 0;
+    final int[] blockIndices = blk.isStriped() ? new int[numMachines] : null;
+    int j = 0, i = 0;
     if (numMachines > 0) {
       for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
         final DatanodeDescriptor d = storage.getDatanodeDescriptor();
         final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
-        if (isCorrupt || (!replicaCorrupt))
+        if (isCorrupt || (!replicaCorrupt)) {
           machines[j++] = storage;
+          // TODO this can be more efficient
+          if (blockIndices != null) {
+            int index = ((BlockInfoStriped) blk).getStorageBlockIndex(storage);
+            assert index >= 0;
+            blockIndices[i++] = index;
+          }
+        }
       }
     }
     assert j == machines.length :
@@ -883,7 +897,9 @@ public class BlockManager {
       " numCorrupt: " + numCorruptNodes +
       " numCorruptRepls: " + numCorruptReplicas;
     final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
-    return new LocatedBlock(eb, machines, pos, isCorrupt);
+    return blockIndices == null ?
+        new LocatedBlock(eb, machines, pos, isCorrupt) :
+        new LocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
   }
 
   /** Create a LocatedBlocks. */
@@ -2383,7 +2399,8 @@ public class BlockManager {
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
       DatanodeStorageInfo storageInfo) throws IOException {
     BlockInfo block = ucBlock.storedBlock;
-    BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
+    BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock,
+        ucBlock.reportedState);
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
         !block.findDatanode(storageInfo.getDatanodeDescriptor())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 a54d46b..86f4158 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
@@ -255,7 +255,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
       return storageMap.get(storageID);
     }
   }
-  DatanodeStorageInfo[] getStorageInfos() {
+
+  @VisibleForTesting
+  public DatanodeStorageInfo[] getStorageInfos() {
     synchronized (storageMap) {
       final Collection<DatanodeStorageInfo> storages = storageMap.values();
       return storages.toArray(new DatanodeStorageInfo[storages.size()]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index 3a5e66e..eb50830 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@ -192,7 +192,8 @@ public class DatanodeStorageInfo {
     return getState() == State.FAILED && numBlocks != 0;
   }
 
-  String getStorageID() {
+  @VisibleForTesting
+  public String getStorageID() {
     return storageID;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 5627788..f293481 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
@@ -364,16 +364,19 @@ public final class FSImageFormatPBINode {
         INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
         file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         BlockInfo lastBlk = file.getLastBlock();
-        // replace the last block of file
-        final BlockInfo ucBlk;
-        if (stripeFeature != null) {
-          BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
-          ucBlk = new BlockInfoStripedUnderConstruction(striped,
-              striped.getDataBlockNum(), striped.getParityBlockNum());
-        } else {
-          ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, replication);
+        if (lastBlk != null) {
+          // replace the last block of file
+          final BlockInfo ucBlk;
+          if (stripeFeature != null) {
+            BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
+            ucBlk = new BlockInfoStripedUnderConstruction(striped,
+                striped.getDataBlockNum(), striped.getParityBlockNum());
+          } else {
+            ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
+                replication);
+          }
+          file.setBlock(file.numBlocks() - 1, ucBlk);
         }
-        file.setBlock(file.numBlocks() - 1, ucBlk);
       }
       return file;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 fa18e1c..667ab74 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
@@ -187,6 +187,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
@@ -205,6 +206,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -1732,8 +1734,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     LocatedBlocks blocks = res.blocks;
     if (blocks != null) {
+      List<LocatedBlock> blkList = blocks.getLocatedBlocks();
+      if (blkList == null || blkList.size() == 0 ||
+          blkList.get(0) instanceof LocatedStripedBlock) {
+        // no need to sort locations for striped blocks
+        return blocks;
+      }
       blockManager.getDatanodeManager().sortLocatedBlocks(
-          clientMachine, blocks.getLocatedBlocks());
+          clientMachine, blkList);
 
       // lastBlock is not part of getLocatedBlocks(), might need to sort it too
       LocatedBlock lastBlock = blocks.getLastLocatedBlock();
@@ -3012,7 +3020,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // Part II.
     // Allocate a new block, add it to the INode and the BlocksMap. 
-    Block newBlock = null;
+    BlockInfo newBlockInfo = null;
     long offset;
     checkOperation(OperationCategory.WRITE);
     waitForLoadingFSImage();
@@ -3045,8 +3053,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                                 ExtendedBlock.getLocalBlock(previous));
 
       // allocate new block, record block locations in INode.
-      newBlock = createNewBlock(isStriped);
-      saveAllocatedBlock(src, fileState.iip, newBlock, targets, isStriped);
+      Block newBlock = createNewBlock(isStriped);
+      newBlockInfo = saveAllocatedBlock(src, fileState.iip, newBlock, targets,
+          isStriped);
 
       persistNewBlock(src, pendingFile);
       offset = pendingFile.computeFileSize();
@@ -3056,7 +3065,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     getEditLog().logSync();
 
     // Return located block
-    return makeLocatedBlock(newBlock, targets, offset);
+    return makeLocatedBlock(newBlockInfo, targets, offset);
   }
 
   /*
@@ -3195,10 +3204,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new FileState(pendingFile, src, iip);
   }
 
-  LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
-                                        long offset) throws IOException {
-    LocatedBlock lBlk = new LocatedBlock(
-        getExtendedBlock(blk), locs, offset, false);
+  LocatedBlock makeLocatedBlock(BlockInfo blk, DatanodeStorageInfo[] locs,
+      long offset) throws IOException {
+    final LocatedBlock lBlk;
+    if (blk.isStriped()) {
+      assert blk instanceof BlockInfoStripedUnderConstruction;
+      lBlk = new LocatedStripedBlock(getExtendedBlock(blk), locs,
+          ((BlockInfoStripedUnderConstruction) blk).getBlockIndices(),
+          offset, false);
+    } else {
+      lBlk = new LocatedBlock(getExtendedBlock(blk), locs, offset, false);
+    }
     getBlockManager().setBlockToken(
         lBlk, BlockTokenSecretManager.AccessMode.WRITE);
     return lBlk;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 1fa3743..d5f636a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -205,6 +205,7 @@ message LocatedBlockProto {
   repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
   repeated StorageTypeProto storageTypes = 7;
   repeated string storageIDs = 8;
+  repeated uint32 blockIndex = 9; // used for striped block to indicate block index for each storage
 }
 
 message DataEncryptionKeyProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 c3dac35..3dc4d5e 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
@@ -79,6 +79,10 @@ import org.apache.hadoop.hdfs.server.namenode.ha
         .ConfiguredFailoverProxyProvider;
 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;
 import org.apache.hadoop.io.nativeio.NativeIO;
@@ -1711,4 +1715,13 @@ public class DFSTestUtil {
     GenericTestUtils.setLogLevel(NameNode.stateChangeLog, level);
     GenericTestUtils.setLogLevel(NameNode.blockStateChangeLog, level);
   }
+
+  public static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock(
+      Block block, BlockStatus blockStatus, DatanodeStorage storage) {
+    ReceivedDeletedBlockInfo[] receivedBlocks = new ReceivedDeletedBlockInfo[1];
+    receivedBlocks[0] = new ReceivedDeletedBlockInfo(block, blockStatus, null);
+    StorageReceivedDeletedBlocks[] reports = new StorageReceivedDeletedBlocks[1];
+    reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks);
+    return reports;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 0236288..19d254c 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
@@ -488,16 +488,16 @@ public class TestPBHelper {
   @Test
   public void testConvertLocatedBlock() {
     LocatedBlock lb = createLocatedBlock();
-    LocatedBlockProto lbProto = PBHelper.convert(lb);
-    LocatedBlock lb2 = PBHelper.convert(lbProto);
+    LocatedBlockProto lbProto = PBHelper.convertLocatedBlock(lb);
+    LocatedBlock lb2 = PBHelper.convertLocatedBlockProto(lbProto);
     compare(lb,lb2);
   }
 
   @Test
   public void testConvertLocatedBlockNoStorageMedia() {
     LocatedBlock lb = createLocatedBlockNoStorageMedia();
-    LocatedBlockProto lbProto = PBHelper.convert(lb);
-    LocatedBlock lb2 = PBHelper.convert(lbProto);
+    LocatedBlockProto lbProto = PBHelper.convertLocatedBlock(lb);
+    LocatedBlock lb2 = PBHelper.convertLocatedBlockProto(lbProto);
     compare(lb,lb2);
   }
 
@@ -507,8 +507,8 @@ public class TestPBHelper {
     for (int i=0;i<3;i++) {
       lbl.add(createLocatedBlock());
     }
-    List<LocatedBlockProto> lbpl = PBHelper.convertLocatedBlock2(lbl);
-    List<LocatedBlock> lbl2 = PBHelper.convertLocatedBlock(lbpl);
+    List<LocatedBlockProto> lbpl = PBHelper.convertLocatedBlocks2(lbl);
+    List<LocatedBlock> lbl2 = PBHelper.convertLocatedBlocks(lbpl);
     assertEquals(lbl.size(), lbl2.size());
     for (int i=0;i<lbl.size();i++) {
       compare(lbl.get(i), lbl2.get(2));
@@ -521,8 +521,8 @@ public class TestPBHelper {
     for (int i=0;i<3;i++) {
       lbl[i] = createLocatedBlock();
     }
-    LocatedBlockProto [] lbpl = PBHelper.convertLocatedBlock(lbl);
-    LocatedBlock [] lbl2 = PBHelper.convertLocatedBlock(lbpl);
+    LocatedBlockProto [] lbpl = PBHelper.convertLocatedBlocks(lbl);
+    LocatedBlock [] lbl2 = PBHelper.convertLocatedBlocks(lbpl);
     assertEquals(lbl.length, lbl2.length);
     for (int i=0;i<lbl.length;i++) {
       compare(lbl[i], lbl2[i]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
index 4e73e6e..5d8d307 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
@@ -220,15 +220,6 @@ public class TestIncrementalBrVariations {
     return new Block(10000000L, 100L, 1048576L);
   }
 
-  private static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock(
-      Block block, DatanodeStorage storage) {
-    ReceivedDeletedBlockInfo[] receivedBlocks = new ReceivedDeletedBlockInfo[1];
-    receivedBlocks[0] = new ReceivedDeletedBlockInfo(block, BlockStatus.RECEIVED_BLOCK, null);
-    StorageReceivedDeletedBlocks[] reports = new StorageReceivedDeletedBlocks[1];
-    reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks);
-    return reports;
-  }
-
   /**
    * Verify that the NameNode can learn about new storages from incremental
    * block reports.
@@ -244,8 +235,9 @@ public class TestIncrementalBrVariations {
     // Generate a report for a fake block on a fake storage.
     final String newStorageUuid = UUID.randomUUID().toString();
     final DatanodeStorage newStorage = new DatanodeStorage(newStorageUuid);
-    StorageReceivedDeletedBlocks[] reports = makeReportForReceivedBlock(
-        getDummyBlock(), newStorage);
+    StorageReceivedDeletedBlocks[] reports = DFSTestUtil.
+        makeReportForReceivedBlock(getDummyBlock(), BlockStatus.RECEIVED_BLOCK,
+            newStorage);
 
     // Send the report to the NN.
     cluster.getNameNodeRpc().blockReceivedAndDeleted(dn0Reg, poolId, reports);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 7226f51..8b51309 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -19,18 +19,29 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 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.BlockStatus;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -38,6 +49,9 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
 
 import static org.junit.Assert.assertEquals;
 
@@ -143,4 +157,131 @@ public class TestAddStripedBlocks {
     }
     return false;
   }
+
+  @Test
+  public void testGetLocatedStripedBlocks() throws Exception {
+    final Path file = new Path("/file1");
+    // create an empty file
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1);
+
+      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+      cluster.getNamesystem().getAdditionalBlock(file.toString(),
+          fileNode.getId(), dfs.getClient().getClientName(), null, null, null);
+      BlockInfoStripedUnderConstruction lastBlk =
+          (BlockInfoStripedUnderConstruction) fileNode.getLastBlock();
+      DatanodeInfo[] expectedDNs = DatanodeStorageInfo
+          .toDatanodeInfos(lastBlk.getExpectedStorageLocations());
+      int[] indices = lastBlk.getBlockIndices();
+
+      LocatedBlocks blks = dfs.getClient().getLocatedBlocks(file.toString(), 0L);
+      Assert.assertEquals(1, blks.locatedBlockCount());
+      LocatedBlock lblk = blks.get(0);
+
+      Assert.assertTrue(lblk instanceof LocatedStripedBlock);
+      DatanodeInfo[] datanodes = lblk.getLocations();
+      int[] blockIndices = ((LocatedStripedBlock) lblk).getBlockIndices();
+      Assert.assertEquals(GROUP_SIZE, datanodes.length);
+      Assert.assertEquals(GROUP_SIZE, blockIndices.length);
+      Assert.assertArrayEquals(indices, blockIndices);
+      Assert.assertArrayEquals(expectedDNs, datanodes);
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+  }
+
+  /**
+   * Test BlockInfoStripedUnderConstruction#addReplicaIfNotPresent in different
+   * scenarios.
+   */
+  @Test
+  public void testAddUCReplica() throws Exception {
+    final Path file = new Path("/file1");
+    final List<String> storageIDs = new ArrayList<>();
+    // create an empty file
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1);
+
+      // 1. create the UC striped block
+      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+      cluster.getNamesystem().getAdditionalBlock(file.toString(),
+          fileNode.getId(), dfs.getClient().getClientName(), null, null, null);
+      BlockInfo lastBlock = fileNode.getLastBlock();
+      BlockInfoStripedUnderConstruction ucBlock =
+          (BlockInfoStripedUnderConstruction) lastBlock;
+
+      DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
+      int[] indices = ucBlock.getBlockIndices();
+      Assert.assertEquals(GROUP_SIZE, locs.length);
+      Assert.assertEquals(GROUP_SIZE, indices.length);
+
+      // 2. mimic incremental block reports and make sure the uc-replica list in
+      // the BlockStripedUC is correct
+      int i = 0;
+      for (DataNode dn : cluster.getDataNodes()) {
+        final Block block = new Block(lastBlock.getBlockId() + i++,
+            lastBlock.getGenerationStamp(), 0);
+        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+        storageIDs.add(storage.getStorageID());
+        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+            .makeReportForReceivedBlock(block, BlockStatus.RECEIVING_BLOCK,
+                storage);
+        for (StorageReceivedDeletedBlocks report : reports) {
+          cluster.getNamesystem().processIncrementalBlockReport(
+              dn.getDatanodeId(), report);
+        }
+      }
+
+      // make sure lastBlock is correct and the storages have been updated
+      locs = ucBlock.getExpectedStorageLocations();
+      indices = ucBlock.getBlockIndices();
+      Assert.assertEquals(GROUP_SIZE, locs.length);
+      Assert.assertEquals(GROUP_SIZE, indices.length);
+      for (DatanodeStorageInfo newstorage : locs) {
+        Assert.assertTrue(storageIDs.contains(newstorage.getStorageID()));
+      }
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+
+    // 3. restart the namenode. mimic the full block reports and check the
+    // uc-replica list again
+    cluster.restartNameNode(true);
+    final String bpId = cluster.getNamesystem().getBlockPoolId();
+    INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
+        .getINode4Write(file.toString()).asFile();
+    BlockInfo lastBlock = fileNode.getLastBlock();
+    int i = GROUP_SIZE - 1;
+    for (DataNode dn : cluster.getDataNodes()) {
+      String storageID = storageIDs.get(i);
+      final Block block = new Block(lastBlock.getBlockId() + i--,
+          lastBlock.getGenerationStamp(), 0);
+      DatanodeStorage storage = new DatanodeStorage(storageID);
+      List<ReplicaBeingWritten> blocks = new ArrayList<>();
+      ReplicaBeingWritten replica = new ReplicaBeingWritten(block, null, null,
+          null);
+      blocks.add(replica);
+      BlockListAsLongs bll = new BlockListAsLongs(null, blocks);
+      StorageBlockReport[] reports = {new StorageBlockReport(storage,
+          bll.getBlockListAsLongs())};
+      cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId),
+          bpId, reports);
+    }
+
+    BlockInfoStripedUnderConstruction ucBlock =
+        (BlockInfoStripedUnderConstruction) lastBlock;
+    DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
+    int[] indices = ucBlock.getBlockIndices();
+    Assert.assertEquals(GROUP_SIZE, locs.length);
+    Assert.assertEquals(GROUP_SIZE, indices.length);
+    for (i = 0; i < GROUP_SIZE; i++) {
+      Assert.assertEquals(storageIDs.get(i),
+          locs[GROUP_SIZE - 1 - i].getStorageID());
+      Assert.assertEquals(GROUP_SIZE - i - 1, indices[i]);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc774e7e/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 913e0a7..4d42911 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
@@ -25,6 +25,8 @@ import java.io.IOException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+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.conf.Configuration;
@@ -219,8 +221,7 @@ public class TestFSImage {
         .format(false)
         .manageDataDfsDirs(false)
         .manageNameDfsDirs(false)
-        .waitSafeMode(false)
-        .startupOption(StartupOption.UPGRADE)
+        .waitSafeMode(false).startupOption(StartupOption.UPGRADE)
         .build();
     try {
       FileSystem fs = cluster.getFileSystem();


[38/50] [abbrv] hadoop git commit: Added the missed entry for commit of HADOOP-11541

Posted by zh...@apache.org.
Added the missed entry for commit of HADOOP-11541


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

Branch: refs/heads/HDFS-7285
Commit: c054b2a9b2ad98b2115e3f0752e5765c77763095
Parents: 435d7b0
Author: drankye <dr...@gmail.com>
Authored: Mon Feb 9 22:04:08 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:08:50 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c054b2a9/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 2124800..9728f97 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -4,4 +4,7 @@
     (Kai Zheng via umamahesh)
 
     HADOOP-11534. Minor improvements for raw erasure coders
-    ( Kai Zheng via vinayakumarb )
\ No newline at end of file
+    ( Kai Zheng via vinayakumarb )
+
+    HADOOP-11541. Raw XOR coder
+    ( Kai Zheng )


[13/50] [abbrv] hadoop git commit: HDFS-7926. NameNode implementation of ClientProtocol.truncate(..) is not idempotent. Contributed by Tsz Wo Nicholas Sze

Posted by zh...@apache.org.
HDFS-7926. NameNode implementation of ClientProtocol.truncate(..) is not idempotent. 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/f446669a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f446669a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f446669a

Branch: refs/heads/HDFS-7285
Commit: f446669afb5c3d31a00c65449f27088b39e11ae3
Parents: 8180e67
Author: Brandon Li <br...@apache.org>
Authored: Fri Mar 13 10:42:22 2015 -0700
Committer: Brandon Li <br...@apache.org>
Committed: Fri Mar 13 10:42:22 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt          |  3 +++
 .../BlockInfoContiguousUnderConstruction.java        |  1 +
 .../hadoop/hdfs/server/namenode/FSNamesystem.java    | 15 +++++++++++++++
 .../hdfs/server/namenode/TestFileTruncate.java       |  2 ++
 4 files changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f446669a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 153453c..909182b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1142,6 +1142,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-6833.  DirectoryScanner should not register a deleting block with
     memory of DataNode.  (Shinichi Yamashita via szetszwo)
 
+    HDFS-7926. NameNode implementation of ClientProtocol.truncate(..) is not 
+    idempotent (Tsz Wo Nicholas Sze via brandonli)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f446669a/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 91b76cc..ae809a5 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
@@ -383,6 +383,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
 
   private void appendUCParts(StringBuilder sb) {
     sb.append("{UCState=").append(blockUCState)
+      .append(", truncateBlock=" + truncateBlock)
       .append(", primaryNodeIndex=").append(primaryNodeIndex)
       .append(", replicas=[");
     if (replicas != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f446669a/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 77b4a27..b384ce6 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
@@ -1966,6 +1966,21 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new UnsupportedOperationException(
           "Cannot truncate lazy persist file " + src);
     }
+
+    // Check if the file is already being truncated with the same length
+    final BlockInfoContiguous last = file.getLastBlock();
+    if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
+      final Block truncateBlock
+          = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock();
+      if (truncateBlock != null) {
+        final long truncateLength = file.computeFileSize(false, false)
+            + truncateBlock.getNumBytes();
+        if (newLength == truncateLength) {
+          return false;
+        }
+      }
+    }
+
     // Opening an existing file for truncate. May need lease recovery.
     recoverLeaseInternal(RecoverLeaseOp.TRUNCATE_FILE,
         iip, src, clientName, clientMachine, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f446669a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index b69d345..260d8bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -166,6 +166,8 @@ public class TestFileTruncate {
       LOG.info("newLength=" + newLength + ", isReady=" + isReady);
       assertEquals("File must be closed for truncating at the block boundary",
           isReady, newLength % BLOCK_SIZE == 0);
+      assertEquals("Truncate is not idempotent",
+          isReady, fs.truncate(p, newLength));
       if (!isReady) {
         checkBlockRecovery(p);
       }


[05/50] [abbrv] hadoop git commit: YARN-3338. Exclude jline dependency from YARN. Contributed by Zhijie Shen

Posted by zh...@apache.org.
YARN-3338. Exclude jline dependency from YARN. Contributed by Zhijie
Shen


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

Branch: refs/heads/HDFS-7285
Commit: 06ce1d9a6cd9bec25e2f478b98264caf96a3ea44
Parents: ff83ae7
Author: Xuan <xg...@apache.org>
Authored: Thu Mar 12 10:25:00 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Thu Mar 12 10:25:00 2015 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml          | 4 ++++
 hadoop-yarn-project/CHANGES.txt | 2 ++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/06ce1d9a/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index a6127c7..6c95cf0 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -833,6 +833,10 @@
             <groupId>org.jboss.netty</groupId>
             <artifactId>netty</artifactId>
           </exclusion>
+          <exclusion>
+            <groupId>jline</groupId>
+            <artifactId>jline</artifactId>
+          </exclusion>
         </exclusions>
       </dependency>
       <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06ce1d9a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 969c6a1..11d1cc9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -755,6 +755,8 @@ Release 2.7.0 - UNRELEASED
     YARN-1884. Added nodeHttpAddress into ContainerReport and fixed the link to NM
     web page. (Xuan Gong via zjshen)
 
+    YARN-3338. Exclude jline dependency from YARN. (Zhijie Shen via xgong)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES


[29/50] [abbrv] hadoop git commit: HADOOP-11692. Amendment to CHANGES.txt.

Posted by zh...@apache.org.
HADOOP-11692. Amendment to CHANGES.txt.


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

Branch: refs/heads/HDFS-7285
Commit: 03b77ede9249d7d16654257035dfc01a7a0a8c50
Parents: 3da9a97
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Mon Mar 16 09:08:41 2015 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Mon Mar 16 09:08:41 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/03b77ede/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index bb08cfe..e161d7d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -449,6 +449,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    HADOOP-11692. Improve authentication failure WARN message to avoid user
+    confusion. (Yongjun Zhang)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -1088,9 +1091,6 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11686. MiniKDC cannot change ORG_NAME or ORG_DOMAIN.
     (Duo Zhang via wheat9)
 
-    HADOOP-11692. Improve authentication failure WARN message to avoid user
-    confusion. (Yongjun Zhang)
-
     HADOOP-11618. DelegateToFileSystem erroneously uses default FS's port in
     constructor. (Brahma Reddy Battula via gera)
 


[24/50] [abbrv] hadoop git commit: HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Jean-Pierre Matsumoto.

Posted by zh...@apache.org.
HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Jean-Pierre Matsumoto.


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

Branch: refs/heads/HDFS-7285
Commit: 79426f3334ade5850fbf169764f540ede00fe366
Parents: b308a8d
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sun Mar 15 14:17:35 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sun Mar 15 14:29:49 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt       |  3 +++
 .../src/site/markdown/SchedulerLoadSimulator.md       |  2 +-
 .../src/site/markdown/HadoopStreaming.md.vm           | 14 +++++++-------
 3 files changed, 11 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79426f33/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 55028cb..bb08cfe 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1100,6 +1100,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt
     synchronization. (Sean Busbey via yliu)
 
+    HADOOP-11558. Fix dead links to doc of hadoop-tools. (Jean-Pierre 
+    Matsumoto via ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79426f33/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
index ca179ee..2cffc86 100644
--- a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
+++ b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
@@ -43,7 +43,7 @@ The Yarn Scheduler Load Simulator (SLS) is such a tool, which can simulate large
 o
 The simulator will exercise the real Yarn `ResourceManager` removing the network factor by simulating `NodeManagers` and `ApplicationMasters` via handling and dispatching `NM`/`AMs` heartbeat events from within the same JVM. To keep tracking of scheduler behavior and performance, a scheduler wrapper will wrap the real scheduler.
 
-The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](https://hadoop.apache.org/docs/stable/rumen.html).
+The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](../hadoop-rumen/Rumen.html).
 
 The simulator will produce real time metrics while executing, including:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79426f33/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
index 0b64586..b4c5e38 100644
--- a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
+++ b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
@@ -201,7 +201,7 @@ To specify additional local temp directories use:
      -D mapred.system.dir=/tmp/system
      -D mapred.temp.dir=/tmp/temp
 
-**Note:** For more details on job configuration parameters see: [mapred-default.xml](./mapred-default.xml)
+**Note:** For more details on job configuration parameters see: [mapred-default.xml](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/mapred-default.xml)
 
 $H4 Specifying Map-Only Jobs
 
@@ -322,7 +322,7 @@ More Usage Examples
 
 $H3 Hadoop Partitioner Class
 
-Hadoop has a library class, [KeyFieldBasedPartitioner](../../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
+Hadoop has a library class, [KeyFieldBasedPartitioner](../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D stream.map.output.field.separator=. \
@@ -372,7 +372,7 @@ Sorting within each partition for the reducer(all 4 fields used for sorting)
 
 $H3 Hadoop Comparator Class
 
-Hadoop has a library class, [KeyFieldBasedComparator](../../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
+Hadoop has a library class, [KeyFieldBasedComparator](../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.job.output.key.comparator.class=org.apache.hadoop.mapreduce.lib.partition.KeyFieldBasedComparator \
@@ -406,7 +406,7 @@ Sorting output for the reducer (where second field used for sorting)
 
 $H3 Hadoop Aggregate Package
 
-Hadoop has a library package called [Aggregate](../../org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
+Hadoop has a library package called [Aggregate](../api/org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
 
 To use Aggregate, simply specify "-reducer aggregate":
 
@@ -441,7 +441,7 @@ The python program myAggregatorForKeyCount.py looks like:
 
 $H3 Hadoop Field Selection Class
 
-Hadoop has a library class, [FieldSelectionMapReduce](../../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
+Hadoop has a library class, [FieldSelectionMapReduce](../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.map.output.key.field.separator=. \
@@ -480,7 +480,7 @@ As an example, consider the problem of zipping (compressing) a set of files acro
 
 $H3 How many reducers should I use?
 
-See MapReduce Tutorial for details: [Reducer](./MapReduceTutorial.html#Reducer)
+See MapReduce Tutorial for details: [Reducer](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Reducer)
 
 $H3 If I set up an alias in my shell script, will that work after -mapper?
 
@@ -556,4 +556,4 @@ A streaming process can use the stderr to emit status information. To set a stat
 
 $H3 How do I get the Job variables in a streaming job's mapper/reducer?
 
-See [Configured Parameters](./MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.
+See [Configured Parameters](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.


[32/50] [abbrv] hadoop git commit: HDFS-7347. Configurable erasure coding policy for individual files and directories ( Contributed by Zhe Zhang )

Posted by zh...@apache.org.
HDFS-7347. Configurable erasure coding policy for individual files and directories ( 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/2e80653f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2e80653f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2e80653f

Branch: refs/heads/HDFS-7285
Commit: 2e80653fb753627c3f4abf912d45c9f75405be88
Parents: d1eebd9
Author: Vinayakumar B <vi...@apache.org>
Authored: Thu Nov 6 10:03:26 2014 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:08:48 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  4 ++
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  2 +
 .../BlockStoragePolicySuite.java                |  5 ++
 .../hadoop/hdfs/TestBlockStoragePolicy.java     | 12 +++-
 .../TestBlockInitialEncoding.java               | 75 ++++++++++++++++++++
 5 files changed, 95 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e80653f/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
new file mode 100644
index 0000000..2ef8527
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -0,0 +1,4 @@
+  BREAKDOWN OF HDFS-7285 SUBTASKS AND RELATED JIRAS
+
+    HDFS-7347. Configurable erasure coding policy for individual files and
+    directories ( Zhe Zhang via vinayakumarb )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e80653f/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 7cf8a47..54c650b 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
@@ -171,6 +171,7 @@ 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;
@@ -178,5 +179,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;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e80653f/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 020cb5f..3d121cc 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
@@ -78,6 +78,11 @@ 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/2e80653f/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 8f99a85..606a332 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,6 +68,7 @@ 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;
@@ -115,6 +116,9 @@ 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], " +
@@ -1157,13 +1161,15 @@ public class TestBlockStoragePolicy {
     final DistributedFileSystem fs = cluster.getFileSystem();
     try {
       BlockStoragePolicy[] policies = fs.getStoragePolicies();
-      Assert.assertEquals(6, policies.length);
+      Assert.assertEquals(7, policies.length);
       Assert.assertEquals(POLICY_SUITE.getPolicy(COLD).toString(),
           policies[0].toString());
-      Assert.assertEquals(POLICY_SUITE.getPolicy(WARM).toString(),
+      Assert.assertEquals(POLICY_SUITE.getPolicy(EC).toString(),
           policies[1].toString());
-      Assert.assertEquals(POLICY_SUITE.getPolicy(HOT).toString(),
+      Assert.assertEquals(POLICY_SUITE.getPolicy(WARM).toString(),
           policies[2].toString());
+      Assert.assertEquals(POLICY_SUITE.getPolicy(HOT).toString(),
+          policies[3].toString());
     } finally {
       IOUtils.cleanup(null, fs);
       cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e80653f/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
new file mode 100644
index 0000000..a84f67b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
@@ -0,0 +1,75 @@
+/**
+ * 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


[45/50] [abbrv] hadoop git commit: HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11643. Define EC schema API for ErasureCodec. 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/fc71e8bd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fc71e8bd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fc71e8bd

Branch: refs/heads/HDFS-7285
Commit: fc71e8bda63cf91a8b31b348fd6e5e3897e91e43
Parents: bc96294
Author: drankye <ka...@intel.com>
Authored: Thu Mar 5 22:51:52 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:13:16 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |   4 +
 .../apache/hadoop/io/erasurecode/ECSchema.java  | 203 +++++++++++++++++++
 .../hadoop/io/erasurecode/TestECSchema.java     |  54 +++++
 3 files changed, 261 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc71e8bd/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 7bbacf7..ee42c84 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -12,3 +12,7 @@
     HADOOP-11542. Raw Reed-Solomon coder in pure Java. Contributed by Kai Zheng
     ( Kai Zheng )
 
+    HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng
+    ( Kai Zheng )
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc71e8bd/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
new file mode 100644
index 0000000..8dc3f45
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
@@ -0,0 +1,203 @@
+/**
+ * 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;
+
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Erasure coding schema to housekeeper relevant information.
+ */
+public 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
+
+  private String schemaName;
+  private String codecName;
+  private Map<String, String> options;
+  private int numDataUnits;
+  private int numParityUnits;
+  private int chunkSize;
+
+  /**
+   * Constructor with schema name and provided options. Note the options may
+   * contain additional information for the erasure codec to interpret further.
+   * @param schemaName schema name
+   * @param options schema options
+   */
+  public ECSchema(String schemaName, Map<String, String> options) {
+    assert (schemaName != null && ! schemaName.isEmpty());
+
+    this.schemaName = schemaName;
+
+    if (options == null || options.isEmpty()) {
+      throw new IllegalArgumentException("No schema options are provided");
+    }
+
+    String codecName = options.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(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
+          " is found. It should be an integer");
+    }
+
+    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(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
+          " is found. It should be an integer");
+    }
+
+    initWith(codecName, dataUnits, parityUnits, options);
+  }
+
+  /**
+   * Constructor with key parameters provided. Note the options may contain
+   * additional information for the erasure codec to interpret further.
+   * @param schemaName
+   * @param codecName
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param options
+   */
+  public ECSchema(String schemaName, String codecName,
+                  int numDataUnits, int numParityUnits,
+                  Map<String, String> options) {
+    assert (schemaName != null && ! schemaName.isEmpty());
+    assert (codecName != null && ! codecName.isEmpty());
+
+    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;
+
+    this.chunkSize = DEFAULT_CHUNK_SIZE;
+    try {
+      if (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 +
+          " 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");
+    }
+  }
+
+  /**
+   * Get the schema name
+   * @return schema name
+   */
+  public String getSchemaName() {
+    return schemaName;
+  }
+
+  /**
+   * Get the codec name
+   * @return codec name
+   */
+  public String getCodecName() {
+    return codecName;
+  }
+
+  /**
+   * Get erasure coding options
+   * @return encoding options
+   */
+  public Map<String, String> getOptions() {
+    return options;
+  }
+
+  /**
+   * Get required data units count in a coding group
+   * @return count of data units
+   */
+  public int getNumDataUnits() {
+    return numDataUnits;
+  }
+
+  /**
+   * Get required parity units count in a coding group
+   * @return count of parity units
+   */
+  public int getNumParityUnits() {
+    return numParityUnits;
+  }
+
+  /**
+   * Get chunk buffer size for the erasure encoding/decoding.
+   * @return chunk buffer size
+   */
+  public int getChunkSize() {
+    return chunkSize;
+  }
+
+  /**
+   * Make a meaningful string representation for log output.
+   * @return string representation
+   */
+  @Override
+  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("]");
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc71e8bd/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
new file mode 100644
index 0000000..4285ef0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode;
+
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestECSchema {
+
+  @Test
+  public void testGoodSchema() {
+    String schemaName = "goodSchema";
+    int numDataUnits = 6;
+    int numParityUnits = 3;
+    int chunkSize = 64 * 1024 * 1024;
+    String codec = "rs";
+    String extraOption = "extraOption";
+    String extraOptionValue = "extraOptionValue";
+
+    Map<String, String> options = new HashMap<String, String>();
+    options.put(ECSchema.NUM_DATA_UNITS_KEY, String.valueOf(numDataUnits));
+    options.put(ECSchema.NUM_PARITY_UNITS_KEY, String.valueOf(numParityUnits));
+    options.put(ECSchema.CODEC_NAME_KEY, codec);
+    options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(chunkSize));
+    options.put(extraOption, extraOptionValue);
+
+    ECSchema schema = new ECSchema(schemaName, options);
+    System.out.println(schema.toString());
+    
+    assertEquals(schemaName, schema.getSchemaName());
+    assertEquals(numDataUnits, schema.getNumDataUnits());
+    assertEquals(numParityUnits, schema.getNumParityUnits());
+    assertEquals(chunkSize, schema.getChunkSize());
+    assertEquals(codec, schema.getCodecName());
+    assertEquals(extraOptionValue, schema.getOptions().get(extraOption));
+  }
+}


[31/50] [abbrv] hadoop git commit: Fix Compilation Error in TestAddBlockgroup.java after the merge

Posted by zh...@apache.org.
Fix Compilation Error in TestAddBlockgroup.java after the merge


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

Branch: refs/heads/HDFS-7285
Commit: 68ecc4e14fe5f24424b8af389adba1eba769ee17
Parents: 86bc54a
Author: Jing Zhao <ji...@apache.org>
Authored: Sun Feb 8 16:01:03 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:08:48 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ecc4e1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
index 95133ce..06dfade 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -75,7 +75,7 @@ public class TestAddBlockgroup {
     final Path file1 = new Path("/file1");
     DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L);
     INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
-    BlockInfo[] file1Blocks = file1Node.getBlocks();
+    BlockInfoContiguous[] file1Blocks = file1Node.getBlocks();
     assertEquals(2, file1Blocks.length);
     assertEquals(GROUP_SIZE, file1Blocks[0].numNodes());
     assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP,


[36/50] [abbrv] hadoop git commit: HADOOP-11534. Minor improvements for raw erasure coders ( Contributed by Kai Zheng )

Posted by zh...@apache.org.
HADOOP-11534. Minor improvements for 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/52c42196
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/52c42196
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/52c42196

Branch: refs/heads/HDFS-7285
Commit: 52c4219645a01561dc13fc8121046a8b8636cde8
Parents: ea10f9b
Author: Vinayakumar B <vi...@intel.com>
Authored: Mon Feb 2 14:39:53 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:08:49 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt           |  5 ++++-
 .../org/apache/hadoop/io/erasurecode/ECChunk.java    | 15 +++++++++++++--
 .../rawcoder/AbstractRawErasureCoder.java            | 12 ++++++------
 3 files changed, 23 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/52c42196/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 8ce5a89..2124800 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -1,4 +1,7 @@
   BREAKDOWN OF HADOOP-11264 SUBTASKS AND RELATED JIRAS (Common part of HDFS-7285)
 
     HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding
-    (Kai Zheng via umamahesh)
\ No newline at end of file
+    (Kai Zheng via umamahesh)
+
+    HADOOP-11534. Minor improvements for raw erasure coders
+    ( Kai Zheng via vinayakumarb )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52c42196/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
index f84eb11..01e8f35 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
@@ -66,15 +66,26 @@ public class ECChunk {
   }
 
   /**
-   * Convert an array of this chunks to an array of byte array
+   * Convert an array of this chunks to an array of byte array.
+   * Note the chunk buffers are not affected.
    * @param chunks
    * @return an array of byte array
    */
   public static byte[][] toArray(ECChunk[] chunks) {
     byte[][] bytesArr = new byte[chunks.length][];
 
+    ByteBuffer buffer;
     for (int i = 0; i < chunks.length; i++) {
-      bytesArr[i] = chunks[i].getBuffer().array();
+      buffer = chunks[i].getBuffer();
+      if (buffer.hasArray()) {
+        bytesArr[i] = buffer.array();
+      } else {
+        bytesArr[i] = new byte[buffer.remaining()];
+        // Avoid affecting the original one
+        buffer.mark();
+        buffer.get(bytesArr[i]);
+        buffer.reset();
+      }
     }
 
     return bytesArr;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52c42196/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
index 474542b..74d2ab6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
@@ -24,26 +24,26 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
  */
 public abstract class AbstractRawErasureCoder implements RawErasureCoder {
 
-  private int dataSize;
-  private int paritySize;
+  private int numDataUnits;
+  private int numParityUnits;
   private int chunkSize;
 
   @Override
   public void initialize(int numDataUnits, int numParityUnits,
                          int chunkSize) {
-    this.dataSize = numDataUnits;
-    this.paritySize = numParityUnits;
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
     this.chunkSize = chunkSize;
   }
 
   @Override
   public int getNumDataUnits() {
-    return dataSize;
+    return numDataUnits;
   }
 
   @Override
   public int getNumParityUnits() {
-    return paritySize;
+    return numParityUnits;
   }
 
   @Override


[21/50] [abbrv] hadoop git commit: Moving CHANGES.txt entry for MAPREDUCE-4742 to branch-2.7.

Posted by zh...@apache.org.
Moving CHANGES.txt entry for MAPREDUCE-4742 to branch-2.7.


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

Branch: refs/heads/HDFS-7285
Commit: bd0a9ba8e3b1c75dd2fc4cc65cb00c3e31d609ce
Parents: 9d38520
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sat Mar 14 16:53:50 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sat Mar 14 16:53:50 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd0a9ba8/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index ab6eef5..28460d3 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -257,8 +257,6 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
-    MAPREDUCE-4742. Fix typo in nnbench#displayUsage. (Liang Xie via ozawa)
-
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -440,6 +438,8 @@ Release 2.7.0 - UNRELEASED
     MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc
     comments. (Akira AJISAKA and Andrew Purtell via ozawa)
 
+    MAPREDUCE-4742. Fix typo in nnbench#displayUsage. (Liang Xie via ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[08/50] [abbrv] hadoop git commit: HDFS-7722. DataNode#checkDiskError should also remove Storage when error is found. (Lei Xu via Colin P. McCabe)

Posted by zh...@apache.org.
HDFS-7722. DataNode#checkDiskError should also remove Storage when error is found. (Lei Xu via Colin P. McCabe)


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

Branch: refs/heads/HDFS-7285
Commit: b49c3a1813aa8c5b05fe6c02a653286c573137ca
Parents: 6dae6d1
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Mar 12 12:00:18 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Mar 12 12:00:18 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hadoop/hdfs/server/datanode/DataNode.java   | 109 ++++++++++++++++---
 .../hdfs/server/datanode/DataStorage.java       |  15 +--
 .../server/datanode/fsdataset/FsDatasetSpi.java |  11 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  81 ++++----------
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |   3 +-
 .../datanode/fsdataset/impl/FsVolumeList.java   |  41 ++++---
 .../server/datanode/SimulatedFSDataset.java     |   6 +-
 .../datanode/TestDataNodeHotSwapVolumes.java    |  65 +++++++++++
 .../datanode/TestDataNodeVolumeFailure.java     |  70 ++++++++++++
 .../TestDataNodeVolumeFailureReporting.java     |  36 +++---
 .../extdataset/ExternalDatasetImpl.java         |   7 +-
 .../fsdataset/impl/TestFsDatasetImpl.java       |  16 +--
 13 files changed, 330 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e52b849..153453c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1162,6 +1162,9 @@ Release 2.7.0 - UNRELEASED
       HDFS-7806. Refactor: move StorageType from hadoop-hdfs to
       hadoop-common. (Xiaoyu Yao via Arpit Agarwal)
 
+      HDFS-7722. DataNode#checkDiskError should also remove Storage when error
+      is found. (Lei Xu via Colin P. McCabe)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/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 92ddb7b..5be6a6d 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
@@ -53,6 +53,7 @@ import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -73,6 +74,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -613,20 +615,16 @@ public class DataNode extends ReconfigurableBase
             errorMessageBuilder.append(
                 String.format("FAILED to ADD: %s: %s%n", volume,
                               e.toString()));
+            LOG.error("Failed to add volume: " + volume, e);
           }
         }
       }
 
-      if (!changedVolumes.deactivateLocations.isEmpty()) {
-        LOG.info("Deactivating volumes: " +
-            Joiner.on(",").join(changedVolumes.deactivateLocations));
-
-        data.removeVolumes(changedVolumes.deactivateLocations);
-        try {
-          storage.removeVolumes(changedVolumes.deactivateLocations);
-        } catch (IOException e) {
-          errorMessageBuilder.append(e.getMessage());
-        }
+      try {
+        removeVolumes(changedVolumes.deactivateLocations);
+      } catch (IOException e) {
+        errorMessageBuilder.append(e.getMessage());
+        LOG.error("Failed to remove volume: " + e.getMessage(), e);
       }
 
       if (errorMessageBuilder.length() > 0) {
@@ -639,6 +637,79 @@ public class DataNode extends ReconfigurableBase
     }
   }
 
+  /**
+   * Remove volumes from DataNode.
+   * See {@link removeVolumes(final Set<File>, boolean)} for details.
+   *
+   * @param locations the StorageLocations of the volumes to be removed.
+   * @throws IOException
+   */
+  private void removeVolumes(final Collection<StorageLocation> locations)
+    throws IOException {
+    if (locations.isEmpty()) {
+      return;
+    }
+    Set<File> volumesToRemove = new HashSet<>();
+    for (StorageLocation loc : locations) {
+      volumesToRemove.add(loc.getFile().getAbsoluteFile());
+    }
+    removeVolumes(volumesToRemove, true);
+  }
+
+  /**
+   * Remove volumes from DataNode.
+   *
+   * It does three things:
+   * <li>
+   *   <ul>Remove volumes and block info from FsDataset.</ul>
+   *   <ul>Remove volumes from DataStorage.</ul>
+   *   <ul>Reset configuration DATA_DIR and {@link dataDirs} to represent
+   *   active volumes.</ul>
+   * </li>
+   * @param absoluteVolumePaths the absolute path of volumes.
+   * @param clearFailure if true, clears the failure information related to the
+   *                     volumes.
+   * @throws IOException
+   */
+  private synchronized void removeVolumes(
+      final Set<File> absoluteVolumePaths, boolean clearFailure)
+      throws IOException {
+    for (File vol : absoluteVolumePaths) {
+      Preconditions.checkArgument(vol.isAbsolute());
+    }
+
+    if (absoluteVolumePaths.isEmpty()) {
+      return;
+    }
+
+    LOG.info(String.format("Deactivating volumes (clear failure=%b): %s",
+        clearFailure, Joiner.on(",").join(absoluteVolumePaths)));
+
+    IOException ioe = null;
+    // Remove volumes and block infos from FsDataset.
+    data.removeVolumes(absoluteVolumePaths, clearFailure);
+
+    // Remove volumes from DataStorage.
+    try {
+      storage.removeVolumes(absoluteVolumePaths);
+    } catch (IOException e) {
+      ioe = e;
+    }
+
+    // Set configuration and dataDirs to reflect volume changes.
+    for (Iterator<StorageLocation> it = dataDirs.iterator(); it.hasNext(); ) {
+      StorageLocation loc = it.next();
+      if (absoluteVolumePaths.contains(loc.getFile().getAbsoluteFile())) {
+        it.remove();
+      }
+    }
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, Joiner.on(",").join(dataDirs));
+
+    if (ioe != null) {
+      throw ioe;
+    }
+  }
+
   private synchronized void setClusterId(final String nsCid, final String bpid
       ) throws IOException {
     if(clusterId != null && !clusterId.equals(nsCid)) {
@@ -3076,10 +3147,20 @@ public class DataNode extends ReconfigurableBase
    * Check the disk error
    */
   private void checkDiskError() {
-    try {
-      data.checkDataDir();
-    } catch (DiskErrorException de) {
-      handleDiskError(de.getMessage());
+    Set<File> unhealthyDataDirs = data.checkDataDir();
+    if (unhealthyDataDirs != null && !unhealthyDataDirs.isEmpty()) {
+      try {
+        // Remove all unhealthy volumes from DataNode.
+        removeVolumes(unhealthyDataDirs, false);
+      } catch (IOException e) {
+        LOG.warn("Error occurred when removing unhealthy storage dirs: "
+            + e.getMessage(), e);
+      }
+      StringBuilder sb = new StringBuilder("DataNode failed volumes:");
+      for (File dataDir : unhealthyDataDirs) {
+        sb.append(dataDir.getAbsolutePath() + ";");
+      }
+      handleDiskError(sb.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 001f684..f979d3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -404,28 +404,23 @@ public class DataStorage extends Storage {
   }
 
   /**
-   * Remove volumes from DataStorage. All volumes are removed even when the
+   * Remove storage dirs from DataStorage. All storage dirs are removed even when the
    * IOException is thrown.
    *
-   * @param locations a collection of volumes.
+   * @param dirsToRemove a set of storage directories to be removed.
    * @throws IOException if I/O error when unlocking storage directory.
    */
-  synchronized void removeVolumes(Collection<StorageLocation> locations)
+  synchronized void removeVolumes(final Set<File> dirsToRemove)
       throws IOException {
-    if (locations.isEmpty()) {
+    if (dirsToRemove.isEmpty()) {
       return;
     }
 
-    Set<File> dataDirs = new HashSet<File>();
-    for (StorageLocation sl : locations) {
-      dataDirs.add(sl.getFile());
-    }
-
     StringBuilder errorMsgBuilder = new StringBuilder();
     for (Iterator<StorageDirectory> it = this.storageDirs.iterator();
          it.hasNext(); ) {
       StorageDirectory sd = it.next();
-      if (dataDirs.contains(sd.getRoot())) {
+      if (dirsToRemove.contains(sd.getRoot())) {
         // Remove the block pool level storage first.
         for (Map.Entry<String, BlockPoolSliceStorage> entry :
             this.bpStorageMap.entrySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 5b183e6..6f4da09 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -27,6 +27,7 @@ import java.io.InputStream;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -113,9 +114,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * If the FSDataset supports block scanning, this function removes
    * the volumes from the block scanner.
    *
-   * @param volumes      The storage locations of the volumes to remove.
+   * @param volumes  The paths of the volumes to be removed.
+   * @param clearFailure set true to clear the failure information about the
+   *                     volumes.
    */
-  public void removeVolumes(Collection<StorageLocation> volumes);
+  public void removeVolumes(Set<File> volumes, boolean clearFailure);
 
   /** @return a storage with the given storage ID */
   public DatanodeStorage getStorage(final String storageUuid);
@@ -388,9 +391,9 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
 
     /**
      * Check if all the data directories are healthy
-     * @throws DiskErrorException
+     * @return A set of unhealthy data directories.
      */
-  public void checkDataDir() throws DiskErrorException;
+  public Set<File> checkDataDir();
 
   /**
    * Shutdown the FSDataset

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 48ac6ca..486acbc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -445,41 +445,42 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   }
 
   /**
-   * Removes a collection of volumes from FsDataset.
-   * @param volumes the root directories of the volumes.
+   * Removes a set of volumes from FsDataset.
+   * @param volumesToRemove a set of absolute root path of each volume.
+   * @param clearFailure set true to clear failure information.
    *
    * DataNode should call this function before calling
    * {@link DataStorage#removeVolumes(java.util.Collection)}.
    */
   @Override
-  public synchronized void removeVolumes(Collection<StorageLocation> volumes) {
-    Set<String> volumeSet = new HashSet<>();
-    for (StorageLocation sl : volumes) {
-      volumeSet.add(sl.getFile().getAbsolutePath());
+  public synchronized void removeVolumes(
+      Set<File> volumesToRemove, boolean clearFailure) {
+    // Make sure that all volumes are absolute path.
+    for (File vol : volumesToRemove) {
+      Preconditions.checkArgument(vol.isAbsolute(),
+          String.format("%s is not absolute path.", vol.getPath()));
     }
     for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
       Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
-      String volume = sd.getRoot().getAbsolutePath();
-      if (volumeSet.contains(volume)) {
-        LOG.info("Removing " + volume + " from FsDataset.");
+      final File absRoot = sd.getRoot().getAbsoluteFile();
+      if (volumesToRemove.contains(absRoot)) {
+        LOG.info("Removing " + absRoot + " from FsDataset.");
 
         // Disable the volume from the service.
         asyncDiskService.removeVolume(sd.getCurrentDir());
-        this.volumes.removeVolume(sd.getRoot());
+        volumes.removeVolume(absRoot, clearFailure);
 
         // Removed all replica information for the blocks on the volume. Unlike
         // updating the volumeMap in addVolume(), this operation does not scan
         // disks.
         for (String bpid : volumeMap.getBlockPoolList()) {
-          List<Block> blocks = new ArrayList<Block>();
           for (Iterator<ReplicaInfo> it = volumeMap.replicas(bpid).iterator();
-              it.hasNext(); ) {
+               it.hasNext(); ) {
             ReplicaInfo block = it.next();
-            String absBasePath =
-                  new File(block.getVolume().getBasePath()).getAbsolutePath();
-            if (absBasePath.equals(volume)) {
+            final File absBasePath =
+                new File(block.getVolume().getBasePath()).getAbsoluteFile();
+            if (absBasePath.equals(absRoot)) {
               invalidate(bpid, block);
-              blocks.add(block);
               it.remove();
             }
           }
@@ -1975,50 +1976,14 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
   /**
    * check if a data directory is healthy
-   * if some volumes failed - make sure to remove all the blocks that belong
-   * to these volumes
-   * @throws DiskErrorException
+   *
+   * if some volumes failed - the caller must emove all the blocks that belong
+   * to these failed volumes.
+   * @return the failed volumes. Returns null if no volume failed.
    */
   @Override // FsDatasetSpi
-  public void checkDataDir() throws DiskErrorException {
-    long totalBlocks=0, removedBlocks=0;
-    List<FsVolumeImpl> failedVols =  volumes.checkDirs();
-    
-    // If there no failed volumes return
-    if (failedVols == null) { 
-      return;
-    }
-    
-    // Otherwise remove blocks for the failed volumes
-    long mlsec = Time.now();
-    synchronized (this) {
-      for (FsVolumeImpl fv: failedVols) {
-        for (String bpid : fv.getBlockPoolList()) {
-          Iterator<ReplicaInfo> ib = volumeMap.replicas(bpid).iterator();
-          while(ib.hasNext()) {
-            ReplicaInfo b = ib.next();
-            totalBlocks++;
-            // check if the volume block belongs to still valid
-            if(b.getVolume() == fv) {
-              LOG.warn("Removing replica " + bpid + ":" + b.getBlockId()
-                  + " on failed volume " + fv.getCurrentDir().getAbsolutePath());
-              ib.remove();
-              removedBlocks++;
-            }
-          }
-        }
-      }
-    } // end of sync
-    mlsec = Time.now() - mlsec;
-    LOG.warn("Removed " + removedBlocks + " out of " + totalBlocks +
-        "(took " + mlsec + " millisecs)");
-
-    // report the error
-    StringBuilder sb = new StringBuilder();
-    for (FsVolumeImpl fv : failedVols) {
-      sb.append(fv.getCurrentDir().getAbsolutePath() + ";");
-    }
-    throw new DiskErrorException("DataNode failed volumes:" + sb);
+  public Set<File> checkDataDir() {
+   return volumes.checkDirs();
   }
     
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 744db62..23efbdf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -289,7 +289,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
     }
   }
 
-  long getDfsUsed() throws IOException {
+  @VisibleForTesting
+  public long getDfsUsed() throws IOException {
     long dfsUsed = 0;
     synchronized(dataset) {
       for(BlockPoolSlice s : bpSlices.values()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
index b38d41f..a5611c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
@@ -24,10 +24,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 
 import com.google.common.collect.Lists;
@@ -218,16 +220,15 @@ class FsVolumeList {
   }
 
   /**
-   * Calls {@link FsVolumeImpl#checkDirs()} on each volume, removing any
-   * volumes from the active list that result in a DiskErrorException.
+   * Calls {@link FsVolumeImpl#checkDirs()} on each volume.
    * 
    * Use checkDirsMutext to allow only one instance of checkDirs() call
    *
-   * @return list of all the removed volumes.
+   * @return list of all the failed volumes.
    */
-  List<FsVolumeImpl> checkDirs() {
+  Set<File> checkDirs() {
     synchronized(checkDirsMutex) {
-      ArrayList<FsVolumeImpl> removedVols = null;
+      Set<File> failedVols = null;
       
       // Make a copy of volumes for performing modification 
       final List<FsVolumeImpl> volumeList = getVolumes();
@@ -238,12 +239,12 @@ class FsVolumeList {
           fsv.checkDirs();
         } catch (DiskErrorException e) {
           FsDatasetImpl.LOG.warn("Removing failed volume " + fsv + ": ", e);
-          if (removedVols == null) {
-            removedVols = new ArrayList<>(1);
+          if (failedVols == null) {
+            failedVols = new HashSet<>(1);
           }
-          removedVols.add(fsv);
-          removeVolume(fsv);
+          failedVols.add(new File(fsv.getBasePath()).getAbsoluteFile());
           addVolumeFailureInfo(fsv);
+          removeVolume(fsv);
         } catch (ClosedChannelException e) {
           FsDatasetImpl.LOG.debug("Caught exception when obtaining " +
             "reference count on closed volume", e);
@@ -252,12 +253,12 @@ class FsVolumeList {
         }
       }
       
-      if (removedVols != null && removedVols.size() > 0) {
-        FsDatasetImpl.LOG.warn("Completed checkDirs. Removed " + removedVols.size()
-            + " volumes. Current volumes: " + this);
+      if (failedVols != null && failedVols.size() > 0) {
+        FsDatasetImpl.LOG.warn("Completed checkDirs. Found " + failedVols.size()
+            + " failure volumes.");
       }
 
-      return removedVols;
+      return failedVols;
     }
   }
 
@@ -290,6 +291,9 @@ class FsVolumeList {
     if (blockScanner != null) {
       blockScanner.addVolumeScanner(ref);
     }
+    // If the volume is used to replace a failed volume, it needs to reset the
+    // volume failure info for this volume.
+    removeVolumeFailureInfo(new File(ref.getVolume().getBasePath()));
     FsDatasetImpl.LOG.info("Added new volume: " +
         ref.getVolume().getStorageID());
   }
@@ -337,8 +341,9 @@ class FsVolumeList {
   /**
    * Dynamically remove volume in the list.
    * @param volume the volume to be removed.
+   * @param clearFailure set true to remove failure info for this volume.
    */
-  void removeVolume(File volume) {
+  void removeVolume(File volume, boolean clearFailure) {
     // Make a copy of volumes to remove one volume.
     final FsVolumeImpl[] curVolumes = volumes.get();
     final List<FsVolumeImpl> volumeList = Lists.newArrayList(curVolumes);
@@ -352,7 +357,9 @@ class FsVolumeList {
         removeVolume(fsVolume);
       }
     }
-    removeVolumeFailureInfo(volume);
+    if (clearFailure) {
+      removeVolumeFailureInfo(volume);
+    }
   }
 
   VolumeFailureInfo[] getVolumeFailureInfos() {
@@ -366,7 +373,9 @@ class FsVolumeList {
   }
 
   private void addVolumeFailureInfo(FsVolumeImpl vol) {
-    addVolumeFailureInfo(new VolumeFailureInfo(vol.getBasePath(), Time.now(),
+    addVolumeFailureInfo(new VolumeFailureInfo(
+        new File(vol.getBasePath()).getAbsolutePath(),
+        Time.now(),
         vol.getCapacity()));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index f0dbd0f..a4ec8d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -30,6 +30,7 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
@@ -959,8 +960,9 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override
-  public void checkDataDir() throws DiskErrorException {
+  public Set<File> checkDataDir() {
     // nothing to check for simulated data set
+    return null;
   }
 
   @Override // FsDatasetSpi
@@ -1281,7 +1283,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override
-  public synchronized void removeVolumes(Collection<StorageLocation> volumes) {
+  public synchronized void removeVolumes(Set<File> volumes, boolean clearFailure) {
     throw new UnsupportedOperationException();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
index ac316e8..466598b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockMissingException;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -38,6 +39,7 @@ import org.apache.hadoop.hdfs.server.common.Storage;
 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.fsdataset.impl.FsDatasetTestUtil;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
@@ -95,6 +97,8 @@ public class TestDataNodeHotSwapVolumes {
     conf.setInt(DFSConfigKeys.DFS_DF_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
         1000);
+    /* Allow 1 volume failure */
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
 
     MiniDFSNNTopology nnTopology =
         MiniDFSNNTopology.simpleFederatedTopology(numNameNodes);
@@ -646,4 +650,65 @@ public class TestDataNodeHotSwapVolumes {
     // this directory were removed from the previous step.
     dn.reconfigurePropertyImpl(DFS_DATANODE_DATA_DIR_KEY, oldDataDir);
   }
+
+  /** Get the FsVolume on the given basePath */
+  private FsVolumeImpl getVolume(DataNode dn, File basePath) {
+    for (FsVolumeSpi vol : dn.getFSDataset().getVolumes()) {
+      if (vol.getBasePath().equals(basePath.getPath())) {
+        return (FsVolumeImpl)vol;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Verify that {@link DataNode#checkDiskErrors()} removes all metadata in
+   * DataNode upon a volume failure. Thus we can run reconfig on the same
+   * configuration to reload the new volume on the same directory as the failed one.
+   */
+  @Test(timeout=60000)
+  public void testDirectlyReloadAfterCheckDiskError()
+      throws IOException, TimeoutException, InterruptedException,
+      ReconfigurationException {
+    startDFSCluster(1, 2);
+    createFile(new Path("/test"), 32, (short)2);
+
+    DataNode dn = cluster.getDataNodes().get(0);
+    final String oldDataDir = dn.getConf().get(DFS_DATANODE_DATA_DIR_KEY);
+    File dirToFail = new File(cluster.getDataDirectory(), "data1");
+
+    FsVolumeImpl failedVolume = getVolume(dn, dirToFail);
+    assertTrue("No FsVolume was found for " + dirToFail,
+        failedVolume != null);
+    long used = failedVolume.getDfsUsed();
+
+    try {
+      assertTrue("Couldn't chmod local vol: " + dirToFail,
+          FileUtil.setExecutable(dirToFail, false));
+      // Call and wait DataNode to detect disk failure.
+      long lastDiskErrorCheck = dn.getLastDiskErrorCheck();
+      dn.checkDiskErrorAsync();
+      while (dn.getLastDiskErrorCheck() == lastDiskErrorCheck) {
+        Thread.sleep(100);
+      }
+
+      createFile(new Path("/test1"), 32, (short)2);
+      assertEquals(used, failedVolume.getDfsUsed());
+    } finally {
+      // Need to restore the mode on dirToFail. Otherwise, if an Exception
+      // is thrown above, the following tests can not delete this data directory
+      // and thus fail to start MiniDFSCluster.
+      assertTrue("Couldn't restore executable for: " + dirToFail,
+          FileUtil.setExecutable(dirToFail, true));
+    }
+
+    dn.reconfigurePropertyImpl(DFS_DATANODE_DATA_DIR_KEY, oldDataDir);
+
+    createFile(new Path("/test2"), 32, (short)2);
+    FsVolumeImpl restoredVolume = getVolume(dn, dirToFail);
+    assertTrue(restoredVolume != null);
+    assertTrue(restoredVolume != failedVolume);
+    // More data has been written to this volume.
+    assertTrue(restoredVolume.getDfsUsed() > used);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index d9ad96b..ba786d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
@@ -31,6 +33,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -57,6 +60,10 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.common.Storage;
+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.fsdataset.impl.FsDatasetTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -201,6 +208,69 @@ public class TestDataNodeVolumeFailure {
   }
 
   /**
+   * Test that DataStorage and BlockPoolSliceStorage remove the failed volume
+   * after failure.
+   */
+  @Test(timeout=150000)
+  public void testFailedVolumeBeingRemovedFromDataNode()
+      throws InterruptedException, IOException, TimeoutException {
+    Path file1 = new Path("/test1");
+    DFSTestUtil.createFile(fs, file1, 1024, (short) 2, 1L);
+    DFSTestUtil.waitReplication(fs, file1, (short) 2);
+
+    File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
+    assertTrue(FileUtil.setExecutable(dn0Vol1, false));
+    DataNode dn0 = cluster.getDataNodes().get(0);
+    long lastDiskErrorCheck = dn0.getLastDiskErrorCheck();
+    dn0.checkDiskErrorAsync();
+    // Wait checkDiskError thread finish to discover volume failure.
+    while (dn0.getLastDiskErrorCheck() == lastDiskErrorCheck) {
+      Thread.sleep(100);
+    }
+
+    // Verify dn0Vol1 has been completely removed from DN0.
+    // 1. dn0Vol1 is removed from DataStorage.
+    DataStorage storage = dn0.getStorage();
+    assertEquals(1, storage.getNumStorageDirs());
+    for (int i = 0; i < storage.getNumStorageDirs(); i++) {
+      Storage.StorageDirectory sd = storage.getStorageDir(i);
+      assertFalse(sd.getRoot().getAbsolutePath().startsWith(
+          dn0Vol1.getAbsolutePath()
+      ));
+    }
+    final String bpid = cluster.getNamesystem().getBlockPoolId();
+    BlockPoolSliceStorage bpsStorage = storage.getBPStorage(bpid);
+    assertEquals(1, bpsStorage.getNumStorageDirs());
+    for (int i = 0; i < bpsStorage.getNumStorageDirs(); i++) {
+      Storage.StorageDirectory sd = bpsStorage.getStorageDir(i);
+      assertFalse(sd.getRoot().getAbsolutePath().startsWith(
+          dn0Vol1.getAbsolutePath()
+      ));
+    }
+
+    // 2. dn0Vol1 is removed from FsDataset
+    FsDatasetSpi<? extends FsVolumeSpi> data = dn0.getFSDataset();
+    for (FsVolumeSpi volume : data.getVolumes()) {
+      assertNotEquals(new File(volume.getBasePath()).getAbsoluteFile(),
+          dn0Vol1.getAbsoluteFile());
+    }
+
+    // 3. all blocks on dn0Vol1 have been removed.
+    for (ReplicaInfo replica : FsDatasetTestUtil.getReplicas(data, bpid)) {
+      assertNotNull(replica.getVolume());
+      assertNotEquals(
+          new File(replica.getVolume().getBasePath()).getAbsoluteFile(),
+          dn0Vol1.getAbsoluteFile());
+    }
+
+    // 4. dn0Vol1 is not in DN0's configuration and dataDirs anymore.
+    String[] dataDirStrs =
+        dn0.getConf().get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY).split(",");
+    assertEquals(1, dataDirStrs.length);
+    assertFalse(dataDirStrs[0].contains(dn0Vol1.getAbsolutePath()));
+  }
+
+  /**
    * Test that there are under replication blocks after vol failures
    */
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
index a8f7990..788ddb3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
@@ -403,23 +403,6 @@ public class TestDataNodeVolumeFailureReporting {
     checkFailuresAtNameNode(dm, dns.get(0), true, dn1Vol1.getAbsolutePath());
     checkFailuresAtNameNode(dm, dns.get(1), true, dn2Vol1.getAbsolutePath());
 
-    // Reconfigure each DataNode to remove its failed volumes.
-    reconfigureDataNode(dns.get(0), dn1Vol2);
-    reconfigureDataNode(dns.get(1), dn2Vol2);
-
-    DataNodeTestUtils.triggerHeartbeat(dns.get(0));
-    DataNodeTestUtils.triggerHeartbeat(dns.get(1));
-
-    checkFailuresAtDataNode(dns.get(0), 1, true);
-    checkFailuresAtDataNode(dns.get(1), 1, true);
-
-    // NN sees reduced capacity, but no volume failures.
-    DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 0,
-        origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
-    checkAggregateFailuresAtNameNode(true, 0);
-    checkFailuresAtNameNode(dm, dns.get(0), true);
-    checkFailuresAtNameNode(dm, dns.get(1), true);
-
     // Reconfigure again to try to add back the failed volumes.
     reconfigureDataNode(dns.get(0), dn1Vol1, dn1Vol2);
     reconfigureDataNode(dns.get(1), dn2Vol1, dn2Vol2);
@@ -460,6 +443,25 @@ public class TestDataNodeVolumeFailureReporting {
     checkAggregateFailuresAtNameNode(false, 2);
     checkFailuresAtNameNode(dm, dns.get(0), false, dn1Vol1.getAbsolutePath());
     checkFailuresAtNameNode(dm, dns.get(1), false, dn2Vol1.getAbsolutePath());
+
+    // Replace failed volume with healthy volume and run reconfigure DataNode.
+    // The failed volume information should be cleared.
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, true));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
+    reconfigureDataNode(dns.get(0), dn1Vol1, dn1Vol2);
+    reconfigureDataNode(dns.get(1), dn2Vol1, dn2Vol2);
+
+    DataNodeTestUtils.triggerHeartbeat(dns.get(0));
+    DataNodeTestUtils.triggerHeartbeat(dns.get(1));
+
+    checkFailuresAtDataNode(dns.get(0), 1, true);
+    checkFailuresAtDataNode(dns.get(1), 1, true);
+
+    DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 0,
+        origCapacity, WAIT_FOR_HEARTBEATS);
+    checkAggregateFailuresAtNameNode(true, 0);
+    checkFailuresAtNameNode(dm, dns.get(0), true);
+    checkFailuresAtNameNode(dm, dns.get(1), true);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index 6653cca..5a440c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -61,8 +61,7 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   }
 
   @Override
-  public void removeVolumes(Collection<StorageLocation> volumes) {
-
+  public void removeVolumes(Set<File> volumes, boolean clearFailure) {
   }
 
   @Override
@@ -243,8 +242,8 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   }
 
   @Override
-  public void checkDataDir() throws DiskErrorException {
-    throw new DiskChecker.DiskErrorException(null);
+  public Set<File> checkDataDir() {
+    return null;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b49c3a18/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
index 403cb2a..8654773 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
@@ -195,10 +195,10 @@ public class TestFsDatasetImpl {
     final String[] dataDirs =
         conf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY).split(",");
     final String volumePathToRemove = dataDirs[0];
-    List<StorageLocation> volumesToRemove = new ArrayList<StorageLocation>();
-    volumesToRemove.add(StorageLocation.parse(volumePathToRemove));
+    Set<File> volumesToRemove = new HashSet<>();
+    volumesToRemove.add(StorageLocation.parse(volumePathToRemove).getFile());
 
-    dataset.removeVolumes(volumesToRemove);
+    dataset.removeVolumes(volumesToRemove, true);
     int expectedNumVolumes = dataDirs.length - 1;
     assertEquals("The volume has been removed from the volumeList.",
         expectedNumVolumes, dataset.getVolumes().size());
@@ -206,7 +206,7 @@ public class TestFsDatasetImpl {
         expectedNumVolumes, dataset.storageMap.size());
 
     try {
-      dataset.asyncDiskService.execute(volumesToRemove.get(0).getFile(),
+      dataset.asyncDiskService.execute(volumesToRemove.iterator().next(),
           new Runnable() {
             @Override
             public void run() {}
@@ -248,8 +248,9 @@ public class TestFsDatasetImpl {
 
     when(storage.getNumStorageDirs()).thenReturn(numExistingVolumes + 1);
     when(storage.getStorageDir(numExistingVolumes)).thenReturn(sd);
-    List<StorageLocation> volumesToRemove = Arrays.asList(loc);
-    dataset.removeVolumes(volumesToRemove);
+    Set<File> volumesToRemove = new HashSet<>();
+    volumesToRemove.add(loc.getFile());
+    dataset.removeVolumes(volumesToRemove, true);
     assertEquals(numExistingVolumes, dataset.getVolumes().size());
   }
 
@@ -278,12 +279,13 @@ public class TestFsDatasetImpl {
     final FsVolumeImpl newVolume = mock(FsVolumeImpl.class);
     final FsVolumeReference newRef = mock(FsVolumeReference.class);
     when(newRef.getVolume()).thenReturn(newVolume);
+    when(newVolume.getBasePath()).thenReturn("data4");
     FsVolumeImpl blockedVolume = volumeList.getVolumes().get(1);
     doAnswer(new Answer() {
       @Override
       public Object answer(InvocationOnMock invocationOnMock)
           throws Throwable {
-        volumeList.removeVolume(new File("data4"));
+        volumeList.removeVolume(new File("data4"), false);
         volumeList.addVolume(newRef);
         return null;
       }


[40/50] [abbrv] hadoop git commit: HDFS-7716. Erasure Coding: extend BlockInfo to handle EC info. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-7716. Erasure Coding: extend BlockInfo to handle EC info. Contributed by Jing Zhao.

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java


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

Branch: refs/heads/HDFS-7285
Commit: 10488c9c27e03ea6692e146319dec10dbc0b4fa9
Parents: c054b2a
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Feb 10 17:54:10 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:09:27 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   1 +
 .../server/blockmanagement/BlockCollection.java |  13 +-
 .../server/blockmanagement/BlockIdManager.java  |   7 +-
 .../hdfs/server/blockmanagement/BlockInfo.java  | 339 +++++++++++++++++
 .../blockmanagement/BlockInfoContiguous.java    | 363 +++----------------
 .../BlockInfoContiguousUnderConstruction.java   | 140 +------
 .../blockmanagement/BlockInfoStriped.java       | 179 +++++++++
 .../server/blockmanagement/BlockManager.java    | 188 +++++-----
 .../hdfs/server/blockmanagement/BlocksMap.java  |  46 +--
 .../CacheReplicationMonitor.java                |  10 +-
 .../blockmanagement/DatanodeDescriptor.java     |  22 +-
 .../blockmanagement/DatanodeStorageInfo.java    |  38 +-
 .../ReplicaUnderConstruction.java               | 119 ++++++
 .../hdfs/server/namenode/FSDirectory.java       |   4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  24 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   3 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |   4 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   4 +-
 .../server/blockmanagement/TestBlockInfo.java   |   6 +-
 .../blockmanagement/TestBlockInfoStriped.java   | 219 +++++++++++
 .../blockmanagement/TestBlockManager.java       |   4 +-
 .../blockmanagement/TestReplicationPolicy.java  |   2 +-
 22 files changed, 1127 insertions(+), 608 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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 de60b6e..245b630 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
@@ -184,5 +184,6 @@ public class HdfsConstants {
 
   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;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
index 1547611..974cac3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
@@ -39,12 +39,12 @@ public interface BlockCollection {
   public ContentSummary computeContentSummary();
 
   /**
-   * @return the number of blocks
+   * @return the number of blocks or block groups
    */ 
   public int numBlocks();
 
   /**
-   * Get the blocks.
+   * Get the blocks or block groups.
    */
   public BlockInfoContiguous[] getBlocks();
 
@@ -55,8 +55,8 @@ public interface BlockCollection {
   public long getPreferredBlockSize();
 
   /**
-   * Get block replication for the collection 
-   * @return block replication value
+   * Get block replication for the collection.
+   * @return block replication value. Return 0 if the file is erasure coded.
    */
   public short getBlockReplication();
 
@@ -71,7 +71,7 @@ public interface BlockCollection {
   public String getName();
 
   /**
-   * Set the block at the given index.
+   * Set the block/block-group at the given index.
    */
   public void setBlock(int index, BlockInfoContiguous blk);
 
@@ -79,7 +79,8 @@ public interface BlockCollection {
    * Convert the last block of the collection to an under-construction block
    * and set the locations.
    */
-  public BlockInfoContiguousUnderConstruction setLastBlock(BlockInfoContiguous lastBlock,
+  public BlockInfoContiguousUnderConstruction setLastBlock(
+      BlockInfoContiguous lastBlock,
       DatanodeStorageInfo[] targets) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index e7f8a05..3ae54ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -217,6 +217,11 @@ public class BlockIdManager {
   }
 
   public static long convertToGroupID(long id) {
-    return id & (~(HdfsConstants.MAX_BLOCKS_IN_GROUP - 1));
+    return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK);
+  }
+
+  public static int getBlockIndex(Block reportedBlock) {
+    return (int) (reportedBlock.getBlockId() &
+        HdfsConstants.BLOCK_GROUP_INDEX_MASK);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
new file mode 100644
index 0000000..f19ad32
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -0,0 +1,339 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.util.LightWeightGSet;
+
+import java.util.LinkedList;
+
+/**
+ * For a given block (or an erasure coding block group), BlockInfo class
+ * maintains 1) the {@link BlockCollection} it is part of, and 2) datanodes
+ * where the replicas of the block, or blocks belonging to the erasure coding
+ * block group, are stored.
+ */
+public abstract class BlockInfo extends Block
+    implements LightWeightGSet.LinkedElement {
+  private BlockCollection bc;
+
+  /** For implementing {@link LightWeightGSet.LinkedElement} interface */
+  private LightWeightGSet.LinkedElement nextLinkedElement;
+
+  /**
+   * This array contains triplets of references. For each i-th storage, the
+   * block belongs to triplets[3*i] is the reference to the
+   * {@link DatanodeStorageInfo} and triplets[3*i+1] and triplets[3*i+2] are
+   * references to the previous and the next blocks, respectively, in the list
+   * of blocks belonging to this storage.
+   *
+   * Using previous and next in Object triplets is done instead of a
+   * {@link LinkedList} list to efficiently use memory. With LinkedList the cost
+   * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
+   * bytes using the triplets.
+   */
+  protected Object[] triplets;
+
+  /**
+   * Construct an entry for blocksmap
+   * @param size the block's replication factor, or the total number of blocks
+   *             in the block group
+   */
+  public BlockInfo(short size) {
+    this.triplets = new Object[3 * size];
+    this.bc = null;
+  }
+
+  public BlockInfo(Block blk, short size) {
+    super(blk);
+    this.triplets = new Object[3 * size];
+    this.bc = null;
+  }
+
+  public BlockCollection getBlockCollection() {
+    return bc;
+  }
+
+  public void setBlockCollection(BlockCollection bc) {
+    this.bc = bc;
+  }
+
+  public DatanodeDescriptor getDatanode(int index) {
+    DatanodeStorageInfo storage = getStorageInfo(index);
+    return storage == null ? null : storage.getDatanodeDescriptor();
+  }
+
+  DatanodeStorageInfo getStorageInfo(int index) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+    return (DatanodeStorageInfo)triplets[index*3];
+  }
+
+  BlockInfo getPrevious(int index) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
+    return (BlockInfo) triplets[index*3+1];
+  }
+
+  BlockInfo getNext(int index) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
+    return (BlockInfo) triplets[index*3+2];
+  }
+
+  void setStorageInfo(int index, DatanodeStorageInfo storage) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+    triplets[index*3] = storage;
+  }
+
+  /**
+   * Return the previous block on the block list for the datanode at
+   * position index. Set the previous block on the list to "to".
+   *
+   * @param index - the datanode index
+   * @param to - block to be set to previous on the list of blocks
+   * @return current previous block on the list of blocks
+   */
+  BlockInfo setPrevious(int index, BlockInfo to) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
+    BlockInfo info = (BlockInfo) triplets[index*3+1];
+    triplets[index*3+1] = to;
+    return info;
+  }
+
+  /**
+   * Return the next block on the block list for the datanode at
+   * position index. Set the next block on the list to "to".
+   *
+   * @param index - the datanode index
+   * @param to - block to be set to next on the list of blocks
+   * @return current next block on the list of blocks
+   */
+  BlockInfo setNext(int index, BlockInfo to) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
+    BlockInfo info = (BlockInfo) triplets[index*3+2];
+    triplets[index*3+2] = to;
+    return info;
+  }
+
+  public int getCapacity() {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
+    return triplets.length / 3;
+  }
+
+  /**
+   * Count the number of data-nodes the block currently belongs to (i.e., NN
+   * has received block reports from the DN).
+   */
+  public abstract int numNodes();
+
+  /**
+   * Add a {@link DatanodeStorageInfo} location for a block
+   * @param storage The storage to add
+   * @param reportedBlock The block reported from the datanode. This is only
+   *                      used by erasure coded blocks, this block's id contains
+   *                      information indicating the index of the block in the
+   *                      corresponding block group.
+   */
+  abstract boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock);
+
+  /**
+   * Remove {@link DatanodeStorageInfo} location for a block
+   */
+  abstract boolean removeStorage(DatanodeStorageInfo storage);
+
+  /**
+   * Replace the current BlockInfo with the new one in corresponding
+   * DatanodeStorageInfo's linked list
+   */
+  abstract void replaceBlock(BlockInfo newBlock);
+
+  /**
+   * Find specified DatanodeDescriptor.
+   * @return index or -1 if not found.
+   */
+  boolean findDatanode(DatanodeDescriptor dn) {
+    int len = getCapacity();
+    for (int idx = 0; idx < len; idx++) {
+      DatanodeDescriptor cur = getDatanode(idx);
+      if(cur == dn) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Find specified DatanodeStorageInfo.
+   * @return DatanodeStorageInfo or null if not found.
+   */
+  DatanodeStorageInfo findStorageInfo(DatanodeDescriptor dn) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeStorageInfo cur = getStorageInfo(idx);
+      if(cur != null && cur.getDatanodeDescriptor() == dn) {
+        return cur;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Find specified DatanodeStorageInfo.
+   * @return index or -1 if not found.
+   */
+  int findStorageInfo(DatanodeStorageInfo storageInfo) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeStorageInfo cur = getStorageInfo(idx);
+      if (cur == storageInfo) {
+        return idx;
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Insert this block into the head of the list of blocks
+   * related to the specified DatanodeStorageInfo.
+   * If the head is null then form a new list.
+   * @return current block as the new head of the list.
+   */
+  BlockInfo listInsert(BlockInfo head, DatanodeStorageInfo storage) {
+    int dnIndex = this.findStorageInfo(storage);
+    assert dnIndex >= 0 : "Data node is not found: current";
+    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
+        "Block is already in the list and cannot be inserted.";
+    this.setPrevious(dnIndex, null);
+    this.setNext(dnIndex, head);
+    if (head != null) {
+      head.setPrevious(head.findStorageInfo(storage), this);
+    }
+    return this;
+  }
+
+  /**
+   * Remove this block from the list of blocks
+   * related to the specified DatanodeStorageInfo.
+   * If this block is the head of the list then return the next block as
+   * the new head.
+   * @return the new head of the list or null if the list becomes
+   * empy after deletion.
+   */
+  BlockInfo listRemove(BlockInfo head, DatanodeStorageInfo storage) {
+    if (head == null) {
+      return null;
+    }
+    int dnIndex = this.findStorageInfo(storage);
+    if (dnIndex < 0) { // this block is not on the data-node list
+      return head;
+    }
+
+    BlockInfo next = this.getNext(dnIndex);
+    BlockInfo prev = this.getPrevious(dnIndex);
+    this.setNext(dnIndex, null);
+    this.setPrevious(dnIndex, null);
+    if (prev != null) {
+      prev.setNext(prev.findStorageInfo(storage), next);
+    }
+    if (next != null) {
+      next.setPrevious(next.findStorageInfo(storage), prev);
+    }
+    if (this == head) { // removing the head
+      head = next;
+    }
+    return head;
+  }
+
+  /**
+   * Remove this block from the list of blocks related to the specified
+   * DatanodeDescriptor. Insert it into the head of the list of blocks.
+   *
+   * @return the new head of the list.
+   */
+  public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage,
+      int curIndex, int headIndex) {
+    if (head == this) {
+      return this;
+    }
+    BlockInfo next = this.setNext(curIndex, head);
+    BlockInfo prev = this.setPrevious(curIndex, null);
+
+    head.setPrevious(headIndex, this);
+    prev.setNext(prev.findStorageInfo(storage), next);
+    if (next != null) {
+      next.setPrevious(next.findStorageInfo(storage), prev);
+    }
+    return this;
+  }
+
+  /**
+   * BlockInfo represents a block that is not being constructed.
+   * In order to start modifying the block, the BlockInfo should be converted
+   * to {@link BlockInfoContiguousUnderConstruction}.
+   * @return {@link HdfsServerConstants.BlockUCState#COMPLETE}
+   */
+  public HdfsServerConstants.BlockUCState getBlockUCState() {
+    return HdfsServerConstants.BlockUCState.COMPLETE;
+  }
+
+  /**
+   * Is this block complete?
+   *
+   * @return true if the state of the block is
+   *         {@link HdfsServerConstants.BlockUCState#COMPLETE}
+   */
+  public boolean isComplete() {
+    return getBlockUCState().equals(HdfsServerConstants.BlockUCState.COMPLETE);
+  }
+
+  @Override
+  public int hashCode() {
+    // Super implementation is sufficient
+    return super.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    // Sufficient to rely on super's implementation
+    return (this == obj) || super.equals(obj);
+  }
+
+  @Override
+  public LightWeightGSet.LinkedElement getNext() {
+    return nextLinkedElement;
+  }
+
+  @Override
+  public void setNext(LightWeightGSet.LinkedElement next) {
+    this.nextLinkedElement = next;
+  }
+
+  static BlockInfo copyOf(BlockInfo b) {
+    if (b instanceof BlockInfoContiguous) {
+      return new BlockInfoContiguous((BlockInfoContiguous) b);
+    } else {
+      return new BlockInfoStriped((BlockInfoStriped) b);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index 48069c1..e54cba3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -17,148 +17,33 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import java.util.LinkedList;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.util.LightWeightGSet;
 
 /**
- * BlockInfo class maintains for a given block
- * the {@link BlockCollection} it is part of and datanodes where the replicas of 
- * the block are stored.
+ * Subclass of {@link BlockInfo}, used for a block with replication scheme.
  */
 @InterfaceAudience.Private
-public class BlockInfoContiguous extends Block
-    implements LightWeightGSet.LinkedElement {
+public class BlockInfoContiguous extends BlockInfo {
   public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
 
-  private BlockCollection bc;
-
-  /** For implementing {@link LightWeightGSet.LinkedElement} interface */
-  private LightWeightGSet.LinkedElement nextLinkedElement;
-
-  /**
-   * This array contains triplets of references. For each i-th storage, the
-   * block belongs to triplets[3*i] is the reference to the
-   * {@link DatanodeStorageInfo} and triplets[3*i+1] and triplets[3*i+2] are
-   * references to the previous and the next blocks, respectively, in the list
-   * of blocks belonging to this storage.
-   * 
-   * Using previous and next in Object triplets is done instead of a
-   * {@link LinkedList} list to efficiently use memory. With LinkedList the cost
-   * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
-   * bytes using the triplets.
-   */
-  private Object[] triplets;
-
-  /**
-   * Construct an entry for blocksmap
-   * @param replication the block's replication factor
-   */
-  public BlockInfoContiguous(short replication) {
-    this.triplets = new Object[3*replication];
-    this.bc = null;
+  public BlockInfoContiguous(short size) {
+    super(size);
   }
-  
-  public BlockInfoContiguous(Block blk, short replication) {
-    super(blk);
-    this.triplets = new Object[3*replication];
-    this.bc = null;
+
+  public BlockInfoContiguous(Block blk, short size) {
+    super(blk, size);
   }
 
   /**
    * Copy construction.
-   * This is used to convert BlockInfoUnderConstruction
-   * @param from BlockInfo to copy from.
+   * This is used to convert BlockReplicationInfoUnderConstruction
+   * @param from BlockReplicationInfo to copy from.
    */
   protected BlockInfoContiguous(BlockInfoContiguous from) {
-    this(from, from.bc.getBlockReplication());
-    this.bc = from.bc;
-  }
-
-  public BlockCollection getBlockCollection() {
-    return bc;
-  }
-
-  public void setBlockCollection(BlockCollection bc) {
-    this.bc = bc;
-  }
-
-  public DatanodeDescriptor getDatanode(int index) {
-    DatanodeStorageInfo storage = getStorageInfo(index);
-    return storage == null ? null : storage.getDatanodeDescriptor();
-  }
-
-  DatanodeStorageInfo getStorageInfo(int index) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-    return (DatanodeStorageInfo)triplets[index*3];
-  }
-
-  private BlockInfoContiguous getPrevious(int index) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
-    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1];
-    assert info == null || 
-        info.getClass().getName().startsWith(BlockInfoContiguous.class.getName()) :
-              "BlockInfo is expected at " + index*3;
-    return info;
-  }
-
-  BlockInfoContiguous getNext(int index) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
-    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2];
-    assert info == null || info.getClass().getName().startsWith(
-        BlockInfoContiguous.class.getName()) :
-        "BlockInfo is expected at " + index*3;
-    return info;
-  }
-
-  private void setStorageInfo(int index, DatanodeStorageInfo storage) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-    triplets[index*3] = storage;
-  }
-
-  /**
-   * Return the previous block on the block list for the datanode at
-   * position index. Set the previous block on the list to "to".
-   *
-   * @param index - the datanode index
-   * @param to - block to be set to previous on the list of blocks
-   * @return current previous block on the list of blocks
-   */
-  private BlockInfoContiguous setPrevious(int index, BlockInfoContiguous to) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
-    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1];
-    triplets[index*3+1] = to;
-    return info;
-  }
-
-  /**
-   * Return the next block on the block list for the datanode at
-   * position index. Set the next block on the list to "to".
-   *
-   * @param index - the datanode index
-   * @param to - block to be set to next on the list of blocks
-   *    * @return current next block on the list of blocks
-   */
-  private BlockInfoContiguous setNext(int index, BlockInfoContiguous to) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
-    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2];
-    triplets[index*3+2] = to;
-    return info;
-  }
-
-  public int getCapacity() {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
-    return triplets.length / 3;
+    this(from, from.getBlockCollection().getBlockReplication());
+    this.setBlockCollection(from.getBlockCollection());
   }
 
   /**
@@ -168,9 +53,10 @@ public class BlockInfoContiguous extends Block
   private int ensureCapacity(int num) {
     assert this.triplets != null : "BlockInfo is not initialized";
     int last = numNodes();
-    if(triplets.length >= (last+num)*3)
+    if (triplets.length >= (last+num)*3) {
       return last;
-    /* Not enough space left. Create a new array. Should normally 
+    }
+    /* Not enough space left. Create a new array. Should normally
      * happen only when replication is manually increased by the user. */
     Object[] old = triplets;
     triplets = new Object[(last+num)*3];
@@ -178,23 +64,8 @@ public class BlockInfoContiguous extends Block
     return last;
   }
 
-  /**
-   * Count the number of data-nodes the block belongs to.
-   */
-  public int numNodes() {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
-    for(int idx = getCapacity()-1; idx >= 0; idx--) {
-      if(getDatanode(idx) != null)
-        return idx+1;
-    }
-    return 0;
-  }
-
-  /**
-   * Add a {@link DatanodeStorageInfo} location for a block
-   */
-  boolean addStorage(DatanodeStorageInfo storage) {
+  @Override
+  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
     // find the last null node
     int lastNode = ensureCapacity(1);
     setStorageInfo(lastNode, storage);
@@ -203,167 +74,53 @@ public class BlockInfoContiguous extends Block
     return true;
   }
 
-  /**
-   * Remove {@link DatanodeStorageInfo} location for a block
-   */
+  @Override
   boolean removeStorage(DatanodeStorageInfo storage) {
     int dnIndex = findStorageInfo(storage);
-    if(dnIndex < 0) // the node is not found
+    if (dnIndex < 0) { // the node is not found
       return false;
-    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
-      "Block is still in the list and must be removed first.";
+    }
+    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
+        "Block is still in the list and must be removed first.";
     // find the last not null node
-    int lastNode = numNodes()-1; 
-    // replace current node triplet by the lastNode one 
+    int lastNode = numNodes()-1;
+    // replace current node triplet by the lastNode one
     setStorageInfo(dnIndex, getStorageInfo(lastNode));
-    setNext(dnIndex, getNext(lastNode)); 
-    setPrevious(dnIndex, getPrevious(lastNode)); 
+    setNext(dnIndex, getNext(lastNode));
+    setPrevious(dnIndex, getPrevious(lastNode));
     // set the last triplet to null
     setStorageInfo(lastNode, null);
-    setNext(lastNode, null); 
-    setPrevious(lastNode, null); 
+    setNext(lastNode, null);
+    setPrevious(lastNode, null);
     return true;
   }
 
-  /**
-   * Find specified DatanodeDescriptor.
-   * @return index or -1 if not found.
-   */
-  boolean findDatanode(DatanodeDescriptor dn) {
-    int len = getCapacity();
-    for(int idx = 0; idx < len; idx++) {
-      DatanodeDescriptor cur = getDatanode(idx);
-      if(cur == dn) {
-        return true;
-      }
-      if(cur == null) {
-        break;
-      }
-    }
-    return false;
-  }
+  @Override
+  public int numNodes() {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
 
-  /**
-   * Find specified DatanodeStorageInfo.
-   * @return DatanodeStorageInfo or null if not found.
-   */
-  DatanodeStorageInfo findStorageInfo(DatanodeDescriptor dn) {
-    int len = getCapacity();
-    for(int idx = 0; idx < len; idx++) {
-      DatanodeStorageInfo cur = getStorageInfo(idx);
-      if(cur == null)
-        break;
-      if(cur.getDatanodeDescriptor() == dn)
-        return cur;
-    }
-    return null;
-  }
-  
-  /**
-   * Find specified DatanodeStorageInfo.
-   * @return index or -1 if not found.
-   */
-  int findStorageInfo(DatanodeStorageInfo storageInfo) {
-    int len = getCapacity();
-    for(int idx = 0; idx < len; idx++) {
-      DatanodeStorageInfo cur = getStorageInfo(idx);
-      if (cur == storageInfo) {
-        return idx;
-      }
-      if (cur == null) {
-        break;
+    for (int idx = getCapacity()-1; idx >= 0; idx--) {
+      if (getDatanode(idx) != null) {
+        return idx + 1;
       }
     }
-    return -1;
-  }
-
-  /**
-   * Insert this block into the head of the list of blocks 
-   * related to the specified DatanodeStorageInfo.
-   * If the head is null then form a new list.
-   * @return current block as the new head of the list.
-   */
-  BlockInfoContiguous listInsert(BlockInfoContiguous head,
-      DatanodeStorageInfo storage) {
-    int dnIndex = this.findStorageInfo(storage);
-    assert dnIndex >= 0 : "Data node is not found: current";
-    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
-            "Block is already in the list and cannot be inserted.";
-    this.setPrevious(dnIndex, null);
-    this.setNext(dnIndex, head);
-    if(head != null)
-      head.setPrevious(head.findStorageInfo(storage), this);
-    return this;
-  }
-
-  /**
-   * Remove this block from the list of blocks 
-   * related to the specified DatanodeStorageInfo.
-   * If this block is the head of the list then return the next block as 
-   * the new head.
-   * @return the new head of the list or null if the list becomes
-   * empy after deletion.
-   */
-  BlockInfoContiguous listRemove(BlockInfoContiguous head,
-      DatanodeStorageInfo storage) {
-    if(head == null)
-      return null;
-    int dnIndex = this.findStorageInfo(storage);
-    if(dnIndex < 0) // this block is not on the data-node list
-      return head;
-
-    BlockInfoContiguous next = this.getNext(dnIndex);
-    BlockInfoContiguous prev = this.getPrevious(dnIndex);
-    this.setNext(dnIndex, null);
-    this.setPrevious(dnIndex, null);
-    if(prev != null)
-      prev.setNext(prev.findStorageInfo(storage), next);
-    if(next != null)
-      next.setPrevious(next.findStorageInfo(storage), prev);
-    if(this == head)  // removing the head
-      head = next;
-    return head;
+    return 0;
   }
 
-  /**
-   * Remove this block from the list of blocks related to the specified
-   * DatanodeDescriptor. Insert it into the head of the list of blocks.
-   *
-   * @return the new head of the list.
-   */
-  public BlockInfoContiguous moveBlockToHead(BlockInfoContiguous head,
-      DatanodeStorageInfo storage, int curIndex, int headIndex) {
-    if (head == this) {
-      return this;
-    }
-    BlockInfoContiguous next = this.setNext(curIndex, head);
-    BlockInfoContiguous prev = this.setPrevious(curIndex, null);
-
-    head.setPrevious(headIndex, this);
-    prev.setNext(prev.findStorageInfo(storage), next);
-    if (next != null) {
-      next.setPrevious(next.findStorageInfo(storage), prev);
+  @Override
+  void replaceBlock(BlockInfo newBlock) {
+    assert newBlock instanceof BlockInfoContiguous;
+    for (int i = this.numNodes() - 1; i >= 0; i--) {
+      final DatanodeStorageInfo storage = this.getStorageInfo(i);
+      final boolean removed = storage.removeBlock(this);
+      assert removed : "currentBlock not found.";
+
+      final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
+          newBlock, newBlock);
+      assert result == DatanodeStorageInfo.AddBlockResult.ADDED :
+          "newBlock already exists.";
     }
-    return this;
-  }
-
-  /**
-   * BlockInfo represents a block that is not being constructed.
-   * In order to start modifying the block, the BlockInfo should be converted
-   * to {@link BlockInfoContiguousUnderConstruction}.
-   * @return {@link BlockUCState#COMPLETE}
-   */
-  public BlockUCState getBlockUCState() {
-    return BlockUCState.COMPLETE;
-  }
-
-  /**
-   * Is this block complete?
-   * 
-   * @return true if the state of the block is {@link BlockUCState#COMPLETE}
-   */
-  public boolean isComplete() {
-    return getBlockUCState().equals(BlockUCState.COMPLETE);
   }
 
   /**
@@ -375,38 +132,16 @@ public class BlockInfoContiguous extends Block
     if(isComplete()) {
       BlockInfoContiguousUnderConstruction ucBlock =
           new BlockInfoContiguousUnderConstruction(this,
-          getBlockCollection().getBlockReplication(), s, targets);
+              getBlockCollection().getBlockReplication(), s, targets);
       ucBlock.setBlockCollection(getBlockCollection());
       return ucBlock;
     }
     // the block is already under construction
     BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction)this;
+        (BlockInfoContiguousUnderConstruction) this;
     ucBlock.setBlockUCState(s);
     ucBlock.setExpectedLocations(targets);
     ucBlock.setBlockCollection(getBlockCollection());
     return ucBlock;
   }
-
-  @Override
-  public int hashCode() {
-    // Super implementation is sufficient
-    return super.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    // Sufficient to rely on super's implementation
-    return (this == obj) || super.equals(obj);
-  }
-
-  @Override
-  public LightWeightGSet.LinkedElement getNext() {
-    return nextLinkedElement;
-  }
-
-  @Override
-  public void setNext(LightWeightGSet.LinkedElement next) {
-    this.nextLinkedElement = next;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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 ae809a5..e75c5d7 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
@@ -60,101 +60,6 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   private Block truncateBlock;
 
   /**
-   * ReplicaUnderConstruction contains information about replicas while
-   * they are under construction.
-   * The GS, the length and the state of the replica is as reported by 
-   * the data-node.
-   * It is not guaranteed, but expected, that data-nodes actually have
-   * corresponding replicas.
-   */
-  static class ReplicaUnderConstruction extends Block {
-    private final DatanodeStorageInfo expectedLocation;
-    private ReplicaState state;
-    private boolean chosenAsPrimary;
-
-    ReplicaUnderConstruction(Block block,
-                             DatanodeStorageInfo target,
-                             ReplicaState state) {
-      super(block);
-      this.expectedLocation = target;
-      this.state = state;
-      this.chosenAsPrimary = false;
-    }
-
-    /**
-     * Expected block replica location as assigned when the block was allocated.
-     * This defines the pipeline order.
-     * It is not guaranteed, but expected, that the data-node actually has
-     * the replica.
-     */
-    private DatanodeStorageInfo getExpectedStorageLocation() {
-      return expectedLocation;
-    }
-
-    /**
-     * Get replica state as reported by the data-node.
-     */
-    ReplicaState getState() {
-      return state;
-    }
-
-    /**
-     * Whether the replica was chosen for recovery.
-     */
-    boolean getChosenAsPrimary() {
-      return chosenAsPrimary;
-    }
-
-    /**
-     * Set replica state.
-     */
-    void setState(ReplicaState s) {
-      state = s;
-    }
-
-    /**
-     * Set whether this replica was chosen for recovery.
-     */
-    void setChosenAsPrimary(boolean chosenAsPrimary) {
-      this.chosenAsPrimary = chosenAsPrimary;
-    }
-
-    /**
-     * Is data-node the replica belongs to alive.
-     */
-    boolean isAlive() {
-      return expectedLocation.getDatanodeDescriptor().isAlive;
-    }
-
-    @Override // Block
-    public int hashCode() {
-      return super.hashCode();
-    }
-
-    @Override // Block
-    public boolean equals(Object obj) {
-      // Sufficient to rely on super's implementation
-      return (this == obj) || super.equals(obj);
-    }
-
-    @Override
-    public String toString() {
-      final StringBuilder b = new StringBuilder(50);
-      appendStringTo(b);
-      return b.toString();
-    }
-    
-    @Override
-    public void appendStringTo(StringBuilder sb) {
-      sb.append("ReplicaUC[")
-        .append(expectedLocation)
-        .append("|")
-        .append(state)
-        .append("]");
-    }
-  }
-
-  /**
    * Create block and set its state to
    * {@link BlockUCState#UNDER_CONSTRUCTION}.
    */
@@ -165,7 +70,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   /**
    * Create a block that is currently being constructed.
    */
-  public BlockInfoContiguousUnderConstruction(Block blk, short replication, BlockUCState state, DatanodeStorageInfo[] targets) {
+  public BlockInfoContiguousUnderConstruction(Block blk, short replication,
+      BlockUCState state, DatanodeStorageInfo[] targets) {
     super(blk, replication);
     assert getBlockUCState() != BlockUCState.COMPLETE :
       "BlockInfoUnderConstruction cannot be in COMPLETE state";
@@ -191,10 +97,11 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   /** Set expected locations */
   public void setExpectedLocations(DatanodeStorageInfo[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
-    this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
-    for(int i = 0; i < numLocations; i++)
-      replicas.add(
-        new ReplicaUnderConstruction(this, targets[i], ReplicaState.RBW));
+    this.replicas = new ArrayList<>(numLocations);
+    for(int i = 0; i < numLocations; i++) {
+      replicas.add(new ReplicaUnderConstruction(this, targets[i],
+          ReplicaState.RBW));
+    }
   }
 
   /**
@@ -204,8 +111,9 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   public DatanodeStorageInfo[] getExpectedStorageLocations() {
     int numLocations = replicas == null ? 0 : replicas.size();
     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
-    for(int i = 0; i < numLocations; i++)
+    for (int i = 0; i < numLocations; i++) {
       storages[i] = replicas.get(i).getExpectedStorageLocation();
+    }
     return storages;
   }
 
@@ -293,17 +201,17 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
         + " No blocks found, lease removed.");
     }
     boolean allLiveReplicasTriedAsPrimary = true;
-    for (int i = 0; i < replicas.size(); i++) {
+    for (ReplicaUnderConstruction replica : replicas) {
       // Check if all replicas have been tried or not.
-      if (replicas.get(i).isAlive()) {
-        allLiveReplicasTriedAsPrimary =
-            (allLiveReplicasTriedAsPrimary && replicas.get(i).getChosenAsPrimary());
+      if (replica.isAlive()) {
+        allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary &&
+            replica.getChosenAsPrimary());
       }
     }
     if (allLiveReplicasTriedAsPrimary) {
       // Just set all the replicas to be chosen whether they are alive or not.
-      for (int i = 0; i < replicas.size(); i++) {
-        replicas.get(i).setChosenAsPrimary(false);
+      for (ReplicaUnderConstruction replica : replicas) {
+        replica.setChosenAsPrimary(false);
       }
     }
     long mostRecentLastUpdate = 0;
@@ -315,7 +223,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
         continue;
       }
       final ReplicaUnderConstruction ruc = replicas.get(i);
-      final long lastUpdate = ruc.getExpectedStorageLocation().getDatanodeDescriptor().getLastUpdate(); 
+      final long lastUpdate = ruc.getExpectedStorageLocation()
+          .getDatanodeDescriptor().getLastUpdate();
       if (lastUpdate > mostRecentLastUpdate) {
         primaryNodeIndex = i;
         primary = ruc;
@@ -323,7 +232,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
       }
     }
     if (primary != null) {
-      primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
+      primary.getExpectedStorageLocation().getDatanodeDescriptor()
+          .addBlockToBeRecovered(this);
       primary.setChosenAsPrimary(true);
       NameNode.blockStateChangeLog.info(
           "BLOCK* {} recovery started, primary={}", this, primary);
@@ -356,18 +266,6 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     replicas.add(new ReplicaUnderConstruction(block, storage, rState));
   }
 
-  @Override // BlockInfo
-  // BlockInfoUnderConstruction participates in maps the same way as BlockInfo
-  public int hashCode() {
-    return super.hashCode();
-  }
-
-  @Override // BlockInfo
-  public boolean equals(Object obj) {
-    // Sufficient to rely on super's implementation
-    return (this == obj) || super.equals(obj);
-  }
-
   @Override
   public String toString() {
     final StringBuilder b = new StringBuilder(100);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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
new file mode 100644
index 0000000..5fff41e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -0,0 +1,179 @@
+/**
+ * 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;
+
+/**
+ * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
+ *
+ * We still use triplets to store DatanodeStorageInfo for each block in the
+ * block group, as well as the previous/next block in the corresponding
+ * DatanodeStorageInfo. For a (m+k) block group, the first (m+k) triplet units
+ * are sorted and strictly mapped to the corresponding block.
+ *
+ * Normally each block belonging to group is stored in only one DataNode.
+ * However, it is possible that some block is over-replicated. Thus the triplet
+ * array's size can be larger than (m+k). Thus currently we use an extra byte
+ * array to record the block index for each triplet.
+ */
+public class BlockInfoStriped extends BlockInfo {
+  private final short dataBlockNum;
+  private final short parityBlockNum;
+  /**
+   * Always the same size with triplets. Record the block index for each triplet
+   * TODO: actually this is only necessary for over-replicated block. Thus can
+   * be further optimized to save memory usage.
+   */
+  private byte[] indices;
+
+  public BlockInfoStriped(Block blk, short dataBlockNum, short parityBlockNum) {
+    super(blk, (short) (dataBlockNum + parityBlockNum));
+    indices = new byte[dataBlockNum + parityBlockNum];
+    initIndices();
+    this.dataBlockNum = dataBlockNum;
+    this.parityBlockNum = parityBlockNum;
+  }
+
+  BlockInfoStriped(BlockInfoStriped b) {
+    this(b, b.dataBlockNum, b.parityBlockNum);
+    this.setBlockCollection(b.getBlockCollection());
+  }
+
+  private short getTotalBlockNum() {
+    return (short) (dataBlockNum + parityBlockNum);
+  }
+
+  private void initIndices() {
+    for (int i = 0; i < indices.length; i++) {
+      indices[i] = -1;
+    }
+  }
+
+  private int findSlot() {
+    int i = getTotalBlockNum();
+    for (; i < getCapacity(); i++) {
+      if (getStorageInfo(i) == null) {
+        return i;
+      }
+    }
+    // need to expand the triplet size
+    ensureCapacity(i + 1, true);
+    return i;
+  }
+
+  @Override
+  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
+    int blockIndex = BlockIdManager.getBlockIndex(reportedBlock);
+    int index = blockIndex;
+    DatanodeStorageInfo old = getStorageInfo(index);
+    if (old != null && !old.equals(storage)) { // over replicated
+      // check if the storage has been stored
+      int i = findStorageInfo(storage);
+      if (i == -1) {
+        index = findSlot();
+      } else {
+        return true;
+      }
+    }
+    addStorage(storage, index, blockIndex);
+    return true;
+  }
+
+  private void addStorage(DatanodeStorageInfo storage, int index,
+      int blockIndex) {
+    setStorageInfo(index, storage);
+    setNext(index, null);
+    setPrevious(index, null);
+    indices[index] = (byte) blockIndex;
+  }
+
+  private int findStorageInfoFromEnd(DatanodeStorageInfo storage) {
+    final int len = getCapacity();
+    for(int idx = len - 1; idx >= 0; idx--) {
+      DatanodeStorageInfo cur = getStorageInfo(idx);
+      if (storage.equals(cur)) {
+        return idx;
+      }
+    }
+    return -1;
+  }
+
+  @Override
+  boolean removeStorage(DatanodeStorageInfo storage) {
+    int dnIndex = findStorageInfoFromEnd(storage);
+    if (dnIndex < 0) { // the node is not found
+      return false;
+    }
+    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
+        "Block is still in the list and must be removed first.";
+    // set the triplet to null
+    setStorageInfo(dnIndex, null);
+    setNext(dnIndex, null);
+    setPrevious(dnIndex, null);
+    indices[dnIndex] = -1;
+    return true;
+  }
+
+  private void ensureCapacity(int totalSize, boolean keepOld) {
+    if (getCapacity() < totalSize) {
+      Object[] old = triplets;
+      byte[] oldIndices = indices;
+      triplets = new Object[totalSize * 3];
+      indices = new byte[totalSize];
+      initIndices();
+
+      if (keepOld) {
+        System.arraycopy(old, 0, triplets, 0, old.length);
+        System.arraycopy(oldIndices, 0, indices, 0, oldIndices.length);
+      }
+    }
+  }
+
+  @Override
+  void replaceBlock(BlockInfo newBlock) {
+    assert newBlock instanceof BlockInfoStriped;
+    BlockInfoStriped newBlockGroup = (BlockInfoStriped) newBlock;
+    final int size = getCapacity();
+    newBlockGroup.ensureCapacity(size, false);
+    for (int i = 0; i < size; i++) {
+      final DatanodeStorageInfo storage = this.getStorageInfo(i);
+      if (storage != null) {
+        final int blockIndex = indices[i];
+        final boolean removed = storage.removeBlock(this);
+        assert removed : "currentBlock not found.";
+
+        newBlockGroup.addStorage(storage, i, blockIndex);
+        storage.insertToList(newBlockGroup);
+      }
+    }
+  }
+
+  @Override
+  public int numNodes() {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
+    int num = 0;
+    for (int idx = getCapacity()-1; idx >= 0; idx--) {
+      if (getStorageInfo(idx) != null) {
+        num++;
+      }
+    }
+    return num;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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 ec07a5f..d9f2772 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
@@ -595,8 +595,8 @@ public class BlockManager {
    * of replicas reported from data-nodes.
    */
   private static boolean commitBlock(
-      final BlockInfoContiguousUnderConstruction block, final Block commitBlock)
-      throws IOException {
+      final BlockInfoContiguousUnderConstruction block,
+      final Block commitBlock) throws IOException {
     if (block.getBlockUCState() == BlockUCState.COMMITTED)
       return false;
     assert block.getNumBytes() <= commitBlock.getNumBytes() :
@@ -627,7 +627,7 @@ public class BlockManager {
       return false; // already completed (e.g. by syncBlock)
     
     final boolean b = commitBlock(
-        (BlockInfoContiguousUnderConstruction) lastBlock, commitBlock);
+        (BlockInfoContiguousUnderConstruction)lastBlock, commitBlock);
     if(countNodes(lastBlock).liveReplicas() >= minReplication)
       completeBlock(bc, bc.numBlocks()-1, false);
     return b;
@@ -640,15 +640,16 @@ public class BlockManager {
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  private BlockInfoContiguous completeBlock(final BlockCollection bc,
+  private BlockInfo completeBlock(final BlockCollection bc,
       final int blkIndex, boolean force) throws IOException {
     if(blkIndex < 0)
       return null;
     BlockInfoContiguous curBlock = bc.getBlocks()[blkIndex];
-    if(curBlock.isComplete())
+    if (curBlock.isComplete())
       return curBlock;
+    // TODO: support BlockInfoStripedUC
     BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction) curBlock;
+        (BlockInfoContiguousUnderConstruction)curBlock;
     int numNodes = ucBlock.numNodes();
     if (!force && numNodes < minReplication)
       throw new IOException("Cannot complete block: " +
@@ -674,13 +675,15 @@ public class BlockManager {
     return blocksMap.replaceBlock(completeBlock);
   }
 
-  private BlockInfoContiguous completeBlock(final BlockCollection bc,
-      final BlockInfoContiguous block, boolean force) throws IOException {
+  // TODO: support BlockInfoStrippedUC
+  private BlockInfo completeBlock(final BlockCollection bc,
+      final BlockInfo block, boolean force) throws IOException {
     BlockInfoContiguous[] fileBlocks = bc.getBlocks();
-    for(int idx = 0; idx < fileBlocks.length; idx++)
-      if(fileBlocks[idx] == block) {
+    for (int idx = 0; idx < fileBlocks.length; idx++) {
+      if (fileBlocks[idx] == block) {
         return completeBlock(bc, idx, force);
       }
+    }
     return block;
   }
   
@@ -689,7 +692,7 @@ public class BlockManager {
    * regardless of whether enough replicas are present. This is necessary
    * when tailing edit logs as a Standby.
    */
-  public BlockInfoContiguous forceCompleteBlock(final BlockCollection bc,
+  public BlockInfo forceCompleteBlock(final BlockCollection bc,
       final BlockInfoContiguousUnderConstruction block) throws IOException {
     block.commitBlock(block);
     return completeBlock(bc, block, true);
@@ -721,8 +724,8 @@ public class BlockManager {
 
     DatanodeStorageInfo[] targets = getStorages(oldBlock);
 
-    BlockInfoContiguousUnderConstruction ucBlock =
-        bc.setLastBlock(oldBlock, targets);
+    BlockInfoContiguousUnderConstruction ucBlock = bc.setLastBlock(oldBlock,
+        targets);
     blocksMap.replaceBlock(ucBlock);
 
     // Remove block from replication queue.
@@ -1022,7 +1025,7 @@ public class BlockManager {
     if(numBlocks == 0) {
       return new BlocksWithLocations(new BlockWithLocations[0]);
     }
-    Iterator<BlockInfoContiguous> iter = node.getBlockIterator();
+    Iterator<BlockInfo> iter = node.getBlockIterator();
     int startBlock = DFSUtil.getRandom().nextInt(numBlocks); // starting from a random block
     // skip blocks
     for(int i=0; i<startBlock; i++) {
@@ -1030,7 +1033,7 @@ public class BlockManager {
     }
     List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
     long totalSize = 0;
-    BlockInfoContiguous curBlock;
+    BlockInfo curBlock;
     while(totalSize<size && iter.hasNext()) {
       curBlock = iter.next();
       if(!curBlock.isComplete())  continue;
@@ -1129,7 +1132,8 @@ public class BlockManager {
   public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
       final DatanodeInfo dn, String storageID, String reason) throws IOException {
     assert namesystem.hasWriteLock();
-    final BlockInfoContiguous storedBlock = getStoredBlock(blk.getLocalBlock());
+    final Block reportedBlock = blk.getLocalBlock();
+    final BlockInfo storedBlock = getStoredBlock(reportedBlock);
     if (storedBlock == null) {
       // Check if the replica is in the blockMap, if not
       // ignore the request for now. This could happen when BlockScanner
@@ -1146,7 +1150,7 @@ public class BlockManager {
           + ") does not exist");
     }
     
-    markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock,
+    markBlockAsCorrupt(new BlockToMarkCorrupt(reportedBlock, storedBlock,
             blk.getGenerationStamp(), reason, Reason.CORRUPTION_REPORTED),
         storageID == null ? null : node.getStorageInfo(storageID),
         node);
@@ -1172,7 +1176,7 @@ public class BlockManager {
 
     // Add replica to the data-node if it is not already there
     if (storageInfo != null) {
-      storageInfo.addBlock(b.stored);
+      storageInfo.addBlock(b.stored, b.reportedBlock);
     }
 
     // Add this replica to corruptReplicas Map
@@ -1717,41 +1721,55 @@ public class BlockManager {
       this.reportedState = reportedState;
     }
   }
-  
+
+  private static class BlockInfoToAdd {
+    final BlockInfo stored;
+    final Block reported;
+
+    BlockInfoToAdd(BlockInfo stored, Block reported) {
+      this.stored = stored;
+      this.reported = reported;
+    }
+  }
+
   /**
    * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a
    * list of blocks that should be considered corrupt due to a block report.
    */
   private static class BlockToMarkCorrupt {
     /** The corrupted block in a datanode. */
-    final BlockInfoContiguous corrupted;
+    final BlockInfo corrupted;
     /** The corresponding block stored in the BlockManager. */
-    final BlockInfoContiguous stored;
+    final BlockInfo stored;
+    /** The block reported from a datanode */
+    final Block reportedBlock;
     /** The reason to mark corrupt. */
     final String reason;
     /** The reason code to be stored */
     final Reason reasonCode;
 
-    BlockToMarkCorrupt(BlockInfoContiguous corrupted,
-        BlockInfoContiguous stored, String reason,
-        Reason reasonCode) {
+    BlockToMarkCorrupt(Block reported, BlockInfo corrupted,
+        BlockInfo stored, String reason, Reason reasonCode) {
+      Preconditions.checkNotNull(reported, "reported is null");
       Preconditions.checkNotNull(corrupted, "corrupted is null");
       Preconditions.checkNotNull(stored, "stored is null");
 
+      this.reportedBlock = reported;
       this.corrupted = corrupted;
       this.stored = stored;
       this.reason = reason;
       this.reasonCode = reasonCode;
     }
 
-    BlockToMarkCorrupt(BlockInfoContiguous stored, String reason,
+    BlockToMarkCorrupt(Block reported, BlockInfo stored, String reason,
         Reason reasonCode) {
-      this(stored, stored, reason, reasonCode);
+      this(reported, stored, stored, reason, reasonCode);
     }
 
-    BlockToMarkCorrupt(BlockInfoContiguous stored, long gs, String reason,
-        Reason reasonCode) {
-      this(new BlockInfoContiguous(stored), stored, reason, reasonCode);
+    BlockToMarkCorrupt(Block reported, BlockInfo stored, long gs,
+        String reason, Reason reasonCode) {
+      this(reported, BlockInfo.copyOf(stored), stored, reason,
+          reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
     }
@@ -1876,7 +1894,7 @@ public class BlockManager {
           break;
         }
 
-        BlockInfoContiguous bi = getStoredBlock(b);
+        BlockInfo bi = getStoredBlock(b);
         if (bi == null) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
@@ -1916,7 +1934,7 @@ public class BlockManager {
     // Modify the (block-->datanode) map, according to the difference
     // between the old and new block report.
     //
-    Collection<BlockInfoContiguous> toAdd = new LinkedList<BlockInfoContiguous>();
+    Collection<BlockInfoToAdd> toAdd = new LinkedList<>();
     Collection<Block> toRemove = new TreeSet<Block>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
@@ -1933,8 +1951,9 @@ public class BlockManager {
       removeStoredBlock(b, node);
     }
     int numBlocksLogged = 0;
-    for (BlockInfoContiguous b : toAdd) {
-      addStoredBlock(b, storageInfo, null, numBlocksLogged < maxNumBlocksToLog);
+    for (BlockInfoToAdd b : toAdd) {
+      addStoredBlock(b.stored, b.reported, storageInfo, null,
+          numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
@@ -1979,7 +1998,7 @@ public class BlockManager {
         continue;
       }
       
-      BlockInfoContiguous storedBlock = getStoredBlock(iblk);
+      BlockInfo storedBlock = getStoredBlock(iblk);
       // If block does not belong to any file, we are done.
       if (storedBlock == null) continue;
       
@@ -2002,7 +2021,7 @@ public class BlockManager {
       
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-        ((BlockInfoContiguousUnderConstruction)storedBlock)
+        ((BlockInfoContiguousUnderConstruction) storedBlock)
             .addReplicaIfNotPresent(storageInfo, iblk, reportedState);
         // OpenFileBlocks only inside snapshots also will be added to safemode
         // threshold. So we need to update such blocks to safemode
@@ -2017,14 +2036,14 @@ public class BlockManager {
       }      
       //add replica if appropriate
       if (reportedState == ReplicaState.FINALIZED) {
-        addStoredBlockImmediate(storedBlock, storageInfo);
+        addStoredBlockImmediate(storedBlock, iblk, storageInfo);
       }
     }
   }
 
   private void reportDiff(DatanodeStorageInfo storageInfo, 
       BlockListAsLongs newReport, 
-      Collection<BlockInfoContiguous> toAdd,              // add to DatanodeDescriptor
+      Collection<BlockInfoToAdd> toAdd,     // add to DatanodeDescriptor
       Collection<Block> toRemove,           // remove from DatanodeDescriptor
       Collection<Block> toInvalidate,       // should be removed from DN
       Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
@@ -2032,8 +2051,10 @@ public class BlockManager {
 
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
-    BlockInfoContiguous delimiter = new BlockInfoContiguous(new Block(), (short) 1);
-    AddBlockResult result = storageInfo.addBlock(delimiter);
+    Block delimiterBlock = new Block();
+    BlockInfoContiguous delimiter = new BlockInfoContiguous(delimiterBlock,
+        (short) 1);
+    AddBlockResult result = storageInfo.addBlock(delimiter, delimiterBlock);
     assert result == AddBlockResult.ADDED 
         : "Delimiting block cannot be present in the node";
     int headIndex = 0; //currently the delimiter is in the head of the list
@@ -2045,7 +2066,7 @@ public class BlockManager {
     // scan the report and process newly reported blocks
     for (BlockReportReplica iblk : newReport) {
       ReplicaState iState = iblk.getState();
-      BlockInfoContiguous storedBlock = processReportedBlock(storageInfo,
+      BlockInfo storedBlock = processReportedBlock(storageInfo,
           iblk, iState, toAdd, toInvalidate, toCorrupt, toUC);
 
       // move block to the head of the list
@@ -2057,8 +2078,7 @@ public class BlockManager {
 
     // collect blocks that have not been reported
     // all of them are next to the delimiter
-    Iterator<BlockInfoContiguous> it =
-        storageInfo.new BlockIterator(delimiter.getNext(0));
+    Iterator<BlockInfo> it = storageInfo.new BlockIterator(delimiter.getNext(0));
     while(it.hasNext())
       toRemove.add(it.next());
     storageInfo.removeBlock(delimiter);
@@ -2095,10 +2115,10 @@ public class BlockManager {
    * @return the up-to-date stored block, if it should be kept.
    *         Otherwise, null.
    */
-  private BlockInfoContiguous processReportedBlock(
+  private BlockInfo processReportedBlock(
       final DatanodeStorageInfo storageInfo,
       final Block block, final ReplicaState reportedState, 
-      final Collection<BlockInfoContiguous> toAdd,
+      final Collection<BlockInfoToAdd> toAdd,
       final Collection<Block> toInvalidate, 
       final Collection<BlockToMarkCorrupt> toCorrupt,
       final Collection<StatefulBlockInfo> toUC) {
@@ -2119,7 +2139,7 @@ public class BlockManager {
     }
     
     // find block by blockId
-    BlockInfoContiguous storedBlock = getStoredBlock(block);
+    BlockInfo storedBlock = getStoredBlock(block);
     if(storedBlock == null) {
       // If blocksMap does not contain reported block id,
       // the replica should be removed from the data-node.
@@ -2173,7 +2193,7 @@ public class BlockManager {
     if (reportedState == ReplicaState.FINALIZED
         && (storedBlock.findStorageInfo(storageInfo) == -1 ||
             corruptReplicas.isReplicaCorrupt(storedBlock, dn))) {
-      toAdd.add(storedBlock);
+      toAdd.add(new BlockInfoToAdd(storedBlock, block));
     }
     return storedBlock;
   }
@@ -2251,7 +2271,7 @@ public class BlockManager {
    */
   private BlockToMarkCorrupt checkReplicaCorrupt(
       Block reported, ReplicaState reportedState, 
-      BlockInfoContiguous storedBlock, BlockUCState ucState,
+      BlockInfo storedBlock, BlockUCState ucState,
       DatanodeDescriptor dn) {
     switch(reportedState) {
     case FINALIZED:
@@ -2260,12 +2280,12 @@ public class BlockManager {
       case COMMITTED:
         if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
           final long reportedGS = reported.getGenerationStamp();
-          return new BlockToMarkCorrupt(storedBlock, reportedGS,
+          return new BlockToMarkCorrupt(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(storedBlock,
+          return new BlockToMarkCorrupt(reported, storedBlock,
               "block is " + ucState + " and reported length " +
               reported.getNumBytes() + " does not match " +
               "length in block map " + storedBlock.getNumBytes(),
@@ -2276,8 +2296,8 @@ public class BlockManager {
       case UNDER_CONSTRUCTION:
         if (storedBlock.getGenerationStamp() > reported.getGenerationStamp()) {
           final long reportedGS = reported.getGenerationStamp();
-          return new BlockToMarkCorrupt(storedBlock, reportedGS, "block is "
-              + ucState + " and reported state " + reportedState
+          return new BlockToMarkCorrupt(reported, storedBlock, reportedGS,
+              "block is " + ucState + " and reported state " + reportedState
               + ", But reported genstamp " + reportedGS
               + " does not match genstamp in block map "
               + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
@@ -2292,7 +2312,7 @@ public class BlockManager {
         return null; // not corrupt
       } else if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
         final long reportedGS = reported.getGenerationStamp();
-        return new BlockToMarkCorrupt(storedBlock, reportedGS,
+        return new BlockToMarkCorrupt(reported, storedBlock, reportedGS,
             "reported " + reportedState + " replica with genstamp " + reportedGS
             + " does not match COMPLETE block's genstamp in block map "
             + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
@@ -2307,7 +2327,7 @@ public class BlockManager {
               "complete with the same genstamp");
           return null;
         } else {
-          return new BlockToMarkCorrupt(storedBlock,
+          return new BlockToMarkCorrupt(reported, storedBlock,
               "reported replica has invalid state " + reportedState,
               Reason.INVALID_STATE);
         }
@@ -2320,11 +2340,12 @@ 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(storedBlock, msg, Reason.INVALID_STATE);
+      return new BlockToMarkCorrupt(reported, storedBlock, msg,
+          Reason.INVALID_STATE);
     }
   }
 
-  private boolean isBlockUnderConstruction(BlockInfoContiguous storedBlock,
+  private boolean isBlockUnderConstruction(BlockInfo storedBlock,
       BlockUCState ucState, ReplicaState reportedState) {
     switch(reportedState) {
     case FINALIZED:
@@ -2353,7 +2374,7 @@ public class BlockManager {
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
         !block.findDatanode(storageInfo.getDatanodeDescriptor())) {
-      addStoredBlock(block, storageInfo, null, true);
+      addStoredBlock(block, ucBlock.reportedBlock, storageInfo, null, true);
     }
   } 
 
@@ -2368,18 +2389,18 @@ public class BlockManager {
    * 
    * @throws IOException
    */
-  private void addStoredBlockImmediate(BlockInfoContiguous storedBlock,
+  private void addStoredBlockImmediate(BlockInfo storedBlock, Block reported,
       DatanodeStorageInfo storageInfo)
   throws IOException {
     assert (storedBlock != null && namesystem.hasWriteLock());
     if (!namesystem.isInStartupSafeMode() 
         || namesystem.isPopulatingReplQueues()) {
-      addStoredBlock(storedBlock, storageInfo, null, false);
+      addStoredBlock(storedBlock, reported, storageInfo, null, false);
       return;
     }
 
     // just add it
-    storageInfo.addBlock(storedBlock);
+    storageInfo.addBlock(storedBlock, reported);
 
     // Now check for completion of blocks and safe block count
     int numCurrentReplica = countLiveNodes(storedBlock);
@@ -2400,13 +2421,14 @@ public class BlockManager {
    * needed replications if this takes care of the problem.
    * @return the block that is stored in blockMap.
    */
-  private Block addStoredBlock(final BlockInfoContiguous block,
+  private Block addStoredBlock(final BlockInfo block,
+                               final Block reportedBlock,
                                DatanodeStorageInfo storageInfo,
                                DatanodeDescriptor delNodeHint,
                                boolean logEveryBlock)
   throws IOException {
     assert block != null && namesystem.hasWriteLock();
-    BlockInfoContiguous storedBlock;
+    BlockInfo storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     if (block instanceof BlockInfoContiguousUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
@@ -2427,7 +2449,7 @@ public class BlockManager {
     assert bc != null : "Block must belong to a file";
 
     // add block to the datanode
-    AddBlockResult result = storageInfo.addBlock(storedBlock);
+    AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock);
 
     int curReplicaDelta;
     if (result == AddBlockResult.ADDED) {
@@ -2502,13 +2524,13 @@ public class BlockManager {
           storedBlock + "blockMap has " + numCorruptNodes + 
           " but corrupt replicas map has " + corruptReplicasCount);
     }
-    if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication))
-      invalidateCorruptReplicas(storedBlock);
+    if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) {
+      invalidateCorruptReplicas(storedBlock, reportedBlock);
+    }
     return storedBlock;
   }
 
-  private void logAddStoredBlock(BlockInfoContiguous storedBlock,
-      DatanodeDescriptor node) {
+  private void logAddStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) {
     if (!blockLog.isInfoEnabled()) {
       return;
     }
@@ -2535,7 +2557,7 @@ public class BlockManager {
    *
    * @param blk Block whose corrupt replicas need to be invalidated
    */
-  private void invalidateCorruptReplicas(BlockInfoContiguous blk) {
+  private void invalidateCorruptReplicas(BlockInfo blk, Block reported) {
     Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
     boolean removedFromBlocksMap = true;
     if (nodes == null)
@@ -2545,7 +2567,7 @@ public class BlockManager {
     DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
     for (DatanodeDescriptor node : nodesCopy) {
       try {
-        if (!invalidateBlock(new BlockToMarkCorrupt(blk, null,
+        if (!invalidateBlock(new BlockToMarkCorrupt(reported, blk, null,
               Reason.ANY), node)) {
           removedFromBlocksMap = false;
         }
@@ -2614,7 +2636,7 @@ public class BlockManager {
     long nrInvalid = 0, nrOverReplicated = 0;
     long nrUnderReplicated = 0, nrPostponed = 0, nrUnderConstruction = 0;
     long startTimeMisReplicatedScan = Time.now();
-    Iterator<BlockInfoContiguous> blocksItr = blocksMap.getBlocks().iterator();
+    Iterator<BlockInfo> blocksItr = blocksMap.getBlocks().iterator();
     long totalBlocks = blocksMap.size();
     replicationQueuesInitProgress = 0;
     long totalProcessed = 0;
@@ -2626,7 +2648,7 @@ public class BlockManager {
       namesystem.writeLockInterruptibly();
       try {
         while (processed < numBlocksPerIteration && blocksItr.hasNext()) {
-          BlockInfoContiguous block = blocksItr.next();
+          BlockInfo block = blocksItr.next();
           MisReplicationResult res = processMisReplicatedBlock(block);
           if (LOG.isTraceEnabled()) {
             LOG.trace("block " + block + ": " + res);
@@ -2700,7 +2722,7 @@ public class BlockManager {
    * appropriate queues if necessary, and returns a result code indicating
    * what happened with it.
    */
-  private MisReplicationResult processMisReplicatedBlock(BlockInfoContiguous block) {
+  private MisReplicationResult processMisReplicatedBlock(BlockInfo block) {
     BlockCollection bc = block.getBlockCollection();
     if (bc == null) {
       // block does not belong to any file
@@ -3029,14 +3051,14 @@ public class BlockManager {
       ReplicaState reportedState, DatanodeDescriptor delHintNode)
       throws IOException {
     // blockReceived reports a finalized block
-    Collection<BlockInfoContiguous> toAdd = new LinkedList<BlockInfoContiguous>();
+    Collection<BlockInfoToAdd> toAdd = new LinkedList<>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
     final DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
 
-    processReportedBlock(storageInfo, block, reportedState,
-                              toAdd, toInvalidate, toCorrupt, toUC);
+    processReportedBlock(storageInfo, block, reportedState, toAdd, toInvalidate,
+        toCorrupt, toUC);
     // the block is only in one of the to-do lists
     // if it is in none then data-node already has it
     assert toUC.size() + toAdd.size() + toInvalidate.size() + toCorrupt.size() <= 1
@@ -3046,8 +3068,9 @@ public class BlockManager {
       addStoredBlockUnderConstruction(b, storageInfo);
     }
     long numBlocksLogged = 0;
-    for (BlockInfoContiguous b : toAdd) {
-      addStoredBlock(b, storageInfo, delHintNode, numBlocksLogged < maxNumBlocksToLog);
+    for (BlockInfoToAdd b : toAdd) {
+      addStoredBlock(b.stored, b.reported, storageInfo, delHintNode,
+          numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
@@ -3170,7 +3193,7 @@ public class BlockManager {
    * @param b - the block being tested
    * @return count of live nodes for this block
    */
-  int countLiveNodes(BlockInfoContiguous b) {
+  int countLiveNodes(BlockInfo b) {
     if (!namesystem.isInStartupSafeMode()) {
       return countNodes(b).liveReplicas();
     }
@@ -3244,7 +3267,7 @@ public class BlockManager {
     return blocksMap.size();
   }
 
-  public DatanodeStorageInfo[] getStorages(BlockInfoContiguous block) {
+  public DatanodeStorageInfo[] getStorages(BlockInfo block) {
     final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()];
     int i = 0;
     for(DatanodeStorageInfo s : blocksMap.getStorages(block)) {
@@ -3274,8 +3297,8 @@ public class BlockManager {
     }
   }
 
-  public BlockInfoContiguous getStoredBlock(Block block) {
-    BlockInfoContiguous info = null;
+  public BlockInfo getStoredBlock(Block block) {
+    BlockInfo info = null;
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
       info = blocksMap.getStoredBlock(
           new Block(BlockIdManager.convertToGroupID(block.getBlockId())));
@@ -3432,7 +3455,8 @@ public class BlockManager {
 
   public BlockInfoContiguous addBlockCollection(BlockInfoContiguous block,
       BlockCollection bc) {
-    return blocksMap.addBlockCollection(block, bc);
+    // TODO
+    return (BlockInfoContiguous) blocksMap.addBlockCollection(block, bc);
   }
 
   public BlockCollection getBlockCollection(Block b) {
@@ -3640,7 +3664,7 @@ public class BlockManager {
 
   /**
    * A simple result enum for the result of
-   * {@link BlockManager#processMisReplicatedBlock(BlockInfoContiguous)}.
+   * {@link BlockManager#processMisReplicatedBlock}.
    */
   enum MisReplicationResult {
     /** The block should be invalidated since it belongs to a deleted file. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index 806a4cb..d383de8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@ -20,12 +20,10 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.util.Iterator;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 
@@ -36,10 +34,10 @@ import com.google.common.collect.Iterables;
  */
 class BlocksMap {
   private static class StorageIterator implements Iterator<DatanodeStorageInfo> {
-    private final BlockInfoContiguous blockInfo;
+    private final BlockInfo blockInfo;
     private int nextIdx = 0;
       
-    StorageIterator(BlockInfoContiguous blkInfo) {
+    StorageIterator(BlockInfo blkInfo) {
       this.blockInfo = blkInfo;
     }
 
@@ -63,14 +61,14 @@ class BlocksMap {
   /** Constant {@link LightWeightGSet} capacity. */
   private final int capacity;
   
-  private GSet<Block, BlockInfoContiguous> blocks;
+  private GSet<Block, BlockInfo> blocks;
 
   BlocksMap(int capacity) {
     // Use 2% of total memory to size the GSet capacity
     this.capacity = capacity;
-    this.blocks = new LightWeightGSet<Block, BlockInfoContiguous>(capacity) {
+    this.blocks = new LightWeightGSet<Block, BlockInfo>(capacity) {
       @Override
-      public Iterator<BlockInfoContiguous> iterator() {
+      public Iterator<BlockInfo> iterator() {
         SetIterator iterator = new SetIterator();
         /*
          * Not tracking any modifications to set. As this set will be used
@@ -97,15 +95,15 @@ class BlocksMap {
   }
 
   BlockCollection getBlockCollection(Block b) {
-    BlockInfoContiguous info = blocks.get(b);
+    BlockInfo info = blocks.get(b);
     return (info != null) ? info.getBlockCollection() : null;
   }
 
   /**
    * Add block b belonging to the specified block collection to the map.
    */
-  BlockInfoContiguous addBlockCollection(BlockInfoContiguous b, BlockCollection bc) {
-    BlockInfoContiguous info = blocks.get(b);
+  BlockInfo addBlockCollection(BlockInfo b, BlockCollection bc) {
+    BlockInfo info = blocks.get(b);
     if (info != b) {
       info = b;
       blocks.put(info);
@@ -120,11 +118,12 @@ class BlocksMap {
    * and remove all data-node locations associated with the block.
    */
   void removeBlock(Block block) {
-    BlockInfoContiguous blockInfo = blocks.remove(block);
+    BlockInfo blockInfo = blocks.remove(block);
     if (blockInfo == null)
       return;
 
     blockInfo.setBlockCollection(null);
+    // TODO: fix this logic for block group
     for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
       dn.removeBlock(blockInfo); // remove from the list and wipe the location
@@ -132,7 +131,7 @@ class BlocksMap {
   }
   
   /** Returns the block object it it exists in the map. */
-  BlockInfoContiguous getStoredBlock(Block b) {
+  BlockInfo getStoredBlock(Block b) {
     return blocks.get(b);
   }
 
@@ -164,7 +163,7 @@ class BlocksMap {
    * For a block that has already been retrieved from the BlocksMap
    * returns {@link Iterable} of the storages the block belongs to.
    */
-  Iterable<DatanodeStorageInfo> getStorages(final BlockInfoContiguous storedBlock) {
+  Iterable<DatanodeStorageInfo> getStorages(final BlockInfo storedBlock) {
     return new Iterable<DatanodeStorageInfo>() {
       @Override
       public Iterator<DatanodeStorageInfo> iterator() {
@@ -175,7 +174,7 @@ class BlocksMap {
 
   /** counts number of containing nodes. Better than using iterator. */
   int numNodes(Block b) {
-    BlockInfoContiguous info = blocks.get(b);
+    BlockInfo info = blocks.get(b);
     return info == null ? 0 : info.numNodes();
   }
 
@@ -185,7 +184,7 @@ class BlocksMap {
    * only if it does not belong to any file and data-nodes.
    */
   boolean removeNode(Block b, DatanodeDescriptor node) {
-    BlockInfoContiguous info = blocks.get(b);
+    BlockInfo info = blocks.get(b);
     if (info == null)
       return false;
 
@@ -203,7 +202,7 @@ class BlocksMap {
     return blocks.size();
   }
 
-  Iterable<BlockInfoContiguous> getBlocks() {
+  Iterable<BlockInfo> getBlocks() {
     return blocks;
   }
   
@@ -218,20 +217,11 @@ class BlocksMap {
    * @param newBlock - block for replacement
    * @return new block
    */
-  BlockInfoContiguous replaceBlock(BlockInfoContiguous newBlock) {
-    BlockInfoContiguous currentBlock = blocks.get(newBlock);
+  BlockInfo replaceBlock(BlockInfo newBlock) {
+    BlockInfo currentBlock = blocks.get(newBlock);
     assert currentBlock != null : "the block if not in blocksMap";
     // replace block in data-node lists
-    for (int i = currentBlock.numNodes() - 1; i >= 0; i--) {
-      final DatanodeDescriptor dn = currentBlock.getDatanode(i);
-      final DatanodeStorageInfo storage = currentBlock.findStorageInfo(dn);
-      final boolean removed = storage.removeBlock(currentBlock);
-      Preconditions.checkState(removed, "currentBlock not found.");
-
-      final AddBlockResult result = storage.addBlock(newBlock);
-      Preconditions.checkState(result == AddBlockResult.ADDED,
-          "newBlock already exists.");
-    }
+    currentBlock.replaceBlock(newBlock);
     // replace block in the map itself
     blocks.put(newBlock);
     return newBlock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
index bf5ece9..79d7713 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
@@ -513,8 +513,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
           iter.remove();
         }
       }
-      BlockInfoContiguous blockInfo = blockManager.
-            getStoredBlock(new Block(cblock.getBlockId()));
+      BlockInfoContiguous blockInfo = namesystem.getStoredBlock(new Block(cblock.getBlockId()));
       String reason = findReasonForNotCaching(cblock, blockInfo);
       int neededCached = 0;
       if (reason != null) {
@@ -628,8 +627,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       List<DatanodeDescriptor> pendingCached) {
     // To figure out which replicas can be cached, we consult the
     // blocksMap.  We don't want to try to cache a corrupt replica, though.
-    BlockInfoContiguous blockInfo = blockManager.
-          getStoredBlock(new Block(cachedBlock.getBlockId()));
+    BlockInfoContiguous blockInfo = namesystem.getStoredBlock(new Block(cachedBlock.getBlockId()));
     if (blockInfo == null) {
       LOG.debug("Block {}: can't add new cached replicas," +
           " because there is no record of this block " +
@@ -668,7 +666,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       while (it.hasNext()) {
         CachedBlock cBlock = it.next();
         BlockInfoContiguous info =
-            blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
+            namesystem.getStoredBlock(new Block(cBlock.getBlockId()));
         if (info != null) {
           pendingBytes -= info.getNumBytes();
         }
@@ -678,7 +676,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       while (it.hasNext()) {
         CachedBlock cBlock = it.next();
         BlockInfoContiguous info =
-            blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
+            namesystem.getStoredBlock(new Block(cBlock.getBlockId()));
         if (info != null) {
           pendingBytes += info.getNumBytes();
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10488c9c/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 c0a17b1..a54d46b 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
@@ -286,7 +286,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Remove block from the list of blocks belonging to the data-node. Remove
    * data-node from the block.
    */
-  boolean removeBlock(BlockInfoContiguous b) {
+  boolean removeBlock(BlockInfo b) {
     final DatanodeStorageInfo s = b.findStorageInfo(this);
     // if block exists on this datanode
     if (s != null) {
@@ -299,12 +299,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Remove block from the list of blocks belonging to the data-node. Remove
    * data-node from the block.
    */
-  boolean removeBlock(String storageID, BlockInfoContiguous b) {
+  boolean removeBlock(String storageID, BlockInfo b) {
     DatanodeStorageInfo s = getStorageInfo(storageID);
-    if (s != null) {
-      return s.removeBlock(b);
-    }
-    return false;
+    return s != null && s.removeBlock(b);
   }
 
   public void resetBlocks() {
@@ -482,12 +479,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  private static class BlockIterator implements Iterator<BlockInfoContiguous> {
+  private static class BlockIterator implements Iterator<BlockInfo> {
     private int index = 0;
-    private final List<Iterator<BlockInfoContiguous>> iterators;
+    private final List<Iterator<BlockInfo>> iterators;
     
     private BlockIterator(final DatanodeStorageInfo... storages) {
-      List<Iterator<BlockInfoContiguous>> iterators = new ArrayList<Iterator<BlockInfoContiguous>>();
+      List<Iterator<BlockInfo>> iterators = new ArrayList<>();
       for (DatanodeStorageInfo e : storages) {
         iterators.add(e.getBlockIterator());
       }
@@ -501,7 +498,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
 
     @Override
-    public BlockInfoContiguous next() {
+    public BlockInfo next() {
       update();
       return iterators.get(index).next();
     }
@@ -518,10 +515,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  Iterator<BlockInfoContiguous> getBlockIterator() {
+  Iterator<BlockInfo> getBlockIterator() {
     return new BlockIterator(getStorageInfos());
   }
-  Iterator<BlockInfoContiguous> getBlockIterator(final String storageID) {
+
+  Iterator<BlockInfo> getBlockIterator(final String storageID) {
     return new BlockIterator(getStorageInfo(storageID));
   }
 


[25/50] [abbrv] hadoop git commit: HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)

Posted by zh...@apache.org.
HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)


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

Branch: refs/heads/HDFS-7285
Commit: bc9cb3e271b22069a15ca110cd60c860250aaab2
Parents: 79426f3
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Sat Mar 14 22:36:46 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Sat Mar 14 22:36:46 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 23 ++++-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  2 +
 .../datatransfer/DataTransferProtocol.java      |  5 +-
 .../hdfs/protocol/datatransfer/Receiver.java    |  2 +-
 .../hdfs/protocol/datatransfer/Sender.java      |  4 +-
 .../hdfs/server/datanode/DataXceiver.java       | 95 ++++++++++++--------
 .../server/datanode/ShortCircuitRegistry.java   | 13 ++-
 .../src/main/proto/datatransfer.proto           | 11 +++
 .../shortcircuit/TestShortCircuitCache.java     | 63 +++++++++++++
 10 files changed, 178 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c3f9367..93237af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1154,6 +1154,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7903. Cannot recover block after truncate and delete snapshot.
     (Plamen Jeliazkov via shv)
 
+    HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and
+    fail to tell the DFSClient about it because of a network error (cmccabe)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
index ba48c79..1e915b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
+
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -69,6 +71,12 @@ import com.google.common.base.Preconditions;
 public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
 
+  public static class FailureInjector {
+    public void injectRequestFileDescriptorsFailure() throws IOException {
+      // do nothing
+    }
+  }
+
   @VisibleForTesting
   static ShortCircuitReplicaCreator
       createShortCircuitReplicaInfoCallback = null;
@@ -76,6 +84,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   private final DFSClient.Conf conf;
 
   /**
+   * Injects failures into specific operations during unit tests.
+   */
+  private final FailureInjector failureInjector;
+
+  /**
    * The file name, for logging and debugging purposes.
    */
   private String fileName;
@@ -169,6 +182,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
 
   public BlockReaderFactory(DFSClient.Conf conf) {
     this.conf = conf;
+    this.failureInjector = conf.brfFailureInjector;
     this.remainingCacheTries = conf.nCachedConnRetry;
   }
 
@@ -518,11 +532,12 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
     SlotId slotId = slot == null ? null : slot.getSlotId();
-    new Sender(out).requestShortCircuitFds(block, token, slotId, 1);
+    new Sender(out).requestShortCircuitFds(block, token, slotId, 1, true);
     DataInputStream in = new DataInputStream(peer.getInputStream());
     BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
         PBHelper.vintPrefixed(in));
     DomainSocket sock = peer.getDomainSocket();
+    failureInjector.injectRequestFileDescriptorsFailure();
     switch (resp.getStatus()) {
     case SUCCESS:
       byte buf[] = new byte[1];
@@ -532,8 +547,13 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       try {
         ExtendedBlockId key =
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+        if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
+          LOG.trace("Sending receipt verification byte for slot " + slot);
+          sock.getOutputStream().write(0);
+        }
         replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
             Time.monotonicNow(), slot);
+        return new ShortCircuitReplicaInfo(replica);
       } catch (IOException e) {
         // This indicates an error reading from disk, or a format error.  Since
         // it's not a socket communication problem, we return null rather than
@@ -545,7 +565,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
           IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
         }
       }
-      return new ShortCircuitReplicaInfo(replica);
     case ERROR_UNSUPPORTED:
       if (!resp.hasShortCircuitAccessVersion()) {
         LOG.warn("short-circuit read access is disabled for " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/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 aac7b51..f970fef 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
@@ -337,6 +337,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     final long shortCircuitCacheStaleThresholdMs;
 
     final long keyProviderCacheExpiryMs;
+    public BlockReaderFactory.FailureInjector brfFailureInjector =
+      new BlockReaderFactory.FailureInjector();
 
     public Conf(Configuration conf) {
       // The hdfsTimeout is currently the same as the ipc timeout 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
index 4be42a8..48e931d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
@@ -138,10 +138,13 @@ public interface DataTransferProtocol {
    *                          to use no slot id.
    * @param maxVersion      Maximum version of the block data the client 
    *                          can understand.
+   * @param supportsReceiptVerification  True if the client supports
+   *                          receipt verification.
    */
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion) throws IOException;
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException;
 
   /**
    * Release a pair of short-circuit FDs requested earlier.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
index 7994027..31bdc5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
@@ -186,7 +186,7 @@ public abstract class Receiver implements DataTransferProtocol {
     try {
       requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
           PBHelper.convert(proto.getHeader().getToken()),
-          slotId, proto.getMaxVersion());
+          slotId, proto.getMaxVersion(), true);
     } finally {
       if (traceScope != null) traceScope.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index 7fea33e..df69125 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
@@ -181,7 +181,8 @@ public class Sender implements DataTransferProtocol {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion) throws IOException {
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException {
     OpRequestShortCircuitAccessProto.Builder builder =
         OpRequestShortCircuitAccessProto.newBuilder()
           .setHeader(DataTransferProtoUtil.buildBaseHeader(
@@ -189,6 +190,7 @@ public class Sender implements DataTransferProtocol {
     if (slotId != null) {
       builder.setSlotId(PBHelper.convert(slotId));
     }
+    builder.setSupportsReceiptVerification(supportsReceiptVerification);
     OpRequestShortCircuitAccessProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index e9547a8..84504fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -22,6 +22,8 @@ import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ER
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_INVALID;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.DO_NOT_USE_RECEIPT_VERIFICATION;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 import static org.apache.hadoop.util.Time.now;
 
@@ -291,64 +293,83 @@ class DataXceiver extends Receiver implements Runnable {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> token,
-      SlotId slotId, int maxVersion) throws IOException {
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException {
     updateCurrentThreadName("Passing file descriptors for block " + blk);
     BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
     FileInputStream fis[] = null;
+    SlotId registeredSlotId = null;
+    boolean success = false;
     try {
-      if (peer.getDomainSocket() == null) {
-        throw new IOException("You cannot pass file descriptors over " +
-            "anything but a UNIX domain socket.");
-      }
-      if (slotId != null) {
-        boolean isCached = datanode.data.
-            isCached(blk.getBlockPoolId(), blk.getBlockId());
-        datanode.shortCircuitRegistry.registerSlot(
-            ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
-      }
       try {
-        fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
-      } finally {
-        if ((fis == null) && (slotId != null)) {
-          datanode.shortCircuitRegistry.unregisterSlot(slotId);
+        if (peer.getDomainSocket() == null) {
+          throw new IOException("You cannot pass file descriptors over " +
+              "anything but a UNIX domain socket.");
         }
+        if (slotId != null) {
+          boolean isCached = datanode.data.
+              isCached(blk.getBlockPoolId(), blk.getBlockId());
+          datanode.shortCircuitRegistry.registerSlot(
+              ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
+          registeredSlotId = slotId;
+        }
+        fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
+        Preconditions.checkState(fis != null);
+        bld.setStatus(SUCCESS);
+        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+      } catch (ShortCircuitFdsVersionException e) {
+        bld.setStatus(ERROR_UNSUPPORTED);
+        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+        bld.setMessage(e.getMessage());
+      } catch (ShortCircuitFdsUnsupportedException e) {
+        bld.setStatus(ERROR_UNSUPPORTED);
+        bld.setMessage(e.getMessage());
+      } catch (InvalidToken e) {
+        bld.setStatus(ERROR_ACCESS_TOKEN);
+        bld.setMessage(e.getMessage());
+      } catch (IOException e) {
+        bld.setStatus(ERROR);
+        bld.setMessage(e.getMessage());
       }
-      bld.setStatus(SUCCESS);
-      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-    } catch (ShortCircuitFdsVersionException e) {
-      bld.setStatus(ERROR_UNSUPPORTED);
-      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-      bld.setMessage(e.getMessage());
-    } catch (ShortCircuitFdsUnsupportedException e) {
-      bld.setStatus(ERROR_UNSUPPORTED);
-      bld.setMessage(e.getMessage());
-    } catch (InvalidToken e) {
-      bld.setStatus(ERROR_ACCESS_TOKEN);
-      bld.setMessage(e.getMessage());
-    } catch (IOException e) {
-      bld.setStatus(ERROR);
-      bld.setMessage(e.getMessage());
-    }
-    try {
       bld.build().writeDelimitedTo(socketOut);
       if (fis != null) {
         FileDescriptor fds[] = new FileDescriptor[fis.length];
         for (int i = 0; i < fds.length; i++) {
           fds[i] = fis[i].getFD();
         }
-        byte buf[] = new byte[] { (byte)0 };
-        peer.getDomainSocket().
-          sendFileDescriptors(fds, buf, 0, buf.length);
+        byte buf[] = new byte[1];
+        if (supportsReceiptVerification) {
+          buf[0] = (byte)USE_RECEIPT_VERIFICATION.getNumber();
+        } else {
+          buf[0] = (byte)DO_NOT_USE_RECEIPT_VERIFICATION.getNumber();
+        }
+        DomainSocket sock = peer.getDomainSocket();
+        sock.sendFileDescriptors(fds, buf, 0, buf.length);
+        if (supportsReceiptVerification) {
+          LOG.trace("Reading receipt verification byte for " + slotId);
+          int val = sock.getInputStream().read();
+          if (val < 0) {
+            throw new EOFException();
+          }
+        } else {
+          LOG.trace("Receipt verification is not enabled on the DataNode.  " +
+                    "Not verifying " + slotId);
+        }
+        success = true;
       }
     } finally {
+      if ((!success) && (registeredSlotId != null)) {
+        LOG.info("Unregistering " + registeredSlotId + " because the " +
+            "requestShortCircuitFdsForRead operation failed.");
+        datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId);
+      }
       if (ClientTraceLog.isInfoEnabled()) {
         DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk
             .getBlockPoolId());
         BlockSender.ClientTraceLog.info(String.format(
             "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
             " blockid: %s, srvID: %s, success: %b",
-            blk.getBlockId(), dnR.getDatanodeUuid(), (fis != null)
-          ));
+            blk.getBlockId(), dnR.getDatanodeUuid(), success));
       }
       if (fis != null) {
         IOUtils.cleanup(LOG, fis);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
index 32906f4..b32c0d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
@@ -30,6 +30,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -83,7 +84,7 @@ public class ShortCircuitRegistry {
 
   private static final int SHM_LENGTH = 8192;
 
-  private static class RegisteredShm extends ShortCircuitShm
+  public static class RegisteredShm extends ShortCircuitShm
       implements DomainSocketWatcher.Handler {
     private final String clientName;
     private final ShortCircuitRegistry registry;
@@ -383,4 +384,14 @@ public class ShortCircuitRegistry {
     }
     IOUtils.closeQuietly(watcher);
   }
+
+  public static interface Visitor {
+    void accept(HashMap<ShmId, RegisteredShm> segments,
+                HashMultimap<ExtendedBlockId, Slot> slots);
+  }
+
+  @VisibleForTesting
+  public synchronized void visit(Visitor visitor) {
+    visitor.accept(segments, slots);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
index d72bb5e1..8426198 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
@@ -179,6 +179,12 @@ message OpRequestShortCircuitAccessProto {
    * The shared memory slot to use, if we are using one.
    */
   optional ShortCircuitShmSlotProto slotId = 3;
+
+  /**
+   * True if the client supports verifying that the file descriptor has been
+   * sent successfully.
+   */
+  optional bool supportsReceiptVerification = 4 [default = false];
 }
 
 message ReleaseShortCircuitAccessRequestProto {
@@ -230,6 +236,11 @@ enum Status {
   IN_PROGRESS = 12;
 }
 
+enum ShortCircuitFdResponse {
+  DO_NOT_USE_RECEIPT_VERIFICATION = 0;
+  USE_RECEIPT_VERIFICATION = 1;
+}
+
 message PipelineAckProto {
   required sint64 seqno = 1;
   repeated uint32 reply = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9cb3e2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
index bfa871c..7daabd0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
@@ -36,13 +36,16 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
+import com.google.common.collect.HashMultimap;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.BlockReaderTestUtil;
+import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -52,11 +55,14 @@ import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.RegisteredShm;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.PerDatanodeVisitorInfo;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.Visitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.CacheVisitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -615,4 +621,61 @@ public class TestShortCircuitCache {
     cluster.shutdown();
     sockDir.close();
   }
+
+  public static class TestCleanupFailureInjector
+        extends BlockReaderFactory.FailureInjector {
+    @Override
+    public void injectRequestFileDescriptorsFailure() throws IOException {
+      throw new IOException("injected I/O error");
+    }
+  }
+
+  // Regression test for HDFS-7915
+  @Test(timeout=60000)
+  public void testDataXceiverCleansUpSlotsOnFailure() throws Exception {
+    BlockReaderTestUtil.enableShortCircuitShmTracing();
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    Configuration conf = createShortCircuitConf(
+        "testDataXceiverCleansUpSlotsOnFailure", sockDir);
+    conf.setLong(DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
+        1000000000L);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    DistributedFileSystem fs = cluster.getFileSystem();
+    final Path TEST_PATH1 = new Path("/test_file1");
+    final Path TEST_PATH2 = new Path("/test_file2");
+    final int TEST_FILE_LEN = 4096;
+    final int SEED = 0xFADE1;
+    DFSTestUtil.createFile(fs, TEST_PATH1, TEST_FILE_LEN,
+        (short)1, SEED);
+    DFSTestUtil.createFile(fs, TEST_PATH2, TEST_FILE_LEN,
+        (short)1, SEED);
+
+    // The first read should allocate one shared memory segment and slot.
+    DFSTestUtil.readFileBuffer(fs, TEST_PATH1);
+
+    // The second read should fail, and we should only have 1 segment and 1 slot
+    // left.
+    fs.getClient().getConf().brfFailureInjector =
+        new TestCleanupFailureInjector();
+    try {
+      DFSTestUtil.readFileBuffer(fs, TEST_PATH2);
+    } catch (Throwable t) {
+      GenericTestUtils.assertExceptionContains("TCP reads were disabled for " +
+          "testing, but we failed to do a non-TCP read.", t);
+    }
+    ShortCircuitRegistry registry =
+      cluster.getDataNodes().get(0).getShortCircuitRegistry();
+    registry.visit(new ShortCircuitRegistry.Visitor() {
+      @Override
+      public void accept(HashMap<ShmId, RegisteredShm> segments,
+                         HashMultimap<ExtendedBlockId, Slot> slots) {
+        Assert.assertEquals(1, segments.size());
+        Assert.assertEquals(1, slots.size());
+      }
+    });
+    cluster.shutdown();
+    sockDir.close();
+  }
 }


[18/50] [abbrv] hadoop git commit: HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)

Posted by zh...@apache.org.
HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and fail to tell the DFSClient about it because of a network error (cmccabe)


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

Branch: refs/heads/HDFS-7285
Commit: 5aa892ed486d42ae6b94c4866b92cd2b382ea640
Parents: 6fdef76
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Mar 13 18:29:49 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Mar 13 18:29:49 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 23 ++++-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  2 +
 .../datatransfer/DataTransferProtocol.java      |  5 +-
 .../hdfs/protocol/datatransfer/Receiver.java    |  2 +-
 .../hdfs/protocol/datatransfer/Sender.java      |  4 +-
 .../hdfs/server/datanode/DataXceiver.java       | 95 ++++++++++++--------
 .../server/datanode/ShortCircuitRegistry.java   | 13 ++-
 .../src/main/proto/datatransfer.proto           | 11 +++
 .../shortcircuit/TestShortCircuitCache.java     | 63 +++++++++++++
 10 files changed, 178 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c3f9367..ff00b0c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1177,6 +1177,9 @@ Release 2.7.0 - UNRELEASED
       HDFS-7722. DataNode#checkDiskError should also remove Storage when error
       is found. (Lei Xu via Colin P. McCabe)
 
+      HDFS-7915. The DataNode can sometimes allocate a ShortCircuitShm slot and
+      fail to tell the DFSClient about it because of a network error (cmccabe)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
index ba48c79..1e915b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
+
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -69,6 +71,12 @@ import com.google.common.base.Preconditions;
 public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
 
+  public static class FailureInjector {
+    public void injectRequestFileDescriptorsFailure() throws IOException {
+      // do nothing
+    }
+  }
+
   @VisibleForTesting
   static ShortCircuitReplicaCreator
       createShortCircuitReplicaInfoCallback = null;
@@ -76,6 +84,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   private final DFSClient.Conf conf;
 
   /**
+   * Injects failures into specific operations during unit tests.
+   */
+  private final FailureInjector failureInjector;
+
+  /**
    * The file name, for logging and debugging purposes.
    */
   private String fileName;
@@ -169,6 +182,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
 
   public BlockReaderFactory(DFSClient.Conf conf) {
     this.conf = conf;
+    this.failureInjector = conf.brfFailureInjector;
     this.remainingCacheTries = conf.nCachedConnRetry;
   }
 
@@ -518,11 +532,12 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
     SlotId slotId = slot == null ? null : slot.getSlotId();
-    new Sender(out).requestShortCircuitFds(block, token, slotId, 1);
+    new Sender(out).requestShortCircuitFds(block, token, slotId, 1, true);
     DataInputStream in = new DataInputStream(peer.getInputStream());
     BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
         PBHelper.vintPrefixed(in));
     DomainSocket sock = peer.getDomainSocket();
+    failureInjector.injectRequestFileDescriptorsFailure();
     switch (resp.getStatus()) {
     case SUCCESS:
       byte buf[] = new byte[1];
@@ -532,8 +547,13 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       try {
         ExtendedBlockId key =
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+        if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
+          LOG.trace("Sending receipt verification byte for slot " + slot);
+          sock.getOutputStream().write(0);
+        }
         replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
             Time.monotonicNow(), slot);
+        return new ShortCircuitReplicaInfo(replica);
       } catch (IOException e) {
         // This indicates an error reading from disk, or a format error.  Since
         // it's not a socket communication problem, we return null rather than
@@ -545,7 +565,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
           IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
         }
       }
-      return new ShortCircuitReplicaInfo(replica);
     case ERROR_UNSUPPORTED:
       if (!resp.hasShortCircuitAccessVersion()) {
         LOG.warn("short-circuit read access is disabled for " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/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 aac7b51..f970fef 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
@@ -337,6 +337,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     final long shortCircuitCacheStaleThresholdMs;
 
     final long keyProviderCacheExpiryMs;
+    public BlockReaderFactory.FailureInjector brfFailureInjector =
+      new BlockReaderFactory.FailureInjector();
 
     public Conf(Configuration conf) {
       // The hdfsTimeout is currently the same as the ipc timeout 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
index 4be42a8..48e931d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
@@ -138,10 +138,13 @@ public interface DataTransferProtocol {
    *                          to use no slot id.
    * @param maxVersion      Maximum version of the block data the client 
    *                          can understand.
+   * @param supportsReceiptVerification  True if the client supports
+   *                          receipt verification.
    */
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion) throws IOException;
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException;
 
   /**
    * Release a pair of short-circuit FDs requested earlier.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
index 7994027..31bdc5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
@@ -186,7 +186,7 @@ public abstract class Receiver implements DataTransferProtocol {
     try {
       requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
           PBHelper.convert(proto.getHeader().getToken()),
-          slotId, proto.getMaxVersion());
+          slotId, proto.getMaxVersion(), true);
     } finally {
       if (traceScope != null) traceScope.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index 7fea33e..df69125 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
@@ -181,7 +181,8 @@ public class Sender implements DataTransferProtocol {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
-      SlotId slotId, int maxVersion) throws IOException {
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException {
     OpRequestShortCircuitAccessProto.Builder builder =
         OpRequestShortCircuitAccessProto.newBuilder()
           .setHeader(DataTransferProtoUtil.buildBaseHeader(
@@ -189,6 +190,7 @@ public class Sender implements DataTransferProtocol {
     if (slotId != null) {
       builder.setSlotId(PBHelper.convert(slotId));
     }
+    builder.setSupportsReceiptVerification(supportsReceiptVerification);
     OpRequestShortCircuitAccessProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index e9547a8..84504fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -22,6 +22,8 @@ import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ER
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_INVALID;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.DO_NOT_USE_RECEIPT_VERIFICATION;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 import static org.apache.hadoop.util.Time.now;
 
@@ -291,64 +293,83 @@ class DataXceiver extends Receiver implements Runnable {
   @Override
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> token,
-      SlotId slotId, int maxVersion) throws IOException {
+      SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
+        throws IOException {
     updateCurrentThreadName("Passing file descriptors for block " + blk);
     BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
     FileInputStream fis[] = null;
+    SlotId registeredSlotId = null;
+    boolean success = false;
     try {
-      if (peer.getDomainSocket() == null) {
-        throw new IOException("You cannot pass file descriptors over " +
-            "anything but a UNIX domain socket.");
-      }
-      if (slotId != null) {
-        boolean isCached = datanode.data.
-            isCached(blk.getBlockPoolId(), blk.getBlockId());
-        datanode.shortCircuitRegistry.registerSlot(
-            ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
-      }
       try {
-        fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
-      } finally {
-        if ((fis == null) && (slotId != null)) {
-          datanode.shortCircuitRegistry.unregisterSlot(slotId);
+        if (peer.getDomainSocket() == null) {
+          throw new IOException("You cannot pass file descriptors over " +
+              "anything but a UNIX domain socket.");
         }
+        if (slotId != null) {
+          boolean isCached = datanode.data.
+              isCached(blk.getBlockPoolId(), blk.getBlockId());
+          datanode.shortCircuitRegistry.registerSlot(
+              ExtendedBlockId.fromExtendedBlock(blk), slotId, isCached);
+          registeredSlotId = slotId;
+        }
+        fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
+        Preconditions.checkState(fis != null);
+        bld.setStatus(SUCCESS);
+        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+      } catch (ShortCircuitFdsVersionException e) {
+        bld.setStatus(ERROR_UNSUPPORTED);
+        bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
+        bld.setMessage(e.getMessage());
+      } catch (ShortCircuitFdsUnsupportedException e) {
+        bld.setStatus(ERROR_UNSUPPORTED);
+        bld.setMessage(e.getMessage());
+      } catch (InvalidToken e) {
+        bld.setStatus(ERROR_ACCESS_TOKEN);
+        bld.setMessage(e.getMessage());
+      } catch (IOException e) {
+        bld.setStatus(ERROR);
+        bld.setMessage(e.getMessage());
       }
-      bld.setStatus(SUCCESS);
-      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-    } catch (ShortCircuitFdsVersionException e) {
-      bld.setStatus(ERROR_UNSUPPORTED);
-      bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
-      bld.setMessage(e.getMessage());
-    } catch (ShortCircuitFdsUnsupportedException e) {
-      bld.setStatus(ERROR_UNSUPPORTED);
-      bld.setMessage(e.getMessage());
-    } catch (InvalidToken e) {
-      bld.setStatus(ERROR_ACCESS_TOKEN);
-      bld.setMessage(e.getMessage());
-    } catch (IOException e) {
-      bld.setStatus(ERROR);
-      bld.setMessage(e.getMessage());
-    }
-    try {
       bld.build().writeDelimitedTo(socketOut);
       if (fis != null) {
         FileDescriptor fds[] = new FileDescriptor[fis.length];
         for (int i = 0; i < fds.length; i++) {
           fds[i] = fis[i].getFD();
         }
-        byte buf[] = new byte[] { (byte)0 };
-        peer.getDomainSocket().
-          sendFileDescriptors(fds, buf, 0, buf.length);
+        byte buf[] = new byte[1];
+        if (supportsReceiptVerification) {
+          buf[0] = (byte)USE_RECEIPT_VERIFICATION.getNumber();
+        } else {
+          buf[0] = (byte)DO_NOT_USE_RECEIPT_VERIFICATION.getNumber();
+        }
+        DomainSocket sock = peer.getDomainSocket();
+        sock.sendFileDescriptors(fds, buf, 0, buf.length);
+        if (supportsReceiptVerification) {
+          LOG.trace("Reading receipt verification byte for " + slotId);
+          int val = sock.getInputStream().read();
+          if (val < 0) {
+            throw new EOFException();
+          }
+        } else {
+          LOG.trace("Receipt verification is not enabled on the DataNode.  " +
+                    "Not verifying " + slotId);
+        }
+        success = true;
       }
     } finally {
+      if ((!success) && (registeredSlotId != null)) {
+        LOG.info("Unregistering " + registeredSlotId + " because the " +
+            "requestShortCircuitFdsForRead operation failed.");
+        datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId);
+      }
       if (ClientTraceLog.isInfoEnabled()) {
         DatanodeRegistration dnR = datanode.getDNRegistrationForBP(blk
             .getBlockPoolId());
         BlockSender.ClientTraceLog.info(String.format(
             "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
             " blockid: %s, srvID: %s, success: %b",
-            blk.getBlockId(), dnR.getDatanodeUuid(), (fis != null)
-          ));
+            blk.getBlockId(), dnR.getDatanodeUuid(), success));
       }
       if (fis != null) {
         IOUtils.cleanup(LOG, fis);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
index 32906f4..b32c0d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
@@ -30,6 +30,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -83,7 +84,7 @@ public class ShortCircuitRegistry {
 
   private static final int SHM_LENGTH = 8192;
 
-  private static class RegisteredShm extends ShortCircuitShm
+  public static class RegisteredShm extends ShortCircuitShm
       implements DomainSocketWatcher.Handler {
     private final String clientName;
     private final ShortCircuitRegistry registry;
@@ -383,4 +384,14 @@ public class ShortCircuitRegistry {
     }
     IOUtils.closeQuietly(watcher);
   }
+
+  public static interface Visitor {
+    void accept(HashMap<ShmId, RegisteredShm> segments,
+                HashMultimap<ExtendedBlockId, Slot> slots);
+  }
+
+  @VisibleForTesting
+  public synchronized void visit(Visitor visitor) {
+    visitor.accept(segments, slots);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
index d72bb5e1..8426198 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
@@ -179,6 +179,12 @@ message OpRequestShortCircuitAccessProto {
    * The shared memory slot to use, if we are using one.
    */
   optional ShortCircuitShmSlotProto slotId = 3;
+
+  /**
+   * True if the client supports verifying that the file descriptor has been
+   * sent successfully.
+   */
+  optional bool supportsReceiptVerification = 4 [default = false];
 }
 
 message ReleaseShortCircuitAccessRequestProto {
@@ -230,6 +236,11 @@ enum Status {
   IN_PROGRESS = 12;
 }
 
+enum ShortCircuitFdResponse {
+  DO_NOT_USE_RECEIPT_VERIFICATION = 0;
+  USE_RECEIPT_VERIFICATION = 1;
+}
+
 message PipelineAckProto {
   required sint64 seqno = 1;
   repeated uint32 reply = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa892ed/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
index bfa871c..7daabd0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
@@ -36,13 +36,16 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
+import com.google.common.collect.HashMultimap;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.BlockReaderTestUtil;
+import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -52,11 +55,14 @@ import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.RegisteredShm;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.PerDatanodeVisitorInfo;
 import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.Visitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.CacheVisitor;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -615,4 +621,61 @@ public class TestShortCircuitCache {
     cluster.shutdown();
     sockDir.close();
   }
+
+  public static class TestCleanupFailureInjector
+        extends BlockReaderFactory.FailureInjector {
+    @Override
+    public void injectRequestFileDescriptorsFailure() throws IOException {
+      throw new IOException("injected I/O error");
+    }
+  }
+
+  // Regression test for HDFS-7915
+  @Test(timeout=60000)
+  public void testDataXceiverCleansUpSlotsOnFailure() throws Exception {
+    BlockReaderTestUtil.enableShortCircuitShmTracing();
+    TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
+    Configuration conf = createShortCircuitConf(
+        "testDataXceiverCleansUpSlotsOnFailure", sockDir);
+    conf.setLong(DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
+        1000000000L);
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    DistributedFileSystem fs = cluster.getFileSystem();
+    final Path TEST_PATH1 = new Path("/test_file1");
+    final Path TEST_PATH2 = new Path("/test_file2");
+    final int TEST_FILE_LEN = 4096;
+    final int SEED = 0xFADE1;
+    DFSTestUtil.createFile(fs, TEST_PATH1, TEST_FILE_LEN,
+        (short)1, SEED);
+    DFSTestUtil.createFile(fs, TEST_PATH2, TEST_FILE_LEN,
+        (short)1, SEED);
+
+    // The first read should allocate one shared memory segment and slot.
+    DFSTestUtil.readFileBuffer(fs, TEST_PATH1);
+
+    // The second read should fail, and we should only have 1 segment and 1 slot
+    // left.
+    fs.getClient().getConf().brfFailureInjector =
+        new TestCleanupFailureInjector();
+    try {
+      DFSTestUtil.readFileBuffer(fs, TEST_PATH2);
+    } catch (Throwable t) {
+      GenericTestUtils.assertExceptionContains("TCP reads were disabled for " +
+          "testing, but we failed to do a non-TCP read.", t);
+    }
+    ShortCircuitRegistry registry =
+      cluster.getDataNodes().get(0).getShortCircuitRegistry();
+    registry.visit(new ShortCircuitRegistry.Visitor() {
+      @Override
+      public void accept(HashMap<ShmId, RegisteredShm> segments,
+                         HashMultimap<ExtendedBlockId, Slot> slots) {
+        Assert.assertEquals(1, segments.size());
+        Assert.assertEquals(1, slots.size());
+      }
+    });
+    cluster.shutdown();
+    sockDir.close();
+  }
 }


[14/50] [abbrv] hadoop git commit: HDFS-7435. PB encoding of block reports is very inefficient. Contributed by Daryn Sharp.

Posted by zh...@apache.org.
HDFS-7435. PB encoding of block reports is very inefficient. Contributed by Daryn Sharp.


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

Branch: refs/heads/HDFS-7285
Commit: d324164a51a43d72c02567248bd9f0f12b244a40
Parents: f446669
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Mar 13 14:13:55 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Mar 13 14:23:37 2015 -0500

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hadoop/hdfs/protocol/BlockListAsLongs.java  | 660 +++++++++++--------
 .../DatanodeProtocolClientSideTranslatorPB.java |  22 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |  14 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |   6 +-
 .../server/blockmanagement/BlockManager.java    |  16 +-
 .../hdfs/server/datanode/BPServiceActor.java    |  13 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  20 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   2 +-
 .../server/protocol/DatanodeRegistration.java   |   9 +
 .../hdfs/server/protocol/NamespaceInfo.java     |  52 ++
 .../server/protocol/StorageBlockReport.java     |   8 +-
 .../src/main/proto/DatanodeProtocol.proto       |   2 +
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |   1 +
 .../hdfs/protocol/TestBlockListAsLongs.java     | 237 +++++++
 .../blockmanagement/TestBlockManager.java       |   8 +-
 .../server/datanode/BlockReportTestBase.java    |  27 +-
 .../server/datanode/SimulatedFSDataset.java     |  11 +-
 .../TestBlockHasMultipleReplicasOnSameDN.java   |   9 +-
 .../datanode/TestDataNodeVolumeFailure.java     |   4 +-
 ...TestDnRespectsBlockReportSplitThreshold.java |   2 +-
 .../extdataset/ExternalDatasetImpl.java         |   2 +-
 .../server/namenode/NNThroughputBenchmark.java  |  23 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |   3 +-
 .../hdfs/server/namenode/TestFSImage.java       |   2 +
 .../TestOfflineEditsViewer.java                 |   9 +-
 26 files changed, 811 insertions(+), 354 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 909182b..ac7e096 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -743,6 +743,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7491. Add incremental blockreport latency to DN metrics.
     (Ming Ma via cnauroth)
 
+    HDFS-7435. PB encoding of block reports is very inefficient.
+    (Daryn Sharp via kihwal)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
index 4389714..1c89ee4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
@@ -17,342 +17,458 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Random;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.Replica;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.CodedOutputStream;
 
-/**
- * This class provides an interface for accessing list of blocks that
- * has been implemented as long[].
- * This class is useful for block report. Rather than send block reports
- * as a Block[] we can send it as a long[].
- *
- * The structure of the array is as follows:
- * 0: the length of the finalized replica list;
- * 1: the length of the under-construction replica list;
- * - followed by finalized replica list where each replica is represented by
- *   3 longs: one for the blockId, one for the block length, and one for
- *   the generation stamp;
- * - followed by the invalid replica represented with three -1s;
- * - followed by the under-construction replica list where each replica is
- *   represented by 4 longs: three for the block id, length, generation 
- *   stamp, and the fourth for the replica state.
- */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class BlockListAsLongs implements Iterable<Block> {
+public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
+  private final static int CHUNK_SIZE = 64*1024; // 64K
+  private static long[] EMPTY_LONGS = new long[]{0, 0};
+
+  public static BlockListAsLongs EMPTY = new BlockListAsLongs() {
+    @Override
+    public int getNumberOfBlocks() {
+      return 0;
+    }
+    @Override
+    public ByteString getBlocksBuffer() {
+      return ByteString.EMPTY;
+    }
+    @Override
+    public long[] getBlockListAsLongs() {
+      return EMPTY_LONGS;
+    }
+    @Override
+    public Iterator<BlockReportReplica> iterator() {
+      return Collections.emptyIterator();
+    }
+  };
+
   /**
-   * A finalized block as 3 longs
-   *   block-id and block length and generation stamp
+   * Prepare an instance to in-place decode the given ByteString buffer
+   * @param numBlocks - blocks in the buffer
+   * @param blocksBuf - ByteString encoded varints
+   * @return BlockListAsLongs
    */
-  private static final int LONGS_PER_FINALIZED_BLOCK = 3;
+  public static BlockListAsLongs decodeBuffer(final int numBlocks,
+      final ByteString blocksBuf) {
+    return new BufferDecoder(numBlocks, blocksBuf);
+  }
 
   /**
-   * An under-construction block as 4 longs
-   *   block-id and block length, generation stamp and replica state
+   * Prepare an instance to in-place decode the given ByteString buffers
+   * @param numBlocks - blocks in the buffers
+   * @param blocksBufs - list of ByteString encoded varints
+   * @return BlockListAsLongs
    */
-  private static final int LONGS_PER_UC_BLOCK = 4;
-
-  /** Number of longs in the header */
-  private static final int HEADER_SIZE = 2;
+  public static BlockListAsLongs decodeBuffers(final int numBlocks,
+      final List<ByteString> blocksBufs) {
+    // this doesn't actually copy the data
+    return decodeBuffer(numBlocks, ByteString.copyFrom(blocksBufs));
+  }
 
   /**
-   * Returns the index of the first long in blockList
-   * belonging to the specified block.
-   * The first long contains the block id.
+   * Prepare an instance to in-place decode the given list of Longs.  Note
+   * it's much more efficient to decode ByteString buffers and only exists
+   * for compatibility.
+   * @param blocksList - list of longs
+   * @return BlockListAsLongs
    */
-  private int index2BlockId(int blockIndex) {
-    if(blockIndex < 0 || blockIndex > getNumberOfBlocks())
-      return -1;
-    int finalizedSize = getNumberOfFinalizedReplicas();
-    if(blockIndex < finalizedSize)
-      return HEADER_SIZE + blockIndex * LONGS_PER_FINALIZED_BLOCK;
-    return HEADER_SIZE + (finalizedSize + 1) * LONGS_PER_FINALIZED_BLOCK
-            + (blockIndex - finalizedSize) * LONGS_PER_UC_BLOCK;
+  public static BlockListAsLongs decodeLongs(List<Long> blocksList) {
+    return blocksList.isEmpty() ? EMPTY : new LongsDecoder(blocksList);
   }
 
-  private final long[] blockList;
-  
   /**
-   * Create block report from finalized and under construction lists of blocks.
-   * 
-   * @param finalized - list of finalized blocks
-   * @param uc - list of under construction blocks
+   * Prepare an instance to encode the collection of replicas into an
+   * efficient ByteString.
+   * @param replicas - replicas to encode
+   * @return BlockListAsLongs
    */
-  public BlockListAsLongs(final List<? extends Replica> finalized,
-                          final List<? extends Replica> uc) {
-    int finalizedSize = finalized == null ? 0 : finalized.size();
-    int ucSize = uc == null ? 0 : uc.size();
-    int len = HEADER_SIZE
-              + (finalizedSize + 1) * LONGS_PER_FINALIZED_BLOCK
-              + ucSize * LONGS_PER_UC_BLOCK;
-
-    blockList = new long[len];
+  public static BlockListAsLongs encode(
+      final Collection<? extends Replica> replicas) {
+    BlockListAsLongs.Builder builder = builder();
+    for (Replica replica : replicas) {
+      builder.add(replica);
+    }
+    return builder.build();
+  }
 
-    // set the header
-    blockList[0] = finalizedSize;
-    blockList[1] = ucSize;
+  public static Builder builder() {
+    return new BlockListAsLongs.Builder();
+  }
 
-    // set finalized blocks
-    for (int i = 0; i < finalizedSize; i++) {
-      setBlock(i, finalized.get(i));
-    }
+  /**
+   * The number of blocks
+   * @return - the number of blocks
+   */
+  abstract public int getNumberOfBlocks();
 
-    // set invalid delimiting block
-    setDelimitingBlock(finalizedSize);
+  /**
+   * Very efficient encoding of the block report into a ByteString to avoid
+   * the overhead of protobuf repeating fields.  Primitive repeating fields
+   * require re-allocs of an ArrayList<Long> and the associated (un)boxing
+   * overhead which puts pressure on GC.
+   * 
+   * The structure of the buffer is as follows:
+   * - each replica is represented by 4 longs:
+   *   blockId, block length, genstamp, replica state
+   *
+   * @return ByteString encoded block report
+   */
+  abstract public ByteString getBlocksBuffer();
 
-    // set under construction blocks
-    for (int i = 0; i < ucSize; i++) {
-      setBlock(finalizedSize + i, uc.get(i));
+  /**
+   * List of ByteStrings that encode this block report
+   *
+   * @return ByteStrings
+   */
+  public List<ByteString> getBlocksBuffers() {
+    final ByteString blocksBuf = getBlocksBuffer();
+    final List<ByteString> buffers;
+    final int size = blocksBuf.size();
+    if (size <= CHUNK_SIZE) {
+      buffers = Collections.singletonList(blocksBuf);
+    } else {
+      buffers = new ArrayList<ByteString>();
+      for (int pos=0; pos < size; pos += CHUNK_SIZE) {
+        // this doesn't actually copy the data
+        buffers.add(blocksBuf.substring(pos, Math.min(pos+CHUNK_SIZE, size)));
+      }
     }
+    return buffers;
   }
 
   /**
-   * Create block report from a list of finalized blocks.  Used by
-   * NNThroughputBenchmark.
-   *
-   * @param blocks - list of finalized blocks
+   * Convert block report to old-style list of longs.  Only used to
+   * re-encode the block report when the DN detects an older NN. This is
+   * inefficient, but in practice a DN is unlikely to be upgraded first
+   * 
+   * The structure of the array is as follows:
+   * 0: the length of the finalized replica list;
+   * 1: the length of the under-construction replica list;
+   * - followed by finalized replica list where each replica is represented by
+   *   3 longs: one for the blockId, one for the block length, and one for
+   *   the generation stamp;
+   * - followed by the invalid replica represented with three -1s;
+   * - followed by the under-construction replica list where each replica is
+   *   represented by 4 longs: three for the block id, length, generation 
+   *   stamp, and the fourth for the replica state.
+   * @return list of longs
    */
-  public BlockListAsLongs(final List<? extends Block> blocks) {
-    int finalizedSize = blocks == null ? 0 : blocks.size();
-    int len = HEADER_SIZE
-              + (finalizedSize + 1) * LONGS_PER_FINALIZED_BLOCK;
+  abstract public long[] getBlockListAsLongs();
 
-    blockList = new long[len];
+  /**
+   * Returns a singleton iterator over blocks in the block report.  Do not
+   * add the returned blocks to a collection.
+   * @return Iterator
+   */
+  abstract public Iterator<BlockReportReplica> iterator();
 
-    // set the header
-    blockList[0] = finalizedSize;
-    blockList[1] = 0;
+  public static class Builder {
+    private final ByteString.Output out;
+    private final CodedOutputStream cos;
+    private int numBlocks = 0;
+    private int numFinalized = 0;
 
-    // set finalized blocks
-    for (int i = 0; i < finalizedSize; i++) {
-      setBlock(i, blocks.get(i));
+    Builder() {
+      out = ByteString.newOutput(64*1024);
+      cos = CodedOutputStream.newInstance(out);
     }
 
-    // set invalid delimiting block
-    setDelimitingBlock(finalizedSize);
-  }
-
-  public BlockListAsLongs() {
-    this((long[])null);
-  }
+    public void add(Replica replica) {
+      try {
+        // zig-zag to reduce size of legacy blocks
+        cos.writeSInt64NoTag(replica.getBlockId());
+        cos.writeRawVarint64(replica.getBytesOnDisk());
+        cos.writeRawVarint64(replica.getGenerationStamp());
+        ReplicaState state = replica.getState();
+        // although state is not a 64-bit value, using a long varint to
+        // allow for future use of the upper bits
+        cos.writeRawVarint64(state.getValue());
+        if (state == ReplicaState.FINALIZED) {
+          numFinalized++;
+        }
+        numBlocks++;
+      } catch (IOException ioe) {
+        // shouldn't happen, ByteString.Output doesn't throw IOE
+        throw new IllegalStateException(ioe);
+      }
+    }
 
-  /**
-   * Constructor
-   * @param iBlockList - BlockListALongs create from this long[] parameter
-   */
-  public BlockListAsLongs(final long[] iBlockList) {
-    if (iBlockList == null) {
-      blockList = new long[HEADER_SIZE];
-      return;
+    public int getNumberOfBlocks() {
+      return numBlocks;
+    }
+    
+    public BlockListAsLongs build() {
+      try {
+        cos.flush();
+      } catch (IOException ioe) {
+        // shouldn't happen, ByteString.Output doesn't throw IOE
+        throw new IllegalStateException(ioe);
+      }
+      return new BufferDecoder(numBlocks, numFinalized, out.toByteString());
     }
-    blockList = iBlockList;
   }
 
-  public long[] getBlockListAsLongs() {
-    return blockList;
-  }
+  // decode new-style ByteString buffer based block report
+  private static class BufferDecoder extends BlockListAsLongs {
+    // reserve upper bits for future use.  decoding masks off these bits to
+    // allow compatibility for the current through future release that may
+    // start using the bits
+    private static long NUM_BYTES_MASK = (-1L) >>> (64 - 48);
+    private static long REPLICA_STATE_MASK = (-1L) >>> (64 - 4);
 
-  /**
-   * Iterates over blocks in the block report.
-   * Avoids object allocation on each iteration.
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Evolving
-  public class BlockReportIterator implements Iterator<Block> {
-    private int currentBlockIndex;
-    private final Block block;
-    private ReplicaState currentReplicaState;
-
-    BlockReportIterator() {
-      this.currentBlockIndex = 0;
-      this.block = new Block();
-      this.currentReplicaState = null;
+    private final ByteString buffer;
+    private final int numBlocks;
+    private int numFinalized;
+
+    BufferDecoder(final int numBlocks, final ByteString buf) {
+      this(numBlocks, -1, buf);
     }
 
-    @Override
-    public boolean hasNext() {
-      return currentBlockIndex < getNumberOfBlocks();
+    BufferDecoder(final int numBlocks, final int numFinalized,
+        final ByteString buf) {
+      this.numBlocks = numBlocks;
+      this.numFinalized = numFinalized;
+      this.buffer = buf;
     }
 
     @Override
-    public Block next() {
-      block.set(blockId(currentBlockIndex),
-                blockLength(currentBlockIndex),
-                blockGenerationStamp(currentBlockIndex));
-      currentReplicaState = blockReplicaState(currentBlockIndex);
-      currentBlockIndex++;
-      return block;
+    public int getNumberOfBlocks() {
+      return numBlocks;
     }
 
     @Override
-    public void remove() {
-      throw new UnsupportedOperationException("Sorry. can't remove.");
+    public ByteString getBlocksBuffer() {
+      return buffer;
     }
 
-    /**
-     * Get the state of the current replica.
-     * The state corresponds to the replica returned
-     * by the latest {@link #next()}. 
-     */
-    public ReplicaState getCurrentReplicaState() {
-      return currentReplicaState;
+    @Override
+    public long[] getBlockListAsLongs() {
+      // terribly inefficient but only occurs if server tries to transcode
+      // an undecoded buffer into longs - ie. it will never happen but let's
+      // handle it anyway
+      if (numFinalized == -1) {
+        int n = 0;
+        for (Replica replica : this) {
+          if (replica.getState() == ReplicaState.FINALIZED) {
+            n++;
+          }
+        }
+        numFinalized = n;
+      }
+      int numUc = numBlocks - numFinalized;
+      int size = 2 + 3*(numFinalized+1) + 4*(numUc);
+      long[] longs = new long[size];
+      longs[0] = numFinalized;
+      longs[1] = numUc;
+
+      int idx = 2;
+      int ucIdx = idx + 3*numFinalized;
+      // delimiter block
+      longs[ucIdx++] = -1;
+      longs[ucIdx++] = -1;
+      longs[ucIdx++] = -1;
+
+      for (BlockReportReplica block : this) {
+        switch (block.getState()) {
+          case FINALIZED: {
+            longs[idx++] = block.getBlockId();
+            longs[idx++] = block.getNumBytes();
+            longs[idx++] = block.getGenerationStamp();
+            break;
+          }
+          default: {
+            longs[ucIdx++] = block.getBlockId();
+            longs[ucIdx++] = block.getNumBytes();
+            longs[ucIdx++] = block.getGenerationStamp();
+            longs[ucIdx++] = block.getState().getValue();
+            break;
+          }
+        }
+      }
+      return longs;
     }
-  }
 
-  /**
-   * Returns an iterator over blocks in the block report. 
-   */
-  @Override
-  public Iterator<Block> iterator() {
-    return getBlockReportIterator();
-  }
-
-  /**
-   * Returns {@link BlockReportIterator}. 
-   */
-  public BlockReportIterator getBlockReportIterator() {
-    return new BlockReportIterator();
-  }
-
-  /**
-   * The number of blocks
-   * @return - the number of blocks
-   */
-  public int getNumberOfBlocks() {
-    assert blockList.length == HEADER_SIZE + 
-            (blockList[0] + 1) * LONGS_PER_FINALIZED_BLOCK +
-            blockList[1] * LONGS_PER_UC_BLOCK :
-              "Number of blocks is inconcistent with the array length";
-    return getNumberOfFinalizedReplicas() + getNumberOfUCReplicas();
-  }
-
-  /**
-   * Returns the number of finalized replicas in the block report.
-   */
-  private int getNumberOfFinalizedReplicas() {
-    return (int)blockList[0];
-  }
-
-  /**
-   * Returns the number of under construction replicas in the block report.
-   */
-  private int getNumberOfUCReplicas() {
-    return (int)blockList[1];
+    @Override
+    public Iterator<BlockReportReplica> iterator() {
+      return new Iterator<BlockReportReplica>() {
+        final BlockReportReplica block = new BlockReportReplica();
+        final CodedInputStream cis = buffer.newCodedInput();
+        private int currentBlockIndex = 0;
+
+        @Override
+        public boolean hasNext() {
+          return currentBlockIndex < numBlocks;
+        }
+
+        @Override
+        public BlockReportReplica next() {
+          currentBlockIndex++;
+          try {
+            // zig-zag to reduce size of legacy blocks and mask off bits
+            // we don't (yet) understand
+            block.setBlockId(cis.readSInt64());
+            block.setNumBytes(cis.readRawVarint64() & NUM_BYTES_MASK);
+            block.setGenerationStamp(cis.readRawVarint64());
+            long state = cis.readRawVarint64() & REPLICA_STATE_MASK;
+            block.setState(ReplicaState.getState((int)state));
+          } catch (IOException e) {
+            throw new IllegalStateException(e);
+          }
+          return block;
+        }
+
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
   }
 
-  /**
-   * Returns the id of the specified replica of the block report.
-   */
-  private long blockId(int index) {
-    return blockList[index2BlockId(index)];
-  }
+  // decode old style block report of longs
+  private static class LongsDecoder extends BlockListAsLongs {
+    private final List<Long> values;
+    private final int finalizedBlocks;
+    private final int numBlocks;
 
-  /**
-   * Returns the length of the specified replica of the block report.
-   */
-  private long blockLength(int index) {
-    return blockList[index2BlockId(index) + 1];
-  }
+    // set the header
+    LongsDecoder(List<Long> values) {
+      this.values = values.subList(2, values.size());
+      this.finalizedBlocks = values.get(0).intValue();
+      this.numBlocks = finalizedBlocks + values.get(1).intValue();
+    }
 
-  /**
-   * Returns the generation stamp of the specified replica of the block report.
-   */
-  private long blockGenerationStamp(int index) {
-    return blockList[index2BlockId(index) + 2];
-  }
+    @Override
+    public int getNumberOfBlocks() {
+      return numBlocks;
+    }
 
-  /**
-   * Returns the state of the specified replica of the block report.
-   */
-  private ReplicaState blockReplicaState(int index) {
-    if(index < getNumberOfFinalizedReplicas())
-      return ReplicaState.FINALIZED;
-    return ReplicaState.getState((int)blockList[index2BlockId(index) + 3]);
-  }
+    @Override
+    public ByteString getBlocksBuffer() {
+      Builder builder = builder();
+      for (Replica replica : this) {
+        builder.add(replica);
+      }
+      return builder.build().getBlocksBuffer();
+    }
 
-  /**
-   * Corrupt the generation stamp of the block with the given index.
-   * Not meant to be used outside of tests.
-   */
-  @VisibleForTesting
-  public long corruptBlockGSForTesting(final int blockIndex, Random rand) {
-    long oldGS = blockList[index2BlockId(blockIndex) + 2];
-    while (blockList[index2BlockId(blockIndex) + 2] == oldGS) {
-      blockList[index2BlockId(blockIndex) + 2] = rand.nextInt();
+    @Override
+    public long[] getBlockListAsLongs() {
+      long[] longs = new long[2+values.size()];
+      longs[0] = finalizedBlocks;
+      longs[1] = numBlocks - finalizedBlocks;
+      for (int i=0; i < longs.length; i++) {
+        longs[i] = values.get(i);
+      }
+      return longs;
     }
-    return oldGS;
-  }
 
-  /**
-   * Corrupt the length of the block with the given index by truncation.
-   * Not meant to be used outside of tests.
-   */
-  @VisibleForTesting
-  public long corruptBlockLengthForTesting(final int blockIndex, Random rand) {
-    long oldLength = blockList[index2BlockId(blockIndex) + 1];
-    blockList[index2BlockId(blockIndex) + 1] =
-        rand.nextInt((int) oldLength - 1);
-    return oldLength;
+    @Override
+    public Iterator<BlockReportReplica> iterator() {
+      return new Iterator<BlockReportReplica>() {
+        private final BlockReportReplica block = new BlockReportReplica();
+        final Iterator<Long> iter = values.iterator();
+        private int currentBlockIndex = 0;
+
+        @Override
+        public boolean hasNext() {
+          return currentBlockIndex < numBlocks;
+        }
+
+        @Override
+        public BlockReportReplica next() {
+          if (currentBlockIndex == finalizedBlocks) {
+            // verify the presence of the delimiter block
+            readBlock();
+            Preconditions.checkArgument(block.getBlockId() == -1 &&
+                                        block.getNumBytes() == -1 &&
+                                        block.getGenerationStamp() == -1,
+                                        "Invalid delimiter block");
+          }
+
+          readBlock();
+          if (currentBlockIndex++ < finalizedBlocks) {
+            block.setState(ReplicaState.FINALIZED);
+          } else {
+            block.setState(ReplicaState.getState(iter.next().intValue()));
+          }
+          return block;
+        }
+
+        private void readBlock() {
+          block.setBlockId(iter.next());
+          block.setNumBytes(iter.next());
+          block.setGenerationStamp(iter.next());
+        }
+
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
   }
   
-  /**
-   * Set the indexTh block
-   * @param index - the index of the block to set
-   * @param r - the block is set to the value of the this Replica
-   */
-  private void setBlock(final int index, final Replica r) {
-    int pos = index2BlockId(index);
-    blockList[pos] = r.getBlockId();
-    blockList[pos + 1] = r.getNumBytes();
-    blockList[pos + 2] = r.getGenerationStamp();
-    if(index < getNumberOfFinalizedReplicas())
-      return;
-    assert r.getState() != ReplicaState.FINALIZED :
-      "Must be under-construction replica.";
-    blockList[pos + 3] = r.getState().getValue();
-  }
-
-  /**
-   * Set the indexTh block
-   * @param index - the index of the block to set
-   * @param b - the block is set to the value of the this Block
-   */
-  private void setBlock(final int index, final Block b) {
-    int pos = index2BlockId(index);
-    blockList[pos] = b.getBlockId();
-    blockList[pos + 1] = b.getNumBytes();
-    blockList[pos + 2] = b.getGenerationStamp();
-  }
-
-  /**
-   * Set the invalid delimiting block between the finalized and
-   * the under-construction lists.
-   * The invalid block has all three fields set to -1.
-   * @param finalizedSzie - the size of the finalized list
-   */
-  private void setDelimitingBlock(final int finalizedSzie) {
-    int idx = HEADER_SIZE + finalizedSzie * LONGS_PER_FINALIZED_BLOCK;
-    blockList[idx] = -1;
-    blockList[idx+1] = -1;
-    blockList[idx+2] = -1;
-  }
-
-  public long getMaxGsInBlockList() {
-    long maxGs = -1;
-    Iterator<Block> iter = getBlockReportIterator();
-    while (iter.hasNext()) {
-      Block b = iter.next();
-      if (b.getGenerationStamp() > maxGs) {
-        maxGs = b.getGenerationStamp();
+  @InterfaceAudience.Private
+  public static class BlockReportReplica extends Block implements Replica {
+    private ReplicaState state;
+    private BlockReportReplica() {
+    }
+    public BlockReportReplica(Block block) {
+      super(block);
+      if (block instanceof BlockReportReplica) {
+        this.state = ((BlockReportReplica)block).getState();
+      } else {
+        this.state = ReplicaState.FINALIZED;
       }
     }
-    return maxGs;
+    public void setState(ReplicaState state) {
+      this.state = state;
+    }
+    @Override
+    public ReplicaState getState() {
+      return state;
+    }
+    @Override
+    public long getBytesOnDisk() {
+      return getNumBytes();
+    }
+    @Override
+    public long getVisibleLength() {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public String getStorageUuid() {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public boolean isOnTransientStorage() {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public boolean equals(Object o) {
+      return super.equals(o);
+    }
+    @Override
+    public int hashCode() {
+      return super.hashCode();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index 192916f..c4003f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -26,6 +26,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -51,6 +52,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo.Capability;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
@@ -64,6 +66,7 @@ import org.apache.hadoop.ipc.RpcClientUtil;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
@@ -83,6 +86,11 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       VersionRequestProto.newBuilder().build();
   private final static RpcController NULL_CONTROLLER = null;
   
+  @VisibleForTesting
+  public DatanodeProtocolClientSideTranslatorPB(DatanodeProtocolPB rpcProxy) {
+    this.rpcProxy = rpcProxy;
+  }
+
   public DatanodeProtocolClientSideTranslatorPB(InetSocketAddress nameNodeAddr,
       Configuration conf) throws IOException {
     RPC.setProtocolEngine(conf, DatanodeProtocolPB.class,
@@ -166,12 +174,20 @@ public class DatanodeProtocolClientSideTranslatorPB implements
         .newBuilder().setRegistration(PBHelper.convert(registration))
         .setBlockPoolId(poolId);
     
+    boolean useBlocksBuffer = registration.getNamespaceInfo()
+        .isCapabilitySupported(Capability.STORAGE_BLOCK_REPORT_BUFFERS);
+
     for (StorageBlockReport r : reports) {
       StorageBlockReportProto.Builder reportBuilder = StorageBlockReportProto
           .newBuilder().setStorage(PBHelper.convert(r.getStorage()));
-      long[] blocks = r.getBlocks();
-      for (int i = 0; i < blocks.length; i++) {
-        reportBuilder.addBlocks(blocks[i]);
+      BlockListAsLongs blocks = r.getBlocks();
+      if (useBlocksBuffer) {
+        reportBuilder.setNumberOfBlocks(blocks.getNumberOfBlocks());
+        reportBuilder.addAllBlocksBuffers(blocks.getBlocksBuffers());
+      } else {
+        for (long value : blocks.getBlockListAsLongs()) {
+          reportBuilder.addBlocks(value);
+        }
       }
       builder.addReports(reportBuilder.build());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 1a89090..e18081f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.protocolPB;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
@@ -58,6 +59,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 
+import com.google.common.base.Preconditions;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
@@ -145,10 +147,14 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     
     int index = 0;
     for (StorageBlockReportProto s : request.getReportsList()) {
-      List<Long> blockIds = s.getBlocksList();
-      long[] blocks = new long[blockIds.size()];
-      for (int i = 0; i < blockIds.size(); i++) {
-        blocks[i] = blockIds.get(i);
+      final BlockListAsLongs blocks;
+      if (s.hasNumberOfBlocks()) { // new style buffer based reports
+        int num = (int)s.getNumberOfBlocks();
+        Preconditions.checkState(s.getBlocksCount() == 0,
+            "cannot send both blocks list and buffers");
+        blocks = BlockListAsLongs.decodeBuffers(num, s.getBlocksBuffersList());
+      } else {
+        blocks = BlockListAsLongs.decodeLongs(s.getBlocksList());
       }
       report[index++] = new StorageBlockReport(PBHelper.convert(s.getStorage()),
           blocks);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/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 ee1603c..c428c2b 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
@@ -573,7 +573,7 @@ public class PBHelper {
     StorageInfoProto storage = info.getStorageInfo();
     return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
         info.getBlockPoolID(), storage.getCTime(), info.getBuildVersion(),
-        info.getSoftwareVersion());
+        info.getSoftwareVersion(), info.getCapabilities());
   }
 
   public static NamenodeCommand convert(NamenodeCommandProto cmd) {
@@ -1233,7 +1233,9 @@ public class PBHelper {
         .setBuildVersion(info.getBuildVersion())
         .setUnused(0)
         .setStorageInfo(PBHelper.convert((StorageInfo)info))
-        .setSoftwareVersion(info.getSoftwareVersion()).build();
+        .setSoftwareVersion(info.getSoftwareVersion())
+        .setCapabilities(info.getCapabilities())
+        .build();
   }
   
   // Located Block Arrays and Lists

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/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 c1a3e05..f155375 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
@@ -47,7 +47,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -1968,11 +1968,9 @@ public class BlockManager {
     if (report == null) return;
     assert (namesystem.hasWriteLock());
     assert (storageInfo.numBlocks() == 0);
-    BlockReportIterator itBR = report.getBlockReportIterator();
 
-    while(itBR.hasNext()) {
-      Block iblk = itBR.next();
-      ReplicaState reportedState = itBR.getCurrentReplicaState();
+    for (BlockReportReplica iblk : report) {
+      ReplicaState reportedState = iblk.getState();
       
       if (shouldPostponeBlocksFromFuture &&
           namesystem.isGenStampInFuture(iblk)) {
@@ -2042,13 +2040,11 @@ public class BlockManager {
     int curIndex;
 
     if (newReport == null) {
-      newReport = new BlockListAsLongs();
+      newReport = BlockListAsLongs.EMPTY;
     }
     // scan the report and process newly reported blocks
-    BlockReportIterator itBR = newReport.getBlockReportIterator();
-    while(itBR.hasNext()) {
-      Block iblk = itBR.next();
-      ReplicaState iState = itBR.getCurrentReplicaState();
+    for (BlockReportReplica iblk : newReport) {
+      ReplicaState iState = iblk.getState();
       BlockInfoContiguous storedBlock = processReportedBlock(storageInfo,
           iblk, iState, toAdd, toInvalidate, toCorrupt, toUC);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index 3ba2f54..3c20f6b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -228,7 +228,7 @@ class BPServiceActor implements Runnable {
     bpos.verifyAndSetNamespaceInfo(nsInfo);
     
     // Second phase of the handshake with the NN.
-    register();
+    register(nsInfo);
   }
 
   // This is useful to make sure NN gets Heartbeat before Blockreport
@@ -468,8 +468,7 @@ class BPServiceActor implements Runnable {
 
     for(Map.Entry<DatanodeStorage, BlockListAsLongs> kvPair : perVolumeBlockLists.entrySet()) {
       BlockListAsLongs blockList = kvPair.getValue();
-      reports[i++] = new StorageBlockReport(
-          kvPair.getKey(), blockList.getBlockListAsLongs());
+      reports[i++] = new StorageBlockReport(kvPair.getKey(), blockList);
       totalBlockCount += blockList.getNumberOfBlocks();
     }
 
@@ -774,10 +773,11 @@ class BPServiceActor implements Runnable {
    *  
    * issued by the namenode to recognize registered datanodes.
    * 
+   * @param nsInfo current NamespaceInfo
    * @see FSNamesystem#registerDatanode(DatanodeRegistration)
    * @throws IOException
    */
-  void register() throws IOException {
+  void register(NamespaceInfo nsInfo) throws IOException {
     // The handshake() phase loaded the block pool storage
     // off disk - so update the bpRegistration object from that info
     bpRegistration = bpos.createRegistration();
@@ -788,6 +788,7 @@ class BPServiceActor implements Runnable {
       try {
         // Use returned registration from namenode with updated fields
         bpRegistration = bpNamenode.registerDatanode(bpRegistration);
+        bpRegistration.setNamespaceInfo(nsInfo);
         break;
       } catch(EOFException e) {  // namenode might have just restarted
         LOG.info("Problem connecting to server: " + nnAddr + " :"
@@ -915,9 +916,9 @@ class BPServiceActor implements Runnable {
     if (shouldRun()) {
       // re-retrieve namespace info to make sure that, if the NN
       // was restarted, we still match its version (HDFS-2120)
-      retrieveNamespaceInfo();
+      NamespaceInfo nsInfo = retrieveNamespaceInfo();
       // and re-register
-      register();
+      register(nsInfo);
       scheduleHeartbeat();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 486acbc..d42c00c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -1575,30 +1575,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     Map<DatanodeStorage, BlockListAsLongs> blockReportsMap =
         new HashMap<DatanodeStorage, BlockListAsLongs>();
 
-    Map<String, ArrayList<ReplicaInfo>> finalized =
-        new HashMap<String, ArrayList<ReplicaInfo>>();
-    Map<String, ArrayList<ReplicaInfo>> uc =
-        new HashMap<String, ArrayList<ReplicaInfo>>();
+    Map<String, BlockListAsLongs.Builder> builders =
+        new HashMap<String, BlockListAsLongs.Builder>();
 
     List<FsVolumeImpl> curVolumes = getVolumes();
     for (FsVolumeSpi v : curVolumes) {
-      finalized.put(v.getStorageID(), new ArrayList<ReplicaInfo>());
-      uc.put(v.getStorageID(), new ArrayList<ReplicaInfo>());
+      builders.put(v.getStorageID(), BlockListAsLongs.builder());
     }
 
     synchronized(this) {
       for (ReplicaInfo b : volumeMap.replicas(bpid)) {
         switch(b.getState()) {
           case FINALIZED:
-            finalized.get(b.getVolume().getStorageID()).add(b);
-            break;
           case RBW:
           case RWR:
-            uc.get(b.getVolume().getStorageID()).add(b);
+            builders.get(b.getVolume().getStorageID()).add(b);
             break;
           case RUR:
             ReplicaUnderRecovery rur = (ReplicaUnderRecovery)b;
-            uc.get(rur.getVolume().getStorageID()).add(rur.getOriginalReplica());
+            builders.get(rur.getVolume().getStorageID())
+                .add(rur.getOriginalReplica());
             break;
           case TEMPORARY:
             break;
@@ -1609,10 +1605,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
 
     for (FsVolumeImpl v : curVolumes) {
-      ArrayList<ReplicaInfo> finalizedList = finalized.get(v.getStorageID());
-      ArrayList<ReplicaInfo> ucList = uc.get(v.getStorageID());
       blockReportsMap.put(v.toDatanodeStorage(),
-                          new BlockListAsLongs(finalizedList, ucList));
+                          builders.get(v.getStorageID()).build());
     }
 
     return blockReportsMap;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/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 f20fb35..059bd28 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
@@ -1302,7 +1302,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     final BlockManager bm = namesystem.getBlockManager(); 
     boolean noStaleStorages = false;
     for(StorageBlockReport r : reports) {
-      final BlockListAsLongs blocks = new BlockListAsLongs(r.getBlocks());
+      final BlockListAsLongs blocks = r.getBlocks();
       //
       // BlockManager.processReport accumulates information of prior calls
       // for the same node and storage, so the value returned by the last

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
index 9db2fca..e788137 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
@@ -40,6 +40,7 @@ public class DatanodeRegistration extends DatanodeID
   private final StorageInfo storageInfo;
   private ExportedBlockKeys exportedKeys;
   private final String softwareVersion;
+  private NamespaceInfo nsInfo;
 
   @VisibleForTesting
   public DatanodeRegistration(String uuid, DatanodeRegistration dnr) {
@@ -77,6 +78,14 @@ public class DatanodeRegistration extends DatanodeID
   public int getVersion() {
     return storageInfo.getLayoutVersion();
   }
+
+  public void setNamespaceInfo(NamespaceInfo nsInfo) {
+    this.nsInfo = nsInfo;
+  }
+
+  public NamespaceInfo getNamespaceInfo() {
+    return nsInfo;
+  }
   
   @Override // NodeRegistration
   public String getRegistrationID() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
index 0733743..a7439a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
@@ -29,6 +29,9 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.util.VersionInfo;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
 /**
  * NamespaceInfo is returned by the name-node in reply 
  * to a data-node handshake.
@@ -40,19 +43,52 @@ public class NamespaceInfo extends StorageInfo {
   final String  buildVersion;
   String blockPoolID = "";    // id of the block pool
   String softwareVersion;
+  long capabilities;
+
+  // only authoritative on the server-side to determine advertisement to
+  // clients.  enum will update the supported values
+  private static long CAPABILITIES_SUPPORTED = 0;
+
+  public enum Capability {
+    UNKNOWN(false),
+    STORAGE_BLOCK_REPORT_BUFFERS(true); // use optimized ByteString buffers
+    private final long mask;
+    Capability(boolean isSupported) {
+      int bits = ordinal() - 1;
+      mask = (bits < 0) ? 0 : (1L << bits);
+      if (isSupported) {
+        CAPABILITIES_SUPPORTED |= mask;
+      }
+    }
+    public long getMask() {
+      return mask;
+    }
+  }
 
+  // defaults to enabled capabilites since this ctor is for server
   public NamespaceInfo() {
     super(NodeType.NAME_NODE);
     buildVersion = null;
+    capabilities = CAPABILITIES_SUPPORTED;
   }
 
+  // defaults to enabled capabilites since this ctor is for server
   public NamespaceInfo(int nsID, String clusterID, String bpID,
       long cT, String buildVersion, String softwareVersion) {
+    this(nsID, clusterID, bpID, cT, buildVersion, softwareVersion,
+        CAPABILITIES_SUPPORTED);
+  }
+
+  // for use by server and/or client
+  public NamespaceInfo(int nsID, String clusterID, String bpID,
+      long cT, String buildVersion, String softwareVersion,
+      long capabilities) {
     super(HdfsConstants.NAMENODE_LAYOUT_VERSION, nsID, clusterID, cT,
         NodeType.NAME_NODE);
     blockPoolID = bpID;
     this.buildVersion = buildVersion;
     this.softwareVersion = softwareVersion;
+    this.capabilities = capabilities;
   }
 
   public NamespaceInfo(int nsID, String clusterID, String bpID, 
@@ -61,6 +97,22 @@ public class NamespaceInfo extends StorageInfo {
         VersionInfo.getVersion());
   }
   
+  public long getCapabilities() {
+    return capabilities;
+  }
+
+  @VisibleForTesting
+  public void setCapabilities(long capabilities) {
+    this.capabilities = capabilities;
+  }
+
+  public boolean isCapabilitySupported(Capability capability) {
+    Preconditions.checkArgument(capability != Capability.UNKNOWN,
+        "cannot test for unknown capability");
+    long mask = capability.getMask();
+    return (capabilities & mask) == mask;
+  }
+
   public String getBuildVersion() {
     return buildVersion;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
index 1521693..4ef5ebc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
@@ -18,14 +18,16 @@
 
 package org.apache.hadoop.hdfs.server.protocol;
 
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+
 /**
  * Block report for a Datanode storage
  */
 public class StorageBlockReport {
   private final DatanodeStorage storage;
-  private final long[] blocks;
+  private final BlockListAsLongs blocks;
   
-  public StorageBlockReport(DatanodeStorage storage, long[] blocks) {
+  public StorageBlockReport(DatanodeStorage storage, BlockListAsLongs blocks) {
     this.storage = storage;
     this.blocks = blocks;
   }
@@ -34,7 +36,7 @@ public class StorageBlockReport {
     return storage;
   }
 
-  public long[] getBlocks() {
+  public BlockListAsLongs getBlocks() {
     return blocks;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/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 348f346..7b3a4a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -237,6 +237,8 @@ message BlockReportRequestProto {
 message StorageBlockReportProto {
   required DatanodeStorageProto storage = 1;    // Storage
   repeated uint64 blocks = 2 [packed=true];
+  optional uint64 numberOfBlocks = 3;
+  repeated bytes blocksBuffers = 4;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/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 97906b1..31e5585 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -517,6 +517,7 @@ message NamespaceInfoProto {
   required string blockPoolID = 3;          // block pool used by the namespace
   required StorageInfoProto storageInfo = 4;// Node information
   required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
+  optional uint64 capabilities = 6 [default = 0]; // feature flags
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java
new file mode 100644
index 0000000..bebde18
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java
@@ -0,0 +1,237 @@
+/**
+ * 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolPB;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
+import org.apache.hadoop.hdfs.server.datanode.Replica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo.Capability;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class TestBlockListAsLongs {
+  static Block b1 = new Block(1, 11, 111);
+  static Block b2 = new Block(2, 22, 222);
+  static Block b3 = new Block(3, 33, 333);
+  static Block b4 = new Block(4, 44, 444);
+
+  @Test
+  public void testEmptyReport() {
+    BlockListAsLongs blocks = checkReport();
+    assertArrayEquals(
+        new long[] {
+            0, 0,
+            -1, -1, -1 }, 
+        blocks.getBlockListAsLongs());
+  }
+
+  @Test
+  public void testFinalized() {
+    BlockListAsLongs blocks = checkReport(
+        new FinalizedReplica(b1, null, null));
+    assertArrayEquals(
+        new long[] {
+            1, 0,
+            1, 11, 111,
+            -1, -1, -1 }, 
+        blocks.getBlockListAsLongs());
+  }
+
+  @Test
+  public void testUc() {
+    BlockListAsLongs blocks = checkReport(
+      new ReplicaBeingWritten(b1, null, null, null));
+    assertArrayEquals(
+        new long[] {
+            0, 1,
+            -1, -1, -1,
+            1, 11, 111, ReplicaState.RBW.getValue() }, 
+        blocks.getBlockListAsLongs());
+  }
+  
+  @Test
+  public void testMix() {
+    BlockListAsLongs blocks = checkReport(
+        new FinalizedReplica(b1, null, null),
+        new FinalizedReplica(b2, null, null),
+        new ReplicaBeingWritten(b3, null, null, null),
+        new ReplicaWaitingToBeRecovered(b4, null, null));
+    assertArrayEquals(
+        new long[] {
+            2, 2,
+            1, 11, 111,
+            2, 22, 222,
+            -1, -1, -1,
+            3, 33, 333, ReplicaState.RBW.getValue(),
+            4, 44, 444, ReplicaState.RWR.getValue() },
+        blocks.getBlockListAsLongs());
+  }
+
+  @Test
+  public void testFuzz() throws InterruptedException {
+    Replica[] replicas = new Replica[100000];
+    Random rand = new Random(0);
+    for (int i=0; i<replicas.length; i++) {
+      Block b = new Block(rand.nextLong(), i, i<<4);
+      switch (rand.nextInt(2)) {
+        case 0:
+          replicas[i] = new FinalizedReplica(b, null, null);
+          break;
+        case 1:
+          replicas[i] = new ReplicaBeingWritten(b, null, null, null);
+          break;
+        case 2:
+          replicas[i] = new ReplicaWaitingToBeRecovered(b, null, null);
+          break;
+      }
+    }
+    checkReport(replicas);
+  }
+
+  private BlockListAsLongs checkReport(Replica...replicas) {
+    Map<Long, Replica> expectedReplicas = new HashMap<>();
+    for (Replica replica : replicas) {
+      expectedReplicas.put(replica.getBlockId(), replica);
+    }
+    expectedReplicas = Collections.unmodifiableMap(expectedReplicas);
+    
+    // encode the blocks and extract the buffers
+    BlockListAsLongs blocks =
+        BlockListAsLongs.encode(expectedReplicas.values());
+    List<ByteString> buffers = blocks.getBlocksBuffers();
+    
+    // convert to old-style list of longs
+    List<Long> longs = new ArrayList<Long>();
+    for (long value : blocks.getBlockListAsLongs()) {
+      longs.add(value);
+    }
+
+    // decode the buffers and verify its contents
+    BlockListAsLongs decodedBlocks =
+        BlockListAsLongs.decodeBuffers(expectedReplicas.size(), buffers);
+    checkReplicas(expectedReplicas, decodedBlocks);
+
+    // decode the long and verify its contents
+    BlockListAsLongs decodedList = BlockListAsLongs.decodeLongs(longs);
+    checkReplicas(expectedReplicas, decodedList);
+    return blocks;
+  }
+  
+  private void checkReplicas(Map<Long,Replica> expectedReplicas,
+                             BlockListAsLongs decodedBlocks) {
+    assertEquals(expectedReplicas.size(), decodedBlocks.getNumberOfBlocks());
+
+    Map<Long, Replica> reportReplicas = new HashMap<>(expectedReplicas);
+    for (BlockReportReplica replica : decodedBlocks) {
+      assertNotNull(replica);
+      Replica expected = reportReplicas.remove(replica.getBlockId());
+      assertNotNull(expected);
+      assertEquals("wrong bytes",
+          expected.getNumBytes(), replica.getNumBytes());
+      assertEquals("wrong genstamp",
+          expected.getGenerationStamp(), replica.getGenerationStamp());
+      assertEquals("wrong replica state",
+          expected.getState(), replica.getState());
+    }
+    assertTrue(reportReplicas.isEmpty());
+  }
+  
+  @Test
+  public void testDatanodeDetect() throws ServiceException, IOException {
+    final AtomicReference<BlockReportRequestProto> request =
+        new AtomicReference<>();
+
+    // just capture the outgoing PB
+    DatanodeProtocolPB mockProxy = mock(DatanodeProtocolPB.class);
+    doAnswer(new Answer<BlockReportResponseProto>() {
+      public BlockReportResponseProto answer(InvocationOnMock invocation) {
+        Object[] args = invocation.getArguments();
+        request.set((BlockReportRequestProto) args[1]);
+        return BlockReportResponseProto.newBuilder().build();
+      }
+    }).when(mockProxy).blockReport(any(RpcController.class),
+                                   any(BlockReportRequestProto.class));
+    
+    @SuppressWarnings("resource")
+    DatanodeProtocolClientSideTranslatorPB nn =
+        new DatanodeProtocolClientSideTranslatorPB(mockProxy);
+
+    DatanodeRegistration reg = DFSTestUtil.getLocalDatanodeRegistration();
+    NamespaceInfo nsInfo = new NamespaceInfo(1, "cluster", "bp", 1);
+    reg.setNamespaceInfo(nsInfo);
+
+    Replica r = new FinalizedReplica(new Block(1, 2, 3), null, null);
+    BlockListAsLongs bbl = BlockListAsLongs.encode(Collections.singleton(r));
+    DatanodeStorage storage = new DatanodeStorage("s1");
+    StorageBlockReport[] sbr = { new StorageBlockReport(storage, bbl) };    
+
+    // check DN sends new-style BR
+    request.set(null);
+    nsInfo.setCapabilities(Capability.STORAGE_BLOCK_REPORT_BUFFERS.getMask());
+    nn.blockReport(reg, "pool", sbr);
+    BlockReportRequestProto proto = request.get();
+    assertNotNull(proto);
+    assertTrue(proto.getReports(0).getBlocksList().isEmpty());
+    assertFalse(proto.getReports(0).getBlocksBuffersList().isEmpty());
+    
+    // back up to prior version and check DN sends old-style BR
+    request.set(null);
+    nsInfo.setCapabilities(Capability.UNKNOWN.getMask());
+    nn.blockReport(reg, "pool", sbr);
+    proto = request.get();
+    assertNotNull(proto);
+    assertFalse(proto.getReports(0).getBlocksList().isEmpty());
+    assertTrue(proto.getReports(0).getBlocksBuffersList().isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 6d67c7d..d9ac9e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -555,12 +555,12 @@ public class TestBlockManager {
     reset(node);
     
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        new BlockListAsLongs(null, null));
+        BlockListAsLongs.EMPTY);
     assertEquals(1, ds.getBlockReportCount());
     // send block report again, should NOT be processed
     reset(node);
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        new BlockListAsLongs(null, null));
+        BlockListAsLongs.EMPTY);
     assertEquals(1, ds.getBlockReportCount());
 
     // re-register as if node restarted, should update existing node
@@ -571,7 +571,7 @@ public class TestBlockManager {
     // send block report, should be processed after restart
     reset(node);
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-                     new BlockListAsLongs(null, null));
+                     BlockListAsLongs.EMPTY);
     // Reinitialize as registration with empty storage list pruned
     // node.storageMap.
     ds = node.getStorageInfos()[0];
@@ -600,7 +600,7 @@ public class TestBlockManager {
     reset(node);
     doReturn(1).when(node).numBlocks();
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        new BlockListAsLongs(null, null));
+        BlockListAsLongs.EMPTY);
     assertEquals(1, ds.getBlockReportCount());
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
index 8d9de7b..37c503c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
@@ -146,22 +147,32 @@ public abstract class BlockReportTestBase {
 
       // Walk the list of blocks until we find one each to corrupt the
       // generation stamp and length, if so requested.
-      for (int i = 0; i < blockList.getNumberOfBlocks(); ++i) {
+      BlockListAsLongs.Builder builder = BlockListAsLongs.builder();
+      for (BlockReportReplica block : blockList) {
         if (corruptOneBlockGs && !corruptedGs) {
-          blockList.corruptBlockGSForTesting(i, rand);
-          LOG.info("Corrupted the GS for block ID " + i);
+          long gsOld = block.getGenerationStamp();
+          long gsNew;
+          do {
+            gsNew = rand.nextInt();
+          } while (gsNew == gsOld);
+          block.setGenerationStamp(gsNew);
+          LOG.info("Corrupted the GS for block ID " + block);
           corruptedGs = true;
         } else if (corruptOneBlockLen && !corruptedLen) {
-          blockList.corruptBlockLengthForTesting(i, rand);
-          LOG.info("Corrupted the length for block ID " + i);
+          long lenOld = block.getNumBytes();
+          long lenNew;
+          do {
+            lenNew = rand.nextInt((int)lenOld - 1);
+          } while (lenNew == lenOld);
+          block.setNumBytes(lenNew);
+          LOG.info("Corrupted the length for block ID " + block);
           corruptedLen = true;
-        } else {
-          break;
         }
+        builder.add(new BlockReportReplica(block));
       }
 
       reports[reportIndex++] =
-          new StorageBlockReport(dnStorage, blockList.getBlockListAsLongs());
+          new StorageBlockReport(dnStorage, builder.build());
     }
 
     return reports;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index a4ec8d5..5c7b4ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -271,7 +270,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
     @Override
     public ReplicaState getState() {
-      return null;
+      return finalized ? ReplicaState.FINALIZED : ReplicaState.RBW;
     }
 
     @Override
@@ -529,7 +528,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   public synchronized void injectBlocks(String bpid,
-      Iterable<Block> injectBlocks) throws IOException {
+      Iterable<? extends Block> injectBlocks) throws IOException {
     ExtendedBlock blk = new ExtendedBlock();
     if (injectBlocks != null) {
       for (Block b: injectBlocks) { // if any blocks in list is bad, reject list
@@ -582,16 +581,16 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   synchronized BlockListAsLongs getBlockReport(String bpid) {
-    final List<Replica> blocks = new ArrayList<Replica>();
+    BlockListAsLongs.Builder report = BlockListAsLongs.builder();
     final Map<Block, BInfo> map = blockMap.get(bpid);
     if (map != null) {
       for (BInfo b : map.values()) {
         if (b.isFinalized()) {
-          blocks.add(b);
+          report.add(b);
         }
       }
     }
-    return new BlockListAsLongs(blocks, null);
+    return report.build();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
index 1152c74..3238d6a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
@@ -107,17 +107,18 @@ public class TestBlockHasMultipleReplicasOnSameDN {
     StorageBlockReport reports[] =
         new StorageBlockReport[cluster.getStoragesPerDatanode()];
 
-    ArrayList<Block> blocks = new ArrayList<Block>();
+    ArrayList<Replica> blocks = new ArrayList<Replica>();
 
     for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
-      blocks.add(locatedBlock.getBlock().getLocalBlock());
+      Block localBlock = locatedBlock.getBlock().getLocalBlock();
+      blocks.add(new FinalizedReplica(localBlock, null, null));
     }
 
+    BlockListAsLongs bll = BlockListAsLongs.encode(blocks);
     for (int i = 0; i < cluster.getStoragesPerDatanode(); ++i) {
-      BlockListAsLongs bll = new BlockListAsLongs(blocks);
       FsVolumeSpi v = dn.getFSDataset().getVolumes().get(i);
       DatanodeStorage dns = new DatanodeStorage(v.getStorageID());
-      reports[i] = new StorageBlockReport(dns, bll.getBlockListAsLongs());
+      reports[i] = new StorageBlockReport(dns, bll);
     }
 
     // Should not assert!

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index ba786d1..9cbad6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -25,11 +25,9 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
 
 import java.io.File;
-import java.io.FilenameFilter;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -188,7 +186,7 @@ public class TestDataNodeVolumeFailure {
         DatanodeStorage dnStorage = kvPair.getKey();
         BlockListAsLongs blockList = kvPair.getValue();
         reports[reportIndex++] =
-            new StorageBlockReport(dnStorage, blockList.getBlockListAsLongs());
+            new StorageBlockReport(dnStorage, blockList);
     }
     
     cluster.getNameNodeRpc().blockReport(dnR, bpid, reports);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
index 7058d71..a5e4d4e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
@@ -98,7 +98,7 @@ public class TestDnRespectsBlockReportSplitThreshold {
       assertThat(reports.length, is(expectedReportsPerCall));
 
       for (StorageBlockReport report : reports) {
-        BlockListAsLongs blockList = new BlockListAsLongs(report.getBlocks());
+        BlockListAsLongs blockList = report.getBlocks();
         numBlocksReported += blockList.getNumberOfBlocks();
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index 5a440c4..0865e11 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -195,7 +195,7 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
     final Map<DatanodeStorage, BlockListAsLongs> result =
 	new HashMap<DatanodeStorage, BlockListAsLongs>();
 
-    result.put(storage, new BlockListAsLongs(null, null));
+    result.put(storage, BlockListAsLongs.EMPTY);
     return result;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index c11abfc..bc3c6b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -26,6 +26,7 @@ import java.util.EnumSet;
 import java.util.List;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -36,6 +37,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -891,9 +893,9 @@ public class NNThroughputBenchmark implements Tool {
     NamespaceInfo nsInfo;
     DatanodeRegistration dnRegistration;
     DatanodeStorage storage; //only one storage 
-    final ArrayList<Block> blocks;
+    final ArrayList<BlockReportReplica> blocks;
     int nrBlocks; // actual number of blocks
-    long[] blockReportList;
+    BlockListAsLongs blockReportList;
     final int dnIdx;
 
     private static int getNodePort(int num) throws IOException {
@@ -904,7 +906,7 @@ public class NNThroughputBenchmark implements Tool {
 
     TinyDatanode(int dnIdx, int blockCapacity) throws IOException {
       this.dnIdx = dnIdx;
-      this.blocks = new ArrayList<Block>(blockCapacity);
+      this.blocks = new ArrayList<BlockReportReplica>(blockCapacity);
       this.nrBlocks = 0;
     }
 
@@ -934,8 +936,7 @@ public class NNThroughputBenchmark implements Tool {
       //first block reports
       storage = new DatanodeStorage(DatanodeStorage.generateUuid());
       final StorageBlockReport[] reports = {
-          new StorageBlockReport(storage,
-              new BlockListAsLongs(null, null).getBlockListAsLongs())
+          new StorageBlockReport(storage, BlockListAsLongs.EMPTY)
       };
       nameNodeProto.blockReport(dnRegistration, 
           nameNode.getNamesystem().getBlockPoolId(), reports);
@@ -968,19 +969,21 @@ public class NNThroughputBenchmark implements Tool {
         }
         return false;
       }
-      blocks.set(nrBlocks, blk);
+      blocks.set(nrBlocks, new BlockReportReplica(blk));
       nrBlocks++;
       return true;
     }
 
     void formBlockReport() {
       // fill remaining slots with blocks that do not exist
-      for(int idx = blocks.size()-1; idx >= nrBlocks; idx--)
-        blocks.set(idx, new Block(blocks.size() - idx, 0, 0));
-      blockReportList = new BlockListAsLongs(blocks).getBlockListAsLongs();
+      for (int idx = blocks.size()-1; idx >= nrBlocks; idx--) {
+        Block block = new Block(blocks.size() - idx, 0, 0);
+        blocks.set(idx, new BlockReportReplica(block));
+      }
+      blockReportList = BlockListAsLongs.EMPTY;
     }
 
-    long[] getBlockReportList() {
+    BlockListAsLongs getBlockReportList() {
       return blockReportList;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index fb1418a..ee80b33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -30,6 +30,7 @@ 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.BlockListAsLongs;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
@@ -104,7 +105,7 @@ public class TestDeadDatanode {
     // Ensure blockReport from dead datanode is rejected with IOException
     StorageBlockReport[] report = { new StorageBlockReport(
         new DatanodeStorage(reg.getDatanodeUuid()),
-        new long[] { 0L, 0L, 0L }) };
+        BlockListAsLongs.EMPTY) };
     try {
       dnp.blockReport(reg, poolId, report);
       fail("Expected IOException is not thrown");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/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 f7dad18..7b9ea93 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
@@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -219,6 +220,7 @@ public class TestFSImage {
         .manageDataDfsDirs(false)
         .manageNameDfsDirs(false)
         .waitSafeMode(false)
+        .startupOption(StartupOption.UPGRADE)
         .build();
     try {
       FileSystem fs = cluster.getFileSystem();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d324164a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
index 0e605ac..2ad7b60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
@@ -32,6 +32,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.OfflineEditsViewerHelper;
 import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer.Flags;
 import org.apache.hadoop.test.PathUtils;
@@ -140,8 +141,8 @@ public class TestOfflineEditsViewer {
     assertEquals(0, runOev(editsReparsed, editsParsedXml2, "xml", false));
 
     // judgment time
-    assertTrue("Test round trip",
-        filesEqualIgnoreTrailingZeros(editsParsedXml, editsParsedXml2));
+    assertTrue("Test round trip", FileUtils.contentEqualsIgnoreEOL(
+        new File(editsParsedXml), new File(editsParsedXml2), "UTF-8"));
 
     os.close();
   }
@@ -238,6 +239,10 @@ public class TestOfflineEditsViewer {
 
     ByteBuffer small = ByteBuffer.wrap(DFSTestUtil.loadFile(filenameSmall));
     ByteBuffer large = ByteBuffer.wrap(DFSTestUtil.loadFile(filenameLarge));
+    // OEV outputs with the latest layout version, so tweak the old file's
+    // contents to have latest version so checkedin binary files don't
+    // require frequent updates
+    small.put(3, (byte)NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
 
     // now correct if it's otherwise
     if (small.capacity() > large.capacity()) {


[11/50] [abbrv] hadoop git commit: HADOOP-11711. Provide a default value for AES/CTR/NoPadding CryptoCodec classes.

Posted by zh...@apache.org.
HADOOP-11711. Provide a default value for AES/CTR/NoPadding CryptoCodec classes.


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

Branch: refs/heads/HDFS-7285
Commit: 387f271c81f7b3bf53bddc5368d5f4486530c2e1
Parents: a852910
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Mar 12 21:40:58 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Mar 12 21:40:58 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/crypto/CryptoCodec.java   | 10 +++++-
 .../fs/CommonConfigurationKeysPublic.java       | 11 ++++++
 .../crypto/TestCryptoStreamsForLocalFS.java     |  8 +----
 ...stCryptoStreamsWithJceAesCtrCryptoCodec.java | 38 ++++++++++++++++++++
 ...yptoStreamsWithOpensslAesCtrCryptoCodec.java |  7 ++++
 5 files changed, 66 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/387f271c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
index c5ac2ae..493e23d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.util.PerformanceAdvisory;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.slf4j.Logger;
@@ -105,7 +106,14 @@ public abstract class CryptoCodec implements Configurable {
     List<Class<? extends CryptoCodec>> result = Lists.newArrayList();
     String configName = HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX + 
         cipherSuite.getConfigSuffix();
-    String codecString = conf.get(configName);
+    String codecString;
+    if (configName.equals(CommonConfigurationKeysPublic
+        .HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY)) {
+      codecString = conf.get(configName, CommonConfigurationKeysPublic
+          .HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_DEFAULT);
+    } else {
+      codecString = conf.get(configName);
+    }
     if (codecString == null) {
       PerformanceAdvisory.LOG.debug(
           "No crypto codec classes with cipher suite configured.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/387f271c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index 470b4d0..87c2aba 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -19,6 +19,9 @@
 package org.apache.hadoop.fs;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.JceAesCtrCryptoCodec;
+import org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec;
 
 /** 
  * This class contains constants for configuration keys used
@@ -299,6 +302,14 @@ public class CommonConfigurationKeysPublic {
     "hadoop.security.saslproperties.resolver.class";
   public static final String HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX = 
     "hadoop.security.crypto.codec.classes";
+  public static final String
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY =
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
+          + CipherSuite.AES_CTR_NOPADDING.getConfigSuffix();
+  public static final String
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_DEFAULT =
+      OpensslAesCtrCryptoCodec.class.getName() + "," +
+          JceAesCtrCryptoCodec.class.getName();
   /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
   public static final String HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY =
     "hadoop.security.crypto.cipher.suite";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/387f271c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java
index 765a364..0a65085 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java
@@ -47,15 +47,9 @@ public class TestCryptoStreamsForLocalFS extends CryptoStreamsTestBase {
   
   @BeforeClass
   public static void init() throws Exception {
-    Configuration conf = new Configuration();
-    conf = new Configuration(false);
+    Configuration conf = new Configuration(false);
     conf.set("fs.file.impl", LocalFileSystem.class.getName());
     fileSys = FileSystem.getLocal(conf);
-    conf.set(
-        CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
-            + CipherSuite.AES_CTR_NOPADDING.getConfigSuffix(),
-        OpensslAesCtrCryptoCodec.class.getName() + ","
-            + JceAesCtrCryptoCodec.class.getName());
     codec = CryptoCodec.getInstance(conf);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/387f271c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithJceAesCtrCryptoCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithJceAesCtrCryptoCodec.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithJceAesCtrCryptoCodec.java
new file mode 100644
index 0000000..76c39d6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithJceAesCtrCryptoCodec.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.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+
+public class TestCryptoStreamsWithJceAesCtrCryptoCodec extends 
+    TestCryptoStreams {
+
+  @BeforeClass
+  public static void init() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY,
+        JceAesCtrCryptoCodec.class.getName());
+    codec = CryptoCodec.getInstance(conf);
+    Assert.assertEquals(JceAesCtrCryptoCodec.class.getCanonicalName(),
+        codec.getClass().getCanonicalName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/387f271c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
index f64e8dc..4f90a0c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.crypto;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 
 public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec 
@@ -26,6 +28,11 @@ public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec
   @BeforeClass
   public static void init() throws Exception {
     Configuration conf = new Configuration();
+    conf.set(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY,
+        OpensslAesCtrCryptoCodec.class.getName());
     codec = CryptoCodec.getInstance(conf);
+    Assert.assertEquals(OpensslAesCtrCryptoCodec.class.getCanonicalName(), 
+        codec.getClass().getCanonicalName());
   }
 }


[15/50] [abbrv] hadoop git commit: HDFS-7903. Cannot recover block after truncate and delete snapshot. Contributed by Plamen Jeliazkov.

Posted by zh...@apache.org.
HDFS-7903. Cannot recover block after truncate and delete snapshot. Contributed by Plamen Jeliazkov.

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

Branch: refs/heads/HDFS-7285
Commit: 6acb7f2110897264241df44d564db2f85260348f
Parents: d324164
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Fri Mar 13 12:39:01 2015 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Fri Mar 13 13:12:51 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../server/namenode/snapshot/FileDiffList.java  | 19 +++++++++++--
 .../hdfs/server/namenode/TestFileTruncate.java  | 30 ++++++++++++++++++++
 3 files changed, 49 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6acb7f21/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ac7e096..a149f18 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1148,6 +1148,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7926. NameNode implementation of ClientProtocol.truncate(..) is not 
     idempotent (Tsz Wo Nicholas Sze via brandonli)
 
+    HDFS-7903. Cannot recover block after truncate and delete snapshot.
+    (Plamen Jeliazkov via shv)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6acb7f21/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index 0c94554..5c9e121 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -20,8 +20,11 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -125,9 +128,19 @@ public class FileDiffList extends
         continue;
       break;
     }
-    // Collect the remaining blocks of the file
-    while(i < removedBlocks.length) {
-      collectedBlocks.addDeleteBlock(removedBlocks[i++]);
+    // Check if last block is part of truncate recovery
+    BlockInfoContiguous lastBlock = file.getLastBlock();
+    Block dontRemoveBlock = null;
+    if(lastBlock != null && lastBlock.getBlockUCState().equals(
+        HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
+      dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
+          .getTruncateBlock();
+    }
+    // Collect the remaining blocks of the file, ignoring truncate block
+    for(;i < removedBlocks.length; i++) {
+      if(dontRemoveBlock == null || !removedBlocks[i].equals(dontRemoveBlock)) {
+        collectedBlocks.addDeleteBlock(removedBlocks[i]);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6acb7f21/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 260d8bb..3b6e107 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -178,6 +178,36 @@ public class TestFileTruncate {
     fs.delete(dir, true);
   }
 
+  /** Truncate the same file multiple times until its size is zero. */
+  @Test
+  public void testSnapshotTruncateThenDeleteSnapshot() throws IOException {
+    Path dir = new Path("/testSnapshotTruncateThenDeleteSnapshot");
+    fs.mkdirs(dir);
+    fs.allowSnapshot(dir);
+    final Path p = new Path(dir, "file");
+    final byte[] data = new byte[BLOCK_SIZE];
+    DFSUtil.getRandom().nextBytes(data);
+    writeContents(data, data.length, p);
+    final String snapshot = "s0";
+    fs.createSnapshot(dir, snapshot);
+    Block lastBlock = getLocatedBlocks(p).getLastLocatedBlock()
+        .getBlock().getLocalBlock();
+    final int newLength = data.length - 1;
+    assert newLength % BLOCK_SIZE != 0 :
+        " newLength must not be multiple of BLOCK_SIZE";
+    final boolean isReady = fs.truncate(p, newLength);
+    LOG.info("newLength=" + newLength + ", isReady=" + isReady);
+    assertEquals("File must be closed for truncating at the block boundary",
+        isReady, newLength % BLOCK_SIZE == 0);
+    fs.deleteSnapshot(dir, snapshot);
+    if (!isReady) {
+      checkBlockRecovery(p);
+    }
+    checkFullFile(p, newLength, data);
+    assertBlockNotPresent(lastBlock);
+    fs.delete(dir, true);
+  }
+
   /**
    * Truncate files and then run other operations such as
    * rename, set replication, set permission, etc.


[17/50] [abbrv] hadoop git commit: YARN-2854. Updated the documentation of the timeline service and the generic history service. Contributed by Naganarasimha G R.

Posted by zh...@apache.org.
YARN-2854. Updated the documentation of the timeline service and the generic history service. Contributed by Naganarasimha G R.


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

Branch: refs/heads/HDFS-7285
Commit: 6fdef76cc3e818856ddcc4d385c2899a8e6ba916
Parents: dfd3201
Author: Zhijie Shen <zj...@apache.org>
Authored: Fri Mar 13 13:58:42 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Mar 13 14:00:09 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../src/site/markdown/TimelineServer.md         | 318 ++++++++++---------
 .../resources/images/timeline_structure.jpg     | Bin 0 -> 23070 bytes
 3 files changed, 165 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fdef76c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 94f992d..77f8819 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -387,6 +387,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3187. Documentation of Capacity Scheduler Queue mapping based on user
     or group. (Gururaj Shetty via jianhe)
 
+    YARN-2854. Updated the documentation of the timeline service and the generic
+    history service. (Naganarasimha G R via zjshen)
+
   OPTIMIZATIONS
 
     YARN-2990. FairScheduler's delay-scheduling always waits for node-local and 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fdef76c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
index 4889936..31fe4ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
@@ -16,144 +16,122 @@ YARN Timeline Server
 ====================
 
 * [Overview](#Overview)
-* [Current Status](#Current_Status)
-* [Basic Configuration](#Basic_Configuration)
-* [Advanced Configuration](#Advanced_Configuration)
-* [Generic-data related Configuration](#Generic-data_related_Configuration)
-* [Per-framework-date related Configuration](#Per-framework-date_related_Configuration)
-* [Running Timeline server](#Running_Timeline_server)
-* [Accessing generic-data via command-line](#Accessing_generic-data_via_command-line)
-* [Publishing of per-framework data by applications](#Publishing_of_per-framework_data_by_applications)
+    * [Introduction](#Introduction)
+    * [Current Status](#Current_Status)
+    * [Timeline Structure](#Timeline_Structure)
+* [Deployment](#Deployment)
+    * [Configurations](#Configurations)
+    * [Running Timeline server](#Running_Timeline_server)
+    * [Accessing generic-data via command-line](#Accessing_generic-data_via_command-line)
+* [Publishing of application specific data](#Publishing_of_application_specific_data)
 
 Overview
---------
+---------
 
-Storage and retrieval of applications' current as well as historic information in a generic fashion is solved in YARN through the Timeline Server (previously also called Generic Application History Server). This serves two responsibilities:
+### Introduction  
 
-* Generic information about completed applications
-    
-    Generic information includes application level data like queue-name, user information etc in the ApplicationSubmissionContext, list of application-attempts that ran for an application, information about each application-attempt, list of containers run under each application-attempt, and information about each container. Generic data is stored by ResourceManager to a history-store (default implementation on a file-system) and used by the web-UI to display information about completed applications.
+ Storage and retrieval of application's current as well as historic information in a generic fashion is solved in YARN through the Timeline Server. This serves two responsibilities:
 
-* Per-framework information of running and completed applications
-    
-    Per-framework information is completely specific to an application or framework. For example, Hadoop MapReduce framework can include pieces of information like number of map tasks, reduce tasks, counters etc. Application developers can publish the specific information to the Timeline server via TimelineClient from within a client, the ApplicationMaster and/or the application's containers. This information is then queryable via REST APIs for rendering by application/framework specific UIs.
+#### Application specific information
 
-Current Status
---------------
+  Supports collection of information completely specific to an application or framework. For example, Hadoop MapReduce framework can include pieces of information like number of map tasks, reduce tasks, counters etc. Application developers can publish the specific information to the Timeline server via TimelineClient, the ApplicationMaster and/or the application's containers. This information is then queryable via REST APIs for rendering by application/framework specific UIs.
 
-Timeline sever is a work in progress. The basic storage and retrieval of information, both generic and framework specific, are in place. Timeline server doesn't work in secure mode yet. The generic information and the per-framework information are today collected and presented separately and thus are not integrated well together. Finally, the per-framework information is only available via RESTful APIs, using JSON type content - ability to install framework specific UIs in YARN isn't supported yet.
+#### Generic information about completed applications
+  
+  Previously this was done by Application History Server but with  timeline server its just one use case of Timeline server functionality. Generic information includes application level data like queue-name, user information etc in the ApplicationSubmissionContext, list of application-attempts that ran for an application, information about each application-attempt, list of containers run under each application-attempt, and information about each container. Generic data is published by ResourceManager to the timeline store and used by the web-UI to display information about completed applications.
+ 
 
-Basic Configuration
--------------------
+### Current Status
 
-Users need to configure the Timeline server before starting it. The simplest configuration you should add in `yarn-site.xml` is to set the hostname of the Timeline server.
+  Current version of Timeline sever has been completed. Essential functionality of the timeline server can work in both secure and non secure modes. The generic history service is also ridden on the timeline store. In subsequent releases we will be rolling out next generation timeline service which is scalable and reliable. Finally, the Application specific information is only available via RESTful APIs, using JSON type content - ability to install framework specific UIs in YARN isn't supported yet.
 
-```xml
-<property>
-  <description>The hostname of the Timeline service web application.</description>
-  <name>yarn.timeline-service.hostname</name>
-  <value>0.0.0.0</value>
-</property>
-```
+### Timeline Structure
 
-Advanced Configuration
-----------------------
+![Timeline Structure] (./images/timeline_structure.jpg)
 
-In addition to the hostname, admins can also configure whether the service is enabled or not, the ports of the RPC and the web interfaces, and the number of RPC handler threads.
+#### TimelineDomain
 
-```xml
-<property>
-  <description>Address for the Timeline server to start the RPC server.</description>
-  <name>yarn.timeline-service.address</name>
-  <value>${yarn.timeline-service.hostname}:10200</value>
-</property>
+  Domain is like namespace for Timeline server and users can host multiple entities, isolating them from others. Timeline server Security is defined at this level. Domain majorly stores owner info, read & write ACL information, created and modified time stamp information. Domain is uniquely identified by ID.
 
-<property>
-  <description>The http address of the Timeline service web application.</description>
-  <name>yarn.timeline-service.webapp.address</name>
-  <value>${yarn.timeline-service.hostname}:8188</value>
-</property>
+#### TimelineEntity
 
-<property>
-  <description>The https address of the Timeline service web application.</description>
-  <name>yarn.timeline-service.webapp.https.address</name>
-  <value>${yarn.timeline-service.hostname}:8190</value>
-</property>
+  Entity contains the the meta information of some conceptual entity and its related events. The entity can be an application, an application attempt, a container or whatever the user-defined object. It contains Primary filters which will be used to index the entities in TimelineStore, such that users should carefully choose the information they want to store as the primary filters. The remaining data can be stored as other information. Entity is uniquely identified by EntityId and EntityType.
 
-<property>
-  <description>Handler thread count to serve the client RPC requests.</description>
-  <name>yarn.timeline-service.handler-thread-count</name>
-  <value>10</value>
-</property>
+#### TimelineEvent
 
-<property>
-  <description>Enables cross-origin support (CORS) for web services where
-  cross-origin web response headers are needed. For example, javascript making
-  a web services request to the timeline server.</description>
-  <name>yarn.timeline-service.http-cross-origin.enabled</name>
-  <value>false</value>
-</property>
+  TimelineEvent contains the information of an event that is related to some conceptual entity of an application. Users are free to define what the event means, such as starting an application, getting allocated a container and etc.
 
-<property>
-  <description>Comma separated list of origins that are allowed for web
-  services needing cross-origin (CORS) support. Wildcards (*) and patterns
-  allowed</description>
-  <name>yarn.timeline-service.http-cross-origin.allowed-origins</name>
-  <value>*</value>
-</property>
+Deployment
+----------
 
-<property>
-  <description>Comma separated list of methods that are allowed for web
-  services needing cross-origin (CORS) support.</description>
-  <name>yarn.timeline-service.http-cross-origin.allowed-methods</name>
-  <value>GET,POST,HEAD</value>
-</property>
+###Configurations
 
-<property>
-  <description>Comma separated list of headers that are allowed for web
-  services needing cross-origin (CORS) support.</description>
-  <name>yarn.timeline-service.http-cross-origin.allowed-headers</name>
-  <value>X-Requested-With,Content-Type,Accept,Origin</value>
-</property>
+#### Basic Configuration
 
-<property>
-  <description>The number of seconds a pre-flighted request can be cached
-  for web services needing cross-origin (CORS) support.</description>
-  <name>yarn.timeline-service.http-cross-origin.max-age</name>
-  <value>1800</value>
-</property>
-```
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.enabled` | Indicate to clients whether Timeline service is enabled or not. If enabled, the TimelineClient library used by end-users will post entities and events to the Timeline server. Defaults to false. |
+| `yarn.resourcemanager.system-metrics-publisher.enabled` | The setting that controls whether yarn system metrics is published on the timeline server or not by RM. Defaults to false. |
+| `yarn.timeline-service.generic-application-history.enabled` | Indicate to clients whether to query generic application data from timeline history-service. If not enabled then application data is only queried from Resource Manager. Defaults to false. |
 
-Generic-data related Configuration
-----------------------------------
+#### Advanced configuration
 
-Users can specify whether the generic data collection is enabled or not, and also choose the storage-implementation class for the generic data. There are more configurations related to generic data collection, and users can refer to `yarn-default.xml` for all of them.
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.ttl-enable` | Enable age off of timeline store data. Defaults to true. |
+| `yarn.timeline-service.ttl-ms` | Time to live for timeline store data in milliseconds. Defaults to 604800000 (7 days). |
+| `yarn.timeline-service.handler-thread-count` | Handler thread count to serve the client RPC requests. Defaults to 10. |
+| `yarn.timeline-service.client.max-retries` | Default maximum number of retires for timeline servive client. Defaults to 30. |
+| `yarn.timeline-service.client.retry-interval-ms` | Default retry time interval for timeline servive client. Defaults to 1000. |
 
-```xml
-<property>
-  <description>Indicate to ResourceManager as well as clients whether
-  history-service is enabled or not. If enabled, ResourceManager starts
-  recording historical data that Timelien service can consume. Similarly,
-  clients can redirect to the history service when applications
-  finish if this is enabled.</description>
-  <name>yarn.timeline-service.generic-application-history.enabled</name>
-  <value>false</value>
-</property>
+#### Timeline store and state store configuration
 
-<property>
-  <description>Store class name for history store, defaulting to file system
-  store</description>
-  <name>yarn.timeline-service.generic-application-history.store-class</name>
-  <value>org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore</value>
-</property>
-```
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.store-class` | Store class name for timeline store. Defaults to org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore. |
+| `yarn.timeline-service.leveldb-timeline-store.path` | Store file name for leveldb timeline store. Defaults to ${hadoop.tmp.dir}/yarn/timeline. |
+| `yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms` | Length of time to wait between deletion cycles of leveldb timeline store in milliseconds. Defaults to 300000. |
+| `yarn.timeline-service.leveldb-timeline-store.read-cache-size` | Size of read cache for uncompressed blocks for leveldb timeline store in bytes. Defaults to 104857600. |
+| `yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size` | Size of cache for recently read entity start times for leveldb timeline store in number of entities. Defaults to 10000. |
+| `yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size` | Size of cache for recently written entity start times for leveldb timeline store in number of entities. Defaults to 10000. |
+| `yarn.timeline-service.recovery.enabled` | Defaults to false. |
+| `yarn.timeline-service.state-store-class` | Store class name for timeline state store. Defaults to org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore. |
+| `yarn.timeline-service.leveldb-state-store.path` | Store file name for leveldb timeline state store. |
+
+#### Web and RPC Configuration
+
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.hostname` | The hostname of the Timeline service web application. Defaults to 0.0.0.0. |
+| `yarn.timeline-service.address` | Address for the Timeline server to start the RPC server. Defaults to ${yarn.timeline-service.hostname}:10200. |
+| `yarn.timeline-service.webapp.address` | The http address of the Timeline service web application. Defaults to ${yarn.timeline-service.hostname}:8188. |
+| `yarn.timeline-service.webapp.https.address` | The https address of the Timeline service web application. Defaults to ${yarn.timeline-service.hostname}:8190. |
+| `yarn.timeline-service.bind-host` | The actual address the server will bind to. If this optional address is set, the RPC and webapp servers will bind to this address and the port specified in yarn.timeline-service.address and yarn.timeline-service.webapp.address, respectively. This is most useful for making the service listen to all interfaces by setting to 0.0.0.0. |
+| `yarn.timeline-service.http-cross-origin.enabled` | Enables cross-origin support (CORS) for web services where cross-origin web response headers are needed. For example, javascript making a web services request to the timeline server. Defaults to false. |
+| `yarn.timeline-service.http-cross-origin.allowed-origins` | Comma separated list of origins that are allowed for web services needing cross-origin (CORS) support. Wildcards `(*)` and patterns allowed. Defaults to `*`. |
+| yarn.timeline-service.http-cross-origin.allowed-methods | Comma separated list of methods that are allowed for web services needing cross-origin (CORS) support. Defaults to GET,POST,HEAD. |
+| `yarn.timeline-service.http-cross-origin.allowed-headers` | Comma separated list of headers that are allowed for web services needing cross-origin (CORS) support. Defaults to X-Requested-With,Content-Type,Accept,Origin. |
+| `yarn.timeline-service.http-cross-origin.max-age` | The number of seconds a pre-flighted request can be cached for web services needing cross-origin (CORS) support. Defaults to 1800. |
+
+#### Security Configuration
+
+ Security can be enabled by setting yarn.timeline-service.http-authentication.type to kerberos and further following configurations can be done.
+
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.http-authentication.type` | Defines authentication used for the timeline server HTTP endpoint. Supported values are: simple / kerberos / #AUTHENTICATION_HANDLER_CLASSNAME#. Defaults to simple. |
+| `yarn.timeline-service.http-authentication.simple.anonymous.allowed` | Indicates if anonymous requests are allowed by the timeline server when using 'simple' authentication. Defaults to true. |
+| `yarn.timeline-service.principal` | The Kerberos principal for the timeline server. |
+| yarn.timeline-service.keytab | The Kerberos keytab for the timeline server. Defaults to /etc/krb5.keytab. |
+| `yarn.timeline-service.delegation.key.update-interval` | Defaults to 86400000 (1 day). |
+| `yarn.timeline-service.delegation.token.renew-interval` | Defaults to 86400000 (1 day). |
+| `yarn.timeline-service.delegation.token.max-lifetime` | Defaults to 604800000 (7 day). |
 
-Per-framework-date related Configuration
-----------------------------------------
+#### Enabling the timeline service and the generic history service
 
-Users can specify whether per-framework data service is enabled or not, choose the store implementation for the per-framework data, and tune the retention of the per-framework data. There are more configurations related to per-framework data service, and users can refer to `yarn-default.xml` for all of them.
+  Following are the basic configuration to start Timeline server.
 
-```xml
+```
 <property>
   <description>Indicate to clients whether Timeline service is enabled or not.
   If enabled, the TimelineClient library used by end-users will post entities
@@ -163,69 +141,97 @@ Users can specify whether per-framework data service is enabled or not, choose t
 </property>
 
 <property>
-  <description>Store class name for timeline store.</description>
-  <name>yarn.timeline-service.store-class</name>
-  <value>org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore</value>
+    <description>The setting that controls whether yarn system metrics is
+    published on the timeline server or not by RM.</description>
+    <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
+    <value>true</value>
 </property>
 
 <property>
-  <description>Enable age off of timeline store data.</description>
-  <name>yarn.timeline-service.ttl-enable</name>
+  <description>Indicate to clients whether to query generic application data from
+  timeline history-service. If not enabled then application data is only queried 
+  from Resource Manager</description>
+  <name>yarn.timeline-service.generic-application-history.enabled</name>
   <value>true</value>
 </property>
-
-<property>
-  <description>Time to live for timeline store data in milliseconds.</description>
-  <name>yarn.timeline-service.ttl-ms</name>
-  <value>604800000</value>
-</property>
 ```
 
-Running Timeline server
------------------------
+### Running Timeline server
 
-Assuming all the aforementioned configurations are set properly, admins can start the Timeline server/history service with the following command:
+  Assuming all the aforementioned configurations are set properly, admins can start the Timeline server/history service with the following command:
 
-      $ yarn timelineserver
+```
+  $ yarn timelineserver
+```
 
-Or users can start the Timeline server / history service as a daemon:
+  Or users can start the Timeline server / history service as a daemon:
 
-      $ yarn --daemon start timelineserver
+```
+  $ yarn --daemon start timelineserver
+```
 
-Accessing generic-data via command-line
----------------------------------------
+### Accessing generic-data via command-line
 
-Users can access applications' generic historic data via the command line as below. Note that the same commands are usable to obtain the corresponding information about running applications.
+  Users can access applications' generic historic data via the command line as below. Note that the same commands are usable to obtain the corresponding information about running applications.
 
 ```
-      $ yarn application -status <Application ID>
-      $ yarn applicationattempt -list <Application ID>
-      $ yarn applicationattempt -status <Application Attempt ID>
-      $ yarn container -list <Application Attempt ID>
-      $ yarn container -status <Container ID>
+  $ yarn application -status <Application ID>
+  $ yarn applicationattempt -list <Application ID>
+  $ yarn applicationattempt -status <Application Attempt ID>
+  $ yarn container -list <Application Attempt ID>
+  $ yarn container -status <Container ID>
 ```
 
-Publishing of per-framework data by applications
+Publishing of application specific data
 ------------------------------------------------
 
-Developers can define what information they want to record for their applications by composing `TimelineEntity` and `TimelineEvent` objects, and put the entities and events to the Timeline server via `TimelineClient`. Following is an example:
-
-```java
-// Create and start the Timeline client
-TimelineClient client = TimelineClient.createTimelineClient();
-client.init(conf);
-client.start();
-
-TimelineEntity entity = null;
-// Compose the entity
-try {
-  TimelinePutResponse response = client.putEntities(entity);
-} catch (IOException e) {
-  // Handle the exception
-} catch (YarnException e) {
-  // Handle the exception
-}
-
-// Stop the Timeline client
-client.stop();
+  Developers can define what information they want to record for their applications by composing `TimelineEntity` and  `TimelineEvent` objects, and put the entities and events to the Timeline server via `TimelineClient`. Below is an example:
+
+```
+  // Create and start the Timeline client
+  TimelineClient client = TimelineClient.createTimelineClient();
+  client.init(conf);
+  client.start();
+
+  try {
+    TimelineDomain myDomain = new TimelineDomain();
+    myDomain.setID("MyDomain");
+    // Compose other Domain info ....
+
+    client.putDomain(myDomain);
+
+    TimelineEntity myEntity = new TimelineEntity();
+    myEntity.setDomainId(myDomain.getId());
+    myEntity.setEntityType("APPLICATION");
+    myEntity.setEntityID("MyApp1")
+    // Compose other entity info
+
+    TimelinePutResponse response = client.putEntities(entity);
+
+    
+    TimelineEvent event = new TimelineEvent();
+    event.setEventType("APP_FINISHED");
+    event.setTimestamp(System.currentTimeMillis());
+    event.addEventInfo("Exit Status", "SUCCESS");
+    // Compose other Event info ....
+
+    myEntity.addEvent(event);
+    timelineClient.putEntities(entity);
+
+  } catch (IOException e) {
+    // Handle the exception
+  } catch (YarnException e) {
+    // Handle the exception
+  }
+
+  // Stop the Timeline client
+  client.stop();
 ```
+
+  **Note** : Following are the points which needs to be observed during updating a entity.
+
+  * Domain ID should not be modified for already existing entity.
+
+  * Its advisable to have same primary filters for all updates on entity. As on modification of primary filter by subsequent updates will result in not fetching the information before the update when queried with updated primary filter.
+
+  * On modification of Primary filter value, new value will be appended with the old value.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6fdef76c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_structure.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_structure.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_structure.jpg
new file mode 100644
index 0000000..dbfce25
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_structure.jpg differ


[23/50] [abbrv] hadoop git commit: Revert "HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Masatake Iwasaki."

Posted by zh...@apache.org.
Revert "HADOOP-11558. Fix dead links to doc of hadoop-tools. Contributed by Masatake Iwasaki."

This reverts commit 7da136ecca4dafc83ef69b5d9980fa5b67ada084.


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

Branch: refs/heads/HDFS-7285
Commit: b308a8d181416b5fe6bf77756e5f2c7b8fbd793c
Parents: 7da136e
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sun Mar 15 14:27:38 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sun Mar 15 14:27:38 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt       |  3 ---
 .../src/site/markdown/SchedulerLoadSimulator.md       |  2 +-
 .../src/site/markdown/HadoopStreaming.md.vm           | 14 +++++++-------
 3 files changed, 8 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b308a8d1/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index e386723..55028cb 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1100,9 +1100,6 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11710. Make CryptoOutputStream behave like DFSOutputStream wrt
     synchronization. (Sean Busbey via yliu)
 
-    HADOOP-11558. Fix dead links to doc of hadoop-tools. (Masatake Iwasaki
-    via ozawa)
-
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b308a8d1/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
index 2cffc86..ca179ee 100644
--- a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
+++ b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
@@ -43,7 +43,7 @@ The Yarn Scheduler Load Simulator (SLS) is such a tool, which can simulate large
 o
 The simulator will exercise the real Yarn `ResourceManager` removing the network factor by simulating `NodeManagers` and `ApplicationMasters` via handling and dispatching `NM`/`AMs` heartbeat events from within the same JVM. To keep tracking of scheduler behavior and performance, a scheduler wrapper will wrap the real scheduler.
 
-The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](../hadoop-rumen/Rumen.html).
+The size of the cluster and the application load can be loaded from configuration files, which are generated from job history files directly by adopting [Apache Rumen](https://hadoop.apache.org/docs/stable/rumen.html).
 
 The simulator will produce real time metrics while executing, including:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b308a8d1/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
index b4c5e38..0b64586 100644
--- a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
+++ b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
@@ -201,7 +201,7 @@ To specify additional local temp directories use:
      -D mapred.system.dir=/tmp/system
      -D mapred.temp.dir=/tmp/temp
 
-**Note:** For more details on job configuration parameters see: [mapred-default.xml](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/mapred-default.xml)
+**Note:** For more details on job configuration parameters see: [mapred-default.xml](./mapred-default.xml)
 
 $H4 Specifying Map-Only Jobs
 
@@ -322,7 +322,7 @@ More Usage Examples
 
 $H3 Hadoop Partitioner Class
 
-Hadoop has a library class, [KeyFieldBasedPartitioner](../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
+Hadoop has a library class, [KeyFieldBasedPartitioner](../../api/org/apache/hadoop/mapred/lib/KeyFieldBasedPartitioner.html), that is useful for many applications. This class allows the Map/Reduce framework to partition the map outputs based on certain key fields, not the whole keys. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D stream.map.output.field.separator=. \
@@ -372,7 +372,7 @@ Sorting within each partition for the reducer(all 4 fields used for sorting)
 
 $H3 Hadoop Comparator Class
 
-Hadoop has a library class, [KeyFieldBasedComparator](../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
+Hadoop has a library class, [KeyFieldBasedComparator](../../api/org/apache/hadoop/mapreduce/lib/partition/KeyFieldBasedComparator.html), that is useful for many applications. This class provides a subset of features provided by the Unix/GNU Sort. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.job.output.key.comparator.class=org.apache.hadoop.mapreduce.lib.partition.KeyFieldBasedComparator \
@@ -406,7 +406,7 @@ Sorting output for the reducer (where second field used for sorting)
 
 $H3 Hadoop Aggregate Package
 
-Hadoop has a library package called [Aggregate](../api/org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
+Hadoop has a library package called [Aggregate](../../org/apache/hadoop/mapred/lib/aggregate/package-summary.html). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
 
 To use Aggregate, simply specify "-reducer aggregate":
 
@@ -441,7 +441,7 @@ The python program myAggregatorForKeyCount.py looks like:
 
 $H3 Hadoop Field Selection Class
 
-Hadoop has a library class, [FieldSelectionMapReduce](../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
+Hadoop has a library class, [FieldSelectionMapReduce](../../api/org/apache/hadoop/mapred/lib/FieldSelectionMapReduce.html), that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
 
     hadoop jar hadoop-streaming-${project.version}.jar \
       -D mapreduce.map.output.key.field.separator=. \
@@ -480,7 +480,7 @@ As an example, consider the problem of zipping (compressing) a set of files acro
 
 $H3 How many reducers should I use?
 
-See MapReduce Tutorial for details: [Reducer](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Reducer)
+See MapReduce Tutorial for details: [Reducer](./MapReduceTutorial.html#Reducer)
 
 $H3 If I set up an alias in my shell script, will that work after -mapper?
 
@@ -556,4 +556,4 @@ A streaming process can use the stderr to emit status information. To set a stat
 
 $H3 How do I get the Job variables in a streaming job's mapper/reducer?
 
-See [Configured Parameters](../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.
+See [Configured Parameters](./MapReduceTutorial.html#Configured_Parameters). During the execution of a streaming job, the names of the "mapred" parameters are transformed. The dots ( . ) become underscores ( \_ ). For example, mapreduce.job.id becomes mapreduce\_job\_id and mapreduce.job.jar becomes mapreduce\_job\_jar. In your code, use the parameter names with the underscores.


[50/50] [abbrv] hadoop git commit: Fixed a compiling issue introduced by HADOOP-11705.

Posted by zh...@apache.org.
Fixed a compiling issue introduced by HADOOP-11705.


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

Branch: refs/heads/HDFS-7285
Commit: 4a6ab375835c149571d1ae59a006b44802d02f34
Parents: 7018997
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Mar 13 00:13:06 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 16 13:14:01 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a6ab375/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 36e061a..d911db9 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
@@ -162,7 +162,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     encoder.initialize(numDataUnits, numParityUnits, chunkSize);
-    encoder.setConf(conf);
+    ((AbstractErasureCoder)encoder).setConf(conf);
     return encoder;
   }
 
@@ -179,7 +179,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     decoder.initialize(numDataUnits, numParityUnits, chunkSize);
-    decoder.setConf(conf);
+    ((AbstractErasureCoder)decoder).setConf(conf);
     return decoder;
   }