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/30 19:24:07 UTC

[01/50] [abbrv] hadoop git commit: HDFS-7990. IBR delete ack should not be delayed. Contributed by Daryn Sharp.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 27cb5701e -> a6543ac97 (forced update)


HDFS-7990. IBR delete ack should not be delayed. 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/60882ab2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/60882ab2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/60882ab2

Branch: refs/heads/HDFS-7285
Commit: 60882ab26d49f05cbf0686944af6559f86b3417d
Parents: af618f2
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Mar 27 09:05:17 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Mar 27 09:05:17 2015 -0500

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt        |  2 ++
 .../hdfs/server/datanode/BPServiceActor.java       | 17 +++++++----------
 .../apache/hadoop/hdfs/server/datanode/DNConf.java |  2 --
 .../hdfs/server/datanode/SimulatedFSDataset.java   | 13 ++++++++++++-
 .../datanode/TestIncrementalBlockReports.java      |  4 ++--
 5 files changed, 23 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/60882ab2/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 dff8bd2..72ea4fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -342,6 +342,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-7928. Scanning blocks from disk during rolling upgrade startup takes
     a lot of time if disks are busy (Rushabh S Shah via kihwal)
 
+    HDFS-7990. IBR delete ack should not be delayed. (daryn via kihwal)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60882ab2/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 10cce45..3b4756c 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
@@ -82,12 +82,11 @@ class BPServiceActor implements Runnable {
 
   final BPOfferService bpos;
   
-  // lastBlockReport, lastDeletedReport and lastHeartbeat may be assigned/read
+  // lastBlockReport and lastHeartbeat may be assigned/read
   // by testing threads (through BPServiceActor#triggerXXX), while also 
   // assigned/read by the actor thread. Thus they should be declared as volatile
   // to make sure the "happens-before" consistency.
   volatile long lastBlockReport = 0;
-  volatile long lastDeletedReport = 0;
 
   boolean resetBlockReportTime = true;
 
@@ -417,10 +416,10 @@ class BPServiceActor implements Runnable {
   @VisibleForTesting
   void triggerDeletionReportForTests() {
     synchronized (pendingIncrementalBRperStorage) {
-      lastDeletedReport = 0;
+      sendImmediateIBR = true;
       pendingIncrementalBRperStorage.notifyAll();
 
-      while (lastDeletedReport == 0) {
+      while (sendImmediateIBR) {
         try {
           pendingIncrementalBRperStorage.wait(100);
         } catch (InterruptedException e) {
@@ -465,7 +464,6 @@ class BPServiceActor implements Runnable {
     // or we will report an RBW replica after the BlockReport already reports
     // a FINALIZED one.
     reportReceivedDeletedBlocks();
-    lastDeletedReport = startTime;
 
     long brCreateStartTime = monotonicNow();
     Map<DatanodeStorage, BlockListAsLongs> perVolumeBlockLists =
@@ -674,7 +672,6 @@ class BPServiceActor implements Runnable {
    */
   private void offerService() throws Exception {
     LOG.info("For namenode " + nnAddr + " using"
-        + " DELETEREPORT_INTERVAL of " + dnConf.deleteReportInterval + " msec "
         + " BLOCKREPORT_INTERVAL of " + dnConf.blockReportInterval + "msec"
         + " CACHEREPORT_INTERVAL of " + dnConf.cacheReportInterval + "msec"
         + " Initial delay: " + dnConf.initialBlockReportDelay + "msec"
@@ -690,7 +687,9 @@ class BPServiceActor implements Runnable {
         //
         // Every so often, send heartbeat or block-report
         //
-        if (startTime - lastHeartbeat >= dnConf.heartBeatInterval) {
+        boolean sendHeartbeat =
+            startTime - lastHeartbeat >= dnConf.heartBeatInterval;
+        if (sendHeartbeat) {
           //
           // All heartbeat messages include following info:
           // -- Datanode name
@@ -729,10 +728,8 @@ class BPServiceActor implements Runnable {
             }
           }
         }
-        if (sendImmediateIBR ||
-            (startTime - lastDeletedReport > dnConf.deleteReportInterval)) {
+        if (sendImmediateIBR || sendHeartbeat) {
           reportReceivedDeletedBlocks();
-          lastDeletedReport = startTime;
         }
 
         List<DatanodeCommand> cmds = blockReport();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60882ab2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
index 67cd1ce..3406f29 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
@@ -82,7 +82,6 @@ public class DNConf {
   final long heartBeatInterval;
   final long blockReportInterval;
   final long blockReportSplitThreshold;
-  final long deleteReportInterval;
   final long initialBlockReportDelay;
   final long cacheReportInterval;
   final long dfsclientSlowIoWarningThresholdMs;
@@ -164,7 +163,6 @@ public class DNConf {
     heartBeatInterval = conf.getLong(DFS_HEARTBEAT_INTERVAL_KEY,
         DFS_HEARTBEAT_INTERVAL_DEFAULT) * 1000L;
     
-    this.deleteReportInterval = 100 * heartBeatInterval;
     // do we need to sync block file contents to disk when blockfile is closed?
     this.syncOnClose = conf.getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY, 
         DFS_DATANODE_SYNCONCLOSE_DEFAULT);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60882ab2/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 5c7b4ac..23fc95b 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
@@ -84,7 +84,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     @Override
     public SimulatedFSDataset newInstance(DataNode datanode,
         DataStorage storage, Configuration conf) throws IOException {
-      return new SimulatedFSDataset(storage, conf);
+      return new SimulatedFSDataset(datanode, storage, conf);
     }
 
     @Override
@@ -509,8 +509,15 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   private final SimulatedStorage storage;
   private final SimulatedVolume volume;
   private final String datanodeUuid;
+  private final DataNode datanode;
   
+
   public SimulatedFSDataset(DataStorage storage, Configuration conf) {
+    this(null, storage, conf);
+  }
+
+  public SimulatedFSDataset(DataNode datanode, DataStorage storage, Configuration conf) {
+    this.datanode = datanode;
     if (storage != null) {
       for (int i = 0; i < storage.getNumStorageDirs(); ++i) {
         storage.createStorageID(storage.getStorageDir(i), false);
@@ -737,6 +744,10 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       }
       storage.free(bpid, binfo.getNumBytes());
       map.remove(b);
+      if (datanode != null) {
+        datanode.notifyNamenodeDeletedBlock(new ExtendedBlock(bpid, b),
+            binfo.getStorageUuid());
+      }
     }
     if (error) {
       throw new IOException("Invalidate: Missing blocks.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60882ab2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java
index b5aa93f..cd2c125 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java
@@ -159,8 +159,8 @@ public class TestIncrementalBlockReports {
           anyString(),
           any(StorageReceivedDeletedBlocks[].class));
 
-      // Trigger a block report, this also triggers an IBR.
-      DataNodeTestUtils.triggerBlockReport(singletonDn);
+      // Trigger a heartbeat, this also triggers an IBR.
+      DataNodeTestUtils.triggerHeartbeat(singletonDn);
       Thread.sleep(2000);
 
       // Ensure that the deleted block is reported.


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

Branch: refs/heads/HDFS-7285
Commit: 808cb1d38bcb143501fdb711e8de9e959f71e856
Parents: 7302ab1
Author: Kai Zheng <dr...@apache.org>
Authored: Sun Feb 8 01:40:27 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:24 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/808cb1d3/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/808cb1d3/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/808cb1d3/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/808cb1d3/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/808cb1d3/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);
+  }
+
+}


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

Branch: refs/heads/HDFS-7285
Commit: 7527a599b839b4e675c4b9f0ca36cbee2ddd2381
Parents: c37d982
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Feb 25 22:10:26 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:25 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     |   8 +-
 .../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(+), 320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7527a599/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 9446b70..f31acc5 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
@@ -172,6 +172,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;
@@ -184,6 +185,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;
@@ -430,6 +432,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/7527a599/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 b14efb4..440a081 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/7527a599/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/7527a599/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 c78c9e2..7a052fd 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/7527a599/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/7527a599/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/7527a599/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 d947425..53ee288 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
@@ -540,8 +540,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 + ": ");
     }
@@ -595,15 +595,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;
   }
   
@@ -621,16 +620,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;
   }
 
@@ -643,22 +642,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);
     
@@ -676,10 +678,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);
@@ -695,6 +696,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);
   }
@@ -716,7 +718,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;
@@ -725,8 +727,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.
@@ -768,11 +772,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++) {
@@ -785,10 +788,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();
@@ -799,9 +802,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++) {
@@ -815,8 +818,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);
@@ -825,8 +829,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(
@@ -839,6 +843,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();
@@ -873,7 +878,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,
@@ -896,7 +901,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;
@@ -1709,12 +1714,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;
@@ -2110,15 +2118,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
@@ -2141,7 +2146,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 
@@ -2271,9 +2276,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;
     }
 
@@ -2457,9 +2461,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())) {
@@ -2519,7 +2522,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 {
@@ -2535,7 +2539,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);
@@ -3494,7 +3497,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);
@@ -3540,21 +3543,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);
   }
@@ -3568,26 +3565,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.
@@ -3701,7 +3678,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/7527a599/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/7527a599/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 31a6af7..576c0b0 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
@@ -143,6 +143,7 @@ class FSDirConcatOp {
         throw new HadoopIllegalArgumentException("concat: source file " + src
             + " is invalid or empty or underConstruction");
       }
+
       // source file's preferred block size cannot be greater than the target
       // file
       if (srcINodeFile.getPreferredBlockSize() >
@@ -152,6 +153,11 @@ class FSDirConcatOp {
             + " which is greater than the target file's preferred block size "
             + targetINode.getPreferredBlockSize());
       }
+      // 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);
     }
 
@@ -228,7 +234,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/7527a599/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 9f41b80..e1b3570 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;
@@ -1064,7 +1065,7 @@ public class FSDirectory implements Closeable {
         unprotectedTruncate(iip, newLength, collectedBlocks, mtime, null);
 
     if(! onBlockBoundary) {
-      BlockInfoContiguous oldBlock = file.getLastBlock();
+      BlockInfo oldBlock = file.getLastBlock();
       Block tBlk =
       getFSNamesystem().prepareFileForTruncate(iip,
           clientName, clientMachine, file.computeFileSize() - newLength,
@@ -1073,7 +1074,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/7527a599/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 df9c585..392a670 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/7527a599/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 f50dc4d..946a37c 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/7527a599/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 cce991f..d62b804 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;
@@ -684,7 +685,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++) {
@@ -952,9 +953,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/7527a599/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/7527a599/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 e8e4758..39fc5fd 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
@@ -204,8 +204,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;
@@ -1980,6 +1982,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(
@@ -2062,8 +2069,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(),
@@ -2078,7 +2089,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: "
@@ -2659,6 +2671,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 &&
@@ -2670,7 +2689,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)) {
@@ -2726,7 +2746,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         }
       }
     } 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]);
@@ -2905,7 +2925,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(
@@ -3092,9 +3112,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);
         }
@@ -3184,7 +3203,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
@@ -3212,7 +3231,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() &&
@@ -3239,8 +3258,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
@@ -3531,6 +3549,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 {
@@ -3564,16 +3583,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;
         }
       }
@@ -3583,16 +3602,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;
@@ -3777,7 +3799,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) {
@@ -4001,10 +4023,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())
@@ -4039,12 +4061,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) {
@@ -4074,6 +4096,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
@@ -4185,14 +4208,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;
@@ -4243,7 +4265,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) {
@@ -4293,11 +4315,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 "
@@ -4310,8 +4330,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         if (remove) {
           blockManager.removeBlock(storedBlock);
         }
-      }
-      else {
+      } else {
         // update last block
         if(!copyTruncate) {
           storedBlock.setGenerationStamp(newgenerationstamp);
@@ -4345,7 +4364,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);
               }
@@ -4359,9 +4378,9 @@ 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) {
             blockManager.markBlockReplicasAsCorrupt(storedBlock,
                 oldGenerationStamp, oldNumBytes, trimmedStorageInfos);
@@ -4371,8 +4390,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       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 {
@@ -4405,7 +4424,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();
@@ -4715,7 +4734,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());
           }
@@ -5668,7 +5687,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());
     }
@@ -6213,7 +6232,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 + 
@@ -6342,8 +6361,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/7527a599/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);


[49/50] [abbrv] hadoop git commit: HADOOP-11664. Loading predefined EC schemas from configuration. Contributed by Kai Zheng.

Posted by zh...@apache.org.
HADOOP-11664. Loading predefined EC schemas from configuration. 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/d50bbd71
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d50bbd71
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d50bbd71

Branch: refs/heads/HDFS-7285
Commit: d50bbd71a08b56bccb4b47d11f131c0deb34bf2f
Parents: 8d49fc3
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Mar 27 14:52:50 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:13:09 2015 -0700

----------------------------------------------------------------------
 .../src/main/conf/ecschema-def.xml              |  40 +++++
 .../hadoop/fs/CommonConfigurationKeys.java      |   5 +
 .../hadoop/io/erasurecode/SchemaLoader.java     | 147 +++++++++++++++++++
 .../hadoop/io/erasurecode/TestSchemaLoader.java |  80 ++++++++++
 4 files changed, 272 insertions(+)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d50bbd71/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
new file mode 100644
index 0000000..c51ed37
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
@@ -0,0 +1,147 @@
+/**
+ * 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.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.w3c.dom.*;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.*;
+
+/**
+ * A EC schema loading utility that loads predefined EC schemas from XML file
+ */
+public class SchemaLoader {
+  private static final Log LOG = LogFactory.getLog(SchemaLoader.class.getName());
+
+  /**
+   * Load predefined ec schemas from configuration file. This file is
+   * expected to be in the XML format.
+   */
+  public List<ECSchema> loadSchema(Configuration conf) {
+    File confFile = getSchemaFile(conf);
+    if (confFile == null) {
+      LOG.warn("Not found any predefined EC schema file");
+      return Collections.emptyList();
+    }
+
+    try {
+      return loadSchema(confFile);
+    } catch (ParserConfigurationException e) {
+      throw new RuntimeException("Failed to load schema file: " + confFile);
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to load schema file: " + confFile);
+    } catch (SAXException e) {
+      throw new RuntimeException("Failed to load schema file: " + confFile);
+    }
+  }
+
+  private List<ECSchema> loadSchema(File schemaFile)
+      throws ParserConfigurationException, IOException, SAXException {
+
+    LOG.info("Loading predefined EC schema file " + schemaFile);
+
+    // Read and parse the schema file.
+    DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+    dbf.setIgnoringComments(true);
+    DocumentBuilder builder = dbf.newDocumentBuilder();
+    Document doc = builder.parse(schemaFile);
+    Element root = doc.getDocumentElement();
+
+    if (!"schemas".equals(root.getTagName())) {
+      throw new RuntimeException("Bad EC schema config file: " +
+          "top-level element not <schemas>");
+    }
+
+    NodeList elements = root.getChildNodes();
+    List<ECSchema> schemas = new ArrayList<ECSchema>();
+    for (int i = 0; i < elements.getLength(); i++) {
+      Node node = elements.item(i);
+      if (node instanceof Element) {
+        Element element = (Element) node;
+        if ("schema".equals(element.getTagName())) {
+          ECSchema schema = loadSchema(element);
+            schemas.add(schema);
+        } else {
+          LOG.warn("Bad element in EC schema configuration file: " +
+              element.getTagName());
+        }
+      }
+    }
+
+    return schemas;
+  }
+
+  /**
+   * Path to the XML file containing predefined ec schemas. If the path is
+   * relative, it is searched for in the classpath.
+   */
+  private File getSchemaFile(Configuration conf) {
+    String schemaFilePath = conf.get(
+        CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_KEY,
+        CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_DEFAULT);
+    File schemaFile = new File(schemaFilePath);
+    if (! schemaFile.isAbsolute()) {
+      URL url = Thread.currentThread().getContextClassLoader()
+          .getResource(schemaFilePath);
+      if (url == null) {
+        LOG.warn(schemaFilePath + " not found on the classpath.");
+        schemaFile = null;
+      } else if (! url.getProtocol().equalsIgnoreCase("file")) {
+        throw new RuntimeException(
+            "EC predefined schema file " + url +
+                " found on the classpath is not on the local filesystem.");
+      } else {
+        schemaFile = new File(url.getPath());
+      }
+    }
+
+    return schemaFile;
+  }
+
+  /**
+   * Loads a schema from a schema element in the configuration file
+   */
+  private ECSchema loadSchema(Element element) {
+    String schemaName = element.getAttribute("name");
+    Map<String, String> ecOptions = new HashMap<String, String>();
+    NodeList fields = element.getChildNodes();
+
+    for (int i = 0; i < fields.getLength(); i++) {
+      Node fieldNode = fields.item(i);
+      if (fieldNode instanceof Element) {
+        Element field = (Element) fieldNode;
+        String tagName = field.getTagName();
+        String value = ((Text) field.getFirstChild()).getData().trim();
+        ecOptions.put(tagName, value);
+      }
+    }
+
+    ECSchema schema = new ECSchema(schemaName, ecOptions);
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d50bbd71/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
new file mode 100644
index 0000000..7bb0a9a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.PrintWriter;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestSchemaLoader {
+
+  final static String TEST_DIR = new File(System.getProperty(
+      "test.build.data", "/tmp")).getAbsolutePath();
+
+  final static String SCHEMA_FILE = new File(TEST_DIR, "test-ecschema")
+      .getAbsolutePath();
+
+  @Test
+  public void testLoadSchema() throws Exception {
+    PrintWriter out = new PrintWriter(new FileWriter(SCHEMA_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<schemas>");
+    out.println("  <schema name=\"RSk6m3\">");
+    out.println("    <k>6</k>");
+    out.println("    <m>3</m>");
+    out.println("    <codec>RS</codec>");
+    out.println("  </schema>");
+    out.println("  <schema name=\"RSk10m4\">");
+    out.println("    <k>10</k>");
+    out.println("    <m>4</m>");
+    out.println("    <codec>RS</codec>");
+    out.println("  </schema>");
+    out.println("</schemas>");
+    out.close();
+
+    Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_KEY,
+        SCHEMA_FILE);
+
+    SchemaLoader schemaLoader = new SchemaLoader();
+    List<ECSchema> schemas = schemaLoader.loadSchema(conf);
+
+    assertEquals(2, schemas.size());
+
+    ECSchema schema1 = schemas.get(0);
+    assertEquals("RSk6m3", schema1.getSchemaName());
+    assertEquals(3, schema1.getOptions().size());
+    assertEquals(6, schema1.getNumDataUnits());
+    assertEquals(3, schema1.getNumParityUnits());
+    assertEquals("RS", schema1.getCodecName());
+
+    ECSchema schema2 = schemas.get(1);
+    assertEquals("RSk10m4", schema2.getSchemaName());
+    assertEquals(3, schema2.getOptions().size());
+    assertEquals(10, schema2.getNumDataUnits());
+    assertEquals(4, schema2.getNumParityUnits());
+    assertEquals("RS", schema2.getCodecName());
+  }
+}
\ No newline at end of file


[48/50] [abbrv] hadoop git commit: HDFS-7827. Erasure Coding: support striped blocks in non-protobuf fsimage. Contributed by Hui Zheng.

Posted by zh...@apache.org.
HDFS-7827. Erasure Coding: support striped blocks in non-protobuf fsimage. 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/37ecc116
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/37ecc116
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/37ecc116

Branch: refs/heads/HDFS-7285
Commit: 37ecc116e38c25e8a94ebd719f27235dee63ff9a
Parents: e238608
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 23 15:10:10 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:13:08 2015 -0700

----------------------------------------------------------------------
 .../blockmanagement/BlockInfoStriped.java       |  11 +-
 .../hdfs/server/namenode/FSImageFormat.java     |  62 ++++++--
 .../server/namenode/FSImageSerialization.java   |  78 +++++++---
 .../blockmanagement/TestBlockInfoStriped.java   |  34 +++++
 .../hdfs/server/namenode/TestFSImage.java       | 148 ++++++++++++++++++-
 5 files changed, 300 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ecc116/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 cef8318..30b5ee7 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
@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import java.io.DataOutput;
+import java.io.IOException;
 
 /**
  * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
@@ -206,6 +208,13 @@ public class BlockInfoStriped extends BlockInfo {
     return num;
   }
 
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeShort(dataBlockNum);
+    out.writeShort(parityBlockNum);
+    super.write(out);
+  }
+
   /**
    * Convert a complete block to an under construction block.
    * @return BlockInfoUnderConstruction -  an under construction block.
@@ -215,7 +224,7 @@ public class BlockInfoStriped extends BlockInfo {
     final BlockInfoStripedUnderConstruction ucBlock;
     if(isComplete()) {
       ucBlock = new BlockInfoStripedUnderConstruction(this, getDataBlockNum(),
-              getParityBlockNum(),  s, targets);
+              getParityBlockNum(), s, targets);
       ucBlock.setBlockCollection(getBlockCollection());
     } else {
       // the block is already under construction

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ecc116/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 2e6e741..ad96863 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
@@ -47,13 +47,16 @@ import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
 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.BlockInfoStriped;
 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.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -752,16 +755,31 @@ public class FSImageFormat {
       atime = in.readLong();
     }
     final long blockSize = in.readLong();
+    final boolean isStriped = NameNodeLayoutVersion.supports(
+            NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
+            && (in.readBoolean());
     final int numBlocks = in.readInt();
 
     if (numBlocks >= 0) {
       // file
       
       // read blocks
-      BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks];
-      for (int j = 0; j < numBlocks; j++) {
-        blocks[j] = new BlockInfoContiguous(replication);
-        blocks[j].readFields(in);
+      Block[] blocks;
+      if (isStriped) {
+        blocks = new Block[numBlocks];
+        for (int j = 0; j < numBlocks; j++) {
+          short dataBlockNum = in.readShort();
+          short parityBlockNum = in.readShort();
+          blocks[j] = new BlockInfoStriped(new Block(),
+                  dataBlockNum, parityBlockNum);
+          blocks[j].readFields(in);
+        }
+      } else {
+        blocks = new BlockInfoContiguous[numBlocks];
+        for (int j = 0; j < numBlocks; j++) {
+          blocks[j] = new BlockInfoContiguous(replication);
+          blocks[j].readFields(in);
+        }
       }
 
       String clientName = "";
@@ -780,9 +798,18 @@ public class FSImageFormat {
             clientMachine = FSImageSerialization.readString(in);
             // convert the last block to BlockUC
             if (blocks.length > 0) {
-              BlockInfoContiguous lastBlk = blocks[blocks.length - 1];
-              blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(
-                  lastBlk, replication);
+              Block lastBlk = blocks[blocks.length - 1];
+              if (isStriped){
+                BlockInfoStriped lastStripedBlk = (BlockInfoStriped) lastBlk;
+                blocks[blocks.length - 1]
+                        = new BlockInfoStripedUnderConstruction(lastBlk,
+                                lastStripedBlk.getDataBlockNum(),
+                                lastStripedBlk.getParityBlockNum());
+              } else {
+                blocks[blocks.length - 1]
+                        = new BlockInfoContiguousUnderConstruction(lastBlk,
+                                replication);
+              }
             }
           }
         }
@@ -795,14 +822,25 @@ public class FSImageFormat {
         counter.increment();
       }
 
-      final INodeFile file = new INodeFile(inodeId, localName, permissions,
-          modificationTime, atime, blocks, replication, blockSize);
+      INodeFile file;
+      if (isStriped) {
+        file = new INodeFile(inodeId, localName, permissions, modificationTime,
+            atime, new BlockInfoContiguous[0], (short) 0, blockSize);
+        file.addStripedBlocksFeature();
+        for (Block block : blocks) {
+          file.getStripedBlocksFeature().addBlock((BlockInfoStriped) block);
+        }
+      } else {
+        file = new INodeFile(inodeId, localName, permissions,
+            modificationTime, atime, (BlockInfoContiguous[]) blocks,
+            replication, blockSize);
+      }
       if (underConstruction) {
         file.toUnderConstruction(clientName, clientMachine);
       }
-        return fileDiffs == null ? file : new INodeFile(file, fileDiffs);
-      } else if (numBlocks == -1) {
-        //directory
+      return fileDiffs == null ? file : new INodeFile(file, fileDiffs);
+    } else if (numBlocks == -1) {
+      //directory
       
       //read quotas
       final long nsQuota = in.readLong();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ecc116/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index 1888d87..1e58858 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -124,21 +126,48 @@ public class FSImageSerialization {
     short blockReplication = in.readShort();
     long modificationTime = in.readLong();
     long preferredBlockSize = in.readLong();
+    final boolean isStriped = NameNodeLayoutVersion.supports(
+        NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
+        && (in.readBoolean());
   
     int numBlocks = in.readInt();
-    BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks];
-    Block blk = new Block();
-    int i = 0;
-    for (; i < numBlocks-1; i++) {
-      blk.readFields(in);
-      blocks[i] = new BlockInfoContiguous(blk, blockReplication);
-    }
-    // last block is UNDER_CONSTRUCTION
-    if(numBlocks > 0) {
-      blk.readFields(in);
-      blocks[i] = new BlockInfoContiguousUnderConstruction(
-        blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
+
+    final BlockInfoContiguous[] blocksContiguous;
+    BlockInfoStriped[] blocksStriped = null;
+    if (isStriped) {
+      blocksContiguous = new BlockInfoContiguous[0];
+      blocksStriped = new BlockInfoStriped[numBlocks];
+      int i = 0;
+      for (; i < numBlocks - 1; i++) {
+        short dataBlockNum = in.readShort();
+        short parityBlockNum = in.readShort();
+        blocksStriped[i] = new BlockInfoStriped(new Block(), dataBlockNum,
+            parityBlockNum);
+        blocksStriped[i].readFields(in);
+      }
+      if (numBlocks > 0) {
+        short dataBlockNum = in.readShort();
+        short parityBlockNum = in.readShort();
+        blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(),
+            dataBlockNum, parityBlockNum, BlockUCState.UNDER_CONSTRUCTION, null);
+        blocksStriped[i].readFields(in);
+      }
+    } else {
+      blocksContiguous = new BlockInfoContiguous[numBlocks];
+      Block blk = new Block();
+      int i = 0;
+      for (; i < numBlocks-1; i++) {
+        blk.readFields(in);
+        blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication);
+      }
+      // last block is UNDER_CONSTRUCTION
+      if(numBlocks > 0) {
+        blk.readFields(in);
+        blocksContiguous[i] = new BlockInfoContiguousUnderConstruction(
+                blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
+      }
     }
+
     PermissionStatus perm = PermissionStatus.read(in);
     String clientName = readString(in);
     String clientMachine = readString(in);
@@ -150,8 +179,19 @@ public class FSImageSerialization {
 
     // Images in the pre-protobuf format will not have the lazyPersist flag,
     // so it is safe to pass false always.
-    INodeFile file = new INodeFile(inodeId, name, perm, modificationTime,
-        modificationTime, blocks, blockReplication, preferredBlockSize);
+    INodeFile file;
+    if (isStriped) {
+      file = new INodeFile(inodeId, name, perm, modificationTime,
+          modificationTime, blocksContiguous, (short) 0, preferredBlockSize);
+      file.addStripedBlocksFeature();
+      for (int i = 0; i < numBlocks; i++) {
+        file.getStripedBlocksFeature().addBlock(blocksStriped[i]);
+      }
+    } else {
+      file = new INodeFile(inodeId, name, perm, modificationTime,
+          modificationTime, blocksContiguous, blockReplication,
+          preferredBlockSize);
+    }
     file.toUnderConstruction(clientName, clientMachine);
     return file;
   }
@@ -166,7 +206,8 @@ public class FSImageSerialization {
     out.writeShort(cons.getFileReplication());
     out.writeLong(cons.getModificationTime());
     out.writeLong(cons.getPreferredBlockSize());
-
+    // whether the file has striped blocks
+    out.writeBoolean(cons.isWithStripedBlocks());
     writeBlocks(cons.getBlocks(), out);
     cons.getPermissionStatus().write(out);
 
@@ -179,9 +220,9 @@ public class FSImageSerialization {
 
   /**
    * Serialize a {@link INodeFile} node
-   * @param node The node to write
+   * @param file The node to write
    * @param out The {@link DataOutputStream} where the fields are written
-   * @param writeBlock Whether to write block information
+   * @param writeUnderConstruction Whether to write block information
    */
   public static void writeINodeFile(INodeFile file, DataOutput out,
       boolean writeUnderConstruction) throws IOException {
@@ -191,7 +232,8 @@ public class FSImageSerialization {
     out.writeLong(file.getModificationTime());
     out.writeLong(file.getAccessTime());
     out.writeLong(file.getPreferredBlockSize());
-
+    // whether the file has striped blocks
+    out.writeBoolean(file.isWithStripedBlocks());
     writeBlocks(file.getBlocks(), out);
     SnapshotFSImageFormat.saveFileDiffList(file, out);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ecc116/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
index 74ddac0..c4db5d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
@@ -25,8 +25,16 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 /**
  * Test {@link BlockInfoStriped}
@@ -216,4 +224,30 @@ public class TestBlockInfoStriped {
       Assert.assertNull(newBlockInfo.getNext());
     }
   }
+
+  @Test
+  public void testWrite() {
+    long blkID = 1;
+    long numBytes = 1;
+    long generationStamp = 1;
+    short dataBlockNum = 6;
+    short parityBlockNum = 3;
+    ByteBuffer byteBuffer = ByteBuffer.allocate(Long.SIZE/Byte.SIZE*3
+            + Short.SIZE/Byte.SIZE*2);
+    byteBuffer.putShort(dataBlockNum).putShort(parityBlockNum)
+            .putLong(blkID).putLong(numBytes).putLong(generationStamp);
+
+    ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+    DataOutput out = new DataOutputStream(byteStream);
+    BlockInfoStriped blk = new BlockInfoStriped(new Block(1,1,1),
+            (short)6,(short)3);
+    try {
+      blk.write(out);
+    } catch(Exception ex) {
+      fail("testWrite error:" + ex.getMessage());
+    }
+    assertEquals(byteBuffer.array().length, byteStream.toByteArray().length);
+    assertArrayEquals(byteBuffer.array(), byteStream.toByteArray());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ecc116/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 4d42911..71dc978 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -17,18 +17,28 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.util.EnumSet;
 
 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.fs.permission.PermissionStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -42,8 +52,8 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.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;
@@ -120,6 +130,140 @@ public class TestFSImage {
     }
   }
 
+  private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf,
+      boolean isUC) throws IOException{
+    // contruct a INode with StripedBlock for saving and loading
+    long id = 123456789;
+    byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
+    PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
+            "testuser_groups", new FsPermission((short)0x755));
+    long mtime = 1426222916-3600;
+    long atime = 1426222916;
+    BlockInfoContiguous[] blks = new BlockInfoContiguous[0];
+    short replication = 3;
+    long preferredBlockSize = 128*1024*1024;
+    byte storagePolicyID = HdfsConstants.EC_STORAGE_POLICY_ID;
+    INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
+        blks, replication, preferredBlockSize, storagePolicyID);
+    ByteArrayOutputStream bs = new ByteArrayOutputStream();
+    file.addStripedBlocksFeature();
+
+    //construct StripedBlocks for the INode
+    BlockInfoStriped[] stripedBlks = new BlockInfoStriped[3];
+    long stripedBlkId = 10000001;
+    long timestamp = mtime+3600;
+    for (int i = 0; i < stripedBlks.length; i++) {
+      stripedBlks[i] = new BlockInfoStriped(
+              new Block(stripedBlkId + i, preferredBlockSize, timestamp),
+              (short) 6, (short) 3);
+      file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
+    }
+
+    final String client = "testClient";
+    final String clientMachine = "testClientMachine";
+    final String path = "testUnderConstructionPath";
+
+    //save the INode to byte array
+    DataOutput out = new DataOutputStream(bs);
+    if (isUC) {
+      file.toUnderConstruction(client, clientMachine);
+      FSImageSerialization.writeINodeUnderConstruction((DataOutputStream) out,
+          file, path);
+    } else {
+      FSImageSerialization.writeINodeFile(file, out, false);
+    }
+    DataInput in = new DataInputStream(
+            new ByteArrayInputStream(bs.toByteArray()));
+
+    // load the INode from the byte array
+    INodeFile fileByLoaded;
+    if (isUC) {
+      fileByLoaded = FSImageSerialization.readINodeUnderConstruction(in,
+              fsn, fsn.getFSImage().getLayoutVersion());
+    } else {
+      fileByLoaded = (INodeFile) new FSImageFormat.Loader(conf, fsn)
+              .loadINodeWithLocalName(false, in, false);
+    }
+
+    assertEquals(id, fileByLoaded.getId() );
+    assertArrayEquals(isUC ? path.getBytes() : name,
+        fileByLoaded.getLocalName().getBytes());
+    assertEquals(permissionStatus.getUserName(),
+        fileByLoaded.getPermissionStatus().getUserName());
+    assertEquals(permissionStatus.getGroupName(),
+        fileByLoaded.getPermissionStatus().getGroupName());
+    assertEquals(permissionStatus.getPermission(),
+        fileByLoaded.getPermissionStatus().getPermission());
+    assertEquals(mtime, fileByLoaded.getModificationTime());
+    assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime());
+    assertEquals(0, fileByLoaded.getContiguousBlocks().length);
+    assertEquals(0, fileByLoaded.getBlockReplication());
+    assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
+
+    //check the BlockInfoStriped
+    BlockInfoStriped[] stripedBlksByLoaded =
+        fileByLoaded.getStripedBlocksFeature().getBlocks();
+    assertEquals(3, stripedBlksByLoaded.length);
+    for (int i = 0; i < 3; i++) {
+      assertEquals(stripedBlks[i].getBlockId(),
+          stripedBlksByLoaded[i].getBlockId());
+      assertEquals(stripedBlks[i].getNumBytes(),
+          stripedBlksByLoaded[i].getNumBytes());
+      assertEquals(stripedBlks[i].getGenerationStamp(),
+          stripedBlksByLoaded[i].getGenerationStamp());
+      assertEquals(stripedBlks[i].getDataBlockNum(),
+          stripedBlksByLoaded[i].getDataBlockNum());
+      assertEquals(stripedBlks[i].getParityBlockNum(),
+          stripedBlksByLoaded[i].getParityBlockNum());
+    }
+
+    if (isUC) {
+      assertEquals(client,
+          fileByLoaded.getFileUnderConstructionFeature().getClientName());
+      assertEquals(clientMachine,
+          fileByLoaded.getFileUnderConstructionFeature().getClientMachine());
+    }
+  }
+
+  /**
+   * Test if a INodeFile with BlockInfoStriped can be saved by
+   * FSImageSerialization and loaded by FSImageFormat#Loader.
+   */
+  @Test
+  public void testSaveAndLoadInodeFile() throws IOException{
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, false);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test if a INodeFileUnderConstruction with BlockInfoStriped can be
+   * saved and loaded by FSImageSerialization
+   */
+  @Test
+  public void testSaveAndLoadInodeFileUC() throws IOException{
+    // construct a INode with StripedBlock for saving and loading
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   /**
    * Ensure that the digest written by the saver equals to the digest of the
    * file.


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

Branch: refs/heads/HDFS-7285
Commit: 80fe23f61c4cc7e53327500002185d558f4a685e
Parents: af9dc8e
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Mar 13 00:13:06 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:27 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/80fe23f6/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;
   }
 


[07/50] [abbrv] hadoop git commit: MAPREDUCE-6291. Correct mapred queue usage command. Contributed by Brahma Reddy Battula.

Posted by zh...@apache.org.
MAPREDUCE-6291. Correct mapred queue usage command. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7285
Commit: 27d49e6714ad7fc6038bc001e70ff5be3755f1ef
Parents: 89fb0f5
Author: Harsh J <ha...@cloudera.com>
Authored: Sat Mar 28 11:57:21 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Sat Mar 28 11:58:17 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../src/main/java/org/apache/hadoop/mapred/JobQueueClient.java    | 2 +-
 .../src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java   | 2 +-
 .../src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java      | 2 +-
 .../src/main/java/org/apache/hadoop/tools/HadoopArchives.java     | 2 +-
 5 files changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/27d49e67/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index ce16510..b0367a7 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -256,6 +256,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-6291. Correct mapred queue usage command.
+    (Brahma Reddu Battula via harsh)
+
     MAPREDUCE-579. Streaming "slowmatch" documentation. (harsh)
 
     MAPREDUCE-6287. Deprecated methods in org.apache.hadoop.examples.Sort

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27d49e67/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java
index 097e338..81f6140 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java
@@ -224,7 +224,7 @@ class JobQueueClient extends Configured implements Tool {
   }
 
   private void displayUsage(String cmd) {
-    String prefix = "Usage: JobQueueClient ";
+    String prefix = "Usage: queue ";
     if ("-queueinfo".equals(cmd)) {
       System.err.println(prefix + "[" + cmd + "<job-queue-name> [-showJobs]]");
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27d49e67/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java
index 8f4259e..4f5b6a1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Submitter.java
@@ -363,7 +363,7 @@ public class Submitter extends Configured implements Tool {
     void printUsage() {
       // The CLI package should do this for us, but I can't figure out how
       // to make it print something reasonable.
-      System.out.println("bin/hadoop pipes");
+      System.out.println("Usage: pipes ");
       System.out.println("  [-input <path>] // Input directory");
       System.out.println("  [-output <path>] // Output directory");
       System.out.println("  [-jar <jar file> // jar filename");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27d49e67/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java
index c3f1564..a350829 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java
@@ -421,7 +421,7 @@ public class CLI extends Configured implements Tool {
    * Display usage of the command-line tool and terminate execution.
    */
   private void displayUsage(String cmd) {
-    String prefix = "Usage: CLI ";
+    String prefix = "Usage: job ";
     String jobPriorityValues = getJobPriorityNames();
     String taskStates = "running, completed";
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27d49e67/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java b/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
index 18cd972..c5c42b1 100644
--- a/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
+++ b/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
@@ -101,7 +101,7 @@ public class HadoopArchives implements Tool {
   /** the desired replication degree; default is 10 **/
   short repl = 10;
 
-  private static final String usage = "archive"
+  private static final String usage = "Usage: archive"
   + " -archiveName <NAME>.har -p <parent path> [-r <replication factor>]" +
       "<src>* <dest>" +
   "\n";


[28/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/7527a599/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/7527a599/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 1858e0a..640fc57 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;
 
@@ -37,12 +38,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;
@@ -174,6 +175,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 */
 
   /**
@@ -208,7 +234,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);
     }
@@ -216,37 +242,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;
   }
 
   /**
@@ -256,19 +301,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 */
@@ -369,13 +420,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;
@@ -413,42 +466,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);
+      }
     }
   }
 
@@ -471,33 +539,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();
@@ -527,13 +595,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());
     }
@@ -725,7 +799,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();
@@ -733,15 +807,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);
@@ -761,21 +826,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
@@ -787,6 +864,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();
   }
 
@@ -796,9 +874,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;
@@ -859,21 +938,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;
@@ -881,7 +975,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/7527a599/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 9ce8ebc..6cd11b5 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/7527a599/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 0b3ed88..92e9f08 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/7527a599/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/7527a599/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/7527a599/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/7527a599/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/7527a599/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 7d94f04..58dbac5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -491,6 +491,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/7527a599/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 6d3bc1e..3eba280 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
@@ -57,6 +57,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;
@@ -1563,7 +1564,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/7527a599/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 fd28ded..6cd0d1c 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
@@ -1240,8 +1240,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/7527a599/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/7527a599/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/7527a599/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/7527a599/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/7527a599/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 fbcc73f..3e27107 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;
@@ -1036,7 +1037,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/7527a599/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/7527a599/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 85072d1..7bffb33 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/7527a599/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));
     }
     


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

Branch: refs/heads/HDFS-7285
Commit: 1d63b947e93001a6e9d4f51f09cfdc2050c44dea
Parents: e7ea2a8
Author: Vinayakumar B <vi...@apache.org>
Authored: Thu Nov 6 10:03:26 2014 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:22 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/1d63b947/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/1d63b947/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/1d63b947/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/1d63b947/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/1d63b947/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


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

Branch: refs/heads/HDFS-7285
Commit: daa78e3134643051204f584a95d97ef72ad17eb2
Parents: 3dbde16
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Mar 5 16:44:38 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:26 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/daa78e31/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 f522850..452c230 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
@@ -876,8 +876,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();
   }
 


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

Branch: refs/heads/HDFS-7285
Commit: ffc4171a750e8b122b29ec32307124d1a4f8e11b
Parents: 7886ed1
Author: Jing Zhao <ji...@apache.org>
Authored: Sun Feb 8 16:01:03 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:23 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/ffc4171a/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,


[09/50] [abbrv] hadoop git commit: HDFS-7501. TransactionsSinceLastCheckpoint can be negative on SBNs. Contributed by Gautam Gopalakrishnan.

Posted by zh...@apache.org.
HDFS-7501. TransactionsSinceLastCheckpoint can be negative on SBNs. Contributed by Gautam Gopalakrishnan.


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

Branch: refs/heads/HDFS-7285
Commit: 3d9132d434c39e9b6e142e5cf9fd7a8afa4190a6
Parents: e0ccea3
Author: Harsh J <ha...@cloudera.com>
Authored: Sun Mar 29 00:45:01 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Sun Mar 29 00:45:01 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/namenode/FSNamesystem.java      |  2 +-
 .../namenode/metrics/TestNameNodeMetrics.java   | 84 ++++++++++++++++++++
 3 files changed, 88 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d9132d4/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 f7cc2bc..496db06 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -351,6 +351,9 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
+    HDFS-7501. TransactionsSinceLastCheckpoint can be negative on SBNs.
+    (Gautam Gopalakrishnan via harsh)
+
     HDFS-5356. MiniDFSCluster should close all open FileSystems when shutdown()
     (Rakesh R via vinayakumarb)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d9132d4/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 d0999b8..0e0f484 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
@@ -4784,7 +4784,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   @Metric({"TransactionsSinceLastCheckpoint",
       "Number of transactions since last checkpoint"})
   public long getTransactionsSinceLastCheckpoint() {
-    return getEditLog().getLastWrittenTxId() -
+    return getFSImage().getLastAppliedOrWrittenTxId() -
         getFSImage().getStorage().getMostRecentCheckpointTxId();
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d9132d4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index 011db3c..64ea1e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -22,12 +22,16 @@ import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
 import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
 import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.Random;
+import com.google.common.collect.ImmutableList;
+import com.google.common.io.Files;
 
+import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
@@ -39,6 +43,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -47,7 +52,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.namenode.top.TopAuditLogger;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsSource;
@@ -69,6 +76,7 @@ public class TestNameNodeMetrics {
     new Path("/testNameNodeMetrics");
   private static final String NN_METRICS = "NameNodeActivity";
   private static final String NS_METRICS = "FSNamesystem";
+  public static final Log LOG = LogFactory.getLog(TestNameNodeMetrics.class);
   
   // Number of datanodes in the cluster
   private static final int DATANODE_COUNT = 3; 
@@ -400,6 +408,82 @@ public class TestNameNodeMetrics {
   }
   
   /**
+   * Testing TransactionsSinceLastCheckpoint. Need a new cluster as
+   * the other tests in here don't use HA. See HDFS-7501.
+   */
+  @Test(timeout = 300000)
+  public void testTransactionSinceLastCheckpointMetrics() throws Exception {
+    Random random = new Random();
+    int retryCount = 0;
+    while (retryCount < 5) {
+      try {
+        int basePort = 10060 + random.nextInt(100) * 2;
+        MiniDFSNNTopology topology = new MiniDFSNNTopology()
+            .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
+            .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(basePort))
+            .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(basePort + 1)));
+
+        HdfsConfiguration conf2 = new HdfsConfiguration();
+        // Lower the checkpoint condition for purpose of testing.
+        conf2.setInt(
+            DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY,
+            100);
+        // Check for checkpoint condition very often, for purpose of testing.
+        conf2.setInt(
+            DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY,
+            1);
+        // Poll and follow ANN txns very often, for purpose of testing.
+        conf2.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+        MiniDFSCluster cluster2 = new MiniDFSCluster.Builder(conf2)
+            .nnTopology(topology).numDataNodes(1).build();
+        cluster2.waitActive();
+        DistributedFileSystem fs2 = cluster2.getFileSystem(0);
+        NameNode nn0 = cluster2.getNameNode(0);
+        NameNode nn1 = cluster2.getNameNode(1);
+        cluster2.transitionToActive(0);
+        fs2.mkdirs(new Path("/tmp-t1"));
+        fs2.mkdirs(new Path("/tmp-t2"));
+        HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+        // Test to ensure tracking works before the first-ever
+        // checkpoint.
+        assertEquals("SBN failed to track 2 transactions pre-checkpoint.",
+            4L, // 2 txns added further when catch-up is called.
+            cluster2.getNameNode(1).getNamesystem()
+              .getTransactionsSinceLastCheckpoint());
+        // Complete up to the boundary required for
+        // an auto-checkpoint. Using 94 to expect fsimage
+        // rounded at 100, as 4 + 94 + 2 (catch-up call) = 100.
+        for (int i = 1; i <= 94; i++) {
+          fs2.mkdirs(new Path("/tmp-" + i));
+        }
+        HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+        // Assert 100 transactions in checkpoint.
+        HATestUtil.waitForCheckpoint(cluster2, 1, ImmutableList.of(100));
+        // Test to ensure number tracks the right state of
+        // uncheckpointed edits, and does not go negative
+        // (as fixed in HDFS-7501).
+        assertEquals("Should be zero right after the checkpoint.",
+            0L,
+            cluster2.getNameNode(1).getNamesystem()
+              .getTransactionsSinceLastCheckpoint());
+        fs2.mkdirs(new Path("/tmp-t3"));
+        fs2.mkdirs(new Path("/tmp-t4"));
+        HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+        // Test to ensure we track the right numbers after
+        // the checkpoint resets it to zero again.
+        assertEquals("SBN failed to track 2 added txns after the ckpt.",
+            4L,
+            cluster2.getNameNode(1).getNamesystem()
+              .getTransactionsSinceLastCheckpoint());
+        cluster2.shutdown();
+        break;
+      } catch (Exception e) {
+        LOG.warn("Unable to set up HA cluster, exception thrown: " + e);
+        retryCount++;
+      }
+    }
+  }
+  /**
    * Test NN checkpoint and transaction-related metrics.
    */
   @Test


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

Branch: refs/heads/HDFS-7285
Commit: 0614729ebfd1fc7d6373874e6a9e810e75643f58
Parents: 808cb1d
Author: drankye <dr...@gmail.com>
Authored: Mon Feb 9 22:04:08 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:24 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/0614729e/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 )


[38/50] [abbrv] hadoop git commit: HADOOP-11706 Refine a little bit erasure coder API

Posted by zh...@apache.org.
HADOOP-11706 Refine a little bit erasure coder API


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

Branch: refs/heads/HDFS-7285
Commit: 671db987ee966e015c47955f96d019602c934c0b
Parents: ace0181
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Mar 18 19:21:37 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:28 2015 -0700

----------------------------------------------------------------------
 .../io/erasurecode/coder/ErasureCoder.java      |  4 +++-
 .../erasurecode/rawcoder/RawErasureCoder.java   |  4 +++-
 .../hadoop/io/erasurecode/TestCoderBase.java    | 17 +++++++++++++---
 .../erasurecode/coder/TestErasureCoderBase.java | 21 +++-----------------
 .../erasurecode/rawcoder/TestJRSRawCoder.java   | 12 +++++------
 .../erasurecode/rawcoder/TestRawCoderBase.java  |  2 ++
 6 files changed, 31 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/671db987/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
index 68875c0..c5922f3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
+import org.apache.hadoop.conf.Configurable;
+
 /**
  * An erasure coder to perform encoding or decoding given a group. Generally it
  * involves calculating necessary internal steps according to codec logic. For
@@ -31,7 +33,7 @@ package org.apache.hadoop.io.erasurecode.coder;
  * of multiple coding steps.
  *
  */
-public interface ErasureCoder {
+public interface ErasureCoder extends Configurable {
 
   /**
    * Initialize with the important parameters for the code.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/671db987/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..9af5b6c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
+import org.apache.hadoop.conf.Configurable;
+
 /**
  * RawErasureCoder is a common interface for {@link RawErasureEncoder} and
  * {@link RawErasureDecoder} as both encoder and decoder share some properties.
@@ -31,7 +33,7 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
  * low level constructs, since it only takes care of the math calculation with
  * a group of byte buffers.
  */
-public interface RawErasureCoder {
+public interface RawErasureCoder extends Configurable {
 
   /**
    * Initialize with the important parameters for the code.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/671db987/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 194413a..22fd98d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -17,11 +17,12 @@
  */
 package org.apache.hadoop.io.erasurecode;
 
+import org.apache.hadoop.conf.Configuration;
+
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Random;
 
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -31,6 +32,7 @@ import static org.junit.Assert.assertTrue;
 public abstract class TestCoderBase {
   protected static Random RAND = new Random();
 
+  private Configuration conf;
   protected int numDataUnits;
   protected int numParityUnits;
   protected int chunkSize = 16 * 1024;
@@ -49,8 +51,9 @@ public abstract class TestCoderBase {
    * @param numParityUnits
    * @param erasedIndexes
    */
-  protected void prepare(int numDataUnits, int numParityUnits,
-                         int[] erasedIndexes) {
+  protected void prepare(Configuration conf, int numDataUnits,
+                         int numParityUnits, int[] erasedIndexes) {
+    this.conf = conf;
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
     this.erasedDataIndexes = erasedIndexes != null ?
@@ -58,6 +61,14 @@ public abstract class TestCoderBase {
   }
 
   /**
+   * Get the conf the test.
+   * @return configuration
+   */
+  protected Configuration getConf() {
+    return this.conf;
+  }
+
+  /**
    * Compare and verify if erased chunks are equal to recovered chunks
    * @param erasedChunks
    * @param recoveredChunks

http://git-wip-us.apache.org/repos/asf/hadoop/blob/671db987/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 d911db9..b963a59 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,10 +17,9 @@
  */
 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;
+import org.apache.hadoop.io.erasurecode.ECChunk;
 import org.apache.hadoop.io.erasurecode.TestCoderBase;
 
 /**
@@ -30,7 +29,6 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   protected Class<? extends ErasureEncoder> encoderClass;
   protected Class<? extends ErasureDecoder> decoderClass;
 
-  private Configuration conf;
   protected int numChunksInBlock = 16;
 
   /**
@@ -48,19 +46,6 @@ 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
@@ -162,7 +147,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     encoder.initialize(numDataUnits, numParityUnits, chunkSize);
-    ((AbstractErasureCoder)encoder).setConf(conf);
+    encoder.setConf(getConf());
     return encoder;
   }
 
@@ -179,7 +164,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     decoder.initialize(numDataUnits, numParityUnits, chunkSize);
-    ((AbstractErasureCoder)decoder).setConf(conf);
+    decoder.setConf(getConf());
     return decoder;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/671db987/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
index e54f647..39e5deb 100644
--- 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
@@ -46,37 +46,37 @@ public class TestJRSRawCoder extends TestRawCoderBase {
 
   @Test
   public void testCodingNoDirectBuffer_10x4() {
-    prepare(10, 4, null);
+    prepare(null, 10, 4, null);
     testCoding(false);
   }
 
   @Test
   public void testCodingDirectBuffer_10x4() {
-    prepare(10, 4, null);
+    prepare(null, 10, 4, null);
     testCoding(true);
   }
 
   @Test
   public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
-    prepare(10, 4, new int[] {2, 4});
+    prepare(null, 10, 4, new int[] {2, 4});
     testCoding(true);
   }
 
   @Test
   public void testCodingDirectBuffer_10x4_erasing_all() {
-    prepare(10, 4, new int[] {0, 1, 2, 3});
+    prepare(null, 10, 4, new int[] {0, 1, 2, 3});
     testCoding(true);
   }
 
   @Test
   public void testCodingNoDirectBuffer_3x3() {
-    prepare(3, 3, null);
+    prepare(null, 3, 3, null);
     testCoding(false);
   }
 
   @Test
   public void testCodingDirectBuffer_3x3() {
-    prepare(3, 3, null);
+    prepare(null, 3, 3, null);
     testCoding(true);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/671db987/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 890f632..b036eed 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
@@ -86,6 +86,7 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     }
 
     encoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    encoder.setConf(getConf());
     return encoder;
   }
 
@@ -102,6 +103,7 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     }
 
     decoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    decoder.setConf(getConf());
     return decoder;
   }
 


[06/50] [abbrv] hadoop git commit: HDFS-7700. Document quota support for storage types. (Contributed by Xiaoyu Yao)

Posted by zh...@apache.org.
HDFS-7700. Document quota support for storage types. (Contributed by Xiaoyu Yao)


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

Branch: refs/heads/HDFS-7285
Commit: 89fb0f57ef5d2e736ad2d50215369e55f3039e40
Parents: e97f8e4
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Mar 27 19:49:26 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Mar 27 19:49:26 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../src/site/markdown/HDFSCommands.md           |  8 ++--
 .../src/site/markdown/HdfsQuotaAdminGuide.md    | 41 ++++++++++++++++++--
 3 files changed, 45 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/89fb0f57/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 af1dd60..f7cc2bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1311,6 +1311,9 @@ Release 2.7.0 - UNRELEASED
       HDFS-7824. GetContentSummary API and its namenode implementation for
       Storage Type Quota/Usage. (Xiaoyu Yao via Arpit Agarwal)
 
+      HDFS-7700. Document quota support for storage types. (Xiaoyu Yao via
+      Arpit Agarwal)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89fb0f57/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 191b5bc..bdb051b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -307,8 +307,8 @@ Usage:
               [-refreshNodes]
               [-setQuota <quota> <dirname>...<dirname>]
               [-clrQuota <dirname>...<dirname>]
-              [-setSpaceQuota <quota> <dirname>...<dirname>]
-              [-clrSpaceQuota <dirname>...<dirname>]
+              [-setSpaceQuota <quota> [-storageType <storagetype>] <dirname>...<dirname>]
+              [-clrSpaceQuota [-storageType <storagetype>] <dirname>...<dirname>]
               [-setStoragePolicy <path> <policyName>]
               [-getStoragePolicy <path>]
               [-finalizeUpgrade]
@@ -342,8 +342,8 @@ Usage:
 | `-refreshNodes` | Re-read the hosts and exclude files to update the set of Datanodes that are allowed to connect to the Namenode and those that should be decommissioned or recommissioned. |
 | `-setQuota` \<quota\> \<dirname\>...\<dirname\> | See [HDFS Quotas Guide](../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands) for the detail. |
 | `-clrQuota` \<dirname\>...\<dirname\> | See [HDFS Quotas Guide](../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands) for the detail. |
-| `-setSpaceQuota` \<quota\> \<dirname\>...\<dirname\> | See [HDFS Quotas Guide](../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands) for the detail. |
-| `-clrSpaceQuota` \<dirname\>...\<dirname\> | See [HDFS Quotas Guide](../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands) for the detail. |
+| `-setSpaceQuota` \<quota\> `[-storageType <storagetype>]` \<dirname\>...\<dirname\> | See [HDFS Quotas Guide](../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands) for the detail. |
+| `-clrSpaceQuota` `[-storageType <storagetype>]` \<dirname\>...\<dirname\> | See [HDFS Quotas Guide](../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands) for the detail. |
 | `-setStoragePolicy` \<path\> \<policyName\> | Set a storage policy to a file or a directory. |
 | `-getStoragePolicy` \<path\> | Get the storage policy of a file or a directory. |
 | `-finalizeUpgrade` | Finalize upgrade of HDFS. Datanodes delete their previous version working directories, followed by Namenode doing the same. This completes the upgrade process. |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89fb0f57/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsQuotaAdminGuide.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsQuotaAdminGuide.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsQuotaAdminGuide.md
index a1bcd78..7c15bb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsQuotaAdminGuide.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsQuotaAdminGuide.md
@@ -19,6 +19,7 @@ HDFS Quotas Guide
     * [Overview](#Overview)
     * [Name Quotas](#Name_Quotas)
     * [Space Quotas](#Space_Quotas)
+    * [Storage Type Quotas](#Storage_Type_Quotas)
     * [Administrative Commands](#Administrative_Commands)
     * [Reporting Command](#Reporting_Command)
 
@@ -41,6 +42,17 @@ The space quota is a hard limit on the number of bytes used by files in the tree
 
 Quotas are persistent with the fsimage. When starting, if the fsimage is immediately in violation of a quota (perhaps the fsimage was surreptitiously modified), a warning is printed for each of such violations. Setting or removing a quota creates a journal entry.
 
+Storage Type Quotas
+------------------
+
+The storage type quota is a hard limit on the usage of specific storage type (SSD, DISK, ARCHIVE) by files in the tree rooted at the directory. It works similar to storage space quota in many aspects but offers fine-grain control over the cluster storage space usage. To set storage type quota on a directory, storage policies must be configured on the directory in order to allow files to be stored in different storage types according to the storage policy. See the [HDFS Storage Policy Documentation](./ArchivalStorage.html) for more information.
+
+The storage type quota can be combined with the space quotas and name quotas to efficiently manage the cluster storage usage. For example,
+
+1. For directories with storage policy configured, administrator should set storage type quotas for resource constraint storage types such as SSD and leave quotas for other storage types and overall space quota with either less restrictive values or default unlimited. HDFS will deduct quotas from both target storage type based on storage policy and the overall space quota.
+2. For directories without storage policy configured, administrator should not configure storage type quota. Storage type quota can be configured even though the specific storage type is unavailable (or available but not configured properly with storage type information). However, overall space quota is recommended in this case as the storage type information is either unavailable or inaccurate for storage type quota enforcement.
+3. Storage type quota on DISK are of limited use except when DISK is not the dominant storage medium. (e.g. cluster with predominantly ARCHIVE storage).
+
 Administrative Commands
 -----------------------
 
@@ -77,17 +89,40 @@ Quotas are managed by a set of commands available only to the administrator.
     directory, with faults reported if the directory does not exist or
     it is a file. It is not a fault if the directory has no quota.
 
+
+*   `hdfs dfsadmin -setSpaceQuota <N> -storageType <storagetype> <directory>...<directory>`
+
+    Set the storage type quota to be N bytes of storage type specified for each directory.
+    This is a hard limit on total storage type usage for all the files under the directory tree.
+    The storage type quota usage reflects the intended usage based on storage policy. For example,
+    one GB of data with replication of 3 and ALL_SSD storage policy consumes 3GB of SSD quota. N
+    can also be specified with a binary prefix for convenience, for e.g. 50g for 50
+    gigabytes and 2t for 2 terabytes etc. Best effort for each
+    directory, with faults reported if N is neither zero nor a positive
+    integer, the directory does not exist or it is a file, or the
+    directory would immediately exceed the new quota.
+
+*   `hdfs dfsadmin -clrSpaceQuota -storageType <storagetype> <directory>...<directory>`
+
+    Remove storage type quota specified for each directory. Best effort
+    for each directory, with faults reported if the directory does not exist or
+    it is a file. It is not a fault if the directory has no storage type quota on
+    for storage type specified.
+
 Reporting Command
 -----------------
 
 An an extension to the count command of the HDFS shell reports quota values and the current count of names and bytes in use.
 
-*   `hadoop fs -count -q [-h] [-v] <directory>...<directory>`
+*   `hadoop fs -count -q [-h] [-v] [-t [comma-separated list of storagetypes]] <directory>...<directory>`
 
     With the -q option, also report the name quota value set for each
     directory, the available name quota remaining, the space quota
     value set, and the available space quota remaining. If the
     directory does not have a quota set, the reported values are `none`
     and `inf`. The -h option shows sizes in human readable format.
-    The -v option displays a header line.
-
+    The -v option displays a header line. The -t option displays the per
+    storage type quota set and the available quota remaining for each directory.
+    If specific storage types are given after -t option, only quota and remaining
+    quota of the types specified will be displayed. Otherwise, quota and
+    remaining quota of all storage types that support quota will be displayed.
\ No newline at end of file


[12/50] [abbrv] hadoop git commit: HDFS-7890. Improve information on Top users for metrics in RollingWindowsManager and lower log level (Contributed by J.Andreina)

Posted by zh...@apache.org.
HDFS-7890. Improve information on Top users for metrics in RollingWindowsManager and lower log level (Contributed by J.Andreina)


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

Branch: refs/heads/HDFS-7285
Commit: 1ed9fb76645ecd195afe0067497dca10a3fb997d
Parents: 232eca9
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Mar 30 10:02:48 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Mon Mar 30 10:02:48 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hdfs/server/namenode/top/window/RollingWindowManager.java     | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ed9fb76/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 f4991da..9b1cc3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -353,6 +353,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-6408. Remove redundant definitions in log4j.properties.
     (Abhiraj Butala via aajisaka)
 
+    HDFS-7890. Improve information on Top users for metrics in
+    RollingWindowsManager and lower log level (J.Andreina via vinayakumarb)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ed9fb76/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/window/RollingWindowManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/window/RollingWindowManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/window/RollingWindowManager.java
index 00e7087..4759cc8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/window/RollingWindowManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/window/RollingWindowManager.java
@@ -245,7 +245,7 @@ public class RollingWindowManager {
           metricName, userName, windowSum);
       topN.offer(new NameValuePair(userName, windowSum));
     }
-    LOG.info("topN size for command {} is: {}", metricName, topN.size());
+    LOG.debug("topN users size for command {} is: {}", metricName, topN.size());
     return topN;
   }
 


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

Branch: refs/heads/HDFS-7285
Commit: 850a9ef88a283be241baff50309acb14fa03b4cf
Parents: 0614729
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Feb 10 17:54:10 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:25 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   | 137 +------
 .../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, 1125 insertions(+), 607 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/850a9ef8/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/850a9ef8/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 e9baf85..b14efb4 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(BlockStoragePolicySuite bsps);
 
   /**
-   * @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/850a9ef8/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/850a9ef8/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/850a9ef8/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/850a9ef8/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 92153ab..c78c9e2 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;
@@ -324,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);
@@ -357,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/850a9ef8/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/850a9ef8/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 3102a08..d947425 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
@@ -596,8 +596,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() :
@@ -628,7 +628,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;
@@ -641,15 +641,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: " +
@@ -675,13 +676,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;
   }
   
@@ -690,7 +693,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);
@@ -722,8 +725,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.
@@ -1023,7 +1026,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++) {
@@ -1031,7 +1034,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;
@@ -1130,7 +1133,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
@@ -1147,7 +1151,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);
@@ -1173,7 +1177,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
@@ -1716,41 +1720,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);
     }
@@ -1925,7 +1943,7 @@ public class BlockManager {
           break;
         }
 
-        BlockInfoContiguous bi = getStoredBlock(b);
+        BlockInfo bi = getStoredBlock(b);
         if (bi == null) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
@@ -1965,7 +1983,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>();
@@ -1982,8 +2000,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) {
@@ -2068,7 +2087,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;
       
@@ -2091,7 +2110,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
@@ -2106,14 +2125,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
@@ -2121,8 +2140,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
@@ -2134,7 +2155,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
@@ -2146,8 +2167,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);
@@ -2184,10 +2204,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) {
@@ -2208,7 +2228,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.
@@ -2262,7 +2282,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;
   }
@@ -2340,7 +2360,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:
@@ -2349,12 +2369,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(),
@@ -2365,8 +2385,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);
@@ -2381,7 +2401,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);
@@ -2396,7 +2416,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);
         }
@@ -2409,11 +2429,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:
@@ -2442,7 +2463,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);
     }
   } 
 
@@ -2457,18 +2478,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);
@@ -2489,13 +2510,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
@@ -2516,7 +2538,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) {
@@ -2588,13 +2610,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;
     }
@@ -2621,7 +2643,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)
@@ -2631,7 +2653,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;
         }
@@ -2700,7 +2722,7 @@ public class BlockManager {
     long nrInvalid = 0, nrOverReplicated = 0;
     long nrUnderReplicated = 0, nrPostponed = 0, nrUnderConstruction = 0;
     long startTimeMisReplicatedScan = Time.monotonicNow();
-    Iterator<BlockInfoContiguous> blocksItr = blocksMap.getBlocks().iterator();
+    Iterator<BlockInfo> blocksItr = blocksMap.getBlocks().iterator();
     long totalBlocks = blocksMap.size();
     replicationQueuesInitProgress = 0;
     long totalProcessed = 0;
@@ -2712,7 +2734,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);
@@ -2787,7 +2809,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
@@ -3116,14 +3138,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
@@ -3133,8 +3155,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) {
@@ -3257,7 +3280,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();
     }
@@ -3331,7 +3354,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)) {
@@ -3361,8 +3384,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())));
@@ -3519,7 +3542,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) {
@@ -3727,7 +3751,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/850a9ef8/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/850a9ef8/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/850a9ef8/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 d0d7a72..a022c81 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
@@ -335,7 +335,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) {
@@ -348,12 +348,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() {
@@ -532,12 +529,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());
       }
@@ -551,7 +548,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
 
     @Override
-    public BlockInfoContiguous next() {
+    public BlockInfo next() {
       update();
       return iterators.get(index).next();
     }
@@ -568,10 +565,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));
   }
 


[40/50] [abbrv] hadoop git commit: HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. 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/ace0181d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ace0181d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ace0181d

Branch: refs/heads/HDFS-7285
Commit: ace0181d97d8d51faa0a5d020d0f7445c231f060
Parents: d2f2604
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Mar 17 10:18:50 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:28 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    | 47 ++++++++---------
 .../PendingReplicationBlocks.java               | 51 +++++++++----------
 .../blockmanagement/UnderReplicatedBlocks.java  | 49 +++++++++---------
 .../hdfs/server/namenode/FSDirAttrOp.java       | 10 ++--
 .../hdfs/server/namenode/FSNamesystem.java      | 21 ++++----
 .../hadoop/hdfs/server/namenode/INode.java      | 12 ++---
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  4 +-
 .../hdfs/server/namenode/NamenodeFsck.java      | 10 ++--
 .../hadoop/hdfs/server/namenode/SafeMode.java   |  3 +-
 .../blockmanagement/BlockManagerTestUtil.java   |  5 +-
 .../blockmanagement/TestBlockManager.java       |  8 +--
 .../server/blockmanagement/TestNodeCount.java   |  3 +-
 .../TestOverReplicatedBlocks.java               |  5 +-
 .../blockmanagement/TestPendingReplication.java | 19 ++++---
 .../TestRBWBlockInvalidation.java               |  4 +-
 .../blockmanagement/TestReplicationPolicy.java  | 53 +++++++++++---------
 .../TestUnderReplicatedBlockQueues.java         | 16 +++---
 .../datanode/TestReadOnlySharedStorage.java     |  9 ++--
 .../namenode/TestProcessCorruptBlocks.java      |  5 +-
 19 files changed, 180 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/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 6481738..1e8ce1f 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
@@ -1336,7 +1336,7 @@ public class BlockManager {
    * @return number of blocks scheduled for replication during this iteration.
    */
   int computeReplicationWork(int blocksToProcess) {
-    List<List<Block>> blocksToReplicate = null;
+    List<List<BlockInfo>> blocksToReplicate = null;
     namesystem.writeLock();
     try {
       // Choose the blocks to be replicated
@@ -1354,7 +1354,7 @@ public class BlockManager {
    * @return the number of blocks scheduled for replication
    */
   @VisibleForTesting
-  int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
+  int computeReplicationWorkForBlocks(List<List<BlockInfo>> blocksToReplicate) {
     int requiredReplication, numEffectiveReplicas;
     List<DatanodeDescriptor> containingNodes;
     DatanodeDescriptor srcNode;
@@ -1368,7 +1368,7 @@ public class BlockManager {
     try {
       synchronized (neededReplications) {
         for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
-          for (Block block : blocksToReplicate.get(priority)) {
+          for (BlockInfo block : blocksToReplicate.get(priority)) {
             // block should belong to a file
             bc = blocksMap.getBlockCollection(block);
             // abandoned block or block reopened for append
@@ -1452,7 +1452,7 @@ public class BlockManager {
         }
 
         synchronized (neededReplications) {
-          Block block = rw.block;
+          BlockInfo block = rw.block;
           int priority = rw.priority;
           // Recheck since global lock was released
           // block should belong to a file
@@ -1710,7 +1710,7 @@ public class BlockManager {
    * and put them back into the neededReplication queue
    */
   private void processPendingReplications() {
-    Block[] timedOutItems = pendingReplications.getTimedOutBlocks();
+    BlockInfo[] timedOutItems = pendingReplications.getTimedOutBlocks();
     if (timedOutItems != null) {
       namesystem.writeLock();
       try {
@@ -2883,13 +2883,13 @@ public class BlockManager {
   
   /** Set replication for the blocks. */
   public void setReplication(final short oldRepl, final short newRepl,
-      final String src, final Block... blocks) {
+      final String src, final BlockInfoContiguous... blocks) {
     if (newRepl == oldRepl) {
       return;
     }
 
     // update needReplication priority queues
-    for(Block b : blocks) {
+    for(BlockInfoContiguous b : blocks) {
       updateNeededReplications(b, 0, newRepl-oldRepl);
     }
       
@@ -2897,7 +2897,7 @@ public class BlockManager {
       // old replication > the new one; need to remove copies
       LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl
           + " for " + src);
-      for(Block b : blocks) {
+      for(BlockInfoContiguous b : blocks) {
         processOverReplicatedBlock(b, newRepl, null, null);
       }
     } else { // replication factor is increased
@@ -3069,7 +3069,8 @@ public class BlockManager {
     blockLog.debug("BLOCK* removeStoredBlock: {} from {}", block, node);
     assert (namesystem.hasWriteLock());
     {
-      if (!blocksMap.removeNode(block, node)) {
+      BlockInfo storedBlock = getStoredBlock(block);
+      if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
         blockLog.debug("BLOCK* removeStoredBlock: {} has already been" +
             " removed from node {}", block, node);
         return;
@@ -3083,8 +3084,8 @@ public class BlockManager {
       //
       BlockCollection bc = blocksMap.getBlockCollection(block);
       if (bc != null) {
-        namesystem.decrementSafeBlockCount(block);
-        updateNeededReplications(block, -1, 0);
+        namesystem.decrementSafeBlockCount(storedBlock);
+        updateNeededReplications(storedBlock, -1, 0);
       }
 
       //
@@ -3158,7 +3159,7 @@ public class BlockManager {
     //
     // Modify the blocks->datanode map and node's map.
     //
-    pendingReplications.decrement(block, node);
+    pendingReplications.decrement(getStoredBlock(block), node);
     processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
         delHintNode);
   }
@@ -3273,7 +3274,7 @@ public class BlockManager {
    * For a striped block, this includes nodes storing blocks belonging to the
    * striped block group.
    */
-  public NumberReplicas countNodes(Block b) {
+  public NumberReplicas countNodes(BlockInfo b) {
     int decommissioned = 0;
     int live = 0;
     int corrupt = 0;
@@ -3303,11 +3304,11 @@ public class BlockManager {
   }
 
   /** 
-   * Simpler, faster form of {@link #countNodes(Block)} that only returns the number
+   * Simpler, faster form of {@link #countNodes} that only returns the number
    * of live nodes.  If in startup safemode (or its 30-sec extension period),
    * then it gains speed by ignoring issues of excess replicas or nodes
    * that are decommissioned or in process of becoming decommissioned.
-   * If not in startup, then it calls {@link #countNodes(Block)} instead.
+   * If not in startup, then it calls {@link #countNodes} instead.
    * 
    * @param b - the block being tested
    * @return count of live nodes for this block
@@ -3337,10 +3338,10 @@ public class BlockManager {
     if (!namesystem.isPopulatingReplQueues()) {
       return;
     }
-    final Iterator<? extends Block> it = srcNode.getBlockIterator();
+    final Iterator<BlockInfo> it = srcNode.getBlockIterator();
     int numOverReplicated = 0;
     while(it.hasNext()) {
-      final Block block = it.next();
+      final BlockInfo block = it.next();
       BlockCollection bc = blocksMap.getBlockCollection(block);
       short expectedReplication = bc.getBlockReplication();
       NumberReplicas num = countNodes(block);
@@ -3399,7 +3400,7 @@ public class BlockManager {
     return blocksMap.size();
   }
 
-  public void removeBlock(Block block) {
+  public void removeBlock(BlockInfo block) {
     assert namesystem.hasWriteLock();
     // No need to ACK blocks that are being removed entirely
     // from the namespace, since the removal of the associated
@@ -3429,7 +3430,7 @@ public class BlockManager {
   }
 
   /** updates a block in under replication queue */
-  private void updateNeededReplications(final Block block,
+  private void updateNeededReplications(final BlockInfo block,
       final int curReplicasDelta, int expectedReplicasDelta) {
     namesystem.writeLock();
     try {
@@ -3461,7 +3462,7 @@ public class BlockManager {
    */
   public void checkReplication(BlockCollection bc) {
     final short expected = bc.getBlockReplication();
-    for (Block block : bc.getBlocks()) {
+    for (BlockInfo block : bc.getBlocks()) {
       final NumberReplicas n = countNodes(block);
       if (isNeededReplication(block, expected, n.liveReplicas())) { 
         neededReplications.add(block, n.liveReplicas(),
@@ -3598,7 +3599,7 @@ public class BlockManager {
   /**
    * Return an iterator over the set of blocks for which there are no replicas.
    */
-  public Iterator<Block> getCorruptReplicaBlockIterator() {
+  public Iterator<BlockInfo> getCorruptReplicaBlockIterator() {
     return neededReplications.iterator(
         UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }
@@ -3712,7 +3713,7 @@ public class BlockManager {
 
   private static class ReplicationWork {
 
-    private final Block block;
+    private final BlockInfo block;
     private final BlockCollection bc;
 
     private final DatanodeDescriptor srcNode;
@@ -3723,7 +3724,7 @@ public class BlockManager {
     private DatanodeStorageInfo targets[];
     private final int priority;
 
-    public ReplicationWork(Block block,
+    public ReplicationWork(BlockInfo block,
         BlockCollection bc,
         DatanodeDescriptor srcNode,
         List<DatanodeDescriptor> containingNodes,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
index 796b878..04232cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
@@ -23,6 +23,7 @@ import java.io.PrintWriter;
 import java.sql.Time;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -46,8 +47,8 @@ import org.slf4j.Logger;
 class PendingReplicationBlocks {
   private static final Logger LOG = BlockManager.LOG;
 
-  private final Map<Block, PendingBlockInfo> pendingReplications;
-  private final ArrayList<Block> timedOutItems;
+  private final Map<BlockInfo, PendingBlockInfo> pendingReplications;
+  private final ArrayList<BlockInfo> timedOutItems;
   Daemon timerThread = null;
   private volatile boolean fsRunning = true;
 
@@ -62,8 +63,8 @@ class PendingReplicationBlocks {
     if ( timeoutPeriod > 0 ) {
       this.timeout = timeoutPeriod;
     }
-    pendingReplications = new HashMap<Block, PendingBlockInfo>();
-    timedOutItems = new ArrayList<Block>();
+    pendingReplications = new HashMap<>();
+    timedOutItems = new ArrayList<>();
   }
 
   void start() {
@@ -76,7 +77,7 @@ class PendingReplicationBlocks {
    * @param block The corresponding block
    * @param targets The DataNodes where replicas of the block should be placed
    */
-  void increment(Block block, DatanodeDescriptor[] targets) {
+  void increment(BlockInfo block, DatanodeDescriptor[] targets) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found == null) {
@@ -93,9 +94,9 @@ class PendingReplicationBlocks {
    * Decrement the number of pending replication requests
    * for this block.
    * 
-   * @param The DataNode that finishes the replication
+   * @param dn The DataNode that finishes the replication
    */
-  void decrement(Block block, DatanodeDescriptor dn) {
+  void decrement(BlockInfo block, DatanodeDescriptor dn) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found != null) {
@@ -115,7 +116,7 @@ class PendingReplicationBlocks {
    * @param block The given block whose pending replication requests need to be
    *              removed
    */
-  void remove(Block block) {
+  void remove(BlockInfo block) {
     synchronized (pendingReplications) {
       pendingReplications.remove(block);
     }
@@ -138,7 +139,7 @@ class PendingReplicationBlocks {
   /**
    * How many copies of this block is pending replication?
    */
-  int getNumReplicas(Block block) {
+  int getNumReplicas(BlockInfo block) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found != null) {
@@ -153,13 +154,13 @@ class PendingReplicationBlocks {
    * replication requests. Returns null if no blocks have
    * timed out.
    */
-  Block[] getTimedOutBlocks() {
+  BlockInfo[] getTimedOutBlocks() {
     synchronized (timedOutItems) {
       if (timedOutItems.size() <= 0) {
         return null;
       }
-      Block[] blockList = timedOutItems.toArray(
-          new Block[timedOutItems.size()]);
+      BlockInfo[] blockList = timedOutItems.toArray(
+          new BlockInfo[timedOutItems.size()]);
       timedOutItems.clear();
       return blockList;
     }
@@ -179,7 +180,7 @@ class PendingReplicationBlocks {
     PendingBlockInfo(DatanodeDescriptor[] targets) {
       this.timeStamp = monotonicNow();
       this.targets = targets == null ? new ArrayList<DatanodeDescriptor>()
-          : new ArrayList<DatanodeDescriptor>(Arrays.asList(targets));
+          : new ArrayList<>(Arrays.asList(targets));
     }
 
     long getTimeStamp() {
@@ -192,9 +193,7 @@ class PendingReplicationBlocks {
 
     void incrementReplicas(DatanodeDescriptor... newTargets) {
       if (newTargets != null) {
-        for (DatanodeDescriptor dn : newTargets) {
-          targets.add(dn);
-        }
+        Collections.addAll(targets, newTargets);
       }
     }
 
@@ -232,17 +231,17 @@ class PendingReplicationBlocks {
      */
     void pendingReplicationCheck() {
       synchronized (pendingReplications) {
-        Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
+        Iterator<Map.Entry<BlockInfo, PendingBlockInfo>> iter =
                                     pendingReplications.entrySet().iterator();
         long now = monotonicNow();
         if(LOG.isDebugEnabled()) {
           LOG.debug("PendingReplicationMonitor checking Q");
         }
         while (iter.hasNext()) {
-          Map.Entry<Block, PendingBlockInfo> entry = iter.next();
+          Map.Entry<BlockInfo, PendingBlockInfo> entry = iter.next();
           PendingBlockInfo pendingBlock = entry.getValue();
           if (now > pendingBlock.getTimeStamp() + timeout) {
-            Block block = entry.getKey();
+            BlockInfo block = entry.getKey();
             synchronized (timedOutItems) {
               timedOutItems.add(block);
             }
@@ -275,16 +274,14 @@ class PendingReplicationBlocks {
     synchronized (pendingReplications) {
       out.println("Metasave: Blocks being replicated: " +
                   pendingReplications.size());
-      Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
-                                  pendingReplications.entrySet().iterator();
-      while (iter.hasNext()) {
-        Map.Entry<Block, PendingBlockInfo> entry = iter.next();
+      for (Map.Entry<BlockInfo, PendingBlockInfo> entry :
+          pendingReplications.entrySet()) {
         PendingBlockInfo pendingBlock = entry.getValue();
         Block block = entry.getKey();
-        out.println(block + 
-                    " StartTime: " + new Time(pendingBlock.timeStamp) +
-                    " NumReplicaInProgress: " + 
-                    pendingBlock.getNumReplicas());
+        out.println(block +
+            " StartTime: " + new Time(pendingBlock.timeStamp) +
+            " NumReplicaInProgress: " +
+            pendingBlock.getNumReplicas());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
index 1daa0ee..f9bce26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
@@ -35,7 +34,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  *
  * <p/>
  * The policy for choosing which priority to give added blocks
- * is implemented in {@link #getPriority(Block, int, int, int)}.
+ * is implemented in {@link #getPriority(int, int, int)}.
  * </p>
  * <p>The queue order is as follows:</p>
  * <ol>
@@ -62,7 +61,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  *   blocks that are not corrupt higher priority.</li>
  * </ol>
  */
-class UnderReplicatedBlocks implements Iterable<Block> {
+class UnderReplicatedBlocks implements Iterable<BlockInfo> {
   /** The total number of queues : {@value} */
   static final int LEVEL = 5;
   /** The queue with the highest priority: {@value} */
@@ -78,8 +77,8 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /** The queue for corrupt blocks: {@value} */
   static final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
   /** the queues themselves */
-  private final List<LightWeightLinkedSet<Block>> priorityQueues
-      = new ArrayList<LightWeightLinkedSet<Block>>(LEVEL);
+  private final List<LightWeightLinkedSet<BlockInfo>> priorityQueues
+      = new ArrayList<>(LEVEL);
 
   /** The number of corrupt blocks with replication factor 1 */
   private int corruptReplOneBlocks = 0;
@@ -87,7 +86,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /** Create an object. */
   UnderReplicatedBlocks() {
     for (int i = 0; i < LEVEL; i++) {
-      priorityQueues.add(new LightWeightLinkedSet<Block>());
+      priorityQueues.add(new LightWeightLinkedSet<BlockInfo>());
     }
   }
 
@@ -131,8 +130,8 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** Check if a block is in the neededReplication queue */
-  synchronized boolean contains(Block block) {
-    for(LightWeightLinkedSet<Block> set : priorityQueues) {
+  synchronized boolean contains(BlockInfo block) {
+    for(LightWeightLinkedSet<BlockInfo> set : priorityQueues) {
       if (set.contains(block)) {
         return true;
       }
@@ -141,13 +140,11 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** Return the priority of a block
-   * @param block a under replicated block
    * @param curReplicas current number of replicas of the block
    * @param expectedReplicas expected number of replicas of the block
    * @return the priority for the blocks, between 0 and ({@link #LEVEL}-1)
    */
-  private int getPriority(Block block,
-                          int curReplicas, 
+  private int getPriority(int curReplicas,
                           int decommissionedReplicas,
                           int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
@@ -183,12 +180,12 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param expectedReplicas expected number of replicas of the block
    * @return true if the block was added to a queue.
    */
-  synchronized boolean add(Block block,
+  synchronized boolean add(BlockInfo block,
                            int curReplicas, 
                            int decomissionedReplicas,
                            int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
-    int priLevel = getPriority(block, curReplicas, decomissionedReplicas,
+    int priLevel = getPriority(curReplicas, decomissionedReplicas,
                                expectedReplicas);
     if(priorityQueues.get(priLevel).add(block)) {
       if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
@@ -207,11 +204,11 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** remove a block from a under replication queue */
-  synchronized boolean remove(Block block, 
+  synchronized boolean remove(BlockInfo block,
                               int oldReplicas, 
                               int decommissionedReplicas,
                               int oldExpectedReplicas) {
-    int priLevel = getPriority(block, oldReplicas, 
+    int priLevel = getPriority(oldReplicas,
                                decommissionedReplicas,
                                oldExpectedReplicas);
     boolean removedBlock = remove(block, priLevel);
@@ -241,7 +238,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param priLevel expected privilege level
    * @return true if the block was found and removed from one of the priority queues
    */
-  boolean remove(Block block, int priLevel) {
+  boolean remove(BlockInfo block, int priLevel) {
     if(priLevel >= 0 && priLevel < LEVEL 
         && priorityQueues.get(priLevel).remove(block)) {
       NameNode.blockStateChangeLog.debug(
@@ -279,14 +276,14 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param curReplicasDelta the change in the replicate count from before
    * @param expectedReplicasDelta the change in the expected replica count from before
    */
-  synchronized void update(Block block, int curReplicas,
+  synchronized void update(BlockInfo block, int curReplicas,
                            int decommissionedReplicas,
                            int curExpectedReplicas,
                            int curReplicasDelta, int expectedReplicasDelta) {
     int oldReplicas = curReplicas-curReplicasDelta;
     int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
-    int curPri = getPriority(block, curReplicas, decommissionedReplicas, curExpectedReplicas);
-    int oldPri = getPriority(block, oldReplicas, decommissionedReplicas, oldExpectedReplicas);
+    int curPri = getPriority(curReplicas, decommissionedReplicas, curExpectedReplicas);
+    int oldPri = getPriority(oldReplicas, decommissionedReplicas, oldExpectedReplicas);
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " + 
         block +
@@ -336,12 +333,12 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @return Return a list of block lists to be replicated. The block list index
    *         represents its replication priority.
    */
-  public synchronized List<List<Block>> chooseUnderReplicatedBlocks(
+  public synchronized List<List<BlockInfo>> chooseUnderReplicatedBlocks(
       int blocksToProcess) {
     // initialize data structure for the return value
-    List<List<Block>> blocksToReplicate = new ArrayList<List<Block>>(LEVEL);
+    List<List<BlockInfo>> blocksToReplicate = new ArrayList<>(LEVEL);
     for (int i = 0; i < LEVEL; i++) {
-      blocksToReplicate.add(new ArrayList<Block>());
+      blocksToReplicate.add(new ArrayList<BlockInfo>());
     }
 
     if (size() == 0) { // There are no blocks to collect.
@@ -364,7 +361,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
       // Loop through all remaining blocks in the list.
       while (blockCount < blocksToProcess
           && neededReplicationsIterator.hasNext()) {
-        Block block = neededReplicationsIterator.next();
+        BlockInfo block = neededReplicationsIterator.next();
         blocksToReplicate.get(priority).add(block);
         blockCount++;
       }
@@ -396,10 +393,10 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /**
    * An iterator over blocks.
    */
-  class BlockIterator implements Iterator<Block> {
+  class BlockIterator implements Iterator<BlockInfo> {
     private int level;
     private boolean isIteratorForLevel = false;
-    private final List<Iterator<Block>> iterators = new ArrayList<Iterator<Block>>();
+    private final List<Iterator<BlockInfo>> iterators = new ArrayList<>();
 
     /**
      * Construct an iterator over all queues.
@@ -431,7 +428,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
     }
 
     @Override
-    public Block next() {
+    public BlockInfo next() {
       if (isIteratorForLevel) {
         return iterators.get(0).next();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index a3881b8..eefbcce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.util.EnumCounters;
@@ -148,8 +150,8 @@ public class FSDirAttrOp {
       }
 
       final short[] blockRepls = new short[2]; // 0: old, 1: new
-      final Block[] blocks = unprotectedSetReplication(fsd, src, replication,
-                                                       blockRepls);
+      final BlockInfoContiguous[] blocks = unprotectedSetReplication(fsd, src,
+          replication, blockRepls);
       isFile = blocks != null;
       if (isFile) {
         fsd.getEditLog().logSetReplication(src, replication);
@@ -375,7 +377,7 @@ public class FSDirAttrOp {
     }
   }
 
-  static Block[] unprotectedSetReplication(
+  static BlockInfoContiguous[] unprotectedSetReplication(
       FSDirectory fsd, String src, short replication, short[] blockRepls)
       throws QuotaExceededException, UnresolvedLinkException,
              SnapshotAccessControlException {
@@ -410,7 +412,7 @@ public class FSDirAttrOp {
       blockRepls[0] = oldBR;
       blockRepls[1] = newBR;
     }
-    return file.getBlocks();
+    return file.getContiguousBlocks();
   }
 
   static void unprotectedSetStoragePolicy(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/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 bf1011a..4e6b5e3 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
@@ -3749,8 +3749,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *          of blocks that need to be removed from blocksMap
    */
   void removeBlocks(BlocksMapUpdateInfo blocks) {
-    List<Block> toDeleteList = blocks.getToDeleteList();
-    Iterator<Block> iter = toDeleteList.iterator();
+    List<BlockInfo> toDeleteList = blocks.getToDeleteList();
+    Iterator<BlockInfo> iter = toDeleteList.iterator();
     while (iter.hasNext()) {
       writeLock();
       try {
@@ -3805,12 +3805,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean trackBlockCounts = isSafeModeTrackingBlocks();
     int numRemovedComplete = 0, numRemovedSafe = 0;
 
-    for (Block b : blocks.getToDeleteList()) {
+    for (BlockInfo b : blocks.getToDeleteList()) {
       if (trackBlockCounts) {
-        BlockInfo bi = getStoredBlock(b);
-        if (bi.isComplete()) {
+        if (b.isComplete()) {
           numRemovedComplete++;
-          if (blockManager.checkMinStorage(bi, bi.numNodes())) {
+          if (blockManager.checkMinStorage(b, b.numNodes())) {
             numRemovedSafe++;
           }
         }
@@ -4739,7 +4738,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       writeLock();
 
       try {
-        final Iterator<Block> it = blockManager.getCorruptReplicaBlockIterator();
+        final Iterator<BlockInfo> it =
+            blockManager.getCorruptReplicaBlockIterator();
 
         while (it.hasNext()) {
           Block b = it.next();
@@ -5689,7 +5689,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override
-  public void decrementSafeBlockCount(Block b) {
+  public void decrementSafeBlockCount(BlockInfo b) {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
@@ -6554,7 +6554,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
       // print a limited # of corrupt files per call
 
-      final Iterator<Block> blkIterator = blockManager.getCorruptReplicaBlockIterator();
+      final Iterator<BlockInfo> blkIterator =
+          blockManager.getCorruptReplicaBlockIterator();
 
       int skip = getIntCookie(cookieTab[0]);
       for (int i = 0; i < skip && blkIterator.hasNext(); i++) {
@@ -6562,7 +6563,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
 
       while (blkIterator.hasNext()) {
-        Block blk = blkIterator.next();
+        BlockInfo blk = blkIterator.next();
         final INode inode = (INode)blockManager.getBlockCollection(blk);
         skip++;
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index 586cce4..d1b40fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -30,9 +30,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
@@ -804,16 +804,16 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     /**
      * The list of blocks that need to be removed from blocksMap
      */
-    private final List<Block> toDeleteList;
+    private final List<BlockInfo> toDeleteList;
     
     public BlocksMapUpdateInfo() {
-      toDeleteList = new ChunkedArrayList<Block>();
+      toDeleteList = new ChunkedArrayList<>();
     }
     
     /**
      * @return The list of blocks that need to be removed from blocksMap
      */
-    public List<Block> getToDeleteList() {
+    public List<BlockInfo> getToDeleteList() {
       return toDeleteList;
     }
     
@@ -822,12 +822,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
      * {@link BlocksMapUpdateInfo#toDeleteList}
      * @param toDelete the to-be-deleted block
      */
-    public void addDeleteBlock(Block toDelete) {
+    public void addDeleteBlock(BlockInfo toDelete) {
       assert toDelete != null : "toDelete is null";
       toDeleteList.add(toDelete);
     }
 
-    public void removeDeleteBlock(Block block) {
+    public void removeDeleteBlock(BlockInfo block) {
       assert block != null : "block is null";
       toDeleteList.remove(block);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/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 b1c57b0..5e2b58f 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
@@ -1070,8 +1070,8 @@ public class INodeFile extends INodeWithAdditionalFields
         getDiffs().findEarlierSnapshotBlocks(snapshotId);
     if(snapshotBlocks == null)
       return;
-    List<Block> toDelete = collectedBlocks.getToDeleteList();
-    for(Block blk : snapshotBlocks) {
+    List<BlockInfo> toDelete = collectedBlocks.getToDeleteList();
+    for(BlockInfo blk : snapshotBlocks) {
       if(toDelete.contains(blk))
         collectedBlocks.removeDeleteBlock(blk);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/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 92e9f08..4cf2185 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
@@ -244,7 +244,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       }
       BlockCollection bc = bm.getBlockCollection(blockInfo);
       INode iNode = (INode) bc;
-      NumberReplicas numberReplicas= bm.countNodes(block);
+      NumberReplicas numberReplicas= bm.countNodes(blockInfo);
       out.println("Block Id: " + blockId);
       out.println("Block belongs to: "+iNode.getFullPathName());
       out.println("No. of Expected Replica: " + bc.getBlockReplication());
@@ -459,7 +459,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     // Get block locations without updating the file access time 
     // and without block access tokens
     LocatedBlocks blocks = null;
-    FSNamesystem fsn = namenode.getNamesystem();
+    final FSNamesystem fsn = namenode.getNamesystem();
     fsn.readLock();
     try {
       blocks = fsn.getBlockLocations(path, 0, fileLen, false, false).blocks;
@@ -507,8 +507,10 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       boolean isCorrupt = lBlk.isCorrupt();
       String blkName = block.toString();
       DatanodeInfo[] locs = lBlk.getLocations();
-      NumberReplicas numberReplicas =
-          namenode.getNamesystem().getBlockManager().countNodes(block.getLocalBlock());
+      final BlockManager blockManager = fsn.getBlockManager();
+      final BlockInfo storedBlock = blockManager.getStoredBlock(
+          block.getLocalBlock());
+      NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
       int liveReplicas = numberReplicas.liveReplicas();
       res.totalReplicas += liveReplicas;
       short targetFileReplication = file.getReplication();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
index 95fc06b..0debb1f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
@@ -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.BlockInfo;
 
 /** SafeMode related operations. */
 @InterfaceAudience.Private
@@ -49,5 +50,5 @@ public interface SafeMode {
   public void incrementSafeBlockCount(int replication);
 
   /** Decrement number of blocks that reached minimal replication. */
-  public void decrementSafeBlockCount(Block b);
+  public void decrementSafeBlockCount(BlockInfo b);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index 23e610f..148135b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -69,9 +69,10 @@ public class BlockManagerTestUtil {
     final BlockManager bm = namesystem.getBlockManager();
     namesystem.readLock();
     try {
+      final BlockInfo storedBlock = bm.getStoredBlock(b);
       return new int[]{getNumberOfRacks(bm, b),
-          bm.countNodes(b).liveReplicas(),
-          bm.neededReplications.contains(b) ? 1 : 0};
+          bm.countNodes(storedBlock).liveReplicas(),
+          bm.neededReplications.contains(storedBlock) ? 1 : 0};
     } finally {
       namesystem.readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/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 1bc6198..2fca5be 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
@@ -435,14 +435,14 @@ public class TestBlockManager {
     return blockInfo;
   }
 
-  private DatanodeStorageInfo[] scheduleSingleReplication(Block block) {
+  private DatanodeStorageInfo[] scheduleSingleReplication(BlockInfo block) {
     // list for priority 1
-    List<Block> list_p1 = new ArrayList<Block>();
+    List<BlockInfo> list_p1 = new ArrayList<>();
     list_p1.add(block);
 
     // list of lists for each priority
-    List<List<Block>> list_all = new ArrayList<List<Block>>();
-    list_all.add(new ArrayList<Block>()); // for priority 0
+    List<List<BlockInfo>> list_all = new ArrayList<>();
+    list_all.add(new ArrayList<BlockInfo>()); // for priority 0
     list_all.add(list_p1); // for priority 1
 
     assertEquals("Block not initially pending replication", 0,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
index c3726f2..1c3f075 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
@@ -166,10 +166,11 @@ public class TestNodeCount {
 
   /* threadsafe read of the replication counts for this block */
   NumberReplicas countNodes(Block block, FSNamesystem namesystem) {
+    BlockManager blockManager = namesystem.getBlockManager();
     namesystem.readLock();
     try {
       lastBlock = block;
-      lastNum = namesystem.getBlockManager().countNodes(block);
+      lastNum = blockManager.countNodes(blockManager.getStoredBlock(block));
       return lastNum;
     }
     finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
index a86b573..2d7bb44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
@@ -117,7 +117,8 @@ public class TestOverReplicatedBlocks {
 
           // corrupt one won't be chosen to be excess one
           // without 4910 the number of live replicas would be 0: block gets lost
-          assertEquals(1, bm.countNodes(block.getLocalBlock()).liveReplicas());
+          assertEquals(1, bm.countNodes(
+              bm.getStoredBlock(block.getLocalBlock())).liveReplicas());
         }
       } finally {
         namesystem.writeUnlock();
@@ -219,7 +220,7 @@ public class TestOverReplicatedBlocks {
       out.close();
       ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, p);
       assertEquals("Expected only one live replica for the block", 1, bm
-          .countNodes(block.getLocalBlock()).liveReplicas());
+          .countNodes(bm.getStoredBlock(block.getLocalBlock())).liveReplicas());
     } finally {
       cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
index c63badc..b9032f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
@@ -53,6 +53,11 @@ public class TestPendingReplication {
   // Number of datanodes in the cluster
   private static final int DATANODE_COUNT = 5;
 
+  private BlockInfo genBlockInfo(long id, long length, long gs) {
+    return new BlockInfoContiguous(new Block(id, length, gs),
+        (short) DATANODE_COUNT);
+  }
+
   @Test
   public void testPendingReplication() {
     PendingReplicationBlocks pendingReplications;
@@ -64,7 +69,7 @@ public class TestPendingReplication {
     //
     DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(10);
     for (int i = 0; i < storages.length; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       DatanodeStorageInfo[] targets = new DatanodeStorageInfo[i];
       System.arraycopy(storages, 0, targets, 0, i);
       pendingReplications.increment(block,
@@ -77,7 +82,7 @@ public class TestPendingReplication {
     //
     // remove one item and reinsert it
     //
-    Block blk = new Block(8, 8, 0);
+    BlockInfo blk = genBlockInfo(8, 8, 0);
     pendingReplications.decrement(blk, storages[7].getDatanodeDescriptor()); // removes one replica
     assertEquals("pendingReplications.getNumReplicas ",
                  7, pendingReplications.getNumReplicas(blk));
@@ -97,7 +102,7 @@ public class TestPendingReplication {
     // are sane.
     //
     for (int i = 0; i < 10; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       int numReplicas = pendingReplications.getNumReplicas(block);
       assertTrue(numReplicas == i);
     }
@@ -116,7 +121,7 @@ public class TestPendingReplication {
     }
 
     for (int i = 10; i < 15; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       pendingReplications.increment(block,
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(i)));
@@ -185,7 +190,7 @@ public class TestPendingReplication {
 
       assertEquals(1, blkManager.pendingReplications.size());
       INodeFile fileNode = fsn.getFSDirectory().getINode4Write(file).asFile();
-      Block[] blocks = fileNode.getBlocks();
+      BlockInfo[] blocks = fileNode.getBlocks();
       assertEquals(DATANODE_COUNT - 1,
           blkManager.pendingReplications.getNumReplicas(blocks[0]));
 
@@ -291,8 +296,8 @@ public class TestPendingReplication {
       BlockManagerTestUtil.computeAllPendingWork(bm);
       BlockManagerTestUtil.updateState(bm);
       assertEquals(bm.getPendingReplicationBlocksCount(), 1L);
-      assertEquals(bm.pendingReplications.getNumReplicas(block.getBlock()
-          .getLocalBlock()), 2);
+      BlockInfo storedBlock = bm.getStoredBlock(block.getBlock().getLocalBlock());
+      assertEquals(bm.pendingReplications.getNumReplicas(storedBlock), 2);
       
       // 4. delete the file
       fs.delete(filePath, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
index 728934d..1a32892 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
@@ -58,7 +58,9 @@ public class TestRBWBlockInvalidation {
   
   private static NumberReplicas countReplicas(final FSNamesystem namesystem,
       ExtendedBlock block) {
-    return namesystem.getBlockManager().countNodes(block.getLocalBlock());
+    final BlockManager blockManager = namesystem.getBlockManager();
+    return blockManager.countNodes(blockManager.getStoredBlock(
+        block.getLocalBlock()));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/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 6cd0d1c..7eeca01 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
@@ -833,6 +833,10 @@ public class TestReplicationPolicy {
     assertEquals(targets.length, 2);
     assertTrue(isOnSameRack(targets[0], dataNodes[2]));
   }
+
+  private BlockInfo genBlockInfo(long id) {
+    return new BlockInfoContiguous(new Block(id), (short) 3);
+  }
   
   /**
    * Test for the high priority blocks are processed before the low priority
@@ -852,14 +856,14 @@ public class TestReplicationPolicy {
           .getNamesystem().getBlockManager().neededReplications;
       for (int i = 0; i < 100; i++) {
         // Adding the blocks directly to normal priority
-        neededReplications.add(new Block(random.nextLong()), 2, 0, 3);
+        neededReplications.add(genBlockInfo(random.nextLong()), 2, 0, 3);
       }
       // Lets wait for the replication interval, to start process normal
       // priority blocks
       Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
       
       // Adding the block directly to high priority list
-      neededReplications.add(new Block(random.nextLong()), 1, 0, 3);
+      neededReplications.add(genBlockInfo(random.nextLong()), 1, 0, 3);
       
       // Lets wait for the replication interval
       Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
@@ -882,25 +886,26 @@ public class TestReplicationPolicy {
 
     for (int i = 0; i < 5; i++) {
       // Adding QUEUE_HIGHEST_PRIORITY block
-      underReplicatedBlocks.add(new Block(random.nextLong()), 1, 0, 3);
+      underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 1, 0, 3);
 
       // Adding QUEUE_VERY_UNDER_REPLICATED block
-      underReplicatedBlocks.add(new Block(random.nextLong()), 2, 0, 7);
+      underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 2, 0, 7);
 
       // Adding QUEUE_REPLICAS_BADLY_DISTRIBUTED block
-      underReplicatedBlocks.add(new Block(random.nextLong()), 6, 0, 6);
+      underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 6, 0, 6);
 
       // Adding QUEUE_UNDER_REPLICATED block
-      underReplicatedBlocks.add(new Block(random.nextLong()), 5, 0, 6);
+      underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 5, 0, 6);
 
       // Adding QUEUE_WITH_CORRUPT_BLOCKS block
-      underReplicatedBlocks.add(new Block(random.nextLong()), 0, 0, 3);
+      underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 0, 0, 3);
     }
 
     // Choose 6 blocks from UnderReplicatedBlocks. Then it should pick 5 blocks
     // from
     // QUEUE_HIGHEST_PRIORITY and 1 block from QUEUE_VERY_UNDER_REPLICATED.
-    List<List<Block>> chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
+    List<List<BlockInfo>> chosenBlocks =
+        underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
     assertTheChosenBlocks(chosenBlocks, 5, 1, 0, 0, 0);
 
     // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 4 blocks from
@@ -910,7 +915,7 @@ public class TestReplicationPolicy {
     assertTheChosenBlocks(chosenBlocks, 0, 4, 5, 1, 0);
 
     // Adding QUEUE_HIGHEST_PRIORITY
-    underReplicatedBlocks.add(new Block(random.nextLong()), 1, 0, 3);
+    underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 1, 0, 3);
 
     // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 1 block from
     // QUEUE_HIGHEST_PRIORITY, 4 blocks from QUEUE_REPLICAS_BADLY_DISTRIBUTED
@@ -928,7 +933,7 @@ public class TestReplicationPolicy {
   
   /** asserts the chosen blocks with expected priority blocks */
   private void assertTheChosenBlocks(
-      List<List<Block>> chosenBlocks, int firstPrioritySize,
+      List<List<BlockInfo>> chosenBlocks, int firstPrioritySize,
       int secondPrioritySize, int thirdPrioritySize, int fourthPrioritySize,
       int fifthPrioritySize) {
     assertEquals(
@@ -1102,9 +1107,9 @@ public class TestReplicationPolicy {
   public void testUpdateDoesNotCauseSkippedReplication() {
     UnderReplicatedBlocks underReplicatedBlocks = new UnderReplicatedBlocks();
 
-    Block block1 = new Block(random.nextLong());
-    Block block2 = new Block(random.nextLong());
-    Block block3 = new Block(random.nextLong());
+    BlockInfo block1 = genBlockInfo(random.nextLong());
+    BlockInfo block2 = genBlockInfo(random.nextLong());
+    BlockInfo block3 = genBlockInfo(random.nextLong());
 
     // Adding QUEUE_VERY_UNDER_REPLICATED block
     final int block1CurReplicas = 2;
@@ -1118,7 +1123,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block3, 2, 0, 6);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1151,8 +1156,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(random.nextLong());
-    Block block2 = new Block(random.nextLong());
+    BlockInfo block1 = genBlockInfo(random.nextLong());
+    BlockInfo block2 = genBlockInfo(random.nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1160,7 +1165,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1197,8 +1202,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(random.nextLong());
-    Block block2 = new Block(random.nextLong());
+    BlockInfo block1 = genBlockInfo(random.nextLong());
+    BlockInfo block2 = genBlockInfo(random.nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1206,7 +1211,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1260,8 +1265,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(random.nextLong());
-    Block block2 = new Block(random.nextLong());
+    BlockInfo block1 = genBlockInfo(random.nextLong());
+    BlockInfo block2 = genBlockInfo(random.nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1269,14 +1274,14 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
     chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
-    bm.setReplication((short)0, (short)1, "", block1);
+    bm.setReplication((short)0, (short)1, "", (BlockInfoContiguous) block1);
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
index e87a043..de36e07 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
@@ -28,6 +28,10 @@ import static org.junit.Assert.fail;
 
 public class TestUnderReplicatedBlockQueues {
 
+  private BlockInfo genBlockInfo(long id) {
+    return new BlockInfoContiguous(new Block(id), (short) 3);
+  }
+
   /**
    * Test that adding blocks with different replication counts puts them
    * into different queues
@@ -36,11 +40,11 @@ public class TestUnderReplicatedBlockQueues {
   @Test
   public void testBlockPriorities() throws Throwable {
     UnderReplicatedBlocks queues = new UnderReplicatedBlocks();
-    Block block1 = new Block(1);
-    Block block2 = new Block(2);
-    Block block_very_under_replicated = new Block(3);
-    Block block_corrupt = new Block(4);
-    Block block_corrupt_repl_one = new Block(5);
+    BlockInfo block1 = genBlockInfo(1);
+    BlockInfo block2 = genBlockInfo(2);
+    BlockInfo block_very_under_replicated = genBlockInfo(3);
+    BlockInfo block_corrupt = genBlockInfo(4);
+    BlockInfo block_corrupt_repl_one = genBlockInfo(5);
 
     //add a block with a single entry
     assertAdded(queues, block1, 1, 0, 3);
@@ -82,7 +86,7 @@ public class TestUnderReplicatedBlockQueues {
   }
 
   private void assertAdded(UnderReplicatedBlocks queues,
-                           Block block,
+                           BlockInfo block,
                            int curReplicas,
                            int decomissionedReplicas,
                            int expectedReplicas) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
index e6bf067..80ad359 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -81,6 +82,7 @@ public class TestReadOnlySharedStorage {
   private DatanodeInfo readOnlyDataNode;
   
   private Block block;
+  private BlockInfo storedBlock;
 
   private ExtendedBlock extendedBlock;
 
@@ -132,6 +134,7 @@ public class TestReadOnlySharedStorage {
     LocatedBlock locatedBlock = getLocatedBlock();
     extendedBlock = locatedBlock.getBlock();
     block = extendedBlock.getLocalBlock();
+    storedBlock = blockManager.getStoredBlock(block);
     
     assertThat(locatedBlock.getLocations().length, is(1));
     normalDataNode = locatedBlock.getLocations()[0];
@@ -188,7 +191,7 @@ public class TestReadOnlySharedStorage {
   }
   
   private void validateNumberReplicas(int expectedReplicas) throws IOException {
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.liveReplicas(), is(expectedReplicas));
     assertThat(numberReplicas.excessReplicas(), is(0));
     assertThat(numberReplicas.corruptReplicas(), is(0));
@@ -230,7 +233,7 @@ public class TestReadOnlySharedStorage {
         cluster.getNameNode(), normalDataNode.getXferAddr());
     
     // The live replica count should now be zero (since the NORMAL replica is offline)
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.liveReplicas(), is(0));
     
     // The block should be reported as under-replicated
@@ -263,7 +266,7 @@ public class TestReadOnlySharedStorage {
     waitForLocations(1);
     
     // However, the corrupt READ_ONLY_SHARED replica should *not* affect the overall corrupt replicas count
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.corruptReplicas(), is(0));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ace0181d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
index 37abc5b..228a667 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.junit.Test;
@@ -260,7 +261,9 @@ public class TestProcessCorruptBlocks {
   }
 
   private static NumberReplicas countReplicas(final FSNamesystem namesystem, ExtendedBlock block) {
-    return namesystem.getBlockManager().countNodes(block.getLocalBlock());
+    final BlockManager blockManager = namesystem.getBlockManager();
+    return blockManager.countNodes(blockManager.getStoredBlock(
+        block.getLocalBlock()));
   }
 
   private void corruptBlock(MiniDFSCluster cluster, FileSystem fs, final Path fileName,


[08/50] [abbrv] hadoop git commit: YARN-3288. Document and fix indentation in the DockerContainerExecutor code

Posted by zh...@apache.org.
YARN-3288. Document and fix indentation in the DockerContainerExecutor code


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

Branch: refs/heads/HDFS-7285
Commit: e0ccea33c9e12f6930b2867e14b1b37569fed659
Parents: 27d49e6
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Sat Mar 28 08:00:41 2015 -0700
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Sat Mar 28 08:00:41 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../server/nodemanager/ContainerExecutor.java   |  18 +-
 .../nodemanager/DockerContainerExecutor.java    | 229 +++++++++++--------
 .../launcher/ContainerLaunch.java               |   8 +-
 .../TestDockerContainerExecutor.java            |  98 ++++----
 .../TestDockerContainerExecutorWithMocks.java   | 110 +++++----
 6 files changed, 277 insertions(+), 188 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0ccea33/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d6ded77..fb233e3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -81,6 +81,8 @@ Release 2.8.0 - UNRELEASED
 
     YARN-3397. yarn rmadmin should skip -failover. (J.Andreina via kasha)
 
+    YARN-3288. Document and fix indentation in the DockerContainerExecutor code
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0ccea33/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 377fd1d..1c670a1 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
@@ -210,8 +210,22 @@ public abstract class ContainerExecutor implements Configurable {
     }
   }
 
-  public void writeLaunchEnv(OutputStream out, Map<String, String> environment, Map<Path, List<String>> resources, List<String> command) throws IOException{
-    ContainerLaunch.ShellScriptBuilder sb = ContainerLaunch.ShellScriptBuilder.create();
+  /**
+   * This method writes out the launch environment of a container. This can be
+   * overridden by extending ContainerExecutors to provide different behaviors
+   * @param out the output stream to which the environment is written (usually
+   * a script file which will be executed by the Launcher)
+   * @param environment The environment variables and their values
+   * @param resources The resources which have been localized for this container
+   * Symlinks will be created to these localized resources
+   * @param command The command that will be run.
+   * @throws IOException if any errors happened writing to the OutputStream,
+   * while creating symlinks
+   */
+  public void writeLaunchEnv(OutputStream out, Map<String, String> environment,
+    Map<Path, List<String>> resources, List<String> command) throws IOException{
+    ContainerLaunch.ShellScriptBuilder sb =
+      ContainerLaunch.ShellScriptBuilder.create();
     if (environment != null) {
       for (Map.Entry<String,String> env : environment.entrySet()) {
         sb.env(env.getKey().toString(), env.getValue().toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0ccea33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.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/DockerContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java
index c854173..71eaa04 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DockerContainerExecutor.java
@@ -18,10 +18,24 @@
 
 package org.apache.hadoop.yarn.server.nodemanager;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
+import static org.apache.hadoop.fs.CreateFlag.CREATE;
+import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
 
 import org.apache.commons.lang.math.RandomUtils;
 import org.apache.commons.logging.Log;
@@ -45,38 +59,35 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.regex.Pattern;
-import java.net.InetSocketAddress;
-import static org.apache.hadoop.fs.CreateFlag.CREATE;
-import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 
 /**
- * This executor will launch a docker container and run the task inside the container.
+ * This executor will launch and run tasks inside Docker containers. It
+ * currently only supports simple authentication mode. It shares a lot of code
+ * with the DefaultContainerExecutor (and it may make sense to pull out those
+ * common pieces later).
  */
 public class DockerContainerExecutor extends ContainerExecutor {
-
   private static final Log LOG = LogFactory
-      .getLog(DockerContainerExecutor.class);
-  public static final String DOCKER_CONTAINER_EXECUTOR_SCRIPT = "docker_container_executor";
-  public static final String DOCKER_CONTAINER_EXECUTOR_SESSION_SCRIPT = "docker_container_executor_session";
-
-  // This validates that the image is a proper docker image and would not crash docker.
-  public static final String DOCKER_IMAGE_PATTERN = "^(([\\w\\.-]+)(:\\d+)*\\/)?[\\w\\.:-]+$";
-
+    .getLog(DockerContainerExecutor.class);
+  //The name of the script file that will launch the Docker containers
+  public static final String DOCKER_CONTAINER_EXECUTOR_SCRIPT =
+    "docker_container_executor";
+  //The name of the session script that the DOCKER_CONTAINER_EXECUTOR_SCRIPT
+  //launches in turn
+  public static final String DOCKER_CONTAINER_EXECUTOR_SESSION_SCRIPT =
+    "docker_container_executor_session";
+
+  //This validates that the image is a proper docker image and would not crash
+  //docker. The image name is not allowed to contain spaces. e.g.
+  //registry.somecompany.com:9999/containername:0.1 or
+  //containername:0.1 or
+  //containername
+  public static final String DOCKER_IMAGE_PATTERN =
+    "^(([\\w\\.-]+)(:\\d+)*\\/)?[\\w\\.:-]+$";
 
   private final FileContext lfs;
   private final Pattern dockerImagePattern;
@@ -96,23 +107,26 @@ public class DockerContainerExecutor extends ContainerExecutor {
 
   @Override
   public void init() throws IOException {
-    String auth = getConf().get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION);
+    String auth =
+      getConf().get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION);
     if (auth != null && !auth.equals("simple")) {
-      throw new IllegalStateException("DockerContainerExecutor only works with simple authentication mode");
+      throw new IllegalStateException(
+        "DockerContainerExecutor only works with simple authentication mode");
     }
-    String dockerExecutor = getConf().get(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME,
+    String dockerExecutor = getConf().get(
+      YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME,
       YarnConfiguration.NM_DEFAULT_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME);
     if (!new File(dockerExecutor).exists()) {
-      throw new IllegalStateException("Invalid docker exec path: " + dockerExecutor);
+      throw new IllegalStateException(
+        "Invalid docker exec path: " + dockerExecutor);
     }
   }
 
   @Override
   public synchronized void startLocalizer(Path nmPrivateContainerTokensPath,
-                                          InetSocketAddress nmAddr, String user, String appId, String locId,
-                                          LocalDirsHandlerService dirsHandler)
+    InetSocketAddress nmAddr, String user, String appId, String locId,
+    LocalDirsHandlerService dirsHandler)
     throws IOException, InterruptedException {
-
     List<String> localDirs = dirsHandler.getLocalDirs();
     List<String> logDirs = dirsHandler.getLogDirs();
 
@@ -128,7 +142,8 @@ public class DockerContainerExecutor extends ContainerExecutor {
     // randomly choose the local directory
     Path appStorageDir = getWorkingDir(localDirs, user, appId);
 
-    String tokenFn = String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId);
+    String tokenFn =
+      String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId);
     Path tokenDst = new Path(appStorageDir, tokenFn);
     copyFile(nmPrivateContainerTokensPath, tokenDst, user);
     LOG.info("Copying from " + nmPrivateContainerTokensPath + " to " + tokenDst);
@@ -140,31 +155,34 @@ public class DockerContainerExecutor extends ContainerExecutor {
 
 
   @Override
-  public int launchContainer(Container container,
-                             Path nmPrivateContainerScriptPath, Path nmPrivateTokensPath,
-                             String userName, String appId, Path containerWorkDir,
-                             List<String> localDirs, List<String> logDirs) throws IOException {
+  public int launchContainer(Container container, Path
+    nmPrivateContainerScriptPath, Path nmPrivateTokensPath, String userName,
+    String appId, Path containerWorkDir, List<String> localDirs, List<String>
+    logDirs) throws IOException {
+    //Variables for the launch environment can be injected from the command-line
+    //while submitting the application
     String containerImageName = container.getLaunchContext().getEnvironment()
-        .get(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME);
+      .get(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME);
     if (LOG.isDebugEnabled()) {
       LOG.debug("containerImageName from launchContext: " + containerImageName);
     }
-    Preconditions.checkArgument(!Strings.isNullOrEmpty(containerImageName), "Container image must not be null");
+    Preconditions.checkArgument(!Strings.isNullOrEmpty(containerImageName),
+      "Container image must not be null");
     containerImageName = containerImageName.replaceAll("['\"]", "");
 
-    Preconditions.checkArgument(saneDockerImage(containerImageName), "Image: " + containerImageName + " is not a proper docker image");
-    String dockerExecutor = getConf().get(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME,
-        YarnConfiguration.NM_DEFAULT_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME);
+    Preconditions.checkArgument(saneDockerImage(containerImageName), "Image: "
+      + containerImageName + " is not a proper docker image");
+    String dockerExecutor = getConf().get(
+      YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME,
+      YarnConfiguration.NM_DEFAULT_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME);
 
     FsPermission dirPerm = new FsPermission(APPDIR_PERM);
     ContainerId containerId = container.getContainerId();
 
     // create container dirs on all disks
     String containerIdStr = ConverterUtils.toString(containerId);
-    String appIdStr =
-        ConverterUtils.toString(
-            containerId.getApplicationAttemptId().
-                getApplicationId());
+    String appIdStr = ConverterUtils.toString(
+      containerId.getApplicationAttemptId().getApplicationId());
     for (String sLocalDir : localDirs) {
       Path usersdir = new Path(sLocalDir, ContainerLocalizer.USERCACHE);
       Path userdir = new Path(usersdir, userName);
@@ -178,46 +196,57 @@ public class DockerContainerExecutor extends ContainerExecutor {
     createContainerLogDirs(appIdStr, containerIdStr, logDirs, userName);
 
     Path tmpDir = new Path(containerWorkDir,
-        YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
+      YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
     createDir(tmpDir, dirPerm, false, userName);
 
     // copy launch script to work dir
     Path launchDst =
-        new Path(containerWorkDir, ContainerLaunch.CONTAINER_SCRIPT);
+      new Path(containerWorkDir, ContainerLaunch.CONTAINER_SCRIPT);
     lfs.util().copy(nmPrivateContainerScriptPath, launchDst);
 
     // copy container tokens to work dir
     Path tokenDst =
-        new Path(containerWorkDir, ContainerLaunch.FINAL_CONTAINER_TOKENS_FILE);
+      new Path(containerWorkDir, ContainerLaunch.FINAL_CONTAINER_TOKENS_FILE);
     lfs.util().copy(nmPrivateTokensPath, tokenDst);
 
-
-
     String localDirMount = toMount(localDirs);
     String logDirMount = toMount(logDirs);
-    String containerWorkDirMount = toMount(Collections.singletonList(containerWorkDir.toUri().getPath()));
+    String containerWorkDirMount = toMount(Collections.singletonList(
+      containerWorkDir.toUri().getPath()));
     StringBuilder commands = new StringBuilder();
+    //Use docker run to launch the docker container. See man pages for
+    //docker-run
+    //--rm removes the container automatically once the container finishes
+    //--net=host allows the container to take on the host's network stack
+    //--name sets the Docker Container name to the YARN containerId string
+    //-v is used to bind mount volumes for local, log and work dirs.
     String commandStr = commands.append(dockerExecutor)
-        .append(" ")
-        .append("run")
-        .append(" ")
-        .append("--rm --net=host")
-        .append(" ")
-        .append(" --name " + containerIdStr)
-        .append(localDirMount)
-        .append(logDirMount)
-        .append(containerWorkDirMount)
-        .append(" ")
-        .append(containerImageName)
-        .toString();
-    String dockerPidScript = "`" + dockerExecutor + " inspect --format {{.State.Pid}} " + containerIdStr + "`";
+      .append(" ")
+      .append("run")
+      .append(" ")
+      .append("--rm --net=host")
+      .append(" ")
+      .append(" --name " + containerIdStr)
+      .append(localDirMount)
+      .append(logDirMount)
+      .append(containerWorkDirMount)
+      .append(" ")
+      .append(containerImageName)
+      .toString();
+    //Get the pid of the process which has been launched as a docker container
+    //using docker inspect
+    String dockerPidScript = "`" + dockerExecutor +
+      " inspect --format {{.State.Pid}} " + containerIdStr + "`";
+
     // Create new local launch wrapper script
-    LocalWrapperScriptBuilder sb =
-      new UnixLocalWrapperScriptBuilder(containerWorkDir, commandStr, dockerPidScript);
+    LocalWrapperScriptBuilder sb = new UnixLocalWrapperScriptBuilder(
+      containerWorkDir, commandStr, dockerPidScript);
     Path pidFile = getPidFilePath(containerId);
     if (pidFile != null) {
       sb.writeLocalWrapperScript(launchDst, pidFile);
     } else {
+      //Although the container was activated by ContainerLaunch before exec()
+      //was called, since then deactivateContainer() has been called.
       LOG.info("Container " + containerIdStr
           + " was marked as inactive. Returning terminated error");
       return ExitCode.TERMINATED.getExitCode();
@@ -234,12 +263,13 @@ public class DockerContainerExecutor extends ContainerExecutor {
       String[] command = getRunCommand(sb.getWrapperScriptPath().toString(),
         containerIdStr, userName, pidFile, this.getConf());
       if (LOG.isDebugEnabled()) {
-        LOG.debug("launchContainer: " + commandStr + " " + Joiner.on(" ").join(command));
+        LOG.debug("launchContainer: " + commandStr + " " +
+          Joiner.on(" ").join(command));
       }
       shExec = new ShellCommandExecutor(
-          command,
-          new File(containerWorkDir.toUri().getPath()),
-          container.getLaunchContext().getEnvironment());      // sanitized env
+        command,
+        new File(containerWorkDir.toUri().getPath()),
+        container.getLaunchContext().getEnvironment());      // sanitized env
       if (isContainerActive(containerId)) {
         shExec.execute();
       } else {
@@ -279,9 +309,17 @@ public class DockerContainerExecutor extends ContainerExecutor {
   }
 
   @Override
-  public void writeLaunchEnv(OutputStream out, Map<String, String> environment, Map<Path, List<String>> resources, List<String> command) throws IOException {
-    ContainerLaunch.ShellScriptBuilder sb = ContainerLaunch.ShellScriptBuilder.create();
+  /**
+   * Filter the environment variables that may conflict with the ones set in
+   * the docker image and write them out to an OutputStream.
+   */
+  public void writeLaunchEnv(OutputStream out, Map<String, String> environment,
+    Map<Path, List<String>> resources, List<String> command)
+    throws IOException {
+    ContainerLaunch.ShellScriptBuilder sb =
+      ContainerLaunch.ShellScriptBuilder.create();
 
+    //Remove environments that may conflict with the ones in Docker image.
     Set<String> exclusionSet = new HashSet<String>();
     exclusionSet.add(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME);
     exclusionSet.add(ApplicationConstants.Environment.HADOOP_YARN_HOME.name());
@@ -427,6 +465,9 @@ public class DockerContainerExecutor extends ContainerExecutor {
     return builder.toString();
   }
 
+  //This class facilitates (only) the creation of platform-specific scripts that
+  //will be used to launch the containers
+  //TODO: This should be re-used from the DefaultContainerExecutor.
   private abstract class LocalWrapperScriptBuilder {
 
     private final Path wrapperScriptPath;
@@ -435,7 +476,8 @@ public class DockerContainerExecutor extends ContainerExecutor {
       return wrapperScriptPath;
     }
 
-    public void writeLocalWrapperScript(Path launchDst, Path pidFile) throws IOException {
+    public void writeLocalWrapperScript(Path launchDst, Path pidFile)
+      throws IOException {
       DataOutputStream out = null;
       PrintStream pout = null;
 
@@ -448,8 +490,8 @@ public class DockerContainerExecutor extends ContainerExecutor {
       }
     }
 
-    protected abstract void writeLocalWrapperScript(Path launchDst, Path pidFile,
-                                                    PrintStream pout);
+    protected abstract void writeLocalWrapperScript(Path launchDst,
+      Path pidFile, PrintStream pout);
 
     protected LocalWrapperScriptBuilder(Path containerWorkDir) {
       this.wrapperScriptPath = new Path(containerWorkDir,
@@ -457,13 +499,15 @@ public class DockerContainerExecutor extends ContainerExecutor {
     }
   }
 
+  //TODO: This class too should be used from DefaultContainerExecutor.
   private final class UnixLocalWrapperScriptBuilder
-      extends LocalWrapperScriptBuilder {
+    extends LocalWrapperScriptBuilder {
     private final Path sessionScriptPath;
     private final String dockerCommand;
     private final String dockerPidScript;
 
-    public UnixLocalWrapperScriptBuilder(Path containerWorkDir, String dockerCommand, String dockerPidScript) {
+    public UnixLocalWrapperScriptBuilder(Path containerWorkDir,
+      String dockerCommand, String dockerPidScript) {
       super(containerWorkDir);
       this.dockerCommand = dockerCommand;
       this.dockerPidScript = dockerPidScript;
@@ -480,8 +524,7 @@ public class DockerContainerExecutor extends ContainerExecutor {
 
     @Override
     public void writeLocalWrapperScript(Path launchDst, Path pidFile,
-                                        PrintStream pout) {
-
+      PrintStream pout) {
       String exitCodeFile = ContainerLaunch.getExitCodeFile(
         pidFile.toString());
       String tmpFile = exitCodeFile + ".tmp";
@@ -505,7 +548,8 @@ public class DockerContainerExecutor extends ContainerExecutor {
         // hence write pid to tmp file first followed by a mv
         pout.println("#!/usr/bin/env bash");
         pout.println();
-        pout.println("echo "+ dockerPidScript +" > " + pidFile.toString() + ".tmp");
+        pout.println("echo "+ dockerPidScript +" > " + pidFile.toString()
+          + ".tmp");
         pout.println("/bin/mv -f " + pidFile.toString() + ".tmp " + pidFile);
         pout.println(dockerCommand + " bash \"" +
           launchDst.toUri().getPath().toString() + "\"");
@@ -518,7 +562,7 @@ public class DockerContainerExecutor extends ContainerExecutor {
   }
 
   protected void createDir(Path dirPath, FsPermission perms,
-                           boolean createParent, String user) throws IOException {
+    boolean createParent, String user) throws IOException {
     lfs.mkdir(dirPath, perms, createParent);
     if (!perms.equals(perms.applyUMask(lfs.getUMask()))) {
       lfs.setPermission(dirPath, perms);
@@ -532,13 +576,14 @@ public class DockerContainerExecutor extends ContainerExecutor {
    * </ul>
    */
   void createUserLocalDirs(List<String> localDirs, String user)
-      throws IOException {
+    throws IOException {
     boolean userDirStatus = false;
     FsPermission userperms = new FsPermission(USER_PERM);
     for (String localDir : localDirs) {
       // create $local.dir/usercache/$user and its immediate parent
       try {
-        createDir(getUserCacheDir(new Path(localDir), user), userperms, true, user);
+        createDir(getUserCacheDir(new Path(localDir), user), userperms, true,
+          user);
       } catch (IOException e) {
         LOG.warn("Unable to create the user directory : " + localDir, e);
         continue;
@@ -633,7 +678,7 @@ public class DockerContainerExecutor extends ContainerExecutor {
    * Create application log directories on all disks.
    */
   void createContainerLogDirs(String appId, String containerId,
-                              List<String> logDirs, String user) throws IOException {
+    List<String> logDirs, String user) throws IOException {
 
     boolean containerLogDirStatus = false;
     FsPermission containerLogDirPerms = new FsPermission(LOGDIR_PERM);
@@ -707,7 +752,7 @@ public class DockerContainerExecutor extends ContainerExecutor {
   }
 
   protected Path getWorkingDir(List<String> localDirs, String user,
-                               String appId) throws IOException {
+    String appId) throws IOException {
     Path appStorageDir = null;
     long totalAvailable = 0L;
     long[] availableOnDisk = new long[localDirs.size()];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0ccea33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.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/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
index a87238d..5a9229b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
@@ -267,11 +267,11 @@ public class ContainerLaunch implements Callable<Integer> {
         // Sanitize the container's environment
         sanitizeEnv(environment, containerWorkDir, appDirs, containerLogDirs,
           localResources, nmPrivateClasspathJarDir);
-        
+
         // Write out the environment
-        exec.writeLaunchEnv(containerScriptOutStream, environment, localResources,
-            launchContext.getCommands());
-        
+        exec.writeLaunchEnv(containerScriptOutStream, environment,
+          localResources, launchContext.getCommands());
+
         // /////////// End of writing out container-script
 
         // /////////// Write out the container-tokens in the nmPrivate space.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0ccea33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutor.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/TestDockerContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutor.java
index ac02542..65e381c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutor.java
@@ -18,7 +18,18 @@
 
 package org.apache.hadoop.yarn.server.nodemanager;
 
-import com.google.common.base.Strings;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -26,48 +37,29 @@ import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.LineNumberReader;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeTrue;
-import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import com.google.common.base.Strings;
 
 /**
- * This is intended to test the DockerContainerExecutor code, but it requires docker
- * to be installed.
+ * This is intended to test the DockerContainerExecutor code, but it requires
+ * docker to be installed.
  * <br><ol>
- * <li>Install docker, and Compile the code with docker-service-url set to the host and port
- * where docker service is running.
+ * <li>Install docker, and Compile the code with docker-service-url set to the
+ * host and port where docker service is running.
  * <br><pre><code>
- * > mvn clean install -Ddocker-service-url=tcp://0.0.0.0:4243
- *                          -DskipTests
+ * > mvn clean install -Ddocker-service-url=tcp://0.0.0.0:4243 -DskipTests
  * </code></pre>
  */
 public class TestDockerContainerExecutor {
   private static final Log LOG = LogFactory
-      .getLog(TestDockerContainerExecutor.class);
+    .getLog(TestDockerContainerExecutor.class);
   private static File workSpace = null;
   private DockerContainerExecutor exec = null;
   private LocalDirsHandlerService dirsHandler;
@@ -75,14 +67,10 @@ public class TestDockerContainerExecutor {
   private FileContext lfs;
   private String yarnImage;
 
-  private int id = 0;
   private String appSubmitter;
   private String dockerUrl;
   private String testImage = "centos:latest";
   private String dockerExec;
-  private String containerIdStr;
-
-
   private ContainerId getNextContainerId() {
     ContainerId cId = mock(ContainerId.class, RETURNS_DEEP_STUBS);
     String id = "CONTAINER_" + System.currentTimeMillis();
@@ -91,6 +79,8 @@ public class TestDockerContainerExecutor {
   }
 
   @Before
+  //Initialize a new DockerContainerExecutor that will be used to launch mocked
+  //containers.
   public void setup() {
     try {
       lfs = FileContext.getLocalFSFileContext();
@@ -113,8 +103,10 @@ public class TestDockerContainerExecutor {
     }
     dockerUrl = " -H " + dockerUrl;
     dockerExec = "docker " + dockerUrl;
-    conf.set(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, yarnImage);
-    conf.set(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME, dockerExec);
+    conf.set(
+      YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, yarnImage);
+    conf.set(
+      YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME, dockerExec);
     exec = new DockerContainerExecutor();
     dirsHandler = new LocalDirsHandlerService();
     dirsHandler.init(conf);
@@ -129,11 +121,10 @@ public class TestDockerContainerExecutor {
 
   private Shell.ShellCommandExecutor shellExec(String command) {
     try {
-
       Shell.ShellCommandExecutor shExec = new Shell.ShellCommandExecutor(
-          command.split("\\s+"),
-          new File(workDir.toUri().getPath()),
-          System.getenv());
+        command.split("\\s+"),
+        new File(workDir.toUri().getPath()),
+        System.getenv());
       shExec.execute();
       return shExec;
     } catch (IOException e) {
@@ -145,14 +136,24 @@ public class TestDockerContainerExecutor {
     return exec != null;
   }
 
-  private int runAndBlock(ContainerId cId, Map<String, String> launchCtxEnv, String... cmd) throws IOException {
+  /**
+   * Test that a docker container can be launched to run a command
+   * @param cId a fake ContainerID
+   * @param launchCtxEnv
+   * @param cmd the command to launch inside the docker container
+   * @return the exit code of the process used to launch the docker container
+   * @throws IOException
+   */
+  private int runAndBlock(ContainerId cId, Map<String, String> launchCtxEnv,
+    String... cmd) throws IOException {
     String appId = "APP_" + System.currentTimeMillis();
     Container container = mock(Container.class);
     ContainerLaunchContext context = mock(ContainerLaunchContext.class);
 
     when(container.getContainerId()).thenReturn(cId);
     when(container.getLaunchContext()).thenReturn(context);
-    when(cId.getApplicationAttemptId().getApplicationId().toString()).thenReturn(appId);
+    when(cId.getApplicationAttemptId().getApplicationId().toString())
+      .thenReturn(appId);
     when(context.getEnvironment()).thenReturn(launchCtxEnv);
 
     String script = writeScriptFile(launchCtxEnv, cmd);
@@ -164,11 +165,13 @@ public class TestDockerContainerExecutor {
 
     exec.activateContainer(cId, pidFile);
     return exec.launchContainer(container, scriptPath, tokensPath,
-        appSubmitter, appId, workDir, dirsHandler.getLocalDirs(),
-        dirsHandler.getLogDirs());
+      appSubmitter, appId, workDir, dirsHandler.getLocalDirs(),
+      dirsHandler.getLogDirs());
   }
 
-  private String writeScriptFile(Map<String, String> launchCtxEnv, String... cmd) throws IOException {
+  // Write the script used to launch the docker container in a temp file
+  private String writeScriptFile(Map<String, String> launchCtxEnv,
+    String... cmd) throws IOException {
     File f = File.createTempFile("TestDockerContainerExecutor", ".sh");
     f.deleteOnExit();
     PrintWriter p = new PrintWriter(new FileOutputStream(f));
@@ -193,6 +196,10 @@ public class TestDockerContainerExecutor {
     }
   }
 
+  /**
+   * Test that a touch command can be launched successfully in a docker
+   * container
+   */
   @Test
   public void testLaunchContainer() throws IOException {
     if (!shouldRun()) {
@@ -201,12 +208,13 @@ public class TestDockerContainerExecutor {
     }
 
     Map<String, String> env = new HashMap<String, String>();
-    env.put(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
+    env.put(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME,
+      testImage);
     String touchFileName = "touch-file-" + System.currentTimeMillis();
     File touchFile = new File(dirsHandler.getLocalDirs().get(0), touchFileName);
     ContainerId cId = getNextContainerId();
-    int ret = runAndBlock(
-        cId, env, "touch", touchFile.getAbsolutePath(), "&&", "cp", touchFile.getAbsolutePath(), "/");
+    int ret = runAndBlock(cId, env, "touch", touchFile.getAbsolutePath(), "&&",
+      "cp", touchFile.getAbsolutePath(), "/");
 
     assertEquals(0, ret);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0ccea33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutorWithMocks.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/TestDockerContainerExecutorWithMocks.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutorWithMocks.java
index 3584fed..8acd9ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutorWithMocks.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDockerContainerExecutorWithMocks.java
@@ -18,7 +18,23 @@
 
 package org.apache.hadoop.yarn.server.nodemanager;
 
-import com.google.common.base.Strings;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.LineNumberReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -36,30 +52,13 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.LineNumberReader;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
-
 /**
  * Mock tests for docker container executor
  */
 public class TestDockerContainerExecutorWithMocks {
 
   private static final Log LOG = LogFactory
-      .getLog(TestDockerContainerExecutorWithMocks.class);
+    .getLog(TestDockerContainerExecutorWithMocks.class);
   public static final String DOCKER_LAUNCH_COMMAND = "/bin/true";
   private DockerContainerExecutor dockerContainerExecutor = null;
   private LocalDirsHandlerService dirsHandler;
@@ -81,8 +80,10 @@ public class TestDockerContainerExecutorWithMocks {
     conf.set(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH, executorPath);
     conf.set(YarnConfiguration.NM_LOCAL_DIRS, "/tmp/nm-local-dir" + time);
     conf.set(YarnConfiguration.NM_LOG_DIRS, "/tmp/userlogs" + time);
-    conf.set(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, yarnImage);
-    conf.set(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME , DOCKER_LAUNCH_COMMAND);
+    conf.set(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME,
+      yarnImage);
+    conf.set(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME,
+      DOCKER_LAUNCH_COMMAND);
     dockerContainerExecutor = new DockerContainerExecutor();
     dirsHandler = new LocalDirsHandlerService();
     dirsHandler.init(conf);
@@ -95,7 +96,6 @@ public class TestDockerContainerExecutorWithMocks {
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
-
   }
 
   @After
@@ -110,13 +110,17 @@ public class TestDockerContainerExecutorWithMocks {
   }
 
   @Test(expected = IllegalStateException.class)
+  //Test that DockerContainerExecutor doesn't successfully init on a secure
+  //cluster
   public void testContainerInitSecure() throws IOException {
     dockerContainerExecutor.getConf().set(
-        CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+      CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
     dockerContainerExecutor.init();
   }
 
   @Test(expected = IllegalArgumentException.class)
+  //Test that when the image name is null, the container launch throws an
+  //IllegalArgumentException
   public void testContainerLaunchNullImage() throws IOException {
     String appSubmitter = "nobody";
     String appId = "APP_ID";
@@ -126,17 +130,19 @@ public class TestDockerContainerExecutorWithMocks {
     Container container = mock(Container.class, RETURNS_DEEP_STUBS);
     ContainerId cId = mock(ContainerId.class, RETURNS_DEEP_STUBS);
     ContainerLaunchContext context = mock(ContainerLaunchContext.class);
-    HashMap<String, String> env = new HashMap<String,String>();
 
+    HashMap<String, String> env = new HashMap<String,String>();
     when(container.getContainerId()).thenReturn(cId);
     when(container.getLaunchContext()).thenReturn(context);
-    when(cId.getApplicationAttemptId().getApplicationId().toString()).thenReturn(appId);
+    when(cId.getApplicationAttemptId().getApplicationId().toString())
+      .thenReturn(appId);
     when(cId.toString()).thenReturn(containerId);
 
     when(context.getEnvironment()).thenReturn(env);
-    env.put(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
-    dockerContainerExecutor.getConf()
-        .set(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
+    env.put(
+      YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
+    dockerContainerExecutor.getConf().set(
+      YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
     Path scriptPath = new Path("file:///bin/echo");
     Path tokensPath = new Path("file:///dev/null");
 
@@ -149,6 +155,8 @@ public class TestDockerContainerExecutorWithMocks {
   }
 
   @Test(expected = IllegalArgumentException.class)
+  //Test that when the image name is invalid, the container launch throws an
+  //IllegalArgumentException
   public void testContainerLaunchInvalidImage() throws IOException {
     String appSubmitter = "nobody";
     String appId = "APP_ID";
@@ -162,13 +170,15 @@ public class TestDockerContainerExecutorWithMocks {
 
     when(container.getContainerId()).thenReturn(cId);
     when(container.getLaunchContext()).thenReturn(context);
-    when(cId.getApplicationAttemptId().getApplicationId().toString()).thenReturn(appId);
+    when(cId.getApplicationAttemptId().getApplicationId().toString())
+      .thenReturn(appId);
     when(cId.toString()).thenReturn(containerId);
 
     when(context.getEnvironment()).thenReturn(env);
-    env.put(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
-    dockerContainerExecutor.getConf()
-      .set(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
+    env.put(
+      YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
+    dockerContainerExecutor.getConf().set(
+      YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
     Path scriptPath = new Path("file:///bin/echo");
     Path tokensPath = new Path("file:///dev/null");
 
@@ -181,6 +191,8 @@ public class TestDockerContainerExecutorWithMocks {
   }
 
   @Test
+  //Test that a container launch correctly wrote the session script with the
+  //commands we expected
   public void testContainerLaunch() throws IOException {
     String appSubmitter = "nobody";
     String appId = "APP_ID";
@@ -194,40 +206,48 @@ public class TestDockerContainerExecutorWithMocks {
 
     when(container.getContainerId()).thenReturn(cId);
     when(container.getLaunchContext()).thenReturn(context);
-    when(cId.getApplicationAttemptId().getApplicationId().toString()).thenReturn(appId);
+    when(cId.getApplicationAttemptId().getApplicationId().toString())
+      .thenReturn(appId);
     when(cId.toString()).thenReturn(containerId);
 
     when(context.getEnvironment()).thenReturn(env);
-    env.put(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
+    env.put(
+      YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME, testImage);
     Path scriptPath = new Path("file:///bin/echo");
     Path tokensPath = new Path("file:///dev/null");
 
     Path pidFile = new Path(workDir, "pid");
 
     dockerContainerExecutor.activateContainer(cId, pidFile);
-    int ret = dockerContainerExecutor.launchContainer(container, scriptPath, tokensPath,
-        appSubmitter, appId, workDir, dirsHandler.getLocalDirs(),
-        dirsHandler.getLogDirs());
+    int ret = dockerContainerExecutor.launchContainer(container, scriptPath,
+      tokensPath, appSubmitter, appId, workDir, dirsHandler.getLocalDirs(),
+      dirsHandler.getLogDirs());
     assertEquals(0, ret);
     //get the script
     Path sessionScriptPath = new Path(workDir,
-        Shell.appendScriptExtension(
-            DockerContainerExecutor.DOCKER_CONTAINER_EXECUTOR_SESSION_SCRIPT));
-    LineNumberReader lnr = new LineNumberReader(new FileReader(sessionScriptPath.toString()));
+      Shell.appendScriptExtension(
+        DockerContainerExecutor.DOCKER_CONTAINER_EXECUTOR_SESSION_SCRIPT));
+    LineNumberReader lnr = new LineNumberReader(new FileReader(
+      sessionScriptPath.toString()));
     boolean cmdFound = false;
     List<String> localDirs = dirsToMount(dirsHandler.getLocalDirs());
     List<String> logDirs = dirsToMount(dirsHandler.getLogDirs());
-    List<String> workDirMount = dirsToMount(Collections.singletonList(workDir.toUri().getPath()));
-    List<String> expectedCommands =  new ArrayList<String>(
-        Arrays.asList(DOCKER_LAUNCH_COMMAND, "run", "--rm", "--net=host",  "--name", containerId));
+    List<String> workDirMount = dirsToMount(Collections.singletonList(
+      workDir.toUri().getPath()));
+    List<String> expectedCommands =  new ArrayList<String>(Arrays.asList(
+      DOCKER_LAUNCH_COMMAND, "run", "--rm", "--net=host",  "--name",
+      containerId));
     expectedCommands.addAll(localDirs);
     expectedCommands.addAll(logDirs);
     expectedCommands.addAll(workDirMount);
     String shellScript =  workDir + "/launch_container.sh";
 
-    expectedCommands.addAll(Arrays.asList(testImage.replaceAll("['\"]", ""), "bash","\"" + shellScript + "\""));
+    expectedCommands.addAll(Arrays.asList(testImage.replaceAll("['\"]", ""),
+      "bash","\"" + shellScript + "\""));
 
-    String expectedPidString = "echo `/bin/true inspect --format {{.State.Pid}} " + containerId+"` > "+ pidFile.toString() + ".tmp";
+    String expectedPidString =
+      "echo `/bin/true inspect --format {{.State.Pid}} " + containerId+"` > "+
+      pidFile.toString() + ".tmp";
     boolean pidSetterFound = false;
     while(lnr.ready()){
       String line = lnr.readLine();


[15/50] [abbrv] hadoop git commit: HDFS-8002. Website refers to /trash directory. Contributd by Brahma Reddy Battula.

Posted by zh...@apache.org.
HDFS-8002. Website refers to /trash directory. Contributd by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7285
Commit: e7ea2a8e8f0a7b428ef10552885757b99b59e4dc
Parents: 04ee18e
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Mar 31 00:27:50 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Mar 31 00:27:50 2015 +0900

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                     | 3 +++
 hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7ea2a8e/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 811ee75..efba80e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -376,6 +376,9 @@ Release 2.8.0 - UNRELEASED
     greater or equal to 1 there is mismatch in the UI report
     (J.Andreina via vinayakumarb)
 
+    HDFS-8002. Website refers to /trash directory. (Brahma Reddy Battula via
+    aajisaka)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7ea2a8e/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
index 87a9fcd..5a8e366 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
@@ -224,9 +224,9 @@ Space Reclamation
 
 ### File Deletes and Undeletes
 
-When a file is deleted by a user or an application, it is not immediately removed from HDFS. Instead, HDFS first renames it to a file in the `/trash` directory. The file can be restored quickly as long as it remains in `/trash`. A file remains in `/trash` for a configurable amount of time. After the expiry of its life in `/trash`, the NameNode deletes the file from the HDFS namespace. The deletion of a file causes the blocks associated with the file to be freed. Note that there could be an appreciable time delay between the time a file is deleted by a user and the time of the corresponding increase in free space in HDFS.
+When a file is deleted by a user or an application, it is not immediately removed from HDFS. Instead, HDFS first renames it to a file in the trash directory(`/user/<username>/.Trash`). The file can be restored quickly as long as it remains in trash. A file remains in trash for a configurable amount of time. After the expiry of its life in trash, the NameNode deletes the file from the HDFS namespace. The deletion of a file causes the blocks associated with the file to be freed. Note that there could be an appreciable time delay between the time a file is deleted by a user and the time of the corresponding increase in free space in HDFS.
 
-A user can Undelete a file after deleting it as long as it remains in the `/trash` directory. If a user wants to undelete a file that he/she has deleted, he/she can navigate the `/trash` directory and retrieve the file. The `/trash` directory contains only the latest copy of the file that was deleted. The `/trash` directory is just like any other directory with one special feature: HDFS applies specified policies to automatically delete files from this directory. Current default trash interval is set to 0 (Deletes file without storing in trash). This value is configurable parameter stored as `fs.trash.interval` stored in core-site.xml.
+A user can Undelete a file after deleting it as long as it remains in the trash directory. If a user wants to undelete a file that he/she has deleted, he/she can navigate the trash directory and retrieve the file. The trash directory contains only the latest copy of the file that was deleted. The trash directory is just like any other directory with one special feature: HDFS applies specified policies to automatically delete files from this directory. Current default trash interval is set to 0 (Deletes file without storing in trash). This value is configurable parameter stored as `fs.trash.interval` stored in core-site.xml.
 
 ### Decrease Replication Factor
 


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

Branch: refs/heads/HDFS-7285
Commit: 67e4d1f6a79e40201204b06f74a5c7fea7fadaf4
Parents: 7527a59
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 2 13:44:33 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:25 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/67e4d1f6/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/67e4d1f6/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/67e4d1f6/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/67e4d1f6/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/67e4d1f6/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 53ee288..0b737b2 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
@@ -579,11 +579,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);
   }
 
   /**
@@ -627,7 +638,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;
@@ -651,7 +662,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.");
     }
@@ -695,9 +706,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);
   }
 
@@ -748,7 +758,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);
 
@@ -1192,8 +1202,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();
@@ -2497,7 +2507,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
@@ -2572,7 +2582,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
@@ -3243,6 +3253,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;
@@ -3391,7 +3403,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/67e4d1f6/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/67e4d1f6/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 9355329..0faf3ad 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/67e4d1f6/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 e1b3570..6b20629 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;
@@ -417,6 +418,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();
     }
@@ -452,6 +457,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);
         }
@@ -473,7 +482,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();
@@ -485,16 +494,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/67e4d1f6/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 946a37c..89cfe05 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/67e4d1f6/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 d62b804..2e6e741 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()
@@ -1264,7 +1271,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/67e4d1f6/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/67e4d1f6/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/67e4d1f6/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 39fc5fd..3e1a0d6 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
@@ -207,7 +207,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;
@@ -2071,7 +2070,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);
@@ -3085,6 +3084,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);
@@ -3125,8 +3125,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();
@@ -3549,13 +3548,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;
@@ -3605,13 +3602,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);
@@ -3802,7 +3794,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++;
           }
         }
@@ -4031,7 +4023,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;
     }
 
@@ -4066,8 +4058,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:
@@ -4075,8 +4067,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*"
@@ -4176,6 +4168,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 {
@@ -5064,8 +5057,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());
@@ -6181,7 +6172,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/67e4d1f6/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 640fc57..f522850 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
@@ -86,7 +86,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);
 
@@ -261,10 +261,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);
     }
@@ -282,12 +282,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);
     }
@@ -546,7 +546,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 {
@@ -558,6 +558,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/67e4d1f6/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/67e4d1f6/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/67e4d1f6/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/67e4d1f6/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;
+  }
+}


[34/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903 and HDFS-7435. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903 and HDFS-7435. 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/c59219ba
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c59219ba
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c59219ba

Branch: refs/heads/HDFS-7285
Commit: c59219ba605e7083d037f6420c2680237a55e60a
Parents: 80fe23f
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 16 14:27:21 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:27 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/blockmanagement/DecommissionManager.java | 2 +-
 .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 2 +-
 .../hadoop/hdfs/server/namenode/snapshot/FileDiffList.java      | 3 ++-
 .../src/test/java/org/apache/hadoop/hdfs/TestDecommission.java  | 5 ++---
 .../hadoop/hdfs/server/namenode/TestAddStripedBlocks.java       | 4 ++--
 5 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59219ba/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 0faf3ad..df31d6e 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
@@ -536,7 +536,7 @@ public class DecommissionManager {
      */
     private void processBlocksForDecomInternal(
         final DatanodeDescriptor datanode,
-        final Iterator<BlockInfoContiguous> it,
+        final Iterator<? extends BlockInfo> it,
         final List<BlockInfoContiguous> insufficientlyReplicated,
         boolean pruneSufficientlyReplicated) {
       boolean firstReplicationLog = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59219ba/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 7bb504ef..bf1011a 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
@@ -2001,7 +2001,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // Check if the file is already being truncated with the same length
-    final BlockInfoContiguous last = file.getLastBlock();
+    final BlockInfo last = file.getLastBlock();
     if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
       final Block truncateBlock
           = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59219ba/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 a1263c5..d0248eb 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
@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -132,7 +133,7 @@ public class FileDiffList extends
       break;
     }
     // Check if last block is part of truncate recovery
-    BlockInfoContiguous lastBlock = file.getLastBlock();
+    BlockInfo lastBlock = file.getLastBlock();
     Block dontRemoveBlock = null;
     if(lastBlock != null && lastBlock.getBlockUCState().equals(
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59219ba/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index 081e40f..c3c1a9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.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.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -671,8 +671,7 @@ public class TestDecommission {
       GenericTestUtils.waitFor(new Supplier<Boolean>() {
         @Override
         public Boolean get() {
-          BlockInfoContiguous info =
-              blockManager.getStoredBlock(b.getLocalBlock());
+          BlockInfo info = blockManager.getStoredBlock(b.getLocalBlock());
           int count = 0;
           StringBuilder sb = new StringBuilder("Replica locations: ");
           for (int i = 0; i < info.numNodes(); i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59219ba/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 8b51309..05aec4b 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
@@ -265,9 +265,9 @@ public class TestAddStripedBlocks {
       ReplicaBeingWritten replica = new ReplicaBeingWritten(block, null, null,
           null);
       blocks.add(replica);
-      BlockListAsLongs bll = new BlockListAsLongs(null, blocks);
+      BlockListAsLongs bll = BlockListAsLongs.encode(blocks);
       StorageBlockReport[] reports = {new StorageBlockReport(storage,
-          bll.getBlockListAsLongs())};
+          bll)};
       cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId),
           bpId, reports);
     }


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

Branch: refs/heads/HDFS-7285
Commit: 7886ed1705a6c082475578e33bb9a715ab888b22
Parents: 42e26e2
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Feb 9 10:27:14 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:23 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/7886ed17/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/7886ed17/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 f6e15a3..3102a08 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
@@ -1925,7 +1925,7 @@ public class BlockManager {
           break;
         }
 
-        BlockInfoContiguous bi = blocksMap.getStoredBlock(b);
+        BlockInfoContiguous bi = getStoredBlock(b);
         if (bi == null) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
@@ -2068,7 +2068,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;
       
@@ -2208,7 +2208,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.
@@ -2499,7 +2499,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;
     }
@@ -3362,7 +3362,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 */


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

Branch: refs/heads/HDFS-7285
Commit: ec4f2243fd2c50ca00db3a609d11feaf177846a9
Parents: 0c6ed98
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 9 14:59:58 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:26 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     |  12 ++
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |  16 +--
 .../datanode/TestIncrementalBrVariations.java   |  14 +-
 .../server/namenode/TestAddStripedBlocks.java   | 141 +++++++++++++++++++
 .../hdfs/server/namenode/TestFSImage.java       |   5 +-
 20 files changed, 444 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec4f2243/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 e729869..a38e8f2 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/ec4f2243/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/ec4f2243/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 e26158b..cf51e3a 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
@@ -419,7 +419,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()));
@@ -495,7 +495,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);
     }
@@ -519,7 +519,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);
@@ -545,8 +545,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);
     }
@@ -950,8 +950,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/ec4f2243/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 4ec6f9e..22da083 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);
@@ -900,7 +901,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/ec4f2243/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 825e835..2db70d1 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
@@ -279,7 +279,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/ec4f2243/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 873eb6d..19b2f83 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
@@ -258,7 +258,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/ec4f2243/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 f31acc5..bee612a 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;
@@ -628,7 +629,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)
@@ -778,7 +779,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();
@@ -799,21 +800,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()];
@@ -833,6 +840,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();
@@ -842,9 +858,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;
@@ -1260,36 +1284,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;
   }
@@ -1299,8 +1323,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);
@@ -1313,14 +1338,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/ec4f2243/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/ec4f2243/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/ec4f2243/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 0b737b2..6481738 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;
@@ -839,21 +840,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();
@@ -868,13 +874,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 :
@@ -884,7 +898,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. */
@@ -2472,7 +2488,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/ec4f2243/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 a022c81..334a416 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
@@ -276,7 +276,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/ec4f2243/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 fa7f263..cf1f896 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
@@ -203,7 +203,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/ec4f2243/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/ec4f2243/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 3e1a0d6..7bb504ef 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
@@ -189,6 +189,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;
@@ -207,6 +208,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;
@@ -1751,8 +1753,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();
@@ -3091,7 +3099,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();
@@ -3124,8 +3132,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();
@@ -3135,7 +3144,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     getEditLog().logSync();
 
     // Return located block
-    return makeLocatedBlock(newBlock, targets, offset);
+    return makeLocatedBlock(newBlockInfo, targets, offset);
   }
 
   /*
@@ -3274,10 +3283,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/ec4f2243/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 58dbac5..2a9dbf1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -220,6 +220,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/ec4f2243/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 3eba280..f17ed0f 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
@@ -80,6 +80,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;
@@ -1732,4 +1736,12 @@ public class DFSTestUtil {
     dn.setLastUpdateMonotonic(Time.monotonicNow() + offset);
   }
 
+  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/ec4f2243/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 c7233bd..4b42f4c 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
@@ -489,16 +489,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);
   }
 
@@ -508,8 +508,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));
@@ -522,8 +522,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/ec4f2243/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/ec4f2243/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/ec4f2243/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();


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

Branch: refs/heads/HDFS-7285
Commit: 42e26e2624b4f733df631c6f8878e8b160b49ed6
Parents: 1d63b94
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Jan 30 16:16:26 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:23 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/42e26e26/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 610932a..eff457c 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
@@ -221,6 +221,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/42e26e26/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/42e26e26/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/42e26e26/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/42e26e26/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/42e26e26/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 7eea343..bbb6b19 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
@@ -473,10 +473,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
@@ -487,7 +491,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/42e26e26/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 0e0f484..c4768ee 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
@@ -2065,7 +2065,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)));
     }
@@ -3010,8 +3010,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;
 
@@ -3049,7 +3050,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();
@@ -3061,7 +3066,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.
@@ -3100,9 +3105,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();
@@ -3523,13 +3528,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;
@@ -3537,10 +3544,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;
@@ -6146,11 +6154,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/42e26e26/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 a6f07f9..1858e0a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -34,12 +34,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;
@@ -918,4 +920,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/42e26e26/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());
+  }
+}


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

Branch: refs/heads/HDFS-7285
Commit: 2fc3e353249eff6bc3d0e24c7d9d4c4121b7051c
Parents: ffc4171
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 30 10:11:24 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/2fc3e353/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/2fc3e353/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/2fc3e353/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/2fc3e353/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/2fc3e353/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/2fc3e353/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/2fc3e353/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/2fc3e353/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);
+
+}


[47/50] [abbrv] hadoop git commit: HDFS-7716. Add a test for BlockGroup support in FSImage. Contributed by Takuya Fukudome

Posted by zh...@apache.org.
HDFS-7716. Add a test for BlockGroup support in FSImage.  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/8d49fc33
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8d49fc33
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8d49fc33

Branch: refs/heads/HDFS-7285
Commit: 8d49fc33900f21e499432edda500fad046e30023
Parents: d0dad77
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Wed Mar 25 19:01:03 2015 +0900
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:13:08 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  6 ++-
 .../hdfs/server/namenode/TestFSImage.java       | 53 ++++++++++++++++++++
 2 files changed, 58 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d49fc33/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 2ef8527..21e4c03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -1,4 +1,8 @@
   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
+    directories ( Zhe Zhang via vinayakumarb )
+
+    HDFS-7716. Add a test for BlockGroup support in FSImage.
+    (Takuya Fukudome via szetszwo)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d49fc33/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 71dc978..440f5cd 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
@@ -31,7 +32,12 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.util.EnumSet;
 
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
 
@@ -46,6 +52,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -378,4 +385,50 @@ public class TestFSImage {
       FileUtil.fullyDelete(dfsDir);
     }
   }
+
+  /**
+   * Ensure that FSImage supports BlockGroup.
+   */
+  @Test
+  public void testSupportBlockGroup() throws IOException {
+    final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
+        HdfsConstants.NUM_PARITY_BLOCKS;
+    final int BLOCK_SIZE = 8 * 1024 * 1024;
+    Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      fs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+      Path file = new Path("/striped");
+      FSDataOutputStream out = fs.create(file);
+      byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE);
+      out.write(bytes);
+      out.close();
+
+      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.saveNamespace();
+      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+      cluster.restartNameNodes();
+      fs = cluster.getFileSystem();
+      assertTrue(fs.exists(file));
+
+      // check the information of striped blocks
+      FSNamesystem fsn = cluster.getNamesystem();
+      INodeFile inode = fsn.dir.getINode(file.toString()).asFile();
+      FileWithStripedBlocksFeature sb = inode.getStripedBlocksFeature();
+      assertNotNull(sb);
+      BlockInfoStriped[] blks = sb.getBlocks();
+      assertEquals(1, blks.length);
+      assertTrue(blks[0].isStriped());
+      assertEquals(HdfsConstants.NUM_DATA_BLOCKS, blks[0].getDataBlockNum());
+      assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, blks[0].getParityBlockNum());
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }


[11/50] [abbrv] hadoop git commit: HDFS-6408. Remove redundant definitions in log4j.properties. Contributed by Abhiraj Butala.

Posted by zh...@apache.org.
HDFS-6408. Remove redundant definitions in log4j.properties. Contributed by Abhiraj Butala.


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

Branch: refs/heads/HDFS-7285
Commit: 232eca944a721c62f37e9012546a7fa814da6e01
Parents: 257c77f
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Mar 30 11:25:35 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Mar 30 11:25:35 2015 +0900

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                     | 3 +++
 .../src/contrib/bkjournal/src/test/resources/log4j.properties   | 5 -----
 2 files changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/232eca94/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 e026f85..f4991da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -350,6 +350,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-6263. Remove DRFA.MaxBackupIndex config from log4j.properties.
     (Abhiraj Butala via aajisaka)
 
+    HDFS-6408. Remove redundant definitions in log4j.properties.
+    (Abhiraj Butala via aajisaka)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/232eca94/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties
index f66c84b..93c22f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties
@@ -53,8 +53,3 @@ log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{
 
 # Max log file size of 10MB
 log4j.appender.ROLLINGFILE.MaxFileSize=10MB
-
-log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
-log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p [%t:%C{1}@%L] - %m%n
-
-


[02/50] [abbrv] hadoop git commit: HADOOP-11760. Fix typo of javadoc in DistCp. Contributed by Brahma Reddy Battula.

Posted by zh...@apache.org.
HADOOP-11760. Fix typo of javadoc in DistCp. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7285
Commit: e074952bd6bedf58d993bbea690bad08c9a0e6aa
Parents: 60882ab
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Fri Mar 27 23:15:51 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Fri Mar 27 23:15:51 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../src/main/java/org/apache/hadoop/tools/DistCp.java             | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e074952b/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 a7d4adc..febbf6b 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -481,6 +481,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11724. DistCp throws NPE when the target directory is root.
     (Lei Eddy Xu via Yongjun Zhang) 
 
+    HADOOP-11760. Fix typo of javadoc in DistCp. (Brahma Reddy Battula via
+    ozawa).
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e074952b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java
index ada4b25..6921a1e 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java
@@ -401,7 +401,7 @@ public class DistCp extends Configured implements Tool {
    * job staging directory
    *
    * @return Returns the working folder information
-   * @throws Exception - EXception if any
+   * @throws Exception - Exception if any
    */
   private Path createMetaFolderPath() throws Exception {
     Configuration configuration = getConf();


[42/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903, HDFS-7435 and HDFS-7930 (this commit is for HDFS-7930 only)

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903, HDFS-7435 and HDFS-7930 (this commit is for HDFS-7930 only)


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

Branch: refs/heads/HDFS-7285
Commit: 9a0f626f74345ffa4622db558edea6244b663109
Parents: 26b5a06
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Mar 23 11:25:40 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:13:07 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/blockmanagement/BlockManager.java      | 7 ++++---
 .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java  | 7 ++++---
 .../org/apache/hadoop/hdfs/server/namenode/INodeFile.java     | 2 +-
 3 files changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a0f626f/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 091c85b..7dfe0a4 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
@@ -2114,17 +2114,18 @@ public class BlockManager {
    * Mark block replicas as corrupt except those on the storages in 
    * newStorages list.
    */
-  public void markBlockReplicasAsCorrupt(BlockInfoContiguous block, 
+  public void markBlockReplicasAsCorrupt(Block oldBlock,
+      BlockInfo block,
       long oldGenerationStamp, long oldNumBytes, 
       DatanodeStorageInfo[] newStorages) throws IOException {
     assert namesystem.hasWriteLock();
     BlockToMarkCorrupt b = null;
     if (block.getGenerationStamp() != oldGenerationStamp) {
-      b = new BlockToMarkCorrupt(block, oldGenerationStamp,
+      b = new BlockToMarkCorrupt(oldBlock, block, oldGenerationStamp,
           "genstamp does not match " + oldGenerationStamp
           + " : " + block.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
     } else if (block.getNumBytes() != oldNumBytes) {
-      b = new BlockToMarkCorrupt(block,
+      b = new BlockToMarkCorrupt(oldBlock, block,
           "length does not match " + oldNumBytes
           + " : " + block.getNumBytes(), Reason.SIZE_MISMATCH);
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a0f626f/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 4e6b5e3..4519cee 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
@@ -2795,7 +2795,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /** Compute quota change for converting a complete block to a UC block */
   private QuotaCounts computeQuotaDeltaForUCBlock(INodeFile file) {
     final QuotaCounts delta = new QuotaCounts.Builder().build();
-    final BlockInfoContiguous lastBlock = file.getLastBlock();
+    final BlockInfo lastBlock = file.getLastBlock();
     if (lastBlock != null) {
       final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes();
       final short repl = file.getBlockReplication();
@@ -4390,8 +4390,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         } else {
           iFile.convertLastBlockToUC(storedBlock, trimmedStorageInfos);
           if (closeFile) {
-            blockManager.markBlockReplicasAsCorrupt(storedBlock,
-                oldGenerationStamp, oldNumBytes, trimmedStorageInfos);
+            blockManager.markBlockReplicasAsCorrupt(oldBlock.getLocalBlock(),
+                storedBlock, oldGenerationStamp, oldNumBytes,
+                trimmedStorageInfos);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a0f626f/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 5014d6d..59aabe4 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
@@ -984,7 +984,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * @return the quota usage delta (not considering replication factor)
    */
   long computeQuotaDeltaForTruncate(final long newLength) {
-    final BlockInfoContiguous[] blocks = getBlocks();
+    final BlockInfo[] blocks = getBlocks();
     if (blocks == null || blocks.length == 0) {
       return 0;
     }


[41/50] [abbrv] hadoop git commit: HDFS-7369. Erasure coding: distribute recovery work for striped blocks to DataNode. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-7369. Erasure coding: distribute recovery work for striped blocks to DataNode. 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/5eb2c926
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5eb2c926
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5eb2c926

Branch: refs/heads/HDFS-7285
Commit: 5eb2c92685602ecb57b521cf75d4821c0d632bd4
Parents: b46621c
Author: Zhe Zhang <zh...@apache.org>
Authored: Wed Mar 18 15:52:36 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:12:45 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockCollection.java |   5 +
 .../server/blockmanagement/BlockManager.java    | 294 +++++++++++++------
 .../blockmanagement/DatanodeDescriptor.java     |  72 ++++-
 .../server/blockmanagement/DatanodeManager.java |  20 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   9 +-
 .../server/protocol/BlockECRecoveryCommand.java |  63 ++++
 .../hdfs/server/protocol/DatanodeProtocol.java  |   1 +
 .../blockmanagement/BlockManagerTestUtil.java   |   2 +-
 .../blockmanagement/TestBlockManager.java       |  22 +-
 .../TestRecoverStripedBlocks.java               | 107 +++++++
 10 files changed, 486 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eb2c926/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 440a081..50dd17b 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
@@ -86,4 +86,9 @@ public interface BlockCollection {
    * @return whether the block collection is under construction.
    */
   public boolean isUnderConstruction();
+
+  /**
+   * @return whether the block collection is in striping format
+   */
+  public boolean isStriped();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eb2c926/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 1e8ce1f..091c85b 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -531,9 +532,9 @@ public class BlockManager {
     
     NumberReplicas numReplicas = new NumberReplicas();
     // source node returned is not used
-    chooseSourceDatanode(block, containingNodes,
+    chooseSourceDatanodes(getStoredBlock(block), containingNodes,
         containingLiveReplicasNodes, numReplicas,
-        UnderReplicatedBlocks.LEVEL);
+        new LinkedList<Short>(), 1, UnderReplicatedBlocks.LEVEL);
     
     // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are 
     // not included in the numReplicas.liveReplicas() count
@@ -1327,15 +1328,15 @@ public class BlockManager {
   }
 
   /**
-   * Scan blocks in {@link #neededReplications} and assign replication
-   * work to data-nodes they belong to.
+   * Scan blocks in {@link #neededReplications} and assign recovery
+   * (replication or erasure coding) work to data-nodes they belong to.
    *
    * The number of process blocks equals either twice the number of live
    * data-nodes or the number of under-replicated blocks whichever is less.
    *
    * @return number of blocks scheduled for replication during this iteration.
    */
-  int computeReplicationWork(int blocksToProcess) {
+  int computeBlockRecoveryWork(int blocksToProcess) {
     List<List<BlockInfo>> blocksToReplicate = null;
     namesystem.writeLock();
     try {
@@ -1345,30 +1346,32 @@ public class BlockManager {
     } finally {
       namesystem.writeUnlock();
     }
-    return computeReplicationWorkForBlocks(blocksToReplicate);
+    return computeRecoveryWorkForBlocks(blocksToReplicate);
   }
 
-  /** Replicate a set of blocks
+  /**
+   * Recover a set of blocks to full strength through replication or
+   * erasure coding
    *
-   * @param blocksToReplicate blocks to be replicated, for each priority
+   * @param blocksToRecover blocks to be recovered, for each priority
    * @return the number of blocks scheduled for replication
    */
   @VisibleForTesting
-  int computeReplicationWorkForBlocks(List<List<BlockInfo>> blocksToReplicate) {
+  int computeRecoveryWorkForBlocks(List<List<BlockInfo>> blocksToRecover) {
     int requiredReplication, numEffectiveReplicas;
     List<DatanodeDescriptor> containingNodes;
-    DatanodeDescriptor srcNode;
     BlockCollection bc = null;
     int additionalReplRequired;
 
     int scheduledWork = 0;
-    List<ReplicationWork> work = new LinkedList<ReplicationWork>();
+    List<BlockRecoveryWork> recovWork = new LinkedList<>();
 
+    // Step 1: categorize at-risk blocks into replication and EC tasks
     namesystem.writeLock();
     try {
       synchronized (neededReplications) {
-        for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
-          for (BlockInfo block : blocksToReplicate.get(priority)) {
+        for (int priority = 0; priority < blocksToRecover.size(); priority++) {
+          for (BlockInfo block : blocksToRecover.get(priority)) {
             // block should belong to a file
             bc = blocksMap.getBlockCollection(block);
             // abandoned block or block reopened for append
@@ -1382,25 +1385,31 @@ public class BlockManager {
             requiredReplication = bc.getBlockReplication();
 
             // get a source data-node
-            containingNodes = new ArrayList<DatanodeDescriptor>();
-            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<DatanodeStorageInfo>();
+            containingNodes = new ArrayList<>();
+            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
             NumberReplicas numReplicas = new NumberReplicas();
-            srcNode = chooseSourceDatanode(
+            List<Short> missingBlockIndices = new LinkedList<>();
+            DatanodeDescriptor[] srcNodes;
+            int numSourceNodes = bc.isStriped() ?
+                HdfsConstants.NUM_DATA_BLOCKS : 1;
+            srcNodes = chooseSourceDatanodes(
                 block, containingNodes, liveReplicaNodes, numReplicas,
-                priority);
-            if(srcNode == null) { // block can not be replicated from any node
-              LOG.debug("Block " + block + " cannot be repl from any node");
+                missingBlockIndices, numSourceNodes, priority);
+            if(srcNodes == null || srcNodes.length == 0) {
+              // block can not be replicated from any node
+              LOG.debug("Block " + block + " cannot be recovered " +
+                  "from any node");
               continue;
             }
 
-            // liveReplicaNodes can include READ_ONLY_SHARED replicas which are 
+            // liveReplicaNodes can include READ_ONLY_SHARED replicas which are
             // not included in the numReplicas.liveReplicas() count
             assert liveReplicaNodes.size() >= numReplicas.liveReplicas();
 
             // do not schedule more if enough replicas is already pending
             numEffectiveReplicas = numReplicas.liveReplicas() +
                                     pendingReplications.getNumReplicas(block);
-      
+
             if (numEffectiveReplicas >= requiredReplication) {
               if ( (pendingReplications.getNumReplicas(block) > 0) ||
                    (blockHasEnoughRacks(block)) ) {
@@ -1417,9 +1426,21 @@ public class BlockManager {
             } else {
               additionalReplRequired = 1; // Needed on a new rack
             }
-            work.add(new ReplicationWork(block, bc, srcNode,
-                containingNodes, liveReplicaNodes, additionalReplRequired,
-                priority));
+            if (bc.isStriped()) {
+              ErasureCodingWork ecw = new ErasureCodingWork(block, bc, srcNodes,
+                  containingNodes, liveReplicaNodes, additionalReplRequired,
+                  priority);
+              short[] missingBlockArray = new short[missingBlockIndices.size()];
+              for (int i = 0 ; i < missingBlockIndices.size(); i++) {
+                missingBlockArray[i] = missingBlockIndices.get(i);
+              }
+              ecw.setMissingBlockIndices(missingBlockArray);
+              recovWork.add(ecw);
+            } else {
+              recovWork.add(new ReplicationWork(block, bc, srcNodes,
+                  containingNodes, liveReplicaNodes, additionalReplRequired,
+                  priority));
+            }
           }
         }
       }
@@ -1427,8 +1448,9 @@ public class BlockManager {
       namesystem.writeUnlock();
     }
 
+    // Step 2: choose target nodes for each recovery task
     final Set<Node> excludedNodes = new HashSet<Node>();
-    for(ReplicationWork rw : work){
+    for(BlockRecoveryWork rw : recovWork){
       // Exclude all of the containing nodes from being targets.
       // This list includes decommissioning or corrupt nodes.
       excludedNodes.clear();
@@ -1442,9 +1464,10 @@ public class BlockManager {
       rw.chooseTargets(blockplacement, storagePolicySuite, excludedNodes);
     }
 
+    // Step 3: add tasks to the DN
     namesystem.writeLock();
     try {
-      for(ReplicationWork rw : work){
+      for(BlockRecoveryWork rw : recovWork){
         final DatanodeStorageInfo[] targets = rw.targets;
         if(targets == null || targets.length == 0){
           rw.targets = null;
@@ -1483,7 +1506,7 @@ public class BlockManager {
 
           if ( (numReplicas.liveReplicas() >= requiredReplication) &&
                (!blockHasEnoughRacks(block)) ) {
-            if (rw.srcNode.getNetworkLocation().equals(
+            if (rw.srcNodes[0].getNetworkLocation().equals(
                 targets[0].getDatanodeDescriptor().getNetworkLocation())) {
               //No use continuing, unless a new rack in this case
               continue;
@@ -1491,7 +1514,17 @@ public class BlockManager {
           }
 
           // Add block to the to be replicated list
-          rw.srcNode.addBlockToBeReplicated(block, targets);
+          if (bc.isStriped()) {
+            assert rw instanceof ErasureCodingWork;
+            assert rw.targets.length > 0;
+            rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
+                new ExtendedBlock(namesystem.getBlockPoolId(), block),
+                rw.srcNodes, rw.targets,
+                ((ErasureCodingWork)rw).getMissingBlockIndicies());
+          }
+          else {
+            rw.srcNodes[0].addBlockToBeReplicated(block, targets);
+          }
           scheduledWork++;
           DatanodeStorageInfo.incrementBlocksScheduled(targets);
 
@@ -1515,7 +1548,7 @@ public class BlockManager {
 
     if (blockLog.isInfoEnabled()) {
       // log which blocks have been scheduled for replication
-      for(ReplicationWork rw : work){
+      for(BlockRecoveryWork rw : recovWork){
         DatanodeStorageInfo[] targets = rw.targets;
         if (targets != null && targets.length != 0) {
           StringBuilder targetList = new StringBuilder("datanode(s)");
@@ -1523,7 +1556,7 @@ public class BlockManager {
             targetList.append(' ');
             targetList.append(targets[k].getDatanodeDescriptor());
           }
-          blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNode,
+          blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNodes,
               rw.block, targetList);
         }
       }
@@ -1609,54 +1642,65 @@ public class BlockManager {
   }
 
   /**
-   * Parse the data-nodes the block belongs to and choose one,
-   * which will be the replication source.
+   * Parse the data-nodes the block belongs to and choose a certain number
+   * from them to be the recovery sources.
    *
    * We prefer nodes that are in DECOMMISSION_INPROGRESS state to other nodes
    * since the former do not have write traffic and hence are less busy.
    * We do not use already decommissioned nodes as a source.
-   * Otherwise we choose a random node among those that did not reach their
-   * replication limits.  However, if the replication is of the highest priority
-   * and all nodes have reached their replication limits, we will choose a
-   * random node despite the replication limit.
+   * Otherwise we randomly choose nodes among those that did not reach their
+   * replication limits. However, if the recovery work is of the highest
+   * priority and all nodes have reached their replication limits, we will
+   * randomly choose the desired number of nodes despite the replication limit.
    *
    * In addition form a list of all nodes containing the block
    * and calculate its replication numbers.
    *
    * @param block Block for which a replication source is needed
-   * @param containingNodes List to be populated with nodes found to contain the 
-   *                        given block
-   * @param nodesContainingLiveReplicas List to be populated with nodes found to
-   *                                    contain live replicas of the given block
-   * @param numReplicas NumberReplicas instance to be initialized with the 
-   *                                   counts of live, corrupt, excess, and
-   *                                   decommissioned replicas of the given
-   *                                   block.
+   * @param containingNodes List to be populated with nodes found to contain
+   *                        the given block
+   * @param nodesContainingLiveReplicas List to be populated with nodes found
+   *                                    to contain live replicas of the given
+   *                                    block
+   * @param numReplicas NumberReplicas instance to be initialized with the
+   *                    counts of live, corrupt, excess, and decommissioned
+   *                    replicas of the given block.
+   * @param missingBlockIndices List to be populated with indices of missing
+   *                            blocks in a striped block group or missing
+   *                            replicas of a replicated block
+   * @param numSourceNodes integer specifying the number of source nodes to
+   *                       choose
    * @param priority integer representing replication priority of the given
    *                 block
-   * @return the DatanodeDescriptor of the chosen node from which to replicate
-   *         the given block
-   */
-   @VisibleForTesting
-   DatanodeDescriptor chooseSourceDatanode(Block block,
-       List<DatanodeDescriptor> containingNodes,
-       List<DatanodeStorageInfo>  nodesContainingLiveReplicas,
-       NumberReplicas numReplicas,
-       int priority) {
+   * @return the array of DatanodeDescriptor of the chosen nodes from which to
+   *         recover the given block
+   */
+  @VisibleForTesting
+  DatanodeDescriptor[] chooseSourceDatanodes(BlockInfo block,
+      List<DatanodeDescriptor> containingNodes,
+      List<DatanodeStorageInfo> nodesContainingLiveReplicas,
+      NumberReplicas numReplicas,
+      List<Short> missingBlockIndices, int numSourceNodes, int priority) {
     containingNodes.clear();
     nodesContainingLiveReplicas.clear();
-    DatanodeDescriptor srcNode = null;
+    LinkedList<DatanodeDescriptor> srcNodes = new LinkedList<>();
     int live = 0;
     int decommissioned = 0;
     int corrupt = 0;
     int excess = 0;
-    
+    missingBlockIndices.clear();
+    Set<Short> healthyIndices = new HashSet<>();
+
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
     for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+      if (block.isStriped()) {
+        healthyIndices.add((short) ((BlockInfoStriped) block).
+            getStorageBlockIndex(storage));
+      }
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       LightWeightLinkedSet<Block> excessBlocks =
         excessReplicateMap.get(node.getDatanodeUuid());
-      int countableReplica = storage.getState() == State.NORMAL ? 1 : 0; 
+      int countableReplica = storage.getState() == State.NORMAL ? 1 : 0;
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
         corrupt += countableReplica;
       else if (node.isDecommissionInProgress() || node.isDecommissioned())
@@ -1690,19 +1734,32 @@ public class BlockManager {
         continue;
 
       // We got this far, current node is a reasonable choice
-      if (srcNode == null) {
-        srcNode = node;
+      if(srcNodes.size() < numSourceNodes) {
+        srcNodes.add(node);
         continue;
       }
       // switch to a different node randomly
       // this to prevent from deterministically selecting the same node even
       // if the node failed to replicate the block on previous iterations
-      if(DFSUtil.getRandom().nextBoolean())
-        srcNode = node;
+      if(DFSUtil.getRandom().nextBoolean()) {
+        int pos = DFSUtil.getRandom().nextInt(numSourceNodes);
+        if(!srcNodes.get(pos).isDecommissionInProgress()) {
+          srcNodes.set(pos, node);
+        }
+      }
     }
-    if(numReplicas != null)
+    if (block.isStriped()) {
+      for (short i = 0; i < HdfsConstants.NUM_DATA_BLOCKS +
+          HdfsConstants.NUM_PARITY_BLOCKS; i++) {
+        if (!healthyIndices.contains(i)) {
+          missingBlockIndices.add(i);
+        }
+      }
+    }
+    if(numReplicas != null) {
       numReplicas.initialize(live, decommissioned, corrupt, excess, 0);
-    return srcNode;
+    }
+    return srcNodes.toArray(new DatanodeDescriptor[srcNodes.size()]);
   }
 
   /**
@@ -1732,7 +1789,7 @@ public class BlockManager {
        */
     }
   }
-  
+
   /**
    * StatefulBlockInfo is used to build the "toUC" list, which is a list of
    * updates to the information about under-construction blocks.
@@ -3624,7 +3681,7 @@ public class BlockManager {
   }
 
   /**
-   * Periodically calls computeReplicationWork().
+   * Periodically calls computeBlockRecoveryWork().
    */
   private class ReplicationMonitor implements Runnable {
 
@@ -3682,7 +3739,7 @@ public class BlockManager {
     final int nodesToProcess = (int) Math.ceil(numlive
         * this.blocksInvalidateWorkPct);
 
-    int workFound = this.computeReplicationWork(blocksToProcess);
+    int workFound = this.computeBlockRecoveryWork(blocksToProcess);
 
     // Update counters
     namesystem.writeLock();
@@ -3709,49 +3766,118 @@ public class BlockManager {
     postponedMisreplicatedBlocks.clear();
     postponedMisreplicatedBlocksCount.set(0);
   }
-  
-
-  private static class ReplicationWork {
 
-    private final BlockInfo block;
-    private final BlockCollection bc;
+  /**
+   * This class is used internally by {@link this#computeRecoveryWorkForBlocks}
+   * to represent a task to recover a block through replication or erasure
+   * coding. Recovery is done by transferring data from {@link srcNodes} to
+   * {@link targets}
+   */
+  private static class BlockRecoveryWork {
+    protected final BlockInfo block;
+    protected final BlockCollection bc;
 
-    private final DatanodeDescriptor srcNode;
-    private final List<DatanodeDescriptor> containingNodes;
-    private final List<DatanodeStorageInfo> liveReplicaStorages;
-    private final int additionalReplRequired;
+    /**
+     * An erasure coding recovery task has multiple source nodes.
+     * A replication task only has 1 source node, stored on top of the array
+     */
+    protected final DatanodeDescriptor[] srcNodes;
+    /** Nodes containing the block; avoid them in choosing new targets */
+    protected final List<DatanodeDescriptor> containingNodes;
+    /** Required by {@link BlockPlacementPolicy#chooseTarget} */
+    protected final List<DatanodeStorageInfo> liveReplicaStorages;
+    protected final int additionalReplRequired;
 
-    private DatanodeStorageInfo targets[];
-    private final int priority;
+    protected DatanodeStorageInfo[] targets;
+    protected final int priority;
 
-    public ReplicationWork(BlockInfo block,
+    public BlockRecoveryWork(BlockInfo block,
         BlockCollection bc,
-        DatanodeDescriptor srcNode,
+        DatanodeDescriptor[] srcNodes,
         List<DatanodeDescriptor> containingNodes,
         List<DatanodeStorageInfo> liveReplicaStorages,
         int additionalReplRequired,
         int priority) {
       this.block = block;
       this.bc = bc;
-      this.srcNode = srcNode;
-      this.srcNode.incrementPendingReplicationWithoutTargets();
+      this.srcNodes = srcNodes;
       this.containingNodes = containingNodes;
       this.liveReplicaStorages = liveReplicaStorages;
       this.additionalReplRequired = additionalReplRequired;
       this.priority = priority;
       this.targets = null;
     }
-    
-    private void chooseTargets(BlockPlacementPolicy blockplacement,
+
+    protected void chooseTargets(BlockPlacementPolicy blockplacement,
+        BlockStoragePolicySuite storagePolicySuite,
+        Set<Node> excludedNodes) {
+    }
+  }
+
+  private static class ReplicationWork extends BlockRecoveryWork {
+
+    public ReplicationWork(BlockInfo block,
+        BlockCollection bc,
+        DatanodeDescriptor[] srcNodes,
+        List<DatanodeDescriptor> containingNodes,
+        List<DatanodeStorageInfo> liveReplicaStorages,
+        int additionalReplRequired,
+        int priority) {
+      super(block, bc, srcNodes, containingNodes,
+          liveReplicaStorages, additionalReplRequired, priority);
+      LOG.debug("Creating a ReplicationWork to recover " + block);
+    }
+
+    protected void chooseTargets(BlockPlacementPolicy blockplacement,
+        BlockStoragePolicySuite storagePolicySuite,
+        Set<Node> excludedNodes) {
+      assert srcNodes.length > 0
+          : "At least 1 source node should have been selected";
+      try {
+        targets = blockplacement.chooseTarget(bc.getName(),
+            additionalReplRequired, srcNodes[0], liveReplicaStorages, false,
+            excludedNodes, block.getNumBytes(),
+            storagePolicySuite.getPolicy(bc.getStoragePolicyID()));
+      } finally {
+        srcNodes[0].decrementPendingReplicationWithoutTargets();
+      }
+    }
+  }
+
+  private static class ErasureCodingWork extends BlockRecoveryWork {
+
+    private short[] missingBlockIndicies = null;
+
+    public ErasureCodingWork(BlockInfo block,
+        BlockCollection bc,
+        DatanodeDescriptor[] srcNodes,
+        List<DatanodeDescriptor> containingNodes,
+        List<DatanodeStorageInfo> liveReplicaStorages,
+        int additionalReplRequired,
+        int priority) {
+      super(block, bc, srcNodes, containingNodes,
+          liveReplicaStorages, additionalReplRequired, priority);
+      LOG.debug("Creating an ErasureCodingWork to recover " + block);
+    }
+
+    public short[] getMissingBlockIndicies() {
+      return missingBlockIndicies;
+    }
+
+    public void setMissingBlockIndices(short[] missingBlockIndicies) {
+      this.missingBlockIndicies = missingBlockIndicies;
+    }
+
+    protected void chooseTargets(BlockPlacementPolicy blockplacement,
         BlockStoragePolicySuite storagePolicySuite,
         Set<Node> excludedNodes) {
       try {
+        // TODO: new placement policy for EC considering multiple writers
         targets = blockplacement.chooseTarget(bc.getName(),
-            additionalReplRequired, srcNode, liveReplicaStorages, false,
+            additionalReplRequired, srcNodes[0], liveReplicaStorages, false,
             excludedNodes, block.getNumBytes(),
             storagePolicySuite.getPolicy(bc.getStoragePolicyID()));
       } finally {
-        srcNode.decrementPendingReplicationWithoutTargets();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eb2c926/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 334a416..9f2a4de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
+import java.util.Arrays;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -41,6 +42,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -97,6 +99,33 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
+  /** Block and targets pair */
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  public static class BlockECRecoveryInfo {
+    public final ExtendedBlock block;
+    public final DatanodeDescriptor[] sources;
+    public final DatanodeStorageInfo[] targets;
+    public final short[] missingBlockIndices;
+
+    BlockECRecoveryInfo(ExtendedBlock block, DatanodeDescriptor[] sources,
+        DatanodeStorageInfo[] targets, short[] missingBlockIndices) {
+      this.block = block;
+      this.sources = sources;
+      this.targets = targets;
+      this.missingBlockIndices = missingBlockIndices;
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().append("BlockECRecoveryInfo(\n  ").
+          append("Recovering ").append(block).
+          append(" From: ").append(Arrays.asList(sources)).
+          append(" To: ").append(Arrays.asList(targets)).append(")\n").
+          toString();
+    }
+  }
+
   /** A BlockTargetPair queue. */
   private static class BlockQueue<E> {
     private final Queue<E> blockq = new LinkedList<E>();
@@ -217,12 +246,17 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private long bandwidth;
 
   /** A queue of blocks to be replicated by this datanode */
-  private final BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
+  private final BlockQueue<BlockTargetPair> replicateBlocks =
+      new BlockQueue<>();
+  /** A queue of blocks to be erasure coded by this datanode */
+  private final BlockQueue<BlockECRecoveryInfo> erasurecodeBlocks =
+      new BlockQueue<>();
   /** A queue of blocks to be recovered by this datanode */
-  private final BlockQueue<BlockInfoContiguousUnderConstruction> recoverBlocks =
-                                new BlockQueue<BlockInfoContiguousUnderConstruction>();
+  private final BlockQueue<BlockInfoContiguousUnderConstruction>
+      recoverBlocks = new BlockQueue<>();
   /** A set of blocks to be invalidated by this datanode */
-  private final LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
+  private final LightWeightHashSet<Block> invalidateBlocks =
+      new LightWeightHashSet<>();
 
   /* Variables for maintaining number of blocks scheduled to be written to
    * this storage. This count is approximate and might be slightly bigger
@@ -375,6 +409,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       this.invalidateBlocks.clear();
       this.recoverBlocks.clear();
       this.replicateBlocks.clear();
+      this.erasurecodeBlocks.clear();
     }
     // pendingCached, cached, and pendingUncached are protected by the
     // FSN lock.
@@ -592,6 +627,20 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
+   * Store block erasure coding work.
+   */
+  void addBlockToBeErasureCoded(ExtendedBlock block, DatanodeDescriptor[] sources,
+      DatanodeStorageInfo[] targets, short[] missingBlockIndicies) {
+    assert(block != null && sources != null && sources.length > 0);
+    BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets,
+        missingBlockIndicies);
+    erasurecodeBlocks.offer(task);
+    BlockManager.LOG.debug("Adding block recovery task " + task +
+        "to " + getName() + ", current queue size is " +
+        erasurecodeBlocks.size());
+  }
+
+  /**
    * Store block recovery work.
    */
   void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction block) {
@@ -623,6 +672,13 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
+   * The number of work items that are pending to be replicated
+   */
+  int getNumberOfBlocksToBeErasureCoded() {
+    return erasurecodeBlocks.size();
+  }
+
+  /**
    * The number of block invalidation items that are pending to 
    * be sent to the datanode
    */
@@ -636,6 +692,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return replicateBlocks.poll(maxTransfers);
   }
 
+  public List<BlockECRecoveryInfo> getErasureCodeCommand(int maxTransfers) {
+    return erasurecodeBlocks.poll(maxTransfers);
+  }
+
   public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
     List<BlockInfoContiguousUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
@@ -836,6 +896,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
     if (repl > 0) {
       sb.append(" ").append(repl).append(" blocks to be replicated;");
     }
+    int ec = erasurecodeBlocks.size();
+    if(ec > 0) {
+      sb.append(" ").append(ec).append(" blocks to be erasure coded;");
+    }
     int inval = invalidateBlocks.size();
     if (inval > 0) {
       sb.append(" ").append(inval).append(" blocks to be invalidated;");      

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eb2c926/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index f68c4fd..6228f86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -1344,7 +1345,7 @@ public class DatanodeManager {
       VolumeFailureSummary volumeFailureSummary) throws IOException {
     synchronized (heartbeatManager) {
       synchronized (datanodeMap) {
-        DatanodeDescriptor nodeinfo = null;
+        DatanodeDescriptor nodeinfo;
         try {
           nodeinfo = getDatanode(nodeReg);
         } catch(UnregisteredNodeException e) {
@@ -1382,10 +1383,10 @@ public class DatanodeManager {
             final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =
-                new ArrayList<DatanodeStorageInfo>(storages.length);
-            for (int i = 0; i < storages.length; i++) {
-              if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) {
-                recoveryLocations.add(storages[i]);
+                new ArrayList<>(storages.length);
+            for (DatanodeStorageInfo storage : storages) {
+              if (!storage.getDatanodeDescriptor().isStale(staleInterval)) {
+                recoveryLocations.add(storage);
               }
             }
             // If we are performing a truncate recovery than set recovery fields
@@ -1424,7 +1425,7 @@ public class DatanodeManager {
           return new DatanodeCommand[] { brCommand };
         }
 
-        final List<DatanodeCommand> cmds = new ArrayList<DatanodeCommand>();
+        final List<DatanodeCommand> cmds = new ArrayList<>();
         //check pending replication
         List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
               maxTransfers);
@@ -1432,6 +1433,13 @@ public class DatanodeManager {
           cmds.add(new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId,
               pendingList));
         }
+        // checking pending erasure coding tasks
+        List<BlockECRecoveryInfo> pendingECList =
+            nodeinfo.getErasureCodeCommand(maxTransfers);
+        if (pendingECList != null) {
+          cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_CODEC,
+              pendingECList));
+        }
         //check block invalidation
         Block[] blks = nodeinfo.getInvalidateBlocks(blockInvalidateLimit);
         if (blks != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eb2c926/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 5e2b58f..5014d6d 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
@@ -407,6 +407,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override // BlockCollection
+  // TODO: rename to reflect both replication and EC
   public short getBlockReplication() {
     short max = getFileReplication(CURRENT_STATE_ID);
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
@@ -417,7 +418,8 @@ public class INodeFile extends INodeWithAdditionalFields
       }
       max = maxInSnapshot > max ? maxInSnapshot : max;
     }
-    return max;
+    return isStriped()?
+        HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max;
   }
 
   /** Set the replication factor of this file. */
@@ -1091,11 +1093,12 @@ public class INodeFile extends INodeWithAdditionalFields
         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)
+  @VisibleForTesting
+  @Override
+  // TODO: move erasure coding policy to file XAttr
   public boolean isStriped() {
     return getStoragePolicyID() == HdfsConstants.EC_STORAGE_POLICY_ID;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eb2c926/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
new file mode 100644
index 0000000..f7f02fd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.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.hdfs.server.protocol;
+
+import com.google.common.base.Joiner;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
+
+import java.util.Collection;
+
+/**
+ * A BlockECRecoveryCommand is an instruction to a DataNode to reconstruct a
+ * striped block group with missing blocks.
+ *
+ * Upon receiving this command, the DataNode pulls data from other DataNodes
+ * hosting blocks in this group and reconstructs the lost blocks through codec
+ * calculation.
+ *
+ * After the reconstruction, the DataNode pushes the reconstructed blocks to
+ * their final destinations if necessary (e.g., the destination is different
+ * from the reconstruction node, or multiple blocks in a group are to be
+ * reconstructed).
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockECRecoveryCommand extends DatanodeCommand {
+  final Collection<BlockECRecoveryInfo> ecTasks;
+
+  /**
+   * Create BlockECRecoveryCommand from a collection of
+   * {@link BlockECRecoveryInfo}, each representing a recovery task
+   */
+  public BlockECRecoveryCommand(int action,
+      Collection<BlockECRecoveryInfo> blockECRecoveryInfoList) {
+    super(action);
+    this.ecTasks = blockECRecoveryInfoList;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("BlockECRecoveryCommand(\n  ");
+    Joiner.on("\n  ").appendTo(sb, ecTasks);
+    sb.append("\n)");
+    return sb.toString();
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eb2c926/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index 148135b..e25ee31 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -161,7 +161,7 @@ public class BlockManagerTestUtil {
    */
   public static int computeAllPendingWork(BlockManager bm) {
     int work = computeInvalidationWork(bm);
-    work += bm.computeReplicationWork(Integer.MAX_VALUE);
+    work += bm.computeBlockRecoveryWork(Integer.MAX_VALUE);
     return work;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eb2c926/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 2fca5be..cbea3d8 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
@@ -448,8 +448,8 @@ public class TestBlockManager {
     assertEquals("Block not initially pending replication", 0,
         bm.pendingReplications.getNumReplicas(block));
     assertEquals(
-        "computeReplicationWork should indicate replication is needed", 1,
-        bm.computeReplicationWorkForBlocks(list_all));
+        "computeBlockRecoveryWork should indicate replication is needed", 1,
+        bm.computeRecoveryWorkForBlocks(list_all));
     assertTrue("replication is pending after work is computed",
         bm.pendingReplications.getNumReplicas(block) > 0);
 
@@ -503,22 +503,22 @@ public class TestBlockManager {
     assertNotNull("Chooses source node for a highest-priority replication"
         + " even if all available source nodes have reached their replication"
         + " limits below the hard limit.",
-        bm.chooseSourceDatanode(
-            aBlock,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY));
+            new LinkedList<Short>(), 1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
 
     assertNull("Does not choose a source node for a less-than-highest-priority"
         + " replication since all available source nodes have reached"
         + " their replication limits.",
-        bm.chooseSourceDatanode(
-            aBlock,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED));
+            new LinkedList<Short>(), 1, UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED)[0]);
 
     // Increase the replication count to test replication count > hard limit
     DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] };
@@ -526,12 +526,12 @@ public class TestBlockManager {
 
     assertNull("Does not choose a source node for a highest-priority"
         + " replication when all available nodes exceed the hard limit.",
-        bm.chooseSourceDatanode(
-            aBlock,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY));
+            new LinkedList<Short>(), 1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5eb2c926/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java
new file mode 100644
index 0000000..d883c9b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java
@@ -0,0 +1,107 @@
+/**
+ * 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.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
+import static org.junit.Assert.assertTrue;
+
+public class TestRecoverStripedBlocks {
+  private final short GROUP_SIZE =
+      HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS;
+  private final short NUM_OF_DATANODES = GROUP_SIZE + 1;
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private static final int BLOCK_SIZE = 1024;
+  private HdfsAdmin dfsAdmin;
+  private FSNamesystem namesystem;
+  private Path ECFilePath;
+
+  @Before
+  public void setupCluster() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    // Large value to make sure the pending replication request can stay in
+    // DatanodeDescriptor.replicateBlocks before test timeout.
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 100);
+    // Make sure BlockManager can pull all blocks from UnderReplicatedBlocks via
+    // chooseUnderReplicatedBlocks at once.
+    conf.setInt(
+        DFSConfigKeys.DFS_NAMENODE_REPLICATION_WORK_MULTIPLIER_PER_ITERATION, 5);
+
+    cluster = new MiniDFSCluster.Builder(conf).
+        numDataNodes(NUM_OF_DATANODES).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
+    namesystem = cluster.getNamesystem();
+    ECFilePath = new Path("/ecfile");
+    DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, GROUP_SIZE, 0);
+    dfsAdmin.setStoragePolicy(ECFilePath, EC_STORAGE_POLICY_NAME);
+  }
+
+  @Test
+  public void testMissingStripedBlock() throws Exception {
+    final BlockManager bm = cluster.getNamesystem().getBlockManager();
+    ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, ECFilePath);
+    Iterator<DatanodeStorageInfo> storageInfos =
+        bm.blocksMap.getStorages(b.getLocalBlock())
+            .iterator();
+
+    DatanodeDescriptor firstDn = storageInfos.next().getDatanodeDescriptor();
+    Iterator<BlockInfo> it = firstDn.getBlockIterator();
+    int missingBlkCnt = 0;
+    while (it.hasNext()) {
+      BlockInfo blk = it.next();
+      BlockManager.LOG.debug("Block " + blk + " will be lost");
+      missingBlkCnt++;
+    }
+    BlockManager.LOG.debug("Missing in total " + missingBlkCnt + " blocks");
+
+    bm.getDatanodeManager().removeDatanode(firstDn);
+
+    bm.computeDatanodeWork();
+
+    short cnt = 0;
+    for (DataNode dn : cluster.getDataNodes()) {
+      DatanodeDescriptor dnDescriptor =
+          bm.getDatanodeManager().getDatanode(dn.getDatanodeUuid());
+      cnt += dnDescriptor.getNumberOfBlocksToBeErasureCoded();
+    }
+
+    assertTrue("Counting the number of outstanding EC tasks", cnt == missingBlkCnt);
+  }
+}


[44/50] [abbrv] hadoop git commit: HDFS-7864. Erasure Coding: Update safemode calculation for striped blocks. Contributed by GAO Rui.

Posted by zh...@apache.org.
HDFS-7864. Erasure Coding: Update safemode calculation for striped blocks. Contributed by GAO Rui.


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

Branch: refs/heads/HDFS-7285
Commit: e238608fe1dc02907efc5243213cf16e762b5fee
Parents: 9a0f626
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 23 15:06:53 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:13:07 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockIdManager.java     |  6 ++++++
 .../hdfs/server/blockmanagement/BlockManager.java  | 12 +++++++-----
 .../hdfs/server/blockmanagement/BlocksMap.java     |  2 +-
 .../hadoop/hdfs/server/namenode/FSNamesystem.java  | 17 ++++++++++++-----
 .../hadoop/hdfs/server/namenode/SafeMode.java      |  5 +++--
 .../java/org/apache/hadoop/hdfs/TestSafeMode.java  | 15 +++++++++++++--
 6 files changed, 42 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e238608f/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 1d69d74..187f8c9 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
@@ -233,6 +233,12 @@ public class BlockIdManager {
     return id < 0;
   }
 
+  /**
+   * The last 4 bits of HdfsConstants.BLOCK_GROUP_INDEX_MASK(15) is 1111,
+   * so the last 4 bits of (~HdfsConstants.BLOCK_GROUP_INDEX_MASK) is 0000
+   * and the other 60 bits are 1. Group ID is the first 60 bits of any
+   * data/parity block id in the same striped block group.
+   */
   public static long convertToStripedID(long id) {
     return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e238608f/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 7dfe0a4..abe44f0 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
@@ -684,8 +684,10 @@ public class BlockManager {
     // a "forced" completion when a file is getting closed by an
     // OP_CLOSE edit on the standby).
     namesystem.adjustSafeModeBlockTotals(0, 1);
+    final int minStorage = curBlock.isStriped() ?
+        ((BlockInfoStriped) curBlock).getDataBlockNum() : minReplication;
     namesystem.incrementSafeBlockCount(
-        Math.min(numNodes, minReplication));
+        Math.min(numNodes, minStorage), curBlock);
     
     // replace block in the blocksMap
     return blocksMap.replaceBlock(completeBlock);
@@ -2208,7 +2210,7 @@ public class BlockManager {
         // refer HDFS-5283
         if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) {
           int numOfReplicas = BlockInfo.getNumExpectedLocations(storedBlock);
-          namesystem.incrementSafeBlockCount(numOfReplicas);
+          namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock);
         }
         //and fall through to next clause
       }      
@@ -2589,14 +2591,14 @@ public class BlockManager {
       // only complete blocks are counted towards that.
       // In the case that the block just became complete above, completeBlock()
       // handles the safe block count maintenance.
-      namesystem.incrementSafeBlockCount(numCurrentReplica);
+      namesystem.incrementSafeBlockCount(numCurrentReplica, storedBlock);
     }
   }
 
   /**
    * Modify (block-->datanode) map. Remove block from set of
    * needed replications if this takes care of the problem.
-   * @return the block that is stored in blockMap.
+   * @return the block that is stored in blocksMap.
    */
   private Block addStoredBlock(final BlockInfo block,
                                final Block reportedBlock,
@@ -2665,7 +2667,7 @@ public class BlockManager {
       // Is no-op if not in safe mode.
       // In the case that the block just became complete above, completeBlock()
       // handles the safe block count maintenance.
-      namesystem.incrementSafeBlockCount(numCurrentReplica);
+      namesystem.incrementSafeBlockCount(numCurrentReplica, storedBlock);
     }
     
     // if file is under construction, then done for now

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e238608f/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 0cf5fe6..e4d7b32 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
@@ -140,7 +140,7 @@ class BlocksMap {
     }
   }
   
-  /** Returns the block object it it exists in the map. */
+  /** Returns the block object if it exists in the map. */
   BlockInfo getStoredBlock(Block b) {
     return blocks.get(b);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e238608f/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 4519cee..92defc9 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
@@ -214,6 +214,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -5385,10 +5386,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     /**
      * Increment number of safe blocks if current block has 
      * reached minimal replication.
-     * @param replication current replication 
+     * @param storageNum current number of replicas or number of internal blocks
+     *                   of a striped block group
+     * @param storedBlock current storedBlock which is either a
+     *                    BlockInfoContiguous or a BlockInfoStriped
      */
-    private synchronized void incrementSafeBlockCount(short replication) {
-      if (replication == safeReplication) {
+    private synchronized void incrementSafeBlockCount(short storageNum,
+        BlockInfo storedBlock) {
+      final int safe = storedBlock.isStriped() ?
+          ((BlockInfoStriped) storedBlock).getDataBlockNum() : safeReplication;
+      if (storageNum == safe) {
         this.blockSafe++;
 
         // Report startup progress only if we haven't completed startup yet.
@@ -5681,12 +5688,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override
-  public void incrementSafeBlockCount(int replication) {
+  public void incrementSafeBlockCount(int storageNum, BlockInfo storedBlock) {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
       return;
-    safeMode.incrementSafeBlockCount((short)replication);
+    safeMode.incrementSafeBlockCount((short) storageNum, storedBlock);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e238608f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
index 0debb1f..e26e727 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
@@ -45,9 +45,10 @@ public interface SafeMode {
     
   /**
    * Increment number of blocks that reached minimal replication.
-   * @param replication current replication 
+   * @param replication current replication
+   * @param storedBlock current stored Block
    */
-  public void incrementSafeBlockCount(int replication);
+  public void incrementSafeBlockCount(int replication, BlockInfo storedBlock);
 
   /** Decrement number of blocks that reached minimal replication. */
   public void decrementSafeBlockCount(BlockInfo b);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e238608f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
index 80fe9ee..a43e371 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
@@ -552,7 +552,18 @@ public class TestSafeMode {
       if(cluster!= null) cluster.shutdown();
     }
   }
-  
+
+  //TODO : test should be added to check safeMode with stripedBloks after stripedBlock related functions have been added in class MiniDFSCluster
+  @Test
+  public void testSafeModeWithCorruptSripedBlock() throws IOException {
+    try {
+
+    } finally {
+      if(fs != null) fs.close();
+      if(cluster!= null) cluster.shutdown();
+    }
+  }
+
   void checkGetBlockLocationsWorks(FileSystem fs, Path fileName) throws IOException {
     FileStatus stat = fs.getFileStatus(fileName);
     try {  
@@ -560,7 +571,7 @@ public class TestSafeMode {
     } catch (SafeModeException e) {
       assertTrue("Should have not got safemode exception", false);
     } catch (RemoteException re) {
-      assertTrue("Should have not got safemode exception", false);   
+      assertTrue("Should have not got remote exception", false);
     }    
   }
 }


[10/50] [abbrv] hadoop git commit: HDFS-6263. Remove DRFA.MaxBackupIndex config from log4j.properties. Contributed by Abhiraj Butala.

Posted by zh...@apache.org.
HDFS-6263. Remove DRFA.MaxBackupIndex config from log4j.properties. Contributed by Abhiraj Butala.


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

Branch: refs/heads/HDFS-7285
Commit: 257c77f895e8e4c3d8748909ebbd3ba7e7f880fc
Parents: 3d9132d
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Mar 30 10:52:15 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Mar 30 10:52:15 2015 +0900

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../src/contrib/bkjournal/src/test/resources/log4j.properties     | 2 --
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/257c77f8/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 496db06..e026f85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -347,6 +347,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8004. Use KeyProviderCryptoExtension#warmUpEncryptedKeys when creating
     an encryption zone. (awang via asuresh)
 
+    HDFS-6263. Remove DRFA.MaxBackupIndex config from log4j.properties.
+    (Abhiraj Butala via aajisaka)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/257c77f8/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties
index 8a6b217..f66c84b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/resources/log4j.properties
@@ -53,8 +53,6 @@ log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{
 
 # Max log file size of 10MB
 log4j.appender.ROLLINGFILE.MaxFileSize=10MB
-# uncomment the next line to limit number of backup files
-#log4j.appender.ROLLINGFILE.MaxBackupIndex=10
 
 log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
 log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p [%t:%C{1}@%L] - %m%n


[37/50] [abbrv] hadoop git commit: HDFS-7826. Erasure Coding: Update INodeFile quota computation for striped blocks. Contributed by Kai Sasaki.

Posted by zh...@apache.org.
HDFS-7826. Erasure Coding: Update INodeFile quota computation for striped blocks. Contributed by Kai Sasaki.


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

Branch: refs/heads/HDFS-7285
Commit: d2f2604118c5aa9cabf20f6bb7d05a6cd432b04c
Parents: c59219b
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 16 16:37:08 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:27 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  3 +
 .../blockmanagement/BlockInfoStriped.java       | 12 ++-
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 89 +++++++++++++++++---
 3 files changed, 90 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2f26041/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 245b630..07b72e6 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
@@ -186,4 +186,7 @@ public class HdfsConstants {
   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;
+
+  // The chunk size for striped block which is used by erasure coding
+  public static final int BLOCK_STRIPED_CHUNK_SIZE = 64 * 1024;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2f26041/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 84c3be6..cef8318 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,7 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
 /**
@@ -34,6 +35,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
  * array to record the block index for each triplet.
  */
 public class BlockInfoStriped extends BlockInfo {
+  private final int   chunkSize = HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
   private final short dataBlockNum;
   private final short parityBlockNum;
   /**
@@ -56,7 +58,7 @@ public class BlockInfoStriped extends BlockInfo {
     this.setBlockCollection(b.getBlockCollection());
   }
 
-  short getTotalBlockNum() {
+  public short getTotalBlockNum() {
     return (short) (dataBlockNum + parityBlockNum);
   }
 
@@ -178,6 +180,14 @@ public class BlockInfoStriped extends BlockInfo {
     }
   }
 
+  public long spaceConsumed() {
+    // In case striped blocks, total usage by this striped blocks should
+    // be the total of data blocks and parity blocks because
+    // `getNumBytes` is the total of actual data block size.
+    return ((getNumBytes() - 1) / (dataBlockNum * chunkSize) + 1)
+        * chunkSize * parityBlockNum + getNumBytes();
+  }
+
   @Override
   public final boolean isStriped() {
     return true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2f26041/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 452c230..b1c57b0 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
@@ -648,6 +649,9 @@ public class INodeFile extends INodeWithAdditionalFields
     long nsDelta = 1;
     final long ssDeltaNoReplication;
     short replication;
+    if (isWithStripedBlocks()) {
+      return computeQuotaUsageWithStriped(bsps, counts);
+    }
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
       FileDiffList fileDiffList = sf.getDiffs();
@@ -685,6 +689,23 @@ public class INodeFile extends INodeWithAdditionalFields
     return counts;
   }
 
+  /**
+   * Compute quota of striped file
+   * @param bsps
+   * @param counts
+   * @param useCache
+   * @param lastSnapshotId
+   * @return quota counts
+   */
+  public final QuotaCounts computeQuotaUsageWithStriped(
+      BlockStoragePolicySuite bsps, QuotaCounts counts) {
+    long nsDelta = 1;
+    final long ssDelta = storagespaceConsumed();
+    counts.addNameSpace(nsDelta);
+    counts.addStorageSpace(ssDelta);
+    return counts;
+  }
+
   @Override
   public final ContentSummaryComputationContext computeContentSummary(
       final ContentSummaryComputationContext summary) {
@@ -762,23 +783,37 @@ public class INodeFile extends INodeWithAdditionalFields
    * @return file size
    */
   public final long computeFileSize(boolean includesLastUcBlock,
-      boolean usePreferredBlockSize4LastUcBlock) {
-    if (blocks == null || blocks.length == 0) {
+                                    boolean usePreferredBlockSize4LastUcBlock) {
+    BlockInfo[] blockInfos = getBlocks();
+    // In case of contiguous blocks
+    if (blockInfos == null || blockInfos.length == 0) {
       return 0;
     }
-    final int last = blocks.length - 1;
+    final int last = blockInfos.length - 1;
     //check if the last block is BlockInfoUnderConstruction
-    long size = blocks[last].getNumBytes();
-    if (blocks[last] instanceof BlockInfoContiguousUnderConstruction) {
-       if (!includesLastUcBlock) {
-         size = 0;
-       } else if (usePreferredBlockSize4LastUcBlock) {
-         size = getPreferredBlockSize();
-       }
+    long size = blockInfos[last].getNumBytes();
+    if (blockInfos[last] instanceof BlockInfoContiguousUnderConstruction) {
+      if (!includesLastUcBlock) {
+        size = 0;
+      } else if (usePreferredBlockSize4LastUcBlock) {
+        size = getPreferredBlockSize();
+      }
+    } else if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
+      if (!includesLastUcBlock) {
+        size = 0;
+      } else if (usePreferredBlockSize4LastUcBlock) {
+        // Striped blocks keeps block group which counts
+        // (data blocks num + parity blocks num). When you
+        // count actual used size by BlockInfoStripedUC must
+        // be multiplied by these blocks number.
+        BlockInfoStripedUnderConstruction blockInfoStripedUC
+            = (BlockInfoStripedUnderConstruction) blockInfos[last];
+        size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum();
+      }
     }
     //sum other blocks
-    for(int i = 0; i < last; i++) {
-      size += blocks[i].getNumBytes();
+    for (int i = 0; i < last; i++) {
+      size += blockInfos[i].getNumBytes();
     }
     return size;
   }
@@ -789,7 +824,35 @@ public class INodeFile extends INodeWithAdditionalFields
    * Use preferred block size for the last block if it is under construction.
    */
   public final long storagespaceConsumed() {
-    return storagespaceConsumedNoReplication() * getBlockReplication();
+    if (isWithStripedBlocks()) {
+      return storagespaceConsumedWithStriped();
+    } else {
+      return storagespaceConsumedNoReplication() * getBlockReplication();
+    }
+  }
+
+  /**
+   * Compute size consumed by striped blocks.
+   * @return long
+   */
+  public final long storagespaceConsumedWithStriped() {
+    BlockInfo[] blockInfos = getBlocks();
+    long size = 0;
+    final int last = blockInfos.length - 1;
+    if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
+      BlockInfoStripedUnderConstruction blockInfoStripedUC
+          =(BlockInfoStripedUnderConstruction)blockInfos[last];
+      size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum();
+    } else {
+      // In case of last block is complete
+      BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[last];
+      size = blockInfoStriped.spaceConsumed();
+    }
+    for (int i = 0; i < last; i++) {
+      BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[i];
+      size += blockInfoStriped.spaceConsumed();
+    }
+    return size;
   }
 
   public final long storagespaceConsumedNoReplication() {


[14/50] [abbrv] hadoop git commit: HDFS-7742. Favoring decommissioning node for replication can cause a block to stay underreplicated for long periods. Contributed by Nathan Roberts.

Posted by zh...@apache.org.
HDFS-7742. Favoring decommissioning node for replication can cause a block to stay
underreplicated for long periods. Contributed by Nathan Roberts.


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

Branch: refs/heads/HDFS-7285
Commit: 04ee18ed48ceef34598f954ff40940abc9fde1d2
Parents: ae3e8c6
Author: Kihwal Lee <ki...@apache.org>
Authored: Mon Mar 30 10:10:11 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Mon Mar 30 10:10:11 2015 -0500

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../server/blockmanagement/BlockManager.java    | 10 ++---
 .../blockmanagement/TestBlockManager.java       | 42 ++++++++++++++++++++
 3 files changed, 50 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/04ee18ed/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 f437ad8..811ee75 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -829,6 +829,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7410. Support CreateFlags with append() to support hsync() for
     appending streams (Vinayakumar B via Colin P. McCabe)
 
+    HDFS-7742. Favoring decommissioning node for replication can cause a block 
+    to stay underreplicated for long periods (Nathan Roberts via kihwal)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04ee18ed/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 ad40782..f6e15a3 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
@@ -1637,7 +1637,8 @@ public class BlockManager {
       // If so, do not select the node as src node
       if ((nodesCorrupt != null) && nodesCorrupt.contains(node))
         continue;
-      if(priority != UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY
+      if(priority != UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY 
+          && !node.isDecommissionInProgress() 
           && node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams)
       {
         continue; // already reached replication limit
@@ -1652,13 +1653,12 @@ public class BlockManager {
       // never use already decommissioned nodes
       if(node.isDecommissioned())
         continue;
-      // we prefer nodes that are in DECOMMISSION_INPROGRESS state
-      if(node.isDecommissionInProgress() || srcNode == null) {
+
+      // We got this far, current node is a reasonable choice
+      if (srcNode == null) {
         srcNode = node;
         continue;
       }
-      if(srcNode.isDecommissionInProgress())
-        continue;
       // switch to a different node randomly
       // this to prevent from deterministically selecting the same node even
       // if the node failed to replicate the block on previous iterations

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04ee18ed/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 707c780..91abb2a 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
@@ -535,6 +535,48 @@ public class TestBlockManager {
   }
 
   @Test
+  public void testFavorDecomUntilHardLimit() throws Exception {
+    bm.maxReplicationStreams = 0;
+    bm.replicationStreamsHardLimit = 1;
+
+    long blockId = 42;         // arbitrary
+    Block aBlock = new Block(blockId, 0, 0);
+    List<DatanodeDescriptor> origNodes = getNodes(0, 1);
+    // Add the block to the first node.
+    addBlockOnNodes(blockId,origNodes.subList(0,1));
+    origNodes.get(0).startDecommission();
+
+    List<DatanodeDescriptor> cntNodes = new LinkedList<DatanodeDescriptor>();
+    List<DatanodeStorageInfo> liveNodes = new LinkedList<DatanodeStorageInfo>();
+
+    assertNotNull("Chooses decommissioning source node for a normal replication"
+        + " if all available source nodes have reached their replication"
+        + " limits below the hard limit.",
+        bm.chooseSourceDatanode(
+            aBlock,
+            cntNodes,
+            liveNodes,
+            new NumberReplicas(),
+            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
+
+
+    // Increase the replication count to test replication count > hard limit
+    DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] };
+    origNodes.get(0).addBlockToBeReplicated(aBlock, targets);
+
+    assertNull("Does not choose a source decommissioning node for a normal"
+        + " replication when all available nodes exceed the hard limit.",
+        bm.chooseSourceDatanode(
+            aBlock,
+            cntNodes,
+            liveNodes,
+            new NumberReplicas(),
+            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
+  }
+
+
+
+  @Test
   public void testSafeModeIBR() throws Exception {
     DatanodeDescriptor node = spy(nodes.get(0));
     DatanodeStorageInfo ds = node.getStorageInfos()[0];


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

Branch: refs/heads/HDFS-7285
Commit: c37d9823d41b3b6a840ac057b39216ba32b94fa3
Parents: 850a9ef
Author: drankye <dr...@gmail.com>
Authored: Thu Feb 12 21:12:44 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:25 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/c37d9823/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/c37d9823/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/c37d9823/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/c37d9823/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/c37d9823/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/c37d9823/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/c37d9823/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/c37d9823/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/c37d9823/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/c37d9823/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() {


[13/50] [abbrv] hadoop git commit: HDFS-4396. Add START_MSG/SHUTDOWN_MSG for ZKFC. Contributed by Liang Xie.

Posted by zh...@apache.org.
HDFS-4396. Add START_MSG/SHUTDOWN_MSG for ZKFC. Contributed by Liang Xie.


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

Branch: refs/heads/HDFS-7285
Commit: ae3e8c61ff4c926ef3e71c782433ed9764d21478
Parents: 1ed9fb7
Author: Harsh J <ha...@cloudera.com>
Authored: Mon Mar 30 15:21:18 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Mon Mar 30 15:21:18 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java     | 2 ++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae3e8c61/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 9b1cc3e..f437ad8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -323,6 +323,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    HDFS-4396. Add START_MSG/SHUTDOWN_MSG for ZKFC
+    (Liang Xie via harsh)
+
     HDFS-7875. Improve log message when wrong value configured for
     dfs.datanode.failed.volumes.tolerated.
     (nijel via harsh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae3e8c61/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
index 85f77f1..4e256a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
@@ -167,6 +167,8 @@ public class DFSZKFailoverController extends ZKFailoverController {
 
   public static void main(String args[])
       throws Exception {
+    StringUtils.startupShutdownMessage(DFSZKFailoverController.class,
+        args, LOG);
     if (DFSUtil.parseHelpArgument(args, 
         ZKFailoverController.USAGE, System.out, true)) {
       System.exit(0);


[50/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts in the branch when merging (this commit is for HDFS-7742)

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts in the branch when merging (this commit is for HDFS-7742)


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

Branch: refs/heads/HDFS-7285
Commit: a6543ac978ac6e0e8dbf7ce32c41fb80525639ab
Parents: d50bbd7
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Mar 30 10:23:09 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:23:09 2015 -0700

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/TestBlockManager.java   | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6543ac9/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 cbea3d8..43f4607 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
@@ -552,11 +552,11 @@ public class TestBlockManager {
     assertNotNull("Chooses decommissioning source node for a normal replication"
         + " if all available source nodes have reached their replication"
         + " limits below the hard limit.",
-        bm.chooseSourceDatanode(
-            aBlock,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
-            new NumberReplicas(),
+            new NumberReplicas(), new LinkedList<Short>(), 1,
             UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
 
 
@@ -566,11 +566,11 @@ public class TestBlockManager {
 
     assertNull("Does not choose a source decommissioning node for a normal"
         + " replication when all available nodes exceed the hard limit.",
-        bm.chooseSourceDatanode(
-            aBlock,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
-            new NumberReplicas(),
+            new NumberReplicas(), new LinkedList<Short>(), 1,
             UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
   }
 


[46/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903, HDFS-7435, HDFS-7930, HDFS-7960 (this commit is for HDFS-7960)

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903, HDFS-7435, HDFS-7930, HDFS-7960 (this commit is for HDFS-7960)


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

Branch: refs/heads/HDFS-7285
Commit: d0dad770197d4ff9352ce285daa121a22e7babaa
Parents: 37ecc11
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Mar 24 11:39:36 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:13:08 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/blockmanagement/BlockManager.java | 4 ++--
 .../blockmanagement/TestNameNodePrunesMissingStorages.java      | 5 ++++-
 .../hadoop/hdfs/server/namenode/TestAddStripedBlocks.java       | 2 +-
 3 files changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0dad770/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 abe44f0..545e66b 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
@@ -1975,10 +1975,10 @@ public class BlockManager {
              "longer exists on the DataNode.",
               Long.toHexString(context.getReportId()), zombie.getStorageID());
     assert(namesystem.hasWriteLock());
-    Iterator<BlockInfoContiguous> iter = zombie.getBlockIterator();
+    Iterator<BlockInfo> iter = zombie.getBlockIterator();
     int prevBlocks = zombie.numBlocks();
     while (iter.hasNext()) {
-      BlockInfoContiguous block = iter.next();
+      BlockInfo block = iter.next();
       // We assume that a block can be on only one storage in a DataNode.
       // That's why we pass in the DatanodeDescriptor rather than the
       // DatanodeStorageInfo.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0dad770/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
index 4b97d01..e9329cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
@@ -171,9 +171,12 @@ public class TestNameNodePrunesMissingStorages {
       String datanodeUuid;
       // Find the first storage which this block is in.
       try {
+        BlockInfo storedBlock =
+            cluster.getNamesystem().getBlockManager().
+                getStoredBlock(block.getLocalBlock());
         Iterator<DatanodeStorageInfo> storageInfoIter =
             cluster.getNamesystem().getBlockManager().
-                getStorages(block.getLocalBlock()).iterator();
+                blocksMap.getStorages(storedBlock).iterator();
         assertTrue(storageInfoIter.hasNext());
         DatanodeStorageInfo info = storageInfoIter.next();
         storageIdToRemove = info.getStorageID();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0dad770/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 05aec4b..7d7c81e 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
@@ -269,7 +269,7 @@ public class TestAddStripedBlocks {
       StorageBlockReport[] reports = {new StorageBlockReport(storage,
           bll)};
       cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId),
-          bpId, reports);
+          bpId, reports, null);
     }
 
     BlockInfoStripedUnderConstruction ucBlock =


[04/50] [abbrv] hadoop git commit: HADOOP-11639. Clean up Windows native code compilation warnings related to Windows Secure Container Executor. Contributed by Remus Rusanu.

Posted by zh...@apache.org.
HADOOP-11639. Clean up Windows native code compilation warnings related to Windows Secure Container Executor. Contributed by Remus Rusanu.


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

Branch: refs/heads/HDFS-7285
Commit: 3836ad6c0b3331cf60286d134157c13985908230
Parents: 05499b1
Author: cnauroth <cn...@apache.org>
Authored: Fri Mar 27 15:03:41 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Fri Mar 27 15:03:41 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../windows_secure_container_executor.c         |  2 +-
 .../hadoop-common/src/main/winutils/client.c    | 17 ++++------
 .../hadoop-common/src/main/winutils/config.cpp  |  2 +-
 .../src/main/winutils/include/winutils.h        | 24 +++++++++++---
 .../src/main/winutils/libwinutils.c             | 18 +++++------
 .../hadoop-common/src/main/winutils/service.c   | 34 ++++++++++----------
 .../src/main/winutils/systeminfo.c              |  3 ++
 .../hadoop-common/src/main/winutils/task.c      | 28 +++++++++-------
 9 files changed, 76 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3836ad6c/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 febbf6b..8643901 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1172,6 +1172,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11691. X86 build of libwinutils is broken.
     (Kiran Kumar M R via cnauroth)
 
+    HADOOP-11639. Clean up Windows native code compilation warnings related to
+    Windows Secure Container Executor. (Remus Rusanu via cnauroth)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3836ad6c/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.c
index 7e65065..b37359d 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.c
@@ -409,7 +409,7 @@ Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00
 
 done:
   if (path)     (*env)->ReleaseStringChars(env, jpath, path);
-  return hFile;
+  return (jlong) hFile;
 #endif
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3836ad6c/hadoop-common-project/hadoop-common/src/main/winutils/client.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/client.c b/hadoop-common-project/hadoop-common/src/main/winutils/client.c
index 047bfb5..e3a2c37 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/client.c
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/client.c
@@ -28,8 +28,6 @@ static ACCESS_MASK CLIENT_MASK = 1;
 VOID ReportClientError(LPWSTR lpszLocation, DWORD dwError) {
   LPWSTR      debugMsg = NULL;
   int         len;
-  WCHAR       hexError[32];
-  HRESULT     hr;
 
   if (IsDebuggerPresent()) {
     len = FormatMessageW(
@@ -49,7 +47,6 @@ DWORD PrepareRpcBindingHandle(
   DWORD       dwError = EXIT_FAILURE;
   RPC_STATUS  status;
   LPWSTR      lpszStringBinding    = NULL;
-  ULONG       ulCode;
   RPC_SECURITY_QOS_V3 qos;
   SID_IDENTIFIER_AUTHORITY authNT = SECURITY_NT_AUTHORITY;
   BOOL rpcBindingInit = FALSE;
@@ -104,7 +101,7 @@ DWORD PrepareRpcBindingHandle(
                   RPC_C_AUTHN_WINNT,              // AuthnSvc
                   NULL,                           // AuthnIdentity (self)
                   RPC_C_AUTHZ_NONE,               // AuthzSvc
-                  &qos);
+                  (RPC_SECURITY_QOS*) &qos);
   if (RPC_S_OK != status) {
     ReportClientError(L"RpcBindingSetAuthInfoEx", status);
     dwError = status;
@@ -375,7 +372,7 @@ DWORD RpcCall_WinutilsCreateFile(
   RpcEndExcept;
 
   if (ERROR_SUCCESS == dwError) {
-    *hFile = response->hFile;
+    *hFile = (HANDLE) response->hFile;
   }
 
 done:
@@ -479,11 +476,11 @@ DWORD RpcCall_TaskCreateAsUser(
     RpcEndExcept;
 
     if (ERROR_SUCCESS == dwError) {
-      *phProcess = response->hProcess;
-      *phThread = response->hThread;
-      *phStdIn = response->hStdIn;
-      *phStdOut = response->hStdOut;
-      *phStdErr = response->hStdErr;
+      *phProcess = (HANDLE) response->hProcess;
+      *phThread = (HANDLE) response->hThread;
+      *phStdIn = (HANDLE) response->hStdIn;
+      *phStdOut = (HANDLE) response->hStdOut;
+      *phStdErr = (HANDLE) response->hStdErr;
     }
 
 done:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3836ad6c/hadoop-common-project/hadoop-common/src/main/winutils/config.cpp
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/config.cpp b/hadoop-common-project/hadoop-common/src/main/winutils/config.cpp
index 1e07b7f..74be689 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/config.cpp
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/config.cpp
@@ -18,7 +18,7 @@
 #include "winutils.h"
 #include <string.h>
 #include <stdlib.h>
-#import "msxml6.dll"
+#import "msxml6.dll" exclude("ISequentialStream", "_FILETIME")
 
 #define ERROR_CHECK_HRESULT_DONE(hr, message)                               \
   if (FAILED(hr))  {                                                        \

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3836ad6c/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h b/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
index f72802c..6c33b5a 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
@@ -248,8 +248,8 @@ DWORD BuildServiceSecurityDescriptor(
   __out PSECURITY_DESCRIPTOR*         pSD);
 
 DWORD AddNodeManagerAndUserACEsToObject(
-  __in HANDLE hObject,
-  __in LPWSTR user,
+  __in HANDLE hProcess,
+  __in LPCWSTR user,
   __in ACCESS_MASK accessMask);
 
 
@@ -283,15 +283,29 @@ DWORD RpcCall_WinutilsCreateFile(
   __out HANDLE* hFile);
 
 DWORD RpcCall_WinutilsMoveFile(
-  __in LPCWSTR    sourcePath, 
-  __in LPCWSTR    destinationPath,
-  __in BOOL       replaceExisting);
+  __in int operation,
+  __in LPCWSTR sourcePath, 
+  __in LPCWSTR destinationPath,
+  __in BOOL replaceExisting);
+
 
 DWORD RpcCall_WinutilsDeletePath(
   __in LPCWSTR    path,
   __in BOOL       isDir,
   __out BOOL*     pDeleted);
 
+DWORD RpcCall_WinutilsChown(
+  __in LPCWSTR filePath, 
+  __in_opt LPCWSTR ownerName, 
+  __in_opt LPCWSTR groupName);
+
+DWORD RpcCall_WinutilsMkDir(
+  __in LPCWSTR filePath);
+
+DWORD RpcCall_WinutilsChmod(
+  __in LPCWSTR filePath, 
+  __in int mode);
+
 #ifdef __cplusplus
 }
 #endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3836ad6c/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c b/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
index 98fe3ab..676f1b2 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
@@ -2596,7 +2596,7 @@ LPCWSTR GetSystemTimeString() {
   QueryPerformanceFrequency(&frequency);
 
   qpc = (double) counter.QuadPart / (double) frequency.QuadPart;
-  subSec = ((qpc - (long)qpc) * 1000000);
+  subSec = (int)((qpc - (long)qpc) * 1000000);
 
   hr = StringCbPrintf(buffer, sizeof(buffer), L"%02d:%02d:%02d.%06d", 
     (int)systime.wHour, (int)systime.wMinute, (int)systime.wSecond, (int)subSec);
@@ -2619,7 +2619,7 @@ done:
 //  Native debugger: windbg, ntsd, cdb, visual studio
 //
 VOID LogDebugMessage(LPCWSTR format, ...) {
-  LPWSTR buffer[8192];
+  wchar_t buffer[8192];
   va_list args;
   HRESULT hr;
 
@@ -2657,8 +2657,8 @@ DWORD SplitStringIgnoreSpaceW(
   size_t tokenCount = 0;
   size_t crtSource;
   size_t crtToken = 0;
-  WCHAR* lpwszTokenStart = NULL;
-  WCHAR* lpwszTokenEnd = NULL;
+  const WCHAR* lpwszTokenStart = NULL;
+  const WCHAR* lpwszTokenEnd = NULL;
   WCHAR* lpwszBuffer = NULL;
   size_t tokenLength = 0;
   size_t cchBufferLength = 0;
@@ -2849,7 +2849,7 @@ DWORD BuildServiceSecurityDescriptor(
     }
   }
 
-  pTokenGroup = (PTOKEN_USER) LocalAlloc(LPTR, dwBufferSize);
+  pTokenGroup = (PTOKEN_PRIMARY_GROUP) LocalAlloc(LPTR, dwBufferSize);
   if (NULL == pTokenGroup) {
     dwError = GetLastError();
     LogDebugMessage(L"LocalAlloc:pTokenGroup: %d\n", dwError);
@@ -2870,11 +2870,11 @@ DWORD BuildServiceSecurityDescriptor(
 
   owner.TrusteeForm = TRUSTEE_IS_SID;
   owner.TrusteeType = TRUSTEE_IS_UNKNOWN;
-  owner.ptstrName = (LPCWSTR) pOwner;
+  owner.ptstrName = (LPWSTR) pOwner;
 
   group.TrusteeForm = TRUSTEE_IS_SID;
   group.TrusteeType = TRUSTEE_IS_UNKNOWN;
-  group.ptstrName = (LPCWSTR) pTokenGroup->PrimaryGroup;
+  group.ptstrName = (LPWSTR) pTokenGroup->PrimaryGroup;
 
   eas = (EXPLICIT_ACCESS*) LocalAlloc(LPTR, sizeof(EXPLICIT_ACCESS) * (grantSidCount + denySidCount));
   if (NULL == eas) {
@@ -2890,7 +2890,7 @@ DWORD BuildServiceSecurityDescriptor(
     eas[crt].grfInheritance = NO_INHERITANCE;
     eas[crt].Trustee.TrusteeForm = TRUSTEE_IS_SID;
     eas[crt].Trustee.TrusteeType = TRUSTEE_IS_UNKNOWN;
-    eas[crt].Trustee.ptstrName = (LPCWSTR) pGrantSids[crt];
+    eas[crt].Trustee.ptstrName = (LPWSTR) pGrantSids[crt];
     eas[crt].Trustee.pMultipleTrustee = NULL;
     eas[crt].Trustee.MultipleTrusteeOperation = NO_MULTIPLE_TRUSTEE;
   }
@@ -2902,7 +2902,7 @@ DWORD BuildServiceSecurityDescriptor(
     eas[crt].grfInheritance = NO_INHERITANCE;
     eas[crt].Trustee.TrusteeForm = TRUSTEE_IS_SID;
     eas[crt].Trustee.TrusteeType = TRUSTEE_IS_UNKNOWN;
-    eas[crt].Trustee.ptstrName = (LPCWSTR) pDenySids[crt - grantSidCount];
+    eas[crt].Trustee.ptstrName = (LPWSTR) pDenySids[crt - grantSidCount];
     eas[crt].Trustee.pMultipleTrustee = NULL;
     eas[crt].Trustee.MultipleTrusteeOperation = NO_MULTIPLE_TRUSTEE;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3836ad6c/hadoop-common-project/hadoop-common/src/main/winutils/service.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/service.c b/hadoop-common-project/hadoop-common/src/main/winutils/service.c
index ba35003..fca5dbc 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/service.c
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/service.c
@@ -206,7 +206,7 @@ DWORD ValidateConfigurationFile() {
   BOOL daclPresent = FALSE;
   BOOL daclDefaulted = FALSE;
   PACL pDacl = NULL;
-  unsigned int crt = 0, crtSid = 0;
+  DWORD crt = 0;
   WELL_KNOWN_SID_TYPE allowedSidTypes[] = {
     WinLocalSystemSid,
     WinBuiltinAdministratorsSid};
@@ -214,7 +214,6 @@ DWORD ValidateConfigurationFile() {
   DWORD cbSid = SECURITY_MAX_SID_SIZE;
   PSID* allowedSids = NULL; 
   int cAllowedSids = 0;
-  BOOL isSidDefaulted;
   PSID sidOwner = NULL;
   PSID sidGroup = NULL;
 
@@ -324,7 +323,7 @@ DWORD InitJobName() {
   int       crt = 0;
 
   // Services can be restarted
-  if (gJobName) LocalFree(gJobName);
+  if (gJobName) LocalFree((HLOCAL)gJobName);
   gJobName = NULL;
     
   dwError = GetConfigValue(
@@ -382,7 +381,7 @@ DWORD InitLocalDirs() {
   }
 
 done:
-  if (value) LocalFree(value);
+  if (value) LocalFree((HLOCAL)value);
   
   return dwError;
 }
@@ -437,7 +436,7 @@ DWORD ValidateLocalPath(LPCWSTR lpszPath) {
       gLocalDirs[crt], gCchLocalDir[crt],
       NULL, // lpVersionInformation
       NULL, // lpReserved
-      NULL); // lParam
+      (LPARAM) NULL); // lParam
     
     if (0 == compareResult) {
       dwError = GetLastError();
@@ -500,7 +499,7 @@ done:
 // Description:
 //  Service main entry point.
 //
-VOID WINAPI SvcMain() {
+VOID WINAPI SvcMain(DWORD dwArg, LPTSTR* lpszArgv) {
   DWORD dwError = ERROR_SUCCESS;
 
   gSvcStatusHandle = RegisterServiceCtrlHandler( 
@@ -693,15 +692,15 @@ done:
 //
 DWORD AuthInit() {
   DWORD       dwError = ERROR_SUCCESS;
-  int         count = 0;
-  int         crt  = 0;
+  size_t      count = 0;
+  size_t      crt  = 0;
   size_t      len = 0;
   LPCWSTR     value = NULL;
   WCHAR**     tokens = NULL;
   LPWSTR      lpszSD = NULL;
   ULONG       cchSD = 0;
   DWORD       dwBufferSize = 0;
-  int         allowedCount = 0;
+  size_t      allowedCount = 0;
   PSID*       allowedSids = NULL;
   
 
@@ -737,7 +736,7 @@ DWORD AuthInit() {
   
 done:
   if (lpszSD) LocalFree(lpszSD);
-  if (value) LocalFree(value);
+  if (value) LocalFree((HLOCAL)value);
   if (tokens) LocalFree(tokens);
   return dwError;
 }
@@ -1167,11 +1166,12 @@ error_status_t WinutilsCreateProcessAsUser(
   // Note that there are no more API calls, only assignments. A failure could occur only if
   // foced (process kill) or hardware error (faulty memory, processort bit flip etc).
 
-  (*response)->hProcess = hDuplicateProcess;
-  (*response)->hThread = hDuplicateThread;
-  (*response)->hStdIn = hDuplicateStdIn;
-  (*response)->hStdOut = hDuplicateStdOut;
-  (*response)->hStdErr = hDuplicateStdErr;
+  // as MIDL has no 'HANDLE' type, the (LONG_PTR) is used instead
+  (*response)->hProcess = (LONG_PTR)hDuplicateProcess;
+  (*response)->hThread = (LONG_PTR)hDuplicateThread;
+  (*response)->hStdIn = (LONG_PTR)hDuplicateStdIn;
+  (*response)->hStdOut = (LONG_PTR)hDuplicateStdOut;
+  (*response)->hStdErr = (LONG_PTR)hDuplicateStdErr;
 
   fMustCleanupProcess = FALSE;
   
@@ -1276,7 +1276,8 @@ error_status_t WinutilsCreateFile(
     goto done;
   }
 
-  (*response)->hFile = hDuplicateFile;
+  // As MIDL has no 'HANDLE' type, (LONG_PTR) is used instead
+  (*response)->hFile = (LONG_PTR)hDuplicateFile;
   hDuplicateFile = INVALID_HANDLE_VALUE;
 
 done:
@@ -1302,7 +1303,6 @@ error_status_t WinutilsKillTask(
     /* [in] */ handle_t IDL_handle,
     /* [in] */ KILLTASK_REQUEST *request) {
   DWORD dwError = ERROR_SUCCESS;
-  HRESULT hr;
   WCHAR bufferName[MAX_PATH];
 
   dwError = GetSecureJobObjectName(request->taskName, MAX_PATH, bufferName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3836ad6c/hadoop-common-project/hadoop-common/src/main/winutils/systeminfo.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/systeminfo.c b/hadoop-common-project/hadoop-common/src/main/winutils/systeminfo.c
index 7fce424..48f03ed 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/systeminfo.c
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/systeminfo.c
@@ -19,6 +19,9 @@
 #include <psapi.h>
 #include <PowrProf.h>
 
+#ifdef PSAPI_VERSION
+#undef PSAPI_VERSION
+#endif
 #define PSAPI_VERSION 1
 #pragma comment(lib, "psapi.lib")
 #pragma comment(lib, "Powrprof.lib")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3836ad6c/hadoop-common-project/hadoop-common/src/main/winutils/task.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/task.c b/hadoop-common-project/hadoop-common/src/main/winutils/task.c
index 37c6ca1..057fd8a 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/task.c
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/task.c
@@ -22,6 +22,9 @@
 #include <authz.h>
 #include <sddl.h>
 
+#ifdef PSAPI_VERSION
+#undef PSAPI_VERSION
+#endif
 #define PSAPI_VERSION 1
 #pragma comment(lib, "psapi.lib")
 
@@ -231,7 +234,7 @@ DWORD BuildImpersonateSecurityDescriptor(__out PSECURITY_DESCRIPTOR* ppSD) {
   LocalFree(tokens);
   tokens = NULL;
 
-  LocalFree(value);
+  LocalFree((HLOCAL)value);
   value = NULL;
   
   dwError = GetConfigValue(wsceConfigRelativePath, NM_WSCE_IMPERSONATE_DENIED, &len, &value); 
@@ -298,18 +301,18 @@ done:
 //
 DWORD AddNodeManagerAndUserACEsToObject(
   __in HANDLE hObject,
-  __in LPWSTR user,
+  __in LPCWSTR user,
   __in ACCESS_MASK accessMask) {
 
   DWORD dwError = ERROR_SUCCESS;
-  int         countTokens = 0;
+  size_t      countTokens = 0;
   size_t      len = 0;
   LPCWSTR     value = NULL;
   WCHAR**     tokens = NULL;
-  int         crt = 0;
+  DWORD       crt = 0;
   PACL        pDacl = NULL;
   PSECURITY_DESCRIPTOR  psdProcess = NULL;
-  LPSTR       lpszOldDacl = NULL, lpszNewDacl = NULL;
+  LPWSTR      lpszOldDacl = NULL, lpszNewDacl = NULL;
   ULONG       daclLen = 0;
   PACL        pNewDacl = NULL;
   ACL_SIZE_INFORMATION si;
@@ -381,8 +384,8 @@ DWORD AddNodeManagerAndUserACEsToObject(
   // ACCESS_ALLOWED_ACE struct contains the first DWORD of the SID 
   //
   dwNewAclSize = si.AclBytesInUse + 
-    (countTokens + 1 + sizeof(forcesSidTypes)/sizeof(forcesSidTypes[0])) * 
-      (sizeof(ACCESS_ALLOWED_ACE) + SECURITY_MAX_SID_SIZE - sizeof(DWORD));
+      (DWORD)(countTokens + 1 + sizeof(forcesSidTypes)/sizeof(forcesSidTypes[0])) * 
+              (sizeof(ACCESS_ALLOWED_ACE) + SECURITY_MAX_SID_SIZE - sizeof(DWORD));
 
   pNewDacl = (PSID) LocalAlloc(LPTR, dwNewAclSize);
   if (!pNewDacl) {
@@ -511,7 +514,7 @@ DWORD AddNodeManagerAndUserACEsToObject(
       goto done;
     }
 
-    LogDebugMessage(L"Old DACL: %s\nNew DACL: %s\n", lpszOldDacl, lpszNewDacl);
+    LogDebugMessage(L"Old DACL: %ls\nNew DACL: %ls\n", lpszOldDacl, lpszNewDacl);
   }
   
 done:
@@ -634,7 +637,7 @@ done:
 // Returns:
 // ERROR_SUCCESS: On success
 // GetLastError: otherwise
-DWORD CreateTaskImpl(__in_opt HANDLE logonHandle, __in PCWSTR jobObjName,__in PCWSTR cmdLine, 
+DWORD CreateTaskImpl(__in_opt HANDLE logonHandle, __in PCWSTR jobObjName,__in PWSTR cmdLine, 
   __in LPCWSTR userName, __in long memory, __in long cpuRate)
 {
   DWORD dwErrorCode = ERROR_SUCCESS;
@@ -912,7 +915,7 @@ DWORD CreateTask(__in PCWSTR jobObjName,__in PWSTR cmdLine, __in long memory, __
 // ERROR_SUCCESS: On success
 // GetLastError: otherwise
 DWORD CreateTaskAsUser(__in PCWSTR jobObjName,
-  __in PCWSTR user, __in PCWSTR pidFilePath, __in PCWSTR cmdLine)
+  __in PCWSTR user, __in PCWSTR pidFilePath, __in PWSTR cmdLine)
 {
   DWORD err = ERROR_SUCCESS;
   DWORD exitCode = EXIT_FAILURE;
@@ -923,6 +926,7 @@ DWORD CreateTaskAsUser(__in PCWSTR jobObjName,
   FILE* pidFile = NULL;
   DWORD retLen = 0;
   HANDLE logonHandle = NULL;
+  errno_t pidErrNo = 0;
 
   err = EnableImpersonatePrivileges();
   if( err != ERROR_SUCCESS ) {
@@ -961,8 +965,8 @@ DWORD CreateTaskAsUser(__in PCWSTR jobObjName,
   profileIsLoaded = TRUE; 
 
   // Create the PID file
-
-  if (!(pidFile = _wfopen(pidFilePath, "w"))) {
+  pidErrNo = _wfopen_s(&pidFile, pidFilePath, L"w");
+  if (pidErrNo) {
       err = GetLastError();
       ReportErrorCode(L"_wfopen:pidFilePath", err);
       goto done;


[45/50] [abbrv] hadoop git commit: HADOOP-11707. Add factory to create raw erasure coder. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11707. Add factory to create raw erasure coder.  Contributed by Kai Zheng


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

Branch: refs/heads/HDFS-7285
Commit: eb86ab788a4ebce924d9bfc19fcb1a6398fb297b
Parents: 5eb2c92
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Mar 20 15:07:00 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:13:07 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  3 +-
 .../rawcoder/JRSRawErasureCoderFactory.java     | 34 ++++++++++++++++++
 .../rawcoder/RawErasureCoderFactory.java        | 38 ++++++++++++++++++++
 .../rawcoder/XorRawErasureCoderFactory.java     | 34 ++++++++++++++++++
 4 files changed, 108 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb86ab78/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 e27ff5c..f566f0e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -24,4 +24,5 @@
     HADOOP-11706. Refine a little bit erasure coder API. Contributed by Kai Zheng
     ( Kai Zheng )
 
-
+    HADOOP-11707. Add factory to create raw erasure coder. Contributed by Kai Zheng
+    ( Kai Zheng )

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb86ab78/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
new file mode 100644
index 0000000..95a1cfe
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+/**
+ * Raw erasure coder factory that can be used to create raw encoder and decoder.
+ * It helps in configuration since only one factory class is needed to be
+ * configured.
+ */
+public interface RawErasureCoderFactory {
+
+  /**
+   * Create raw erasure encoder.
+   * @return raw erasure encoder
+   */
+  public RawErasureEncoder createEncoder();
+
+  /**
+   * Create raw erasure decoder.
+   * @return raw erasure decoder
+   */
+  public RawErasureDecoder createDecoder();
+}

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


[03/50] [abbrv] hadoop git commit: MAPREDUCE-6294. Remove an extra parameter described in Javadoc of TockenCache. Contributed by Brahma Reddy Battula.

Posted by zh...@apache.org.
MAPREDUCE-6294. Remove an extra parameter described in Javadoc of TockenCache. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7285
Commit: 05499b1093ea6ba6a39a1354d67b0a46a2982824
Parents: e074952
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Sat Mar 28 00:08:35 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Sat Mar 28 00:08:35 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../java/org/apache/hadoop/mapreduce/security/TokenCache.java     | 1 -
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/05499b10/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 9d6f1d4..ce16510 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -305,6 +305,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6242. Progress report log is incredibly excessive in 
     application master. (Varun Saxena via devaraj)
 
+    MAPREDUCE-6294. Remove an extra parameter described in Javadoc of
+    TockenCache. (Brahma Reddy Battula via ozawa)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05499b10/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
index 7b1f657..6c0de1b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
@@ -105,7 +105,6 @@ public class TokenCache {
    * get delegation token for a specific FS
    * @param fs
    * @param credentials
-   * @param p
    * @param conf
    * @throws IOException
    */


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

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


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

Branch: refs/heads/HDFS-7285
Commit: b46621c95245d65a8790a99120df2d97e8bf0a84
Parents: 671db98
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Mar 18 19:24:24 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:28 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/b46621c9/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 a97dc34..e27ff5c 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -19,6 +19,9 @@
     ( Kai Zheng via vinayakumarb )
 
     HADOOP-11705. Make erasure coder configurable. Contributed by Kai Zheng
-    ( Kai Zheng )    
+    ( Kai Zheng )
+
+    HADOOP-11706. Refine a little bit erasure coder API. Contributed by Kai Zheng
+    ( Kai Zheng )
 
 


[05/50] [abbrv] hadoop git commit: HDFS-8004. Use KeyProviderCryptoExtension#warmUpEncryptedKeys when creating an encryption zone. (awang via asuresh)

Posted by zh...@apache.org.
HDFS-8004. Use KeyProviderCryptoExtension#warmUpEncryptedKeys when creating an encryption zone. (awang via asuresh)


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

Branch: refs/heads/HDFS-7285
Commit: e97f8e44af9dffc42c030278425cffe0c9da723b
Parents: 3836ad6
Author: Arun Suresh <as...@apache.org>
Authored: Fri Mar 27 19:23:45 2015 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Fri Mar 27 19:23:45 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e97f8e44/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 72ea4fb..af1dd60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -344,6 +344,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-7990. IBR delete ack should not be delayed. (daryn via kihwal)
 
+    HDFS-8004. Use KeyProviderCryptoExtension#warmUpEncryptedKeys when creating
+    an encryption zone. (awang via asuresh)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e97f8e44/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 1226a26..d0999b8 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
@@ -7957,7 +7957,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         throw new IOException("Key " + keyName + " doesn't exist.");
       }
       // If the provider supports pool for EDEKs, this will fill in the pool
-      generateEncryptedDataEncryptionKey(keyName);
+      provider.warmUpEncryptedKeys(keyName);
       createEncryptionZoneInt(src, metadata.getCipher(),
           keyName, logRetryCache);
     } catch (AccessControlException e) {


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

Branch: refs/heads/HDFS-7285
Commit: 7302ab1281058bc927b8148a88669fc38df77f75
Parents: 2fc3e35
Author: Vinayakumar B <vi...@intel.com>
Authored: Mon Feb 2 14:39:53 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:24 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/7302ab12/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/7302ab12/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/7302ab12/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


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

Branch: refs/heads/HDFS-7285
Commit: 0c6ed987fbb59fe210fae7dd2144ba0c16ace517
Parents: daa78e3
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Mar 9 12:32:26 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:26 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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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/0c6ed987/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);
+  }
+
+}


[25/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/850a9ef8/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 be16a87..fa7f263 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 ID of the last full block report which updated this storage.
@@ -226,7 +227,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;
@@ -245,13 +246,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--;
@@ -265,16 +274,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;
   }
@@ -284,7 +292,7 @@ public class DatanodeStorageInfo {
    * @return the head of the blockList
    */
   @VisibleForTesting
-  BlockInfoContiguous getBlockListHeadForTesting(){
+  BlockInfo getBlockListHeadForTesting(){
     return blockList;
   }
 
@@ -371,6 +379,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/850a9ef8/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/850a9ef8/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 bbb6b19..9f41b80 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
@@ -473,8 +473,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/850a9ef8/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 c4768ee..e8e4758 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
@@ -3239,8 +3239,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
@@ -3583,9 +3583,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
@@ -4072,7 +4074,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;
@@ -4182,9 +4185,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
@@ -4343,9 +4345,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);
               }
             }
           }
@@ -4713,7 +4715,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/850a9ef8/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 669f68a..0b3ed88 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/850a9ef8/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/850a9ef8/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 5fc78d1..6d3bc1e 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
@@ -1562,8 +1562,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/850a9ef8/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/850a9ef8/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/850a9ef8/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 91abb2a..1bc6198 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/850a9ef8/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 32fae45..fd28ded 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
@@ -1238,7 +1238,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);


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

Branch: refs/heads/HDFS-7285
Commit: 3dbde16cb763bc642bd89e76e95d76fc39353c46
Parents: 67e4d1f
Author: drankye <ka...@intel.com>
Authored: Thu Mar 5 22:51:52 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:26 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/3dbde16c/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/3dbde16c/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/3dbde16c/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));
+  }
+}


[43/50] [abbrv] hadoop git commit: HADOOP-11647. Reed-Solomon ErasureCoder. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11647. Reed-Solomon ErasureCoder. 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/26b5a06c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/26b5a06c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/26b5a06c

Branch: refs/heads/HDFS-7285
Commit: 26b5a06c222a89b4fdc165a7e648a4d0f42384a2
Parents: eb86ab7
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Mar 20 19:15:52 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:13:07 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  3 +
 .../hadoop/fs/CommonConfigurationKeys.java      | 15 ++++
 .../erasurecode/coder/AbstractErasureCoder.java | 65 ++++++++++++++
 .../coder/AbstractErasureDecoder.java           |  6 +-
 .../coder/AbstractErasureEncoder.java           |  6 +-
 .../io/erasurecode/coder/RSErasureDecoder.java  | 83 ++++++++++++++++++
 .../io/erasurecode/coder/RSErasureEncoder.java  | 47 ++++++++++
 .../io/erasurecode/coder/XorErasureDecoder.java |  2 +-
 .../io/erasurecode/coder/XorErasureEncoder.java |  2 +-
 .../erasurecode/coder/TestRSErasureCoder.java   | 92 ++++++++++++++++++++
 10 files changed, 315 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/26b5a06c/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 f566f0e..b69e69a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -26,3 +26,6 @@
 
     HADOOP-11707. Add factory to create raw erasure coder. Contributed by Kai Zheng
     ( Kai Zheng )
+
+    HADOOP-11647. Reed-Solomon ErasureCoder. Contributed by Kai Zheng
+    ( Kai Zheng )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26b5a06c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 7575496..70fea01 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -135,6 +135,21 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
       false;
 
   /**
+   * Erasure Coding configuration family
+   */
+
+  /** Supported erasure codec classes */
+  public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs";
+
+  /** Use XOR raw coder when possible for the RS codec */
+  public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY =
+      "io.erasurecode.codec.rs.usexor";
+
+  /** Raw coder factory for the RS codec */
+  public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY =
+      "io.erasurecode.codec.rs.rawcoder";
+
+  /**
    * Service Authorization
    */
   public static final String 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26b5a06c/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 8d3bc34..0e4de89 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,7 +17,12 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 
 /**
  * A common class of basic facilities to be shared by encoder and decoder
@@ -31,6 +36,66 @@ public abstract class AbstractErasureCoder
   private int numParityUnits;
   private int chunkSize;
 
+  /**
+   * Create raw decoder using the factory specified by rawCoderFactoryKey
+   * @param rawCoderFactoryKey
+   * @return raw decoder
+   */
+  protected RawErasureDecoder createRawDecoder(String rawCoderFactoryKey) {
+    RawErasureCoder rawCoder = createRawCoder(getConf(),
+        rawCoderFactoryKey, false);
+    return (RawErasureDecoder) rawCoder;
+  }
+
+  /**
+   * Create raw encoder using the factory specified by rawCoderFactoryKey
+   * @param rawCoderFactoryKey
+   * @return raw encoder
+   */
+  protected RawErasureEncoder createRawEncoder(String rawCoderFactoryKey) {
+    RawErasureCoder rawCoder = createRawCoder(getConf(),
+        rawCoderFactoryKey, true);
+    return (RawErasureEncoder) rawCoder;
+  }
+
+  /**
+   * Create raw coder using specified conf and raw coder factory key.
+   * @param conf
+   * @param rawCoderFactoryKey
+   * @param isEncoder
+   * @return raw coder
+   */
+  protected static RawErasureCoder createRawCoder(Configuration conf,
+      String rawCoderFactoryKey, boolean isEncoder) {
+
+    if (conf == null) {
+      return null;
+    }
+
+    Class<? extends RawErasureCoderFactory> factClass = null;
+    factClass = conf.getClass(rawCoderFactoryKey,
+        factClass, RawErasureCoderFactory.class);
+
+    if (factClass == null) {
+      return null;
+    }
+
+    RawErasureCoderFactory fact;
+    try {
+      fact = factClass.newInstance();
+    } catch (InstantiationException e) {
+      throw new RuntimeException("Failed to create raw coder", e);
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException("Failed to create raw coder", e);
+    }
+
+    if (fact != null) {
+      return isEncoder ? fact.createEncoder() : fact.createDecoder();
+    }
+
+    return null;
+  }
+
   @Override
   public void initialize(int numDataUnits, int numParityUnits,
                          int chunkSize) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26b5a06c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
index df71528..54a6d1e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
@@ -30,7 +30,8 @@ public abstract class AbstractErasureDecoder extends AbstractErasureCoder
 
   @Override
   public ErasureCodingStep decode(ECBlockGroup blockGroup) {
-    return performDecoding(blockGroup);
+    // We may have more than this when considering complicate cases. HADOOP-11550
+    return prepareDecodingStep(blockGroup);
   }
 
   /**
@@ -38,7 +39,8 @@ public abstract class AbstractErasureDecoder extends AbstractErasureCoder
    * @param blockGroup
    * @return decoding step for caller to do the real work
    */
-  protected abstract ErasureCodingStep performDecoding(ECBlockGroup blockGroup);
+  protected abstract ErasureCodingStep prepareDecodingStep(
+      ECBlockGroup blockGroup);
 
   /**
    * We have all the data blocks and parity blocks as input blocks for

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26b5a06c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
index 80b9875..09b31e5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
@@ -30,7 +30,8 @@ public abstract class AbstractErasureEncoder extends AbstractErasureCoder
 
   @Override
   public ErasureCodingStep encode(ECBlockGroup blockGroup) {
-    return performEncoding(blockGroup);
+    // We may have more than this when considering complicate cases. HADOOP-11550
+    return prepareEncodingStep(blockGroup);
   }
 
   /**
@@ -38,7 +39,8 @@ public abstract class AbstractErasureEncoder extends AbstractErasureCoder
    * @param blockGroup
    * @return encoding step for caller to do the real work
    */
-  protected abstract ErasureCodingStep performEncoding(ECBlockGroup blockGroup);
+  protected abstract ErasureCodingStep prepareEncodingStep(
+      ECBlockGroup blockGroup);
 
   protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
     return blockGroup.getDataBlocks();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26b5a06c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
new file mode 100644
index 0000000..ba32f04
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -0,0 +1,83 @@
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
+
+/**
+ * Reed-Solomon erasure decoder that decodes a block group.
+ *
+ * It implements {@link ErasureDecoder}.
+ */
+public class RSErasureDecoder extends AbstractErasureDecoder {
+  private RawErasureDecoder rsRawDecoder;
+  private RawErasureDecoder xorRawDecoder;
+  private boolean useXorWhenPossible = true;
+
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+
+    if (conf != null) {
+      this.useXorWhenPossible = conf.getBoolean(
+          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, true);
+    }
+  }
+
+    @Override
+  protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
+
+    RawErasureDecoder rawDecoder;
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+    ECBlock[] outputBlocks = getOutputBlocks(blockGroup);
+
+    /**
+     * Optimization: according to some benchmark, when only one block is erased
+     * and to be recovering, the most simple XOR scheme can be much efficient.
+     * We will have benchmark tests to verify this opt is effect or not.
+     */
+    if (outputBlocks.length == 1 && useXorWhenPossible) {
+      rawDecoder = checkCreateXorRawDecoder();
+    } else {
+      rawDecoder = checkCreateRSRawDecoder();
+    }
+
+    return new ErasureDecodingStep(inputBlocks,
+        getErasedIndexes(inputBlocks), outputBlocks, rawDecoder);
+  }
+
+  private RawErasureDecoder checkCreateRSRawDecoder() {
+    if (rsRawDecoder == null) {
+      rsRawDecoder = createRawDecoder(
+          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
+      if (rsRawDecoder == null) {
+        rsRawDecoder = new JRSRawDecoder();
+      }
+      rsRawDecoder.initialize(getNumDataUnits(),
+          getNumParityUnits(), getChunkSize());
+    }
+    return rsRawDecoder;
+  }
+
+  private RawErasureDecoder checkCreateXorRawDecoder() {
+    if (xorRawDecoder == null) {
+      xorRawDecoder = new XorRawDecoder();
+      xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize());
+    }
+    return xorRawDecoder;
+  }
+
+  @Override
+  public void release() {
+    if (xorRawDecoder != null) {
+      xorRawDecoder.release();
+    } else if (rsRawDecoder != null) {
+      rsRawDecoder.release();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26b5a06c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
new file mode 100644
index 0000000..430749d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
@@ -0,0 +1,47 @@
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+
+/**
+ * Reed-Solomon erasure encoder that encodes a block group.
+ *
+ * It implements {@link ErasureEncoder}.
+ */
+public class RSErasureEncoder extends AbstractErasureEncoder {
+  private RawErasureEncoder rawEncoder;
+
+  @Override
+  protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
+
+    RawErasureEncoder rawEncoder = checkCreateRSRawEncoder();
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureEncodingStep(inputBlocks,
+        getOutputBlocks(blockGroup), rawEncoder);
+  }
+
+  private RawErasureEncoder checkCreateRSRawEncoder() {
+    if (rawEncoder == null) {
+      rawEncoder = createRawEncoder(
+          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
+      if (rawEncoder == null) {
+        rawEncoder = new JRSRawEncoder();
+      }
+      rawEncoder.initialize(getNumDataUnits(),
+          getNumParityUnits(), getChunkSize());
+    }
+    return rawEncoder;
+  }
+
+  @Override
+  public void release() {
+    if (rawEncoder != null) {
+      rawEncoder.release();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26b5a06c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
index 9a6f6e2..33f5386 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
 public class XorErasureDecoder extends AbstractErasureDecoder {
 
   @Override
-  protected ErasureCodingStep performDecoding(final ECBlockGroup blockGroup) {
+  protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
     // May be configured
     RawErasureDecoder rawDecoder = new XorRawDecoder();
     rawDecoder.initialize(getNumDataUnits(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26b5a06c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
index db6e3db..f8d67c3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XorRawEncoder;
 public class XorErasureEncoder extends AbstractErasureEncoder {
 
   @Override
-  protected ErasureCodingStep performEncoding(final ECBlockGroup blockGroup) {
+  protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
     // May be configured
     RawErasureEncoder rawEncoder = new XorRawEncoder();
     rawEncoder.initialize(getNumDataUnits(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26b5a06c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
new file mode 100644
index 0000000..8a7561c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawErasureCoderFactory;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test Reed-Solomon encoding and decoding.
+ */
+public class TestRSErasureCoder extends TestErasureCoderBase {
+
+  @Before
+  public void setup() {
+    this.encoderClass = RSErasureEncoder.class;
+    this.decoderClass = RSErasureDecoder.class;
+
+    this.numDataUnits = 10;
+    this.numParityUnits = 1;
+
+    this.numChunksInBlock = 10;
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_10x4() {
+    prepare(null, 10, 4, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4() {
+    prepare(null, 10, 4, null);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBufferWithConf_10x4() {
+    /**
+     * This tests if the two configuration items work or not.
+     */
+    Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
+        JRSRawErasureCoderFactory.class.getCanonicalName());
+    conf.setBoolean(
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false);
+    prepare(conf, 10, 4, null);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
+    prepare(null, 10, 4, new int[] {2, 4});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_all() {
+    prepare(null, 10, 4, new int[] {0, 1, 2, 3});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_3x3() {
+    prepare(null, 3, 3, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_3x3() {
+    prepare(null, 3, 3, null);
+    testCoding(true);
+  }
+
+}


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

Branch: refs/heads/HDFS-7285
Commit: af9dc8e50f34c226ce03a7b9517f9af5352921b3
Parents: ec4f224
Author: drankye <ka...@intel.com>
Authored: Thu Mar 12 23:35:22 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 30 10:11:27 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/af9dc8e5/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/af9dc8e5/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/af9dc8e5/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/af9dc8e5/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/af9dc8e5/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/af9dc8e5/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);