Merge remote-tracking branch 'apache/trunk' into HDFS-7285

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java

Change-Id: I53ec1c426dc988d6c4a2c87b00caef49c4057010
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..1f3006e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -0,0 +1,74 @@
+  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)
+
+    HADOOP-11534. Minor improvements for raw erasure coders
+    ( Kai Zheng via vinayakumarb )
+
+    HADOOP-11541. Raw XOR coder
+    ( Kai Zheng )
+
+    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 )
+
+    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 )
+
+    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 )
+
+    HADOOP-11647. Reed-Solomon ErasureCoder. Contributed by Kai Zheng
+    ( Kai Zheng )
+
+    HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin
+    ( Xinwei Qin via Kai Zheng )
+
+    HADOOP-11740. Combine erasure encoder and decoder interfaces (Zhe Zhang)
+
+    HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng
+    ( Kai Zheng )
+
+    HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code
+    ( Kai Zheng via vinayakumarb )
+  
+    HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng)
+
+    HADOOP-11841. Remove unused ecschema-def.xml files.  (szetszwo)
+
+    HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng via Zhe Zhang)
+
+    HADOOP-11920. Refactor some codes for erasure coders. (Kai Zheng via Zhe Zhang)
+
+    HADOOP-11566. Add tests and fix for erasure coders to recover erased parity 
+    units. (Kai Zheng via Zhe Zhang)
+
+    HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure 
+    coder. (Kai Zheng via Zhe Zhang)
+
+    HADOOP-12013. Generate fixed data to perform erasure coder test. (Kai Zheng)
+
+    HADOOP-12029. Remove chunkSize from ECSchema as its not required for coders
+    (vinayakumarb)
+
+    HADOOP-11847. Enhance raw coder allowing to read least required inputs in decoding.
+    (Kai Zheng)
+
+    HADOOP-12011. Allow to dump verbose information to ease debugging in raw erasure coders
+    (Kai Zheng)
+
+    HADOOP-12065. Using more meaningful keys in EC schema. (Kai Zheng)
+
+    HDFS-8557. Allow to configure RS and XOR raw coders (Kai Zheng)
+
+    HADOOP-12060. Fix ByteBuffer usage for raw erasure coders. (Kai Zheng via
+    jing9)
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 09d2550..ea517df 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
@@ -137,6 +137,21 @@
       false;
 
   /**
+   * Erasure Coding configuration family
+   */
+
+  /** Supported erasure codec classes */
+  public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs";
+
+  /** Raw coder factory for the RS codec. */
+  public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY =
+      "io.erasurecode.codec.rs.rawcoder";
+
+  /** Raw coder factory for the XOR codec. */
+  public static final String IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY =
+      "io.erasurecode.codec.xor.rawcoder";
+
+  /**
    * Service Authorization
    */
   public static final String 
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
index 68ff557..2458b2f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
@@ -195,6 +195,10 @@
     return sum.getChecksumSize();
   }
 
+  protected DataChecksum getDataChecksum() {
+    return sum;
+  }
+
   protected TraceScope createWriteTraceScope() {
     return null;
   }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
index d91866e..334b6bc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
@@ -121,6 +121,10 @@
     return getTrash().getCurrentTrashDir();
   }
 
+  protected String getUsagePrefix() {
+    return usagePrefix;
+  }
+
   // NOTE: Usage/Help are inner classes to allow access to outer methods
   // that access commandFactory
   
@@ -204,7 +208,7 @@
       }
     } else {
       // display help or usage for all commands 
-      out.println(usagePrefix);
+      out.println(getUsagePrefix());
       
       // display list of short usages
       ArrayList<Command> instances = new ArrayList<Command>();
@@ -228,7 +232,7 @@
   }
 
   private void printInstanceUsage(PrintStream out, Command instance) {
-    out.println(usagePrefix + " " + instance.getUsage());
+    out.println(getUsagePrefix() + " " + instance.getUsage());
   }
 
   private void printInstanceHelp(PrintStream out, Command instance) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
new file mode 100644
index 0000000..027d58b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
@@ -0,0 +1,144 @@
+/**
+ * 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.apache.hadoop.io.erasurecode.rawcoder.*;
+
+/**
+ * A codec & coder utility to help create raw coders conveniently.
+ */
+public final class CodecUtil {
+
+  private CodecUtil() { }
+
+  /**
+   * Create RS raw encoder according to configuration.
+   * @param conf configuration possibly with some items to configure the coder
+   * @param numDataUnits number of data units in a coding group
+   * @param numParityUnits number of parity units in a coding group
+   * @return raw encoder
+   */
+  public static RawErasureEncoder createRSRawEncoder(
+      Configuration conf, int numDataUnits, int numParityUnits) {
+    RawErasureCoder rawCoder = createRawCoder(conf,
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
+        true, numDataUnits, numParityUnits);
+    if (rawCoder == null) {
+      rawCoder = new RSRawEncoder(numDataUnits, numParityUnits);
+    }
+
+    return (RawErasureEncoder) rawCoder;
+  }
+
+  /**
+   * Create RS raw decoder according to configuration.
+   * @param conf configuration possibly with some items to configure the coder
+   * @param numDataUnits number of data units in a coding group
+   * @param numParityUnits number of parity units in a coding group
+   * @return raw decoder
+   */
+  public static RawErasureDecoder createRSRawDecoder(
+      Configuration conf, int numDataUnits, int numParityUnits) {
+    RawErasureCoder rawCoder = createRawCoder(conf,
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
+        false, numDataUnits, numParityUnits);
+    if (rawCoder == null) {
+      rawCoder = new RSRawDecoder(numDataUnits, numParityUnits);
+    }
+
+    return (RawErasureDecoder) rawCoder;
+  }
+
+  /**
+   * Create XOR raw encoder according to configuration.
+   * @param conf configuration possibly with some items to configure the coder
+   * @param numDataUnits number of data units in a coding group
+   * @param numParityUnits number of parity units in a coding group
+   * @return raw encoder
+   */
+  public static RawErasureEncoder createXORRawEncoder(
+      Configuration conf, int numDataUnits, int numParityUnits) {
+    RawErasureCoder rawCoder = createRawCoder(conf,
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY,
+        true, numDataUnits, numParityUnits);
+    if (rawCoder == null) {
+      rawCoder = new XORRawEncoder(numDataUnits, numParityUnits);
+    }
+
+    return (RawErasureEncoder) rawCoder;
+  }
+
+  /**
+   * Create XOR raw decoder according to configuration.
+   * @param conf configuration possibly with some items to configure the coder
+   * @param numDataUnits number of data units in a coding group
+   * @param numParityUnits number of parity units in a coding group
+   * @return raw decoder
+   */
+  public static RawErasureDecoder createXORRawDecoder(
+      Configuration conf, int numDataUnits, int numParityUnits) {
+    RawErasureCoder rawCoder = createRawCoder(conf,
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY,
+        false, numDataUnits, numParityUnits);
+    if (rawCoder == null) {
+      rawCoder = new XORRawDecoder(numDataUnits, numParityUnits);
+    }
+
+    return (RawErasureDecoder) rawCoder;
+  }
+
+  /**
+   * Create raw coder using specified conf and raw coder factory key.
+   * @param conf configuration possibly with some items to configure the coder
+   * @param rawCoderFactoryKey configuration key to find the raw coder factory
+   * @param isEncoder is encoder or not we're going to create
+   * @param numDataUnits number of data units in a coding group
+   * @param numParityUnits number of parity units in a coding group
+   * @return raw coder
+   */
+  public static RawErasureCoder createRawCoder(Configuration conf,
+      String rawCoderFactoryKey, boolean isEncoder, int numDataUnits,
+                                               int numParityUnits) {
+
+    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);
+    }
+
+    return isEncoder ? fact.createEncoder(numDataUnits, numParityUnits) :
+            fact.createDecoder(numDataUnits, numParityUnits);
+  }
+}
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..5c0a160
--- /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 is a parity block
+   * @param isErased is erased or not
+   */
+  public ECBlock(boolean isParity, boolean isErased) {
+    this.isParity = isParity;
+    this.isErased = isErased;
+  }
+
+  /**
+   * Set true if it's for a parity block.
+   * @param isParity is parity or not
+   */
+  public void setParity(boolean isParity) {
+    this.isParity = isParity;
+  }
+
+  /**
+   * Set true if the block is missing.
+   * @param isErased is erased or not
+   */
+  public void setErased(boolean isErased) {
+    this.isErased = isErased;
+  }
+
+  /**
+   *
+   * @return true if it's parity block, otherwise false
+   */
+  public boolean isParity() {
+    return isParity;
+  }
+
+  /**
+   *
+   * @return true if it's erased due to erasure, otherwise false
+   */
+  public boolean isErased() {
+    return isErased;
+  }
+
+}
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..91e4fb8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
@@ -0,0 +1,100 @@
+/**
+ * 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 data blocks in the group
+   * @param parityBlocks parity blocks in the group
+   */
+  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;
+  }
+
+  /**
+   * Get erased blocks count
+   * @return erased count of blocks
+   */
+  public int getErasedCount() {
+    int erasedCount = 0;
+
+    for (ECBlock dataBlock : dataBlocks) {
+      if (dataBlock.isErased()) erasedCount++;
+    }
+
+    for (ECBlock parityBlock : parityBlocks) {
+      if (parityBlock.isErased()) erasedCount++;
+    }
+
+    return erasedCount;
+  }
+
+}
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..d0120d8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
@@ -0,0 +1,87 @@
+/**
+ * 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 buffer to be wrapped by the chunk
+   */
+  public ECChunk(ByteBuffer buffer) {
+    this.chunkBuffer = buffer;
+  }
+
+  /**
+   * Wrapping a bytes array
+   * @param buffer buffer to be wrapped by the chunk
+   */
+  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 chunks to convert into buffers
+   * @return an array of ByteBuffers
+   */
+  public static ByteBuffer[] toBuffers(ECChunk[] chunks) {
+    ByteBuffer[] buffers = new ByteBuffer[chunks.length];
+
+    ECChunk chunk;
+    for (int i = 0; i < chunks.length; i++) {
+      chunk = chunks[i];
+      if (chunk == null) {
+        buffers[i] = null;
+      } else {
+        buffers[i] = chunk.getBuffer();
+      }
+    }
+
+    return buffers;
+  }
+
+  /**
+   * Convert to a bytes array, just for test usage.
+   * @return bytes array
+   */
+  public byte[] toBytesArray() {
+    byte[] bytesArr = new byte[chunkBuffer.remaining()];
+    // Avoid affecting the original one
+    chunkBuffer.mark();
+    chunkBuffer.get(bytesArr);
+    chunkBuffer.reset();
+
+    return bytesArr;
+  }
+}
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..0d5bf8f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
@@ -0,0 +1,227 @@
+/**
+ * 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.HashMap;
+import java.util.Map;
+
+/**
+ * Erasure coding schema to housekeeper relevant information.
+ */
+public final class ECSchema {
+  public static final String NUM_DATA_UNITS_KEY = "numDataUnits";
+  public static final String NUM_PARITY_UNITS_KEY = "numParityUnits";
+  public static final String CODEC_NAME_KEY = "codec";
+
+  /**
+   * The erasure codec name associated.
+   */
+  private final String codecName;
+
+  /**
+   * Number of source data units coded
+   */
+  private final int numDataUnits;
+
+  /**
+   * Number of parity units generated in a coding
+   */
+  private final int numParityUnits;
+
+  /*
+   * An erasure code can have its own specific advanced parameters, subject to
+   * itself to interpret these key-value settings.
+   */
+  private final Map<String, String> extraOptions;
+
+  /**
+   * Constructor with schema name and provided all options. Note the options may
+   * contain additional information for the erasure codec to interpret further.
+   * @param allOptions all schema options
+   */
+  public ECSchema(Map<String, String> allOptions) {
+    if (allOptions == null || allOptions.isEmpty()) {
+      throw new IllegalArgumentException("No schema options are provided");
+    }
+
+    this.codecName = allOptions.get(CODEC_NAME_KEY);
+    if (codecName == null || codecName.isEmpty()) {
+      throw new IllegalArgumentException("No codec option is provided");
+    }
+
+    int tmpNumDataUnits = extractIntOption(NUM_DATA_UNITS_KEY, allOptions);
+    int tmpNumParityUnits = extractIntOption(NUM_PARITY_UNITS_KEY, allOptions);
+    if (tmpNumDataUnits < 0 || tmpNumParityUnits < 0) {
+      throw new IllegalArgumentException(
+          "No good option for numDataUnits or numParityUnits found ");
+    }
+    this.numDataUnits = tmpNumDataUnits;
+    this.numParityUnits = tmpNumParityUnits;
+
+    allOptions.remove(CODEC_NAME_KEY);
+    allOptions.remove(NUM_DATA_UNITS_KEY);
+    allOptions.remove(NUM_PARITY_UNITS_KEY);
+    // After some cleanup
+    this.extraOptions = Collections.unmodifiableMap(allOptions);
+  }
+
+  /**
+   * Constructor with key parameters provided.
+   * @param codecName codec name
+   * @param numDataUnits number of data units used in the schema
+   * @param numParityUnits number os parity units used in the schema
+   */
+  public ECSchema(String codecName, int numDataUnits, int numParityUnits) {
+    this(codecName, numDataUnits, numParityUnits, null);
+  }
+
+  /**
+   * Constructor with key parameters provided. Note the extraOptions may contain
+   * additional information for the erasure codec to interpret further.
+   * @param codecName codec name
+   * @param numDataUnits number of data units used in the schema
+   * @param numParityUnits number os parity units used in the schema
+   * @param extraOptions extra options to configure the codec
+   */
+  public ECSchema(String codecName, int numDataUnits, int numParityUnits,
+      Map<String, String> extraOptions) {
+    assert (codecName != null && ! codecName.isEmpty());
+    assert (numDataUnits > 0 && numParityUnits > 0);
+
+    this.codecName = codecName;
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+
+    if (extraOptions == null) {
+      extraOptions = new HashMap<>();
+    }
+
+    // After some cleanup
+    this.extraOptions = Collections.unmodifiableMap(extraOptions);
+  }
+
+  private int extractIntOption(String optionKey, Map<String, String> options) {
+    int result = -1;
+
+    try {
+      if (options.containsKey(optionKey)) {
+        result = Integer.parseInt(options.get(optionKey));
+        if (result <= 0) {
+          throw new IllegalArgumentException("Bad option value " + result +
+              " found for " + optionKey);
+        }
+      }
+    } catch (NumberFormatException e) {
+      throw new IllegalArgumentException("Option value " +
+          options.get(optionKey) + " for " + optionKey +
+          " is found. It should be an integer");
+    }
+
+    return result;
+  }
+
+  /**
+   * Get the codec name
+   * @return codec name
+   */
+  public String getCodecName() {
+    return codecName;
+  }
+
+  /**
+   * Get extra options specific to a erasure code.
+   * @return extra options
+   */
+  public Map<String, String> getExtraOptions() {
+    return extraOptions;
+  }
+
+  /**
+   * 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;
+  }
+
+  /**
+   * Make a meaningful string representation for log output.
+   * @return string representation
+   */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ECSchema=[");
+
+    sb.append("Codec=" + codecName + ", ");
+    sb.append(NUM_DATA_UNITS_KEY + "=" + numDataUnits + ", ");
+    sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits);
+    sb.append((extraOptions.isEmpty() ? "" : ", "));
+
+    int i = 0;
+    for (String opt : extraOptions.keySet()) {
+      sb.append(opt + "=" + extraOptions.get(opt) +
+          (++i < extraOptions.size() ? ", " : ""));
+    }
+
+    sb.append("]");
+
+    return sb.toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    ECSchema ecSchema = (ECSchema) o;
+
+    if (numDataUnits != ecSchema.numDataUnits) {
+      return false;
+    }
+    if (numParityUnits != ecSchema.numParityUnits) {
+      return false;
+    }
+    if (!codecName.equals(ecSchema.codecName)) {
+      return false;
+    }
+    return extraOptions.equals(ecSchema.extraOptions);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = codecName.hashCode();
+    result = 31 * result + extraOptions.hashCode();
+    result = 31 * result + numDataUnits;
+    result = 31 * result + numParityUnits;
+
+    return result;
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
new file mode 100644
index 0000000..0cacfbc
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.codec;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper;
+
+/**
+ * Abstract Erasure Codec that implements {@link ErasureCodec}.
+ */
+public abstract class AbstractErasureCodec extends Configured
+    implements ErasureCodec {
+
+  private final ECSchema schema;
+
+  public AbstractErasureCodec(ECSchema schema) {
+    this.schema = schema;
+  }
+
+  public String getName() {
+    return schema.getCodecName();
+  }
+
+  public ECSchema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public BlockGrouper createBlockGrouper() {
+    BlockGrouper blockGrouper = new BlockGrouper();
+    blockGrouper.setSchema(getSchema());
+
+    return blockGrouper;
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
new file mode 100644
index 0000000..9aa3db2
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.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.codec;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper;
+
+/**
+ * Erasure Codec API that's to cover the essential specific aspects of a code.
+ * Currently it cares only block grouper and erasure coder. In future we may
+ * add more aspects here to make the behaviors customizable.
+ */
+public interface ErasureCodec extends Configurable {
+
+  /**
+   * Create block grouper
+   * @return block grouper
+   */
+  public BlockGrouper createBlockGrouper();
+
+  /**
+   * Create Erasure Encoder
+   * @return erasure encoder
+   */
+  public ErasureCoder createEncoder();
+
+  /**
+   * Create Erasure Decoder
+   * @return erasure decoder
+   */
+  public ErasureCoder createDecoder();
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
new file mode 100644
index 0000000..6edd638
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.codec;
+
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.io.erasurecode.coder.RSErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.RSErasureEncoder;
+
+/**
+ * A Reed-Solomon erasure codec.
+ */
+public class RSErasureCodec extends AbstractErasureCodec {
+
+  public RSErasureCodec(ECSchema schema) {
+    super(schema);
+  }
+
+  @Override
+  public ErasureCoder createEncoder() {
+    return new RSErasureEncoder(getSchema());
+  }
+
+  @Override
+  public ErasureCoder createDecoder() {
+    return new RSErasureDecoder(getSchema());
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
new file mode 100644
index 0000000..e2dcfa7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.codec;
+
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.io.erasurecode.coder.XORErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.XORErasureEncoder;
+
+/**
+ * A XOR erasure codec.
+ */
+public class XORErasureCodec extends AbstractErasureCodec {
+
+  public XORErasureCodec(ECSchema schema) {
+    super(schema);
+    assert(schema.getNumParityUnits() == 1);
+  }
+
+  @Override
+  public ErasureCoder createEncoder() {
+    return new XORErasureEncoder(getSchema());
+  }
+
+  @Override
+  public ErasureCoder createDecoder() {
+    return new XORErasureDecoder(getSchema());
+  }
+}
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..5cd0ee8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+/**
+ * A common class of basic facilities to be shared by encoder and decoder
+ *
+ * It implements the {@link ErasureCoder} interface.
+ */
+public abstract class AbstractErasureCoder
+    extends Configured implements ErasureCoder {
+
+  private final int numDataUnits;
+  private final int numParityUnits;
+
+  public AbstractErasureCoder(int numDataUnits, int numParityUnits) {
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+  }
+
+  public AbstractErasureCoder(ECSchema schema) {
+    this(schema.getNumDataUnits(), schema.getNumParityUnits());
+  }
+
+  @Override
+  public int getNumDataUnits() {
+    return numDataUnits;
+  }
+
+  @Override
+  public int getNumParityUnits() {
+    return numParityUnits;
+  }
+
+  @Override
+  public boolean preferDirectBuffer() {
+    return false;
+  }
+
+  @Override
+  public void release() {
+    // Nothing to do by default
+  }
+}
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
+  }
+
+}
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..3ea9311
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
@@ -0,0 +1,167 @@
+/**
+ * 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.ECSchema;
+
+/**
+ * An abstract erasure decoder that's to be inherited by new decoders.
+ *
+ * It implements the {@link ErasureCoder} interface.
+ */
+public abstract class AbstractErasureDecoder extends AbstractErasureCoder {
+
+  public AbstractErasureDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public AbstractErasureDecoder(ECSchema schema) {
+    super(schema);
+  }
+
+  @Override
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
+    // We may have more than this when considering complicate cases. HADOOP-11550
+    return prepareDecodingStep(blockGroup);
+  }
+
+  /**
+   * Perform decoding against a block blockGroup.
+   * @param blockGroup
+   * @return decoding step for caller to do the real work
+   */
+  protected abstract ErasureCodingStep prepareDecodingStep(
+      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 ?
+   * @param blockGroup
+   * @return output blocks to recover
+   */
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    ECBlock[] outputBlocks = new ECBlock[getNumErasedBlocks(blockGroup)];
+
+    int idx = 0;
+
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      if (blockGroup.getParityBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getParityBlocks()[i];
+      }
+    }
+
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      if (blockGroup.getDataBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
+      }
+    }
+
+    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;
+  }
+
+}
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..7c887e8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
@@ -0,0 +1,60 @@
+/**
+ * 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.ECSchema;
+
+/**
+ * An abstract erasure encoder that's to be inherited by new encoders.
+ *
+ * It implements the {@link ErasureCoder} interface.
+ */
+public abstract class AbstractErasureEncoder extends AbstractErasureCoder {
+
+  public AbstractErasureEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public AbstractErasureEncoder(ECSchema schema) {
+    super(schema);
+  }
+
+  @Override
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
+    // We may have more than this when considering complicate cases. HADOOP-11550
+    return prepareEncodingStep(blockGroup);
+  }
+
+  /**
+   * Perform encoding against a block group.
+   * @param blockGroup
+   * @return encoding step for caller to do the real work
+   */
+  protected abstract ErasureCodingStep prepareEncodingStep(
+      ECBlockGroup blockGroup);
+
+  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
+    return blockGroup.getDataBlocks();
+  }
+
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    return blockGroup.getParityBlocks();
+  }
+}
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..f05ea41
--- /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;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+
+/**
+ * An erasure coder to perform encoding or decoding given a group. Generally it
+ * 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 extends Configurable {
+
+  /**
+   * 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();
+
+  /**
+   * Calculate the encoding or decoding steps given a block blockGroup.
+   *
+   * Note, currently only one coding step is supported. Will support complex
+   * cases of multiple coding steps.
+   *
+   * @param blockGroup the erasure coding block group containing all necessary
+   *                   information for codec calculation
+   */
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup);
+
+  /**
+   * Tell if direct or off-heap buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, either on heap or off heap.
+   * It will return false by default.
+   * @return true if direct buffer is preferred for performance consideration,
+   * otherwise false.
+   */
+  public boolean preferDirectBuffer();
+
+  /**
+   * Release the resources if any. Good chance to invoke RawErasureCoder#release.
+   */
+  public void release();
+}
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();
+}
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);
+  }
+
+}
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);
+  }
+
+}
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..f56674d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -0,0 +1,67 @@
+/**
+ * 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.CodecUtil;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+
+/**
+ * Reed-Solomon erasure decoder that decodes a block group.
+ *
+ * It implements {@link ErasureCoder}.
+ */
+public class RSErasureDecoder extends AbstractErasureDecoder {
+  private RawErasureDecoder rsRawDecoder;
+
+  public RSErasureDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public RSErasureDecoder(ECSchema schema) {
+    super(schema);
+  }
+
+  @Override
+  protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+    ECBlock[] outputBlocks = getOutputBlocks(blockGroup);
+
+    RawErasureDecoder rawDecoder = checkCreateRSRawDecoder();
+    return new ErasureDecodingStep(inputBlocks,
+        getErasedIndexes(inputBlocks), outputBlocks, rawDecoder);
+  }
+
+  private RawErasureDecoder checkCreateRSRawDecoder() {
+    if (rsRawDecoder == null) {
+      rsRawDecoder = CodecUtil.createRSRawDecoder(getConf(),
+          getNumDataUnits(), getNumParityUnits());
+    }
+    return rsRawDecoder;
+  }
+
+  @Override
+  public void release() {
+    if (rsRawDecoder != null) {
+      rsRawDecoder.release();
+    }
+  }
+}
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..3ed3e20
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
@@ -0,0 +1,67 @@
+/**
+ * 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.CodecUtil;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+
+/**
+ * Reed-Solomon erasure encoder that encodes a block group.
+ *
+ * It implements {@link ErasureCoder}.
+ */
+public class RSErasureEncoder extends AbstractErasureEncoder {
+  private RawErasureEncoder rawEncoder;
+
+  public RSErasureEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public RSErasureEncoder(ECSchema schema) {
+    super(schema);
+  }
+
+  @Override
+  protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
+
+    RawErasureEncoder rawEncoder = checkCreateRSRawEncoder();
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureEncodingStep(inputBlocks,
+        getOutputBlocks(blockGroup), rawEncoder);
+  }
+
+  private RawErasureEncoder checkCreateRSRawEncoder() {
+    if (rawEncoder == null) {
+      rawEncoder = CodecUtil.createRSRawEncoder(getConf(),
+              getNumDataUnits(), getNumParityUnits());
+    }
+    return rawEncoder;
+  }
+
+  @Override
+  public void release() {
+    if (rawEncoder != null) {
+      rawEncoder.release();
+    }
+  }
+}
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..a847418
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
@@ -0,0 +1,86 @@
+/**
+ * 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.CodecUtil;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+
+/**
+ * Xor erasure decoder that decodes a block group.
+ *
+ * It implements {@link ErasureCoder}.
+ */
+public class XORErasureDecoder extends AbstractErasureDecoder {
+
+  public XORErasureDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public XORErasureDecoder(ECSchema schema) {
+    super(schema);
+  }
+
+  @Override
+  protected ErasureCodingStep prepareDecodingStep(
+      final ECBlockGroup blockGroup) {
+    RawErasureDecoder rawDecoder = CodecUtil.createXORRawDecoder(getConf(),
+        getNumDataUnits(), getNumParityUnits());
+
+    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;
+  }
+}
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..5c4bcdd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
@@ -0,0 +1,53 @@
+/**
+ * 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.CodecUtil;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+
+/**
+ * Xor erasure encoder that encodes a block group.
+ *
+ * It implements {@link ErasureCoder}.
+ */
+public class XORErasureEncoder extends AbstractErasureEncoder {
+
+  public XORErasureEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  public XORErasureEncoder(ECSchema schema) {
+    super(schema);
+  }
+
+  @Override
+  protected ErasureCodingStep prepareEncodingStep(
+      final ECBlockGroup blockGroup) {
+    RawErasureEncoder rawEncoder = CodecUtil.createXORRawEncoder(getConf(),
+        getNumDataUnits(), getNumParityUnits());
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureEncodingStep(inputBlocks,
+        getOutputBlocks(blockGroup), rawEncoder);
+  }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java
new file mode 100644
index 0000000..bdc1624
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.grouper;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+/**
+ * As part of a codec, to handle how to form a block group for encoding
+ * and provide instructions on how to recover erased blocks from a block group
+ */
+public class BlockGrouper {
+
+  private ECSchema schema;
+
+  /**
+   * Set EC schema.
+   * @param schema
+   */
+  public void setSchema(ECSchema schema) {
+    this.schema = schema;
+  }
+
+  /**
+   * Get EC schema.
+   * @return
+   */
+  protected ECSchema getSchema() {
+    return schema;
+  }
+
+  /**
+   * Get required data blocks count in a BlockGroup.
+   * @return count of required data blocks
+   */
+  public int getRequiredNumDataBlocks() {
+    return schema.getNumDataUnits();
+  }
+
+  /**
+   * Get required parity blocks count in a BlockGroup.
+   * @return count of required parity blocks
+   */
+  public int getRequiredNumParityBlocks() {
+    return schema.getNumParityUnits();
+  }
+
+  /**
+   * Calculating and organizing BlockGroup, to be called by ECManager
+   * @param dataBlocks Data blocks to compute parity blocks against
+   * @param parityBlocks To be computed parity blocks
+   * @return
+   */
+  public ECBlockGroup makeBlockGroup(ECBlock[] dataBlocks,
+                                     ECBlock[] parityBlocks) {
+
+    ECBlockGroup blockGroup = new ECBlockGroup(dataBlocks, parityBlocks);
+    return blockGroup;
+  }
+
+  /**
+   * Given a BlockGroup, tell if any of the missing blocks can be recovered,
+   * to be called by ECManager
+   * @param blockGroup a blockGroup that may contain erased blocks but not sure
+   *                   recoverable or not
+   * @return true if any erased block recoverable, false otherwise
+   */
+  public boolean anyRecoverable(ECBlockGroup blockGroup) {
+    int erasedCount = blockGroup.getErasedCount();
+
+    return erasedCount > 0 && erasedCount <= getRequiredNumParityBlocks();
+  }
+
+}
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..4b7461e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.conf.Configured;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * A common class of basic facilities to be shared by encoder and decoder
+ *
+ * It implements the {@link RawErasureCoder} interface.
+ */
+public abstract class AbstractRawErasureCoder
+    extends Configured implements RawErasureCoder {
+
+  private final int numDataUnits;
+  private final int numParityUnits;
+
+  public AbstractRawErasureCoder(int numDataUnits, int numParityUnits) {
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+  }
+
+  @Override
+  public int getNumDataUnits() {
+    return numDataUnits;
+  }
+
+  @Override
+  public int getNumParityUnits() {
+    return numParityUnits;
+  }
+
+  @Override
+  public boolean preferDirectBuffer() {
+    return false;
+  }
+
+  @Override
+  public void release() {
+    // Nothing to do by default
+  }
+
+  /**
+   * Ensure a buffer filled with ZERO bytes from current readable/writable
+   * position.
+   * @param buffer a buffer ready to read / write certain size bytes
+   * @return the buffer itself, with ZERO bytes written, the position and limit
+   *         are not changed after the call
+   */
+  protected ByteBuffer resetBuffer(ByteBuffer buffer) {
+    int pos = buffer.position();
+    for (int i = pos; i < buffer.limit(); ++i) {
+      buffer.put((byte) 0);
+    }
+    buffer.position(pos);
+
+    return buffer;
+  }
+
+  /**
+   * Ensure the buffer (either input or output) ready to read or write with ZERO
+   * bytes fully in specified length of len.
+   * @param buffer bytes array buffer
+   * @return the buffer itself
+   */
+  protected byte[] resetBuffer(byte[] buffer, int offset, int len) {
+    for (int i = offset; i < len; ++i) {
+      buffer[i] = (byte) 0;
+    }
+
+    return buffer;
+  }
+
+  /**
+   * Check and ensure the buffers are of the length specified by dataLen, also
+   * ensure the buffers are direct buffers or not according to isDirectBuffer.
+   * @param buffers the buffers to check
+   * @param allowNull whether to allow any element to be null or not
+   * @param dataLen the length of data available in the buffer to ensure with
+   * @param isDirectBuffer is direct buffer or not to ensure with
+   */
+  protected void ensureLengthAndType(ByteBuffer[] buffers, boolean allowNull,
+                                     int dataLen, boolean isDirectBuffer) {
+    for (ByteBuffer buffer : buffers) {
+      if (buffer == null && !allowNull) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer found, not allowing null");
+      } else if (buffer != null) {
+        if (buffer.remaining() != dataLen) {
+          throw new HadoopIllegalArgumentException(
+              "Invalid buffer, not of length " + dataLen);
+        }
+        if (buffer.isDirect() != isDirectBuffer) {
+          throw new HadoopIllegalArgumentException(
+              "Invalid buffer, isDirect should be " + isDirectBuffer);
+        }
+      }
+    }
+  }
+
+  /**
+   * Check and ensure the buffers are of the length specified by dataLen.
+   * @param buffers the buffers to check
+   * @param allowNull whether to allow any element to be null or not
+   * @param dataLen the length of data available in the buffer to ensure with
+   */
+  protected void ensureLength(byte[][] buffers,
+                              boolean allowNull, int dataLen) {
+    for (byte[] buffer : buffers) {
+      if (buffer == null && !allowNull) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer found, not allowing null");
+      } else if (buffer != null && buffer.length != dataLen) {
+        throw new HadoopIllegalArgumentException(
+            "Invalid buffer not of length " + dataLen);
+      }
+    }
+  }
+}
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..931cda1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
@@ -0,0 +1,207 @@
+/**
+ * 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.HadoopIllegalArgumentException;
+import org.apache.hadoop.io.erasurecode.ECChunk;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * An abstract raw erasure decoder that's to be inherited by new decoders.
+ *
+ * It implements the {@link RawErasureDecoder} interface.
+ */
+public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
+    implements RawErasureDecoder {
+
+  public AbstractRawErasureDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  @Override
+  public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
+                     ByteBuffer[] outputs) {
+    checkParameters(inputs, erasedIndexes, outputs);
+
+    ByteBuffer validInput = findFirstValidInput(inputs);
+    boolean usingDirectBuffer = validInput.isDirect();
+    int dataLen = validInput.remaining();
+    if (dataLen == 0) {
+      return;
+    }
+    ensureLengthAndType(inputs, true, dataLen, usingDirectBuffer);
+    ensureLengthAndType(outputs, false, dataLen, usingDirectBuffer);
+
+    if (usingDirectBuffer) {
+      doDecode(inputs, erasedIndexes, outputs);
+      return;
+    }
+
+    int[] inputOffsets = new int[inputs.length];
+    int[] outputOffsets = new int[outputs.length];
+    byte[][] newInputs = new byte[inputs.length][];
+    byte[][] newOutputs = new byte[outputs.length][];
+
+    ByteBuffer buffer;
+    for (int i = 0; i < inputs.length; ++i) {
+      buffer = inputs[i];
+      if (buffer != null) {
+        inputOffsets[i] = buffer.arrayOffset() + buffer.position();
+        newInputs[i] = buffer.array();
+      }
+    }
+
+    for (int i = 0; i < outputs.length; ++i) {
+      buffer = outputs[i];
+      outputOffsets[i] = buffer.arrayOffset() + buffer.position();
+      newOutputs[i] = buffer.array();
+    }
+
+    doDecode(newInputs, inputOffsets, dataLen,
+        erasedIndexes, newOutputs, outputOffsets);
+
+    for (int i = 0; i < inputs.length; ++i) {
+      buffer = inputs[i];
+      if (buffer != null) {
+        // dataLen bytes consumed
+        buffer.position(buffer.position() + dataLen);
+      }
+    }
+  }
+
+  /**
+   * Perform the real decoding using Direct ByteBuffer.
+   * @param inputs Direct ByteBuffers expected
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs Direct ByteBuffers expected
+   */
+  protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                                   ByteBuffer[] outputs);
+
+  @Override
+  public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) {
+    checkParameters(inputs, erasedIndexes, outputs);
+
+    byte[] validInput = findFirstValidInput(inputs);
+    int dataLen = validInput.length;
+    if (dataLen == 0) {
+      return;
+    }
+    ensureLength(inputs, true, dataLen);
+    ensureLength(outputs, false, dataLen);
+
+    int[] inputOffsets = new int[inputs.length]; // ALL ZERO
+    int[] outputOffsets = new int[outputs.length]; // ALL ZERO
+
+    doDecode(inputs, inputOffsets, dataLen, erasedIndexes, outputs,
+        outputOffsets);
+  }
+
+  /**
+   * Perform the real decoding using bytes array, supporting offsets and
+   * lengths.
+   * @param inputs the input byte arrays to read data from
+   * @param inputOffsets offsets for the input byte arrays to read data from
+   * @param dataLen how much data are to be read from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs the output byte arrays to write resultant data into
+   * @param outputOffsets offsets from which to write resultant data into
+   */
+  protected abstract void doDecode(byte[][] inputs, int[] inputOffsets,
+                                   int dataLen, int[] erasedIndexes,
+                                   byte[][] outputs, int[] outputOffsets);
+
+  @Override
+  public void decode(ECChunk[] inputs, int[] erasedIndexes,
+                     ECChunk[] outputs) {
+    ByteBuffer[] newInputs = ECChunk.toBuffers(inputs);
+    ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs);
+    decode(newInputs, erasedIndexes, newOutputs);
+  }
+
+  /**
+   * Check and validate decoding parameters, throw exception accordingly. The
+   * checking assumes it's a MDS code. Other code  can override this.
+   * @param inputs input buffers to check
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs output buffers to check
+   */
+  protected <T> void checkParameters(T[] inputs, int[] erasedIndexes,
+                                 T[] outputs) {
+    if (inputs.length != getNumParityUnits() + getNumDataUnits()) {
+      throw new IllegalArgumentException("Invalid inputs length");
+    }
+
+    if (erasedIndexes.length != outputs.length) {
+      throw new HadoopIllegalArgumentException(
+          "erasedIndexes and outputs mismatch in length");
+    }
+
+    if (erasedIndexes.length > getNumParityUnits()) {
+      throw new HadoopIllegalArgumentException(
+          "Too many erased, not recoverable");
+    }
+
+    int validInputs = 0;
+    for (T input : inputs) {
+      if (input != null) {
+        validInputs += 1;
+      }
+    }
+
+    if (validInputs < getNumDataUnits()) {
+      throw new HadoopIllegalArgumentException(
+          "No enough valid inputs are provided, not recoverable");
+    }
+  }
+
+  /**
+   * Get indexes into inputs array for items marked as null, either erased or
+   * not to read.
+   * @return indexes into inputs array
+   */
+  protected <T> int[] getErasedOrNotToReadIndexes(T[] inputs) {
+    int[] invalidIndexes = new int[inputs.length];
+    int idx = 0;
+    for (int i = 0; i < inputs.length; i++) {
+      if (inputs[i] == null) {
+        invalidIndexes[idx++] = i;
+      }
+    }
+
+    return Arrays.copyOf(invalidIndexes, idx);
+  }
+
+  /**
+   * Find the valid input from all the inputs.
+   * @param inputs input buffers to look for valid input
+   * @return the first valid input
+   */
+  protected static <T> T findFirstValidInput(T[] inputs) {
+    for (T input : inputs) {
+      if (input != null) {
+        return input;
+      }
+    }
+
+    throw new HadoopIllegalArgumentException(
+        "Invalid inputs are found, all being null");
+  }
+}
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..a0b3cfe
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
@@ -0,0 +1,136 @@
+/**
+ * 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.HadoopIllegalArgumentException;
+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 {
+
+  public AbstractRawErasureEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  @Override
+  public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+    checkParameters(inputs, outputs);
+
+    boolean usingDirectBuffer = inputs[0].isDirect();
+    int dataLen = inputs[0].remaining();
+    if (dataLen == 0) {
+      return;
+    }
+    ensureLengthAndType(inputs, false, dataLen, usingDirectBuffer);
+    ensureLengthAndType(outputs, false, dataLen, usingDirectBuffer);
+
+    if (usingDirectBuffer) {
+      doEncode(inputs, outputs);
+      return;
+    }
+
+    int[] inputOffsets = new int[inputs.length];
+    int[] outputOffsets = new int[outputs.length];
+    byte[][] newInputs = new byte[inputs.length][];
+    byte[][] newOutputs = new byte[outputs.length][];
+
+    ByteBuffer buffer;
+    for (int i = 0; i < inputs.length; ++i) {
+      buffer = inputs[i];
+      inputOffsets[i] = buffer.arrayOffset() + buffer.position();
+      newInputs[i] = buffer.array();
+    }
+
+    for (int i = 0; i < outputs.length; ++i) {
+      buffer = outputs[i];
+      outputOffsets[i] = buffer.arrayOffset() + buffer.position();
+      newOutputs[i] = buffer.array();
+    }
+
+    doEncode(newInputs, inputOffsets, dataLen, newOutputs, outputOffsets);
+
+    for (int i = 0; i < inputs.length; ++i) {
+      buffer = inputs[i];
+      buffer.position(buffer.position() + dataLen); // dataLen bytes consumed
+    }
+  }
+
+  /**
+   * Perform the real encoding work using direct ByteBuffer
+   * @param inputs Direct ByteBuffers expected
+   * @param outputs Direct ByteBuffers expected
+   */
+  protected abstract void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs);
+
+  @Override
+  public void encode(byte[][] inputs, byte[][] outputs) {
+    checkParameters(inputs, outputs);
+    int dataLen = inputs[0].length;
+    if (dataLen == 0) {
+      return;
+    }
+    ensureLength(inputs, false, dataLen);
+    ensureLength(outputs, false, dataLen);
+
+    int[] inputOffsets = new int[inputs.length]; // ALL ZERO
+    int[] outputOffsets = new int[outputs.length]; // ALL ZERO
+
+    doEncode(inputs, inputOffsets, dataLen, outputs, outputOffsets);
+  }
+
+  /**
+   * Perform the real encoding work using bytes array, supporting offsets
+   * and lengths.
+   * @param inputs the input byte arrays to read data from
+   * @param inputOffsets offsets for the input byte arrays to read data from
+   * @param dataLen how much data are to be read from
+   * @param outputs the output byte arrays to write resultant data into
+   * @param outputOffsets offsets from which to write resultant data into
+   */
+  protected abstract void doEncode(byte[][] inputs, int[] inputOffsets,
+                                   int dataLen, byte[][] outputs,
+                                   int[] outputOffsets);
+
+  @Override
+  public void encode(ECChunk[] inputs, ECChunk[] outputs) {
+    ByteBuffer[] newInputs = ECChunk.toBuffers(inputs);
+    ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs);
+    encode(newInputs, newOutputs);
+  }
+
+  /**
+   * Check and validate decoding parameters, throw exception accordingly.
+   * @param inputs input buffers to check
+   * @param outputs output buffers to check
+   */
+  protected <T> void checkParameters(T[] inputs, T[] outputs) {
+    if (inputs.length != getNumDataUnits()) {
+      throw new HadoopIllegalArgumentException("Invalid inputs length");
+    }
+    if (outputs.length != getNumParityUnits()) {
+      throw new HadoopIllegalArgumentException("Invalid outputs length");
+    }
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
new file mode 100644
index 0000000..57e6957
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
@@ -0,0 +1,216 @@
+/**
+ * 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.HadoopIllegalArgumentException;
+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.
+ *
+ * Currently this implementation will compute and decode not to read units
+ * unnecessarily due to the underlying implementation limit in GF. This will be
+ * addressed in HADOOP-11871.
+ */
+public class RSRawDecoder extends AbstractRawErasureDecoder {
+  // To describe and calculate the needed Vandermonde matrix
+  private int[] errSignature;
+  private int[] primitivePower;
+
+  /**
+   * We need a set of reusable buffers either for the bytes array
+   * decoding version or direct buffer decoding version. Normally not both.
+   *
+   * For output, in addition to the valid buffers from the caller
+   * passed from above, we need to provide extra buffers for the internal
+   * decoding implementation. For output, the caller should provide no more
+   * than numParityUnits but at least one buffers. And the left buffers will be
+   * borrowed from either bytesArrayBuffers, for the bytes array version.
+   *
+   */
+  // Reused buffers for decoding with bytes arrays
+  private byte[][] bytesArrayBuffers = new byte[getNumParityUnits()][];
+  private byte[][] adjustedByteArrayOutputsParameter =
+      new byte[getNumParityUnits()][];
+  private int[] adjustedOutputOffsets = new int[getNumParityUnits()];
+
+  // Reused buffers for decoding with direct ByteBuffers
+  private ByteBuffer[] directBuffers = new ByteBuffer[getNumParityUnits()];
+  private ByteBuffer[] adjustedDirectBufferOutputsParameter =
+      new ByteBuffer[getNumParityUnits()];
+
+  public RSRawDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+    if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
+      throw new HadoopIllegalArgumentException(
+              "Invalid numDataUnits and numParityUnits");
+    }
+
+    this.errSignature = new int[numParityUnits];
+    this.primitivePower = RSUtil.getPrimitivePower(numDataUnits,
+        numParityUnits);
+  }
+
+  private void doDecodeImpl(ByteBuffer[] inputs, int[] erasedIndexes,
+                          ByteBuffer[] outputs) {
+    ByteBuffer valid = findFirstValidInput(inputs);
+    int dataLen = valid.remaining();
+    for (int i = 0; i < erasedIndexes.length; i++) {
+      errSignature[i] = primitivePower[erasedIndexes[i]];
+      RSUtil.GF.substitute(inputs, dataLen, outputs[i], primitivePower[i]);
+    }
+
+    RSUtil.GF.solveVandermondeSystem(errSignature,
+        outputs, erasedIndexes.length);
+  }
+
+  private void doDecodeImpl(byte[][] inputs, int[] inputOffsets,
+                          int dataLen, int[] erasedIndexes,
+                          byte[][] outputs, int[] outputOffsets) {
+    for (int i = 0; i < erasedIndexes.length; i++) {
+      errSignature[i] = primitivePower[erasedIndexes[i]];
+      RSUtil.GF.substitute(inputs, inputOffsets, dataLen, outputs[i],
+          outputOffsets[i], primitivePower[i]);
+    }
+
+    RSUtil.GF.solveVandermondeSystem(errSignature, outputs, outputOffsets,
+            erasedIndexes.length, dataLen);
+  }
+
+  @Override
+  protected void doDecode(byte[][] inputs, int[] inputOffsets,
+                          int dataLen, int[] erasedIndexes,
+                          byte[][] outputs, int[] outputOffsets) {
+    /**
+     * As passed parameters are friendly to callers but not to the underlying
+     * implementations, so we have to adjust them before calling doDecodeImpl.
+     */
+
+    int[] erasedOrNotToReadIndexes = getErasedOrNotToReadIndexes(inputs);
+
+    // Prepare for adjustedOutputsParameter
+
+    // First reset the positions needed this time
+    for (int i = 0; i < erasedOrNotToReadIndexes.length; i++) {
+      adjustedByteArrayOutputsParameter[i] = null;
+      adjustedOutputOffsets[i] = 0;
+    }
+    // Use the caller passed buffers in erasedIndexes positions
+    for (int outputIdx = 0, i = 0; i < erasedIndexes.length; i++) {
+      boolean found = false;
+      for (int j = 0; j < erasedOrNotToReadIndexes.length; j++) {
+        // If this index is one requested by the caller via erasedIndexes, then
+        // we use the passed output buffer to avoid copying data thereafter.
+        if (erasedIndexes[i] == erasedOrNotToReadIndexes[j]) {
+          found = true;
+          adjustedByteArrayOutputsParameter[j] = resetBuffer(
+                  outputs[outputIdx], outputOffsets[outputIdx], dataLen);
+          adjustedOutputOffsets[j] = outputOffsets[outputIdx];
+          outputIdx++;
+        }
+      }
+      if (!found) {
+        throw new HadoopIllegalArgumentException(
+            "Inputs not fully corresponding to erasedIndexes in null places");
+      }
+    }
+    // Use shared buffers for other positions (not set yet)
+    for (int bufferIdx = 0, i = 0; i < erasedOrNotToReadIndexes.length; i++) {
+      if (adjustedByteArrayOutputsParameter[i] == null) {
+        adjustedByteArrayOutputsParameter[i] = resetBuffer(
+            checkGetBytesArrayBuffer(bufferIdx, dataLen), 0, dataLen);
+        adjustedOutputOffsets[i] = 0; // Always 0 for such temp output
+        bufferIdx++;
+      }
+    }
+
+    doDecodeImpl(inputs, inputOffsets, dataLen, erasedOrNotToReadIndexes,
+            adjustedByteArrayOutputsParameter, adjustedOutputOffsets);
+  }
+
+  @Override
+  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                          ByteBuffer[] outputs) {
+    ByteBuffer validInput = findFirstValidInput(inputs);
+    int dataLen = validInput.remaining();
+
+    /**
+     * As passed parameters are friendly to callers but not to the underlying
+     * implementations, so we have to adjust them before calling doDecodeImpl.
+     */
+
+    int[] erasedOrNotToReadIndexes = getErasedOrNotToReadIndexes(inputs);
+
+    // Prepare for adjustedDirectBufferOutputsParameter
+
+    // First reset the positions needed this time
+    for (int i = 0; i < erasedOrNotToReadIndexes.length; i++) {
+      adjustedDirectBufferOutputsParameter[i] = null;
+    }
+    // Use the caller passed buffers in erasedIndexes positions
+    for (int outputIdx = 0, i = 0; i < erasedIndexes.length; i++) {
+      boolean found = false;
+      for (int j = 0; j < erasedOrNotToReadIndexes.length; j++) {
+        // If this index is one requested by the caller via erasedIndexes, then
+        // we use the passed output buffer to avoid copying data thereafter.
+        if (erasedIndexes[i] == erasedOrNotToReadIndexes[j]) {
+          found = true;
+          adjustedDirectBufferOutputsParameter[j] =
+              resetBuffer(outputs[outputIdx++]);
+        }
+      }
+      if (!found) {
+        throw new HadoopIllegalArgumentException(
+            "Inputs not fully corresponding to erasedIndexes in null places");
+      }
+    }
+    // Use shared buffers for other positions (not set yet)
+    for (int bufferIdx = 0, i = 0; i < erasedOrNotToReadIndexes.length; i++) {
+      if (adjustedDirectBufferOutputsParameter[i] == null) {
+        ByteBuffer buffer = checkGetDirectBuffer(bufferIdx, dataLen);
+        buffer.position(0);
+        buffer.limit(dataLen);
+        adjustedDirectBufferOutputsParameter[i] = resetBuffer(buffer);
+        bufferIdx++;
+      }
+    }
+
+    doDecodeImpl(inputs, erasedOrNotToReadIndexes,
+        adjustedDirectBufferOutputsParameter);
+  }
+
+  private byte[] checkGetBytesArrayBuffer(int idx, int bufferLen) {
+    if (bytesArrayBuffers[idx] == null ||
+            bytesArrayBuffers[idx].length < bufferLen) {
+      bytesArrayBuffers[idx] = new byte[bufferLen];
+    }
+    return bytesArrayBuffers[idx];
+  }
+
+  private ByteBuffer checkGetDirectBuffer(int idx, int bufferLen) {
+    if (directBuffers[idx] == null ||
+        directBuffers[idx].capacity() < bufferLen) {
+      directBuffers[idx] = ByteBuffer.allocateDirect(bufferLen);
+    }
+    return directBuffers[idx];
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
new file mode 100644
index 0000000..efeee90
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw erasure encoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible.
+ */
+public class RSRawEncoder extends AbstractRawErasureEncoder {
+  private int[] generatingPolynomial;
+
+  public RSRawEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+
+    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
+
+    int[] primitivePower = RSUtil.getPrimitivePower(numDataUnits,
+        numParityUnits);
+    // compute generating polynomial
+    int[] gen = {1};
+    int[] poly = new int[2];
+    for (int i = 0; i < numParityUnits; 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) {
+    // parity units + data units
+    ByteBuffer[] all = new ByteBuffer[outputs.length + inputs.length];
+    System.arraycopy(outputs, 0, all, 0, outputs.length);
+    System.arraycopy(inputs, 0, all, outputs.length, inputs.length);
+
+    // Compute the remainder
+    RSUtil.GF.remainder(all, generatingPolynomial);
+  }
+
+  @Override
+  protected void doEncode(byte[][] inputs, int[] inputOffsets,
+                          int dataLen, byte[][] outputs,
+                          int[] outputOffsets) {
+    // parity units + data units
+    byte[][] all = new byte[outputs.length + inputs.length][];
+    System.arraycopy(outputs, 0, all, 0, outputs.length);
+    System.arraycopy(inputs, 0, all, outputs.length, inputs.length);
+
+    int[] offsets = new int[inputOffsets.length + outputOffsets.length];
+    System.arraycopy(outputOffsets, 0, offsets, 0, outputOffsets.length);
+    System.arraycopy(inputOffsets, 0, offsets,
+        outputOffsets.length, inputOffsets.length);
+
+    // Compute the remainder
+    RSUtil.GF.remainder(all, offsets, dataLen, generatingPolynomial);
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
new file mode 100644
index 0000000..5db49e3
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+/**
+ * A raw coder factory for raw Reed-Solomon coder in Java.
+ */
+public class RSRawErasureCoderFactory implements RawErasureCoderFactory {
+
+  @Override
+  public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits) {
+    return new RSRawEncoder(numDataUnits, numParityUnits);
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) {
+    return new RSRawDecoder(numDataUnits, numParityUnits);
+  }
+}
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..0ddb460
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.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.
+ *
+ * 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 extends Configurable {
+
+  /**
+   * 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();
+
+  /**
+   * Tell if direct buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, using DirectByteBuffer or
+   * bytes array. It will return false by default.
+   * @return true if native buffer is preferred for performance consideration,
+   * otherwise false.
+   */
+  public boolean preferDirectBuffer();
+
+  /**
+   * Should be called when release this coder. Good chance to release encoding
+   * or decoding buffers
+   */
+  public void release();
+}
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..280daf3
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
@@ -0,0 +1,42 @@
+/**
+ * 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.
+   * @param numDataUnits number of data units in a coding group
+   * @param numParityUnits number of parity units in a coding group
+   * @return raw erasure encoder
+   */
+  public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits);
+
+  /**
+   * Create raw erasure decoder.
+   * @param numDataUnits number of data units in a coding group
+   * @param numParityUnits number of parity units in a coding group
+   * @return raw erasure decoder
+   */
+  public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits);
+}
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..e2d01d9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.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.
+   * How to prepare for inputs:
+   * 1. Create an array containing parity units + data units. Please note the
+   *    parity units should be first or before the data units.
+   * 2. Set null in the array locations specified via erasedIndexes to indicate
+   *    they're erased and no data are to read from;
+   * 3. Set null in the array locations for extra redundant items, as they're
+   *    not necessary to read when decoding. For example in RS-6-3, if only 1
+   *    unit is really erased, then we have 2 extra items as redundant. They can
+   *    be set as null to indicate no data will be used from them.
+   *
+   * For an example using RS (6, 3), assuming sources (d0, d1, d2, d3, d4, d5)
+   * and parities (p0, p1, p2), d2 being erased. We can and may want to use only
+   * 6 units like (d1, d3, d4, d5, p0, p2) to recover d2. We will have:
+   *     inputs = [p0, null(p1), p2, null(d0), d1, null(d2), d3, d4, d5]
+   *     erasedIndexes = [5] // index of d2 into inputs array
+   *     outputs = [a-writable-buffer]
+   *
+   * Note, for both inputs and outputs, no mixing of on-heap buffers and direct
+   * buffers are allowed.
+   *
+   * @param inputs inputs to read data from, contents may change after the call
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes, ready for reading the result data from after
+   *                the call
+   */
+  public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
+                     ByteBuffer[] outputs);
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs. More see above.
+   * @param inputs inputs to read data from, contents may change after the call
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes, ready for reading the result data from after
+   *                the call
+   */
+  public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs);
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs. More see above.
+   *
+   * Note, for both input and output ECChunks, no mixing of on-heap buffers and
+   * direct buffers are allowed.
+   *
+   * @param inputs inputs to read data from, contents may change after the call
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes, ready for reading the result data from after
+   *                the call
+   */
+  public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs);
+
+}
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..7571f09
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.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.
+   *
+   * Note, for both inputs and outputs, no mixing of on-heap buffers and direct
+   * buffers are allowed.
+   *
+   * @param inputs inputs to read data from, contents may change after the call
+   * @param outputs
+   */
+  public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs);
+
+  /**
+   * Encode with inputs and generates outputs
+   * @param inputs inputs to read data from, contents may change after the call
+   * @param outputs outputs to write into for data generated, ready for reading
+   *                the result data from after the call
+   */
+  public void encode(byte[][] inputs, byte[][] outputs);
+
+  /**
+   * Encode with inputs and generates outputs.
+   *
+   * Note, for both input and output ECChunks, no mixing of on-heap buffers and
+   * direct buffers are allowed.
+   *
+   * @param inputs inputs to read data from, contents may change after the call
+   * @param outputs outputs to write into for data generated, ready for reading
+   *                the result data from after the call
+   */
+  public void encode(ECChunk[] inputs, ECChunk[] outputs);
+
+}
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..e20e543
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ *
+ * XOR code is an important primitive code scheme in erasure coding and often
+ * used in advanced codes, like HitchHiker and LRC, though itself is rarely
+ * deployed independently.
+ */
+public class XORRawDecoder extends AbstractRawErasureDecoder {
+
+  public XORRawDecoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  @Override
+  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                          ByteBuffer[] outputs) {
+    ByteBuffer output = outputs[0];
+    resetBuffer(output);
+
+    int erasedIdx = erasedIndexes[0];
+
+    // Process the inputs.
+    int iIdx, oIdx;
+    for (int i = 0; i < inputs.length; i++) {
+      // Skip the erased location.
+      if (i == erasedIdx) {
+        continue;
+      }
+
+      for (iIdx = inputs[i].position(), oIdx = output.position();
+           iIdx < inputs[i].limit();
+           iIdx++, oIdx++) {
+        output.put(oIdx, (byte) (output.get(oIdx) ^ inputs[i].get(iIdx)));
+      }
+    }
+  }
+
+  @Override
+  protected void doDecode(byte[][] inputs, int[] inputOffsets, int dataLen,
+                          int[] erasedIndexes, byte[][] outputs,
+                          int[] outputOffsets) {
+    byte[] output = outputs[0];
+    resetBuffer(output, outputOffsets[0], dataLen);
+
+    int erasedIdx = erasedIndexes[0];
+
+    // Process the inputs.
+    int iIdx, oIdx;
+    for (int i = 0; i < inputs.length; i++) {
+      // Skip the erased location.
+      if (i == erasedIdx) {
+        continue;
+      }
+
+      for (iIdx = inputOffsets[i], oIdx = outputOffsets[0];
+           iIdx < inputOffsets[i] + dataLen; iIdx++, oIdx++) {
+        output[oIdx] ^= inputs[i][iIdx];
+      }
+    }
+  }
+
+}
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..f4d242e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.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.rawcoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ *
+ * XOR code is an important primitive code scheme in erasure coding and often
+ * used in advanced codes, like HitchHiker and LRC, though itself is rarely
+ * deployed independently.
+ */
+public class XORRawEncoder extends AbstractRawErasureEncoder {
+
+  public XORRawEncoder(int numDataUnits, int numParityUnits) {
+    super(numDataUnits, numParityUnits);
+  }
+
+  protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+    ByteBuffer output = outputs[0];
+    resetBuffer(output);
+
+    // Get the first buffer's data.
+    int iIdx, oIdx;
+    for (iIdx = inputs[0].position(), oIdx = output.position();
+         iIdx < inputs[0].limit(); iIdx++, oIdx++) {
+      output.put(oIdx, inputs[0].get(iIdx));
+    }
+
+    // XOR with everything else.
+    for (int i = 1; i < inputs.length; i++) {
+      for (iIdx = inputs[i].position(), oIdx = output.position();
+           iIdx < inputs[i].limit();
+           iIdx++, oIdx++) {
+        output.put(oIdx, (byte) (output.get(oIdx) ^ inputs[i].get(iIdx)));
+      }
+    }
+  }
+
+  @Override
+  protected void doEncode(byte[][] inputs, int[] inputOffsets, int dataLen,
+                          byte[][] outputs, int[] outputOffsets) {
+    byte[] output = outputs[0];
+    resetBuffer(output, outputOffsets[0], dataLen);
+
+    // Get the first buffer's data.
+    int iIdx, oIdx;
+    for (iIdx = inputOffsets[0], oIdx = outputOffsets[0];
+         iIdx < inputOffsets[0] + dataLen; iIdx++, oIdx++) {
+      output[oIdx] = inputs[0][iIdx];
+    }
+
+    // XOR with everything else.
+    for (int i = 1; i < inputs.length; i++) {
+      for (iIdx = inputOffsets[i], oIdx = outputOffsets[0];
+           iIdx < inputOffsets[i] + dataLen; iIdx++, oIdx++) {
+        output[oIdx] ^= inputs[i][iIdx];
+      }
+    }
+  }
+}
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..de20c95
--- /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(int numDataUnits, int numParityUnits) {
+    return new XORRawEncoder(numDataUnits, numParityUnits);
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) {
+    return new XORRawDecoder(numDataUnits, numParityUnits);
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java
new file mode 100644
index 0000000..c8f133f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java
@@ -0,0 +1,85 @@
+/**
+ * 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 org.apache.hadoop.io.erasurecode.ECChunk;
+
+/**
+ * A dump utility class for debugging data erasure coding/decoding issues. Don't
+ * suggest they are used in runtime production codes.
+ */
+public final class DumpUtil {
+  private static final String HEX_CHARS_STR = "0123456789ABCDEF";
+  private static final char[] HEX_CHARS = HEX_CHARS_STR.toCharArray();
+
+  private DumpUtil() {
+    // No called
+  }
+
+  /**
+   * Convert bytes into format like 0x02 02 00 80.
+   */
+  public static String bytesToHex(byte[] bytes, int limit) {
+    if (limit > bytes.length) {
+      limit = bytes.length;
+    }
+    int len = limit * 2;
+    len += limit; // for ' ' appended for each char
+    len += 2; // for '0x' prefix
+    char[] hexChars = new char[len];
+    hexChars[0] = '0';
+    hexChars[1] = 'x';
+    for (int j = 0; j < limit; j++) {
+      int v = bytes[j] & 0xFF;
+      hexChars[j * 3 + 2] = HEX_CHARS[v >>> 4];
+      hexChars[j * 3 + 3] = HEX_CHARS[v & 0x0F];
+      hexChars[j * 3 + 4] = ' ';
+    }
+
+    return new String(hexChars);
+  }
+
+  /**
+   * Print data in hex format in an array of chunks.
+   * @param header
+   * @param chunks
+   */
+  public static void dumpChunks(String header, ECChunk[] chunks) {
+    System.out.println();
+    System.out.println(header);
+    for (int i = 0; i < chunks.length; i++) {
+      dumpChunk(chunks[i]);
+    }
+    System.out.println();
+  }
+
+  /**
+   * Print data in hex format in a chunk.
+   * @param chunk
+   */
+  public static void dumpChunk(ECChunk chunk) {
+    String str;
+    if (chunk == null) {
+      str = "<EMPTY>";
+    } else {
+      byte[] bytes = chunk.toBytesArray();
+      str = DumpUtil.bytesToHex(bytes, 16);
+    }
+    System.out.println(str);
+  }
+}
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..03683b0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
@@ -0,0 +1,561 @@
+/**
+ * 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[] outputOffsets,
+                                     int len, int dataLen) {
+    int idx1, idx2;
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        for (idx2 = outputOffsets[j-1], idx1 = outputOffsets[j];
+             idx1 < outputOffsets[j] + dataLen; idx1++, idx2++) {
+          y[j][idx1] = (byte) (y[j][idx1] ^ mulTable[x[i]][y[j - 1][idx2] &
+              0x000000FF]);
+        }
+      }
+    }
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        for (idx1 = outputOffsets[j];
+             idx1 < outputOffsets[j] + dataLen; idx1++) {
+          y[j][idx1] = (byte) (divTable[y[j][idx1] & 0x000000FF][x[j] ^
+              x[j - i - 1]]);
+        }
+      }
+      for (int j = i; j < len - 1; j++) {
+        for (idx2 = outputOffsets[j+1], idx1 = outputOffsets[j];
+             idx1 < outputOffsets[j] + dataLen; idx1++, idx2++) {
+          y[j][idx1] = (byte) (y[j][idx1] ^ y[j + 1][idx2]);
+        }
+      }
+    }
+  }
+
+  /**
+   * A "bulk" version of the solveVandermondeSystem, using ByteBuffer.
+   */
+  public void solveVandermondeSystem(int[] x, ByteBuffer[] y, int len) {
+    ByteBuffer p;
+    int idx1, idx2;
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        p = y[j];
+        for (idx1 = p.position(), idx2 = y[j-1].position();
+             idx1 < p.limit(); idx1++, idx2++) {
+          p.put(idx1, (byte) (p.get(idx1) ^ mulTable[x[i]][y[j-1].get(idx2) &
+              0x000000FF]));
+        }
+      }
+    }
+
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        p = y[j];
+        for (idx1 = p.position(); idx1 < p.limit(); idx1++) {
+          p.put(idx1, (byte) (divTable[p.get(idx1) &
+              0x000000FF][x[j] ^ x[j - i - 1]]));
+        }
+      }
+
+      for (int j = i; j < len - 1; j++) {
+        p = y[j];
+        for (idx1 = p.position(), idx2 = y[j+1].position();
+             idx1 < p.limit(); idx1++, idx2++) {
+          p.put(idx1, (byte) (p.get(idx1) ^ y[j+1].get(idx2)));
+        }
+      }
+    }
+  }
+
+  /**
+   * 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.
+   * Tends to be 2X faster than the "int" substitute in a loop.
+   *
+   * @param p input polynomial
+   * @param offsets
+   * @param len
+   * @param q store the return result
+   * @param offset
+   * @param x input field
+   */
+  public void substitute(byte[][] p, int[] offsets,
+                         int len, byte[] q, int offset, int x) {
+    int y = 1, iIdx, oIdx;
+    for (int i = 0; i < p.length; i++) {
+      byte[] pi = p[i];
+      for (iIdx = offsets[i], oIdx = offset;
+           iIdx < offsets[i] + len; iIdx++, oIdx++) {
+        int pij = pi != null ? pi[iIdx] & 0x000000FF : 0;
+        q[oIdx] = (byte) (q[oIdx] ^ mulTable[pij][y]);
+      }
+      y = mulTable[x][y];
+    }
+  }
+
+  /**
+   * A "bulk" version of the substitute, using ByteBuffer.
+   * Tends to be 2X faster than the "int" substitute in a loop.
+   *
+   * @param p input polynomial
+   * @param q store the return result
+   * @param x input field
+   */
+  public void substitute(ByteBuffer[] p, int len, ByteBuffer q, int x) {
+    int y = 1, iIdx, oIdx;
+    for (int i = 0; i < p.length; i++) {
+      ByteBuffer pi = p[i];
+      int pos = pi != null ? pi.position() : 0;
+      int limit = pi != null ? pi.limit() : len;
+      for (oIdx = q.position(), iIdx = pos;
+           iIdx < limit; iIdx++, oIdx++) {
+        int pij = pi != null ? pi.get(iIdx) & 0x000000FF : 0;
+        q.put(oIdx, (byte) (q.get(oIdx) ^ mulTable[pij][y]));
+      }
+      y = mulTable[x][y];
+    }
+  }
+
+  /**
+   * 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.
+   * Warning: This function will modify the "dividend" inputs.
+   */
+  public void remainder(byte[][] dividend, int[] offsets,
+                        int len, int[] divisor) {
+    int idx1, idx2;
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      for (int j = 0; j < divisor.length; j++) {
+        for (idx2 = offsets[j + i], idx1 = offsets[i + divisor.length - 1];
+             idx1 < offsets[i + divisor.length - 1] + len;
+             idx1++, idx2++) {
+          int ratio = divTable[dividend[i + divisor.length - 1][idx1] &
+              0x00FF][divisor[divisor.length - 1]];
+          dividend[j + i][idx2] = (byte) ((dividend[j + i][idx2] & 0x00FF) ^
+              mulTable[ratio][divisor[j]]);
+        }
+      }
+    }
+  }
+
+  /**
+   * The "bulk" version of the remainder, using ByteBuffer.
+   * Warning: This function will modify the "dividend" inputs.
+   */
+  public void remainder(ByteBuffer[] dividend, int[] divisor) {
+    int idx1, idx2;
+    ByteBuffer b1, b2;
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      for (int j = 0; j < divisor.length; j++) {
+        b1 = dividend[i + divisor.length - 1];
+        b2 = dividend[j + i];
+        for (idx1 = b1.position(), idx2 = b2.position();
+             idx1 < b1.limit(); idx1++, idx2++) {
+          int ratio = divTable[b1.get(idx1) &
+              0x00FF][divisor[divisor.length - 1]];
+          b2.put(idx2, (byte) ((b2.get(idx2) & 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]));
+        }
+      }
+    }
+  }
+
+}
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..8badf02
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.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.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;
+  }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/BufferAllocator.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/BufferAllocator.java
new file mode 100644
index 0000000..8f552b7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/BufferAllocator.java
@@ -0,0 +1,91 @@
+/**
+ * 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;
+
+/**
+ * An abstract buffer allocator used for test.
+ */
+public abstract class BufferAllocator {
+  private boolean usingDirect = false;
+
+  public BufferAllocator(boolean usingDirect) {
+    this.usingDirect = usingDirect;
+  }
+
+  protected boolean isUsingDirect() {
+    return usingDirect;
+  }
+
+  /**
+   * Allocate and return a ByteBuffer of specified length.
+   * @param bufferLen
+   * @return
+   */
+  public abstract ByteBuffer allocate(int bufferLen);
+
+  /**
+   * A simple buffer allocator that just uses ByteBuffer's
+   * allocate/allocateDirect API.
+   */
+  public static class SimpleBufferAllocator extends BufferAllocator {
+
+    public SimpleBufferAllocator(boolean usingDirect) {
+      super(usingDirect);
+    }
+
+    @Override
+    public ByteBuffer allocate(int bufferLen) {
+      return isUsingDirect() ? ByteBuffer.allocateDirect(bufferLen) :
+          ByteBuffer.allocate(bufferLen);
+    }
+  }
+
+  /**
+   * A buffer allocator that allocates a buffer from an existing large buffer by
+   * slice calling, but if no available space just degrades as
+   * SimpleBufferAllocator. So please ensure enough space for it.
+   */
+  public static class SlicedBufferAllocator extends BufferAllocator {
+    private ByteBuffer overallBuffer;
+
+    public SlicedBufferAllocator(boolean usingDirect, int totalBufferLen) {
+      super(usingDirect);
+      overallBuffer = isUsingDirect() ?
+          ByteBuffer.allocateDirect(totalBufferLen) :
+          ByteBuffer.allocate(totalBufferLen);
+    }
+
+    @Override
+    public ByteBuffer allocate(int bufferLen) {
+      if (bufferLen > overallBuffer.capacity() - overallBuffer.position()) {
+        // If no available space for the requested length, then allocate new
+        return isUsingDirect() ? ByteBuffer.allocateDirect(bufferLen) :
+            ByteBuffer.allocate(bufferLen);
+      }
+
+      overallBuffer.limit(overallBuffer.position() + bufferLen);
+      ByteBuffer result = overallBuffer.slice();
+      overallBuffer.position(overallBuffer.position() + bufferLen);
+      return result;
+    }
+  }
+
+}
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..8f277f4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -0,0 +1,500 @@
+/**
+ * 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.io.erasurecode.BufferAllocator.SimpleBufferAllocator;
+import org.apache.hadoop.io.erasurecode.BufferAllocator.SlicedBufferAllocator;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * 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();
+
+  private boolean allowDump = true;
+
+  private Configuration conf;
+  protected int numDataUnits;
+  protected int numParityUnits;
+  protected int baseChunkSize = 513;
+  private int chunkSize = baseChunkSize;
+  private BufferAllocator allocator;
+
+  private byte[] zeroChunkBytes;
+
+  private boolean startBufferWithZero = true;
+
+  // Indexes of erased data units.
+  protected int[] erasedDataIndexes = new int[] {0};
+
+  // Indexes of erased parity units.
+  protected int[] erasedParityIndexes = new int[] {0};
+
+  // Data buffers are either direct or on-heap, for performance the two cases
+  // may go to different coding implementations.
+  protected boolean usingDirectBuffer = true;
+
+  protected boolean usingFixedData = true;
+  // Using this the generated data can be repeatable across multiple calls to
+  // encode(), in order for troubleshooting.
+  private static int FIXED_DATA_GENERATOR = 0;
+  protected byte[][] fixedData;
+
+  protected int getChunkSize() {
+    return chunkSize;
+  }
+
+  protected void setChunkSize(int chunkSize) {
+    this.chunkSize = chunkSize;
+    this.zeroChunkBytes = new byte[chunkSize]; // With ZERO by default
+  }
+
+  protected void prepareBufferAllocator(boolean usingSlicedBuffer) {
+    if (usingSlicedBuffer) {
+      int roughEstimationSpace =
+          chunkSize * (numDataUnits + numParityUnits) * 10;
+      allocator = new SlicedBufferAllocator(usingDirectBuffer,
+          roughEstimationSpace);
+    } else {
+      allocator = new SimpleBufferAllocator(usingDirectBuffer);
+    }
+  }
+
+  /**
+   * Set true during setup if want to dump test settings and coding data,
+   * useful in debugging.
+   * @param allowDump
+   */
+  protected void setAllowDump(boolean allowDump) {
+    this.allowDump = allowDump;
+  }
+
+  /**
+   * Prepare before running the case.
+   * @param conf
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedDataIndexes
+   * @param erasedParityIndexes
+   * @param usingFixedData Using fixed or pre-generated data to test instead of
+   *                       generating data
+   */
+  protected void prepare(Configuration conf, int numDataUnits,
+                         int numParityUnits, int[] erasedDataIndexes,
+                         int[] erasedParityIndexes, boolean usingFixedData) {
+    this.conf = conf;
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+    this.erasedDataIndexes = erasedDataIndexes != null ?
+        erasedDataIndexes : new int[] {0};
+    this.erasedParityIndexes = erasedParityIndexes != null ?
+        erasedParityIndexes : new int[] {0};
+    this.usingFixedData = usingFixedData;
+    if (usingFixedData) {
+      prepareFixedData();
+    }
+  }
+
+  /**
+   * Prepare before running the case.
+   * @param conf
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedDataIndexes
+   * @param erasedParityIndexes
+   */
+  protected void prepare(Configuration conf, int numDataUnits,
+                         int numParityUnits, int[] erasedDataIndexes,
+                         int[] erasedParityIndexes) {
+    prepare(conf, numDataUnits, numParityUnits, erasedDataIndexes,
+        erasedParityIndexes, false);
+  }
+
+  /**
+   * Prepare before running the case.
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedDataIndexes
+   * @param erasedParityIndexes
+   */
+  protected void prepare(int numDataUnits, int numParityUnits,
+                         int[] erasedDataIndexes, int[] erasedParityIndexes) {
+    prepare(null, numDataUnits, numParityUnits, erasedDataIndexes,
+        erasedParityIndexes, false);
+  }
+
+  /**
+   * 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
+   */
+  protected void compareAndVerify(ECChunk[] erasedChunks,
+                                  ECChunk[] recoveredChunks) {
+    byte[][] erased = toArrays(erasedChunks);
+    byte[][] recovered = toArrays(recoveredChunks);
+    boolean result = Arrays.deepEquals(erased, recovered);
+    assertTrue("Decoding and comparing failed.", result);
+  }
+
+  /**
+   * Adjust and return erased indexes altogether, including erased data indexes
+   * and parity indexes.
+   * @return erased indexes altogether
+   */
+  protected int[] getErasedIndexesForDecoding() {
+    int[] erasedIndexesForDecoding =
+        new int[erasedParityIndexes.length + erasedDataIndexes.length];
+
+    int idx = 0;
+
+    for (int i = 0; i < erasedParityIndexes.length; i++) {
+      erasedIndexesForDecoding[idx ++] = erasedParityIndexes[i];
+    }
+
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
+      erasedIndexesForDecoding[idx ++] = erasedDataIndexes[i] + numParityUnits;
+    }
+
+    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;
+  }
+
+  /**
+   * Erase some data chunks to test the recovering of them. As they're erased,
+   * we don't need to read them and will not have the buffers at all, so just
+   * set them as null.
+   * @param dataChunks
+   * @param parityChunks
+   * @return clone of erased chunks
+   */
+  protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks,
+                                      ECChunk[] parityChunks) {
+    ECChunk[] toEraseChunks = new ECChunk[erasedParityIndexes.length +
+        erasedDataIndexes.length];
+
+    int idx = 0;
+
+    for (int i = 0; i < erasedParityIndexes.length; i++) {
+      toEraseChunks[idx ++] = parityChunks[erasedParityIndexes[i]];
+      parityChunks[erasedParityIndexes[i]] = null;
+    }
+
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
+      toEraseChunks[idx ++] = dataChunks[erasedDataIndexes[i]];
+      dataChunks[erasedDataIndexes[i]] = null;
+    }
+
+    return toEraseChunks;
+  }
+
+  /**
+   * Erase data from the specified chunks, just setting them as null.
+   * @param chunks
+   */
+  protected void eraseDataFromChunks(ECChunk[] chunks) {
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = null;
+    }
+  }
+
+  /**
+   * 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 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 ECChunk cloneChunkWithData(ECChunk chunk) {
+    ByteBuffer srcBuffer = chunk.getBuffer();
+
+    byte[] bytesArr = new byte[srcBuffer.remaining()];
+    srcBuffer.mark();
+    srcBuffer.get(bytesArr, 0, bytesArr.length);
+    srcBuffer.reset();
+
+    ByteBuffer destBuffer = allocateOutputBuffer(bytesArr.length);
+    int pos = destBuffer.position();
+    destBuffer.put(bytesArr);
+    destBuffer.flip();
+    destBuffer.position(pos);
+
+    return new ECChunk(destBuffer);
+  }
+
+  /**
+   * Allocate a chunk for output or writing.
+   * @return
+   */
+  protected ECChunk allocateOutputChunk() {
+    ByteBuffer buffer = allocateOutputBuffer(chunkSize);
+
+    return new ECChunk(buffer);
+  }
+
+  /**
+   * Allocate a buffer for output or writing. It can prepare for two kinds of
+   * data buffers: one with position as 0, the other with position > 0
+   * @return a buffer ready to write chunkSize bytes from current position
+   */
+  protected ByteBuffer allocateOutputBuffer(int bufferLen) {
+    /**
+     * When startBufferWithZero, will prepare a buffer as:---------------
+     * otherwise, the buffer will be like:             ___TO--BE--WRITTEN___,
+     * and in the beginning, dummy data are prefixed, to simulate a buffer of
+     * position > 0.
+     */
+    int startOffset = startBufferWithZero ? 0 : 11; // 11 is arbitrary
+    int allocLen = startOffset + bufferLen + startOffset;
+    ByteBuffer buffer = allocator.allocate(allocLen);
+    buffer.limit(startOffset + bufferLen);
+    fillDummyData(buffer, startOffset);
+    startBufferWithZero = ! startBufferWithZero;
+
+    return buffer;
+  }
+
+  /**
+   * Prepare data chunks for each data unit, by generating random data.
+   * @return
+   */
+  protected ECChunk[] prepareDataChunksForEncoding() {
+    if (usingFixedData) {
+      ECChunk[] chunks = new ECChunk[numDataUnits];
+      for (int i = 0; i < chunks.length; i++) {
+        chunks[i] = makeChunkUsingData(fixedData[i]);
+      }
+      return chunks;
+    }
+
+    return generateDataChunks();
+  }
+
+  private ECChunk makeChunkUsingData(byte[] data) {
+    ECChunk chunk = allocateOutputChunk();
+    ByteBuffer buffer = chunk.getBuffer();
+    int pos = buffer.position();
+    buffer.put(data, 0, chunkSize);
+    buffer.flip();
+    buffer.position(pos);
+
+    return chunk;
+  }
+
+  private ECChunk[] generateDataChunks() {
+    ECChunk[] chunks = new ECChunk[numDataUnits];
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = generateDataChunk();
+    }
+
+    return chunks;
+  }
+
+  private void prepareFixedData() {
+    // We may load test data from a resource, or just generate randomly.
+    // The generated data will be used across subsequent encode/decode calls.
+    this.fixedData = new byte[numDataUnits][];
+    for (int i = 0; i < numDataUnits; i++) {
+      fixedData[i] = generateFixedData(baseChunkSize * 2);
+    }
+  }
+
+  /**
+   * Generate data chunk by making random data.
+   * @return
+   */
+  protected ECChunk generateDataChunk() {
+    ByteBuffer buffer = allocateOutputBuffer(chunkSize);
+    int pos = buffer.position();
+    buffer.put(generateData(chunkSize));
+    buffer.flip();
+    buffer.position(pos);
+
+    return new ECChunk(buffer);
+  }
+
+  /**
+   * Fill len of dummy data in the buffer at the current position.
+   * @param buffer
+   * @param len
+   */
+  protected void fillDummyData(ByteBuffer buffer, int len) {
+    byte[] dummy = new byte[len];
+    RAND.nextBytes(dummy);
+    buffer.put(dummy);
+  }
+
+  protected byte[] generateData(int len) {
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) RAND.nextInt(256);
+    }
+    return buffer;
+  }
+
+  protected byte[] generateFixedData(int len) {
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) FIXED_DATA_GENERATOR++;
+      if (FIXED_DATA_GENERATOR == 256) {
+        FIXED_DATA_GENERATOR = 0;
+      }
+    }
+    return buffer;
+  }
+
+  /**
+   * Prepare parity chunks for encoding, each chunk for each parity unit.
+   * @return
+   */
+  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 +
+        erasedParityIndexes.length];
+
+    for (int i = 0; i < chunks.length; i++) {
+      chunks[i] = allocateOutputChunk();
+    }
+
+    return chunks;
+  }
+
+  /**
+   * Convert an array of this chunks to an array of byte array.
+   * Note the chunk buffers are not affected.
+   * @param chunks
+   * @return an array of byte array
+   */
+  protected byte[][] toArrays(ECChunk[] chunks) {
+    byte[][] bytesArr = new byte[chunks.length][];
+
+    for (int i = 0; i < chunks.length; i++) {
+      bytesArr[i] = chunks[i].toBytesArray();
+    }
+
+    return bytesArr;
+  }
+
+  /**
+   * Dump all the settings used in the test case if allowDump is enabled.
+   */
+  protected void dumpSetting() {
+    if (allowDump) {
+      StringBuilder sb = new StringBuilder("Erasure coder test settings:\n");
+      sb.append(" numDataUnits=").append(numDataUnits);
+      sb.append(" numParityUnits=").append(numParityUnits);
+      sb.append(" chunkSize=").append(chunkSize).append("\n");
+
+      sb.append(" erasedDataIndexes=").
+              append(Arrays.toString(erasedDataIndexes));
+      sb.append(" erasedParityIndexes=").
+              append(Arrays.toString(erasedParityIndexes));
+      sb.append(" usingDirectBuffer=").append(usingDirectBuffer).append("\n");
+
+      System.out.println(sb.toString());
+    }
+  }
+
+  /**
+   * Dump chunks prefixed with a header if allowDump is enabled.
+   * @param header
+   * @param chunks
+   */
+  protected void dumpChunks(String header, ECChunk[] chunks) {
+    if (allowDump) {
+      DumpUtil.dumpChunks(header, chunks);
+    }
+  }
+
+  /**
+   * Make some chunk messy or not correct any more
+   * @param chunks
+   */
+  protected void corruptSomeChunk(ECChunk[] chunks) {
+    int idx = new Random().nextInt(chunks.length);
+    ByteBuffer buffer = chunks[idx].getBuffer();
+    if (buffer.hasRemaining()) {
+      buffer.position(buffer.position() + 1);
+    }
+  }
+}
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..1d39901
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.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;
+
+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() {
+    int numDataUnits = 6;
+    int numParityUnits = 3;
+    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(extraOption, extraOptionValue);
+
+    ECSchema schema = new ECSchema(options);
+    System.out.println(schema.toString());
+
+    assertEquals(numDataUnits, schema.getNumDataUnits());
+    assertEquals(numParityUnits, schema.getNumParityUnits());
+    assertEquals(codec, schema.getCodecName());
+    assertEquals(extraOptionValue, schema.getExtraOptions().get(extraOption));
+  }
+}
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..738d28e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
@@ -0,0 +1,297 @@
+/**
+ * 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.ECChunk;
+import org.apache.hadoop.io.erasurecode.TestCoderBase;
+
+import java.lang.reflect.Constructor;
+
+/**
+ * Erasure coder test base with utilities.
+ */
+public abstract class TestErasureCoderBase extends TestCoderBase {
+  protected Class<? extends ErasureCoder> encoderClass;
+  protected Class<? extends ErasureCoder> decoderClass;
+
+  private ErasureCoder encoder;
+  private ErasureCoder decoder;
+
+  protected int numChunksInBlock = 16;
+
+  /**
+   * 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;
+    prepareCoders();
+
+    /**
+     * The following runs will use 3 different chunkSize for inputs and outputs,
+     * to verify the same encoder/decoder can process variable width of data.
+     */
+    performTestCoding(baseChunkSize, true);
+    performTestCoding(baseChunkSize - 17, false);
+    performTestCoding(baseChunkSize + 16, true);
+  }
+
+  private void performTestCoding(int chunkSize, boolean usingSlicedBuffer) {
+    setChunkSize(chunkSize);
+    prepareBufferAllocator(usingSlicedBuffer);
+
+    // 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());
+    TestBlock[] parityBlocks = (TestBlock[]) blockGroup.getParityBlocks();
+
+    ErasureCodingStep codingStep;
+    codingStep = encoder.calculateCoding(blockGroup);
+    performCodingStep(codingStep);
+    // Erase specified sources but return copies of them for later comparing
+    TestBlock[] backupBlocks = backupAndEraseBlocks(clonedDataBlocks, parityBlocks);
+
+    // Decode
+    blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
+    codingStep = decoder.calculateCoding(blockGroup);
+    performCodingStep(codingStep);
+
+    // Compare
+    compareAndVerify(backupBlocks, 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);
+    }
+  }
+
+  private void prepareCoders() {
+    if (encoder == null) {
+      encoder = createEncoder();
+    }
+
+    if (decoder == null) {
+      decoder = createDecoder();
+    }
+  }
+
+  /**
+   * Create the raw erasure encoder to test
+   * @return
+   */
+  protected ErasureCoder createEncoder() {
+    ErasureCoder encoder;
+    try {
+      Constructor<? extends ErasureCoder> constructor =
+          (Constructor<? extends ErasureCoder>)
+              encoderClass.getConstructor(int.class, int.class);
+      encoder = constructor.newInstance(numDataUnits, numParityUnits);
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create encoder", e);
+    }
+
+    encoder.setConf(getConf());
+    return encoder;
+  }
+
+  /**
+   * create the raw erasure decoder to test
+   * @return
+   */
+  protected ErasureCoder createDecoder() {
+    ErasureCoder decoder;
+    try {
+      Constructor<? extends ErasureCoder> constructor =
+          (Constructor<? extends ErasureCoder>)
+              decoderClass.getConstructor(int.class, int.class);
+      decoder = constructor.newInstance(numDataUnits, numParityUnits);
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create decoder", e);
+    }
+
+    decoder.setConf(getConf());
+    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);
+  }
+
+  /**
+   * Erase blocks to test the recovering of them. Before erasure clone them
+   * first so could return themselves.
+   * @param dataBlocks
+   * @return clone of erased dataBlocks
+   */
+  protected TestBlock[] backupAndEraseBlocks(TestBlock[] dataBlocks,
+                                             TestBlock[] parityBlocks) {
+    TestBlock[] toEraseBlocks = new TestBlock[erasedDataIndexes.length +
+                                          erasedParityIndexes.length];
+    int idx = 0;
+    TestBlock block;
+
+    for (int i = 0; i < erasedParityIndexes.length; i++) {
+      block = parityBlocks[erasedParityIndexes[i]];
+      toEraseBlocks[idx ++] = cloneBlockWithData(block);
+      eraseDataFromBlock(block);
+    }
+
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
+      block = dataBlocks[erasedDataIndexes[i]];
+      toEraseBlocks[idx ++] = cloneBlockWithData(block);
+      eraseDataFromBlock(block);
+    }
+
+    return toEraseBlocks;
+  }
+
+  /**
+   * 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 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 TestBlock cloneBlockWithData(TestBlock block) {
+    ECChunk[] newChunks = cloneChunksWithData(block.chunks);
+
+    return new TestBlock(newChunks);
+  }
+
+  /**
+   * Erase data from a block.
+   */
+  protected void eraseDataFromBlock(TestBlock theBlock) {
+    eraseDataFromChunks(theBlock.chunks);
+    theBlock.setErased(true);
+  }
+}
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..94f77db
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
@@ -0,0 +1,126 @@
+/**
+ * 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.RSRawErasureCoderFactory;
+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_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
+    /**
+     * Doing twice to test if the coders can be repeatedly reused. This matters
+     * as the underlying coding buffers are shared, which may have bugs.
+     */
+    testCoding(false);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBufferWithConf_10x4_erasing_d0() {
+    /**
+     * This tests if the configuration items work or not.
+     */
+    Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
+        RSRawErasureCoderFactory.class.getCanonicalName());
+    prepare(conf, 10, 4, new int[]{0}, new int[0]);
+
+    testCoding(true);
+    testCoding(true);
+  }
+  
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_p1() {
+    prepare(null, 10, 4, new int[]{}, new int[]{1});
+    testCoding(true);
+    testCoding(true);
+  }
+  
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_d2() {
+    prepare(null, 10, 4, new int[] {2}, new int[] {});
+    testCoding(true);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
+    testCoding(true);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingBothBuffers_10x4_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
+
+    /**
+     * Doing in mixed buffer usage model to test if the coders can be repeatedly
+     * reused with different buffer usage model. This matters as the underlying
+     * coding buffers are shared, which may have bugs.
+     */
+    testCoding(true);
+    testCoding(false);
+    testCoding(true);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasure_of_d2_d4_p0() {
+    prepare(null, 10, 4, new int[] {2, 4}, new int[] {0});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_d0_d1_p0_p1() {
+    prepare(null, 10, 4, new int[] {0, 1}, new int[] {0, 1});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_3x3_erasing_d0_p0() {
+    prepare(null, 3, 3, new int[] {0}, new int[] {0});
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_3x3_erasing_d0_p0() {
+    prepare(null, 3, 3, new int[] {0}, new int[] {0});
+    testCoding(true);
+  }
+}
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..06e0087
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.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.numChunksInBlock = 10;
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_erasing_p0() {
+    prepare(null, 10, 1, new int[0], new int[] {0});
+
+    /**
+     * Doing twice to test if the coders can be repeatedly reused. This matters
+     * as the underlying coding buffers are shared, which may have bugs.
+     */
+    testCoding(false);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingBothBuffers_erasing_d5() {
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
+
+    /**
+     * Doing in mixed buffer usage model to test if the coders can be repeatedly
+     * reused with different buffer usage model. This matters as the underlying
+     * coding buffers are shared, which may have bugs.
+     */
+    testCoding(true);
+    testCoding(false);
+    testCoding(true);
+    testCoding(false);
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
new file mode 100644
index 0000000..a35a4dd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test raw Reed-solomon coder implemented in Java.
+ */
+public class TestRSRawCoder extends TestRSRawCoderBase {
+
+  @Before
+  public void setup() {
+    this.encoderClass = RSRawEncoder.class;
+    this.decoderClass = RSRawDecoder.class;
+    setAllowDump(false); // Change to true to allow verbose dump for debugging
+  }
+
+  @Test
+  public void testCoding_6x3_erasing_all_d() {
+    prepare(null, 6, 3, new int[]{0, 1, 2}, new int[0], true);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_6x3_erasing_d0_d2() {
+    prepare(null, 6, 3, new int[] {0, 2}, new int[]{});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_6x3_erasing_d0() {
+    prepare(null, 6, 3, new int[]{0}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_6x3_erasing_d2() {
+    prepare(null, 6, 3, new int[]{2}, new int[]{});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_6x3_erasing_d0_p0() {
+    prepare(null, 6, 3, new int[]{0}, new int[]{0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_6x3_erasing_all_p() {
+    prepare(null, 6, 3, new int[0], new int[]{0, 1, 2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_6x3_erasing_p0() {
+    prepare(null, 6, 3, new int[0], new int[]{0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_6x3_erasing_p2() {
+    prepare(null, 6, 3, new int[0], new int[]{2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_6x3_erasure_p0_p2() {
+    prepare(null, 6, 3, new int[0], new int[]{0, 2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_6x3_erasing_d0_p0_p1() {
+    prepare(null, 6, 3, new int[]{0}, new int[]{0, 1});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_6x3_erasing_d0_d2_p2() {
+    prepare(null, 6, 3, new int[]{0, 2}, new int[]{2});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCodingNegative_6x3_erasing_d2_d4() {
+    prepare(null, 6, 3, new int[]{2, 4}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCodingNegative_6x3_erasing_too_many() {
+    prepare(null, 6, 3, new int[]{2, 4}, new int[]{0, 1});
+    testCodingWithErasingTooMany();
+  }
+
+  @Test
+  public void testCoding_10x4_erasing_d0_p0() {
+    prepare(null, 10, 4, new int[] {0}, new int[] {0});
+    testCodingDoMixAndTwice();
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java
new file mode 100644
index 0000000..efde332
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java
@@ -0,0 +1,58 @@
+/**
+ * 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;
+
+/**
+ * Test base for raw Reed-solomon coders.
+ */
+public abstract class TestRSRawCoderBase extends TestRawCoderBase {
+
+  private static int symbolSize = 0;
+  private static int symbolMax = 0;
+
+  private static int RS_FIXED_DATA_GENERATOR = 0;
+
+  static {
+    symbolSize = (int) Math.round(Math.log(
+        RSUtil.GF.getFieldSize()) / Math.log(2));
+    symbolMax = (int) Math.pow(2, symbolSize);
+  }
+
+  @Override
+  protected byte[] generateData(int len) {
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) RAND.nextInt(symbolMax);
+    }
+    return buffer;
+  }
+
+  @Override
+  protected byte[] generateFixedData(int len) {
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) RS_FIXED_DATA_GENERATOR++;
+      if (RS_FIXED_DATA_GENERATOR == symbolMax) {
+        RS_FIXED_DATA_GENERATOR = 0;
+      }
+    }
+    return buffer;
+  }
+}
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..2b7a3c4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
@@ -0,0 +1,232 @@
+/**
+ * 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;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.lang.reflect.Constructor;
+
+/**
+ * Raw coder test base with utilities.
+ */
+public abstract class TestRawCoderBase extends TestCoderBase {
+  protected Class<? extends RawErasureEncoder> encoderClass;
+  protected Class<? extends RawErasureDecoder> decoderClass;
+  private RawErasureEncoder encoder;
+  private RawErasureDecoder decoder;
+
+  /**
+   * Doing twice to test if the coders can be repeatedly reused. This matters
+   * as the underlying coding buffers are shared, which may have bugs.
+   */
+  protected void testCodingDoMixAndTwice() {
+    testCodingDoMixed();
+    testCodingDoMixed();
+  }
+
+  /**
+   * Doing in mixed buffer usage model to test if the coders can be repeatedly
+   * reused with different buffer usage model. This matters as the underlying
+   * coding buffers are shared, which may have bugs.
+   */
+  protected void testCodingDoMixed() {
+    testCoding(true);
+    testCoding(false);
+  }
+
+  /**
+   * Generating source data, encoding, recovering and then verifying.
+   * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
+   * it supports two kinds of ByteBuffers, one is array backed, the other is
+   * direct ByteBuffer. Use usingDirectBuffer indicate which case to test.
+   *
+   * @param usingDirectBuffer
+   */
+  protected void testCoding(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders();
+
+    /**
+     * The following runs will use 3 different chunkSize for inputs and outputs,
+     * to verify the same encoder/decoder can process variable width of data.
+     */
+    performTestCoding(baseChunkSize, true, false, false);
+    performTestCoding(baseChunkSize - 17, false, false, false);
+    performTestCoding(baseChunkSize + 16, true, false, false);
+  }
+
+  /**
+   * Similar to above, but perform negative cases using bad input for encoding.
+   * @param usingDirectBuffer
+   */
+  protected void testCodingWithBadInput(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders();
+
+    try {
+      performTestCoding(baseChunkSize, false, true, false);
+      Assert.fail("Encoding test with bad input should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  /**
+   * Similar to above, but perform negative cases using bad output for decoding.
+   * @param usingDirectBuffer
+   */
+  protected void testCodingWithBadOutput(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+    prepareCoders();
+
+    try {
+      performTestCoding(baseChunkSize, false, false, true);
+      Assert.fail("Decoding test with bad output should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  @Test
+  public void testCodingWithErasingTooMany() {
+    try {
+      testCoding(true);
+      Assert.fail("Decoding test erasing too many should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+
+    try {
+      testCoding(false);
+      Assert.fail("Decoding test erasing too many should fail");
+    } catch (Exception e) {
+      // Expected
+    }
+  }
+
+  private void performTestCoding(int chunkSize, boolean usingSlicedBuffer,
+                                 boolean useBadInput, boolean useBadOutput) {
+    setChunkSize(chunkSize);
+    prepareBufferAllocator(usingSlicedBuffer);
+
+    dumpSetting();
+
+    // Generate data and encode
+    ECChunk[] dataChunks = prepareDataChunksForEncoding();
+    if (useBadInput) {
+      corruptSomeChunk(dataChunks);
+    }
+    dumpChunks("Testing data chunks", dataChunks);
+
+    ECChunk[] parityChunks = prepareParityChunksForEncoding();
+
+    // Backup all the source chunks for later recovering because some coders
+    // may affect the source data.
+    ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks);
+
+    encoder.encode(dataChunks, parityChunks);
+    dumpChunks("Encoded parity chunks", parityChunks);
+
+    // Backup and erase some chunks
+    ECChunk[] backupChunks = backupAndEraseChunks(clonedDataChunks, parityChunks);
+
+    // Decode
+    ECChunk[] inputChunks = prepareInputChunksForDecoding(
+        clonedDataChunks, parityChunks);
+
+    // Remove unnecessary chunks, allowing only least required chunks to be read.
+    ensureOnlyLeastRequiredChunks(inputChunks);
+
+    ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
+    if (useBadOutput) {
+      corruptSomeChunk(recoveredChunks);
+    }
+
+    dumpChunks("Decoding input chunks", inputChunks);
+    decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks);
+    dumpChunks("Decoded/recovered chunks", recoveredChunks);
+
+    // Compare
+    compareAndVerify(backupChunks, recoveredChunks);
+  }
+
+  private void prepareCoders() {
+    if (encoder == null) {
+      encoder = createEncoder();
+    }
+
+    if (decoder == null) {
+      decoder = createDecoder();
+    }
+  }
+
+  private void ensureOnlyLeastRequiredChunks(ECChunk[] inputChunks) {
+    int leastRequiredNum = numDataUnits;
+    int erasedNum = erasedDataIndexes.length + erasedParityIndexes.length;
+    int goodNum = inputChunks.length - erasedNum;
+    int redundantNum = goodNum - leastRequiredNum;
+
+    for (int i = 0; i < inputChunks.length && redundantNum > 0; i++) {
+      if (inputChunks[i] != null) {
+        inputChunks[i] = null; // Setting it null, not needing it actually
+        redundantNum--;
+      }
+    }
+  }
+
+  /**
+   * Create the raw erasure encoder to test
+   * @return
+   */
+  protected RawErasureEncoder createEncoder() {
+    RawErasureEncoder encoder;
+    try {
+      Constructor<? extends RawErasureEncoder> constructor =
+              (Constructor<? extends RawErasureEncoder>)
+                      encoderClass.getConstructor(int.class, int.class);
+      encoder = constructor.newInstance(numDataUnits, numParityUnits);
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create encoder", e);
+    }
+
+    encoder.setConf(getConf());
+    return encoder;
+  }
+
+  /**
+   * create the raw erasure decoder to test
+   * @return
+   */
+  protected RawErasureDecoder createDecoder() {
+    RawErasureDecoder decoder;
+    try {
+      Constructor<? extends RawErasureDecoder> constructor =
+              (Constructor<? extends RawErasureDecoder>)
+                      decoderClass.getConstructor(int.class, int.class);
+      decoder = constructor.newInstance(numDataUnits, numParityUnits);
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create decoder", e);
+    }
+
+    decoder.setConf(getConf());
+    return decoder;
+  }
+}
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..48463ad
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test XOR encoding and decoding.
+ */
+public class TestXORRawCoder extends TestRawCoderBase {
+
+  @Before
+  public void setup() {
+    this.encoderClass = XORRawEncoder.class;
+    this.decoderClass = XORRawDecoder.class;
+  }
+
+  @Test
+  public void testCoding_10x1_erasing_d0() {
+    prepare(null, 10, 1, new int[] {0}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_10x1_erasing_p0() {
+    prepare(null, 10, 1, new int[0], new int[] {0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_10x1_erasing_d5() {
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCodingNegative_10x1_erasing_too_many() {
+    prepare(null, 10, 1, new int[]{2}, new int[]{0});
+    testCodingWithErasingTooMany();
+  }
+
+  @Test
+  public void testCodingNegative_10x1_erasing_d5() {
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
+    testCodingWithBadInput(true);
+    testCodingWithBadOutput(false);
+    testCodingWithBadInput(true);
+    testCodingWithBadOutput(false);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
index 68bd289..63c16d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
@@ -96,6 +96,7 @@
                   <include>hdfs.proto</include>
                   <include>encryption.proto</include>
                   <include>inotify.proto</include>
+                  <include>erasurecoding.proto</include>
                 </includes>
               </source>
               <output>${project.build.directory}/generated-sources/java</output>
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
index aa3e8ba..8f988af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.util.EnumSet;
 
@@ -24,13 +25,14 @@
 import org.apache.hadoop.fs.ByteBufferReadable;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+import org.apache.hadoop.util.DataChecksum;
 
 /**
  * A BlockReader is responsible for reading a single block
  * from a single datanode.
  */
 @InterfaceAudience.Private
-public interface BlockReader extends ByteBufferReadable {
+public interface BlockReader extends ByteBufferReadable, Closeable {
   
 
   /* same interface as inputStream java.io.InputStream#read()
@@ -62,6 +64,7 @@
    *
    * @throws IOException
    */
+  @Override // java.io.Closeable
   void close() throws IOException;
 
   /**
@@ -99,4 +102,9 @@
    *                      supported.
    */
   ClientMmap getClientMmap(EnumSet<ReadOption> opts);
+
+  /**
+   * @return              The DataChecksum used by the read block
+   */
+  DataChecksum getDataChecksum();
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
index 10dc35c..7d0822e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
@@ -724,4 +724,9 @@
   void forceUnanchorable() {
     replica.getSlot().makeUnanchorable();
   }
+
+  @Override
+  public DataChecksum getDataChecksum() {
+    return checksum;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
index 4a1828e..0afd4ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
@@ -727,4 +727,9 @@
   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
     return null;
   }
+
+  @Override
+  public DataChecksum getDataChecksum() {
+    return checksum;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 8f3df81..183602a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -157,6 +157,7 @@
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
@@ -229,6 +230,7 @@
   private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
       new DFSHedgedReadMetrics();
   private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
+  private static volatile ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
   private final int smallBufferSize;
 
   public DfsClientConf getConf() {
@@ -364,8 +366,12 @@
         dfsClientConf);
 
     if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) {
-      this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize());
+      this.initThreadsNumForHedgedReads(dfsClientConf.
+          getHedgedReadThreadpoolSize());
     }
+
+    this.initThreadsNumForStripedReads(dfsClientConf.
+        getStripedReadThreadpoolSize());
     this.saslClient = new SaslDataTransferClient(
       conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf),
       TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth);
@@ -1050,7 +1056,17 @@
     //    Get block info from namenode
     TraceScope scope = newPathTraceScope("newDFSInputStream", src);
     try {
-      return new DFSInputStream(this, src, verifyChecksum, null);
+      LocatedBlocks locatedBlocks = getLocatedBlocks(src, 0);
+      if (locatedBlocks != null) {
+        ErasureCodingPolicy ecPolicy = locatedBlocks.getErasureCodingPolicy();
+        if (ecPolicy != null) {
+          return new DFSStripedInputStream(this, src, verifyChecksum, ecPolicy,
+              locatedBlocks);
+        }
+        return new DFSInputStream(this, src, verifyChecksum, locatedBlocks);
+      } else {
+        throw new IOException("Cannot open filename " + src);
+      }
     } finally {
       scope.close();
     }
@@ -1182,7 +1198,7 @@
                              Progressable progress,
                              int buffersize,
                              ChecksumOpt checksumOpt) throws IOException {
-    return create(src, permission, flag, createParent, replication, blockSize, 
+    return create(src, permission, flag, createParent, replication, blockSize,
         progress, buffersize, checksumOpt, null);
   }
 
@@ -2867,6 +2883,22 @@
     return new EncryptionZoneIterator(namenode, tracer);
   }
 
+
+  public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
+      throws IOException {
+    checkOpen();
+    TraceScope scope = newPathTraceScope("setErasureCodingPolicy", src);
+    try {
+      namenode.setErasureCodingPolicy(src, ecPolicy);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class,
+          UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
   public void setXAttr(String src, String name, byte[] value, 
       EnumSet<XAttrSetFlag> flag) throws IOException {
     checkOpen();
@@ -2979,6 +3011,16 @@
     }
   }
 
+  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
+    checkOpen();
+    TraceScope scope = tracer.newScope("getErasureCodingPolicies");
+    try {
+      return namenode.getErasureCodingPolicies();
+    } finally {
+      scope.close();
+    }
+  }
+
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
     checkOpen();
     return new DFSInotifyEventInputStream(namenode, tracer);
@@ -3052,10 +3094,51 @@
     LOG.debug("Using hedged reads; pool threads={}", num);
   }
 
+  /**
+   * Create thread pool for parallel reading in striped layout,
+   * STRIPED_READ_THREAD_POOL, if it does not already exist.
+   * @param num Number of threads for striped reads thread pool.
+   */
+  private void initThreadsNumForStripedReads(int num) {
+    assert num > 0;
+    if (STRIPED_READ_THREAD_POOL != null) {
+      return;
+    }
+    synchronized (DFSClient.class) {
+      if (STRIPED_READ_THREAD_POOL == null) {
+        STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
+            TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new Daemon.DaemonFactory() {
+          private final AtomicInteger threadIndex = new AtomicInteger(0);
+
+          @Override
+          public Thread newThread(Runnable r) {
+            Thread t = super.newThread(r);
+            t.setName("stripedRead-" + threadIndex.getAndIncrement());
+            return t;
+          }
+        }, new ThreadPoolExecutor.CallerRunsPolicy() {
+          @Override
+          public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {
+            LOG.info("Execution for striped reading rejected, "
+                + "Executing in current thread");
+            // will run in the current thread
+            super.rejectedExecution(runnable, e);
+          }
+        });
+        STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
+      }
+    }
+  }
+
   ThreadPoolExecutor getHedgedReadsThreadPool() {
     return HEDGED_READ_THREAD_POOL;
   }
 
+  ThreadPoolExecutor getStripedReadsThreadPool() {
+    return STRIPED_READ_THREAD_POOL;
+  }
+
   boolean isHedgedReadsEnabled() {
     return (HEDGED_READ_THREAD_POOL != null) &&
       HEDGED_READ_THREAD_POOL.getMaximumPoolSize() > 0;
@@ -3115,6 +3198,28 @@
     return scope;
   }
 
+  /**
+   * Get the erasure coding policy information for the specified path
+   *
+   * @param src path to get the information for
+   * @return Returns the policy information if file or directory on the path is
+   * erasure coded, null otherwise
+   * @throws IOException
+   */
+
+  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
+    checkOpen();
+    TraceScope scope = newPathTraceScope("getErasureCodingPolicy", src);
+    try {
+      return namenode.getErasureCodingPolicy(src);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(FileNotFoundException.class,
+          AccessControlException.class, UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
   Tracer getTracer() {
     return tracer;
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 81e8c27..ab5faae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -44,7 +44,6 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import com.google.common.base.Preconditions;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ByteBufferReadable;
@@ -1056,9 +1055,7 @@
       }
     }
     if (chosenNode == null) {
-      DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() +
-          " after checking nodes = " + Arrays.toString(nodes) +
-          ", ignoredNodes = " + ignoredNodes);
+      reportLostBlock(block, ignoredNodes);
       return null;
     }
     final String dnAddr =
@@ -1068,6 +1065,17 @@
     return new DNAddrPair(chosenNode, targetAddr, storageType);
   }
 
+  /**
+   * Warn the user of a lost block
+   */
+  protected void reportLostBlock(LocatedBlock lostBlock,
+      Collection<DatanodeInfo> ignoredNodes) {
+    DatanodeInfo[] nodes = lostBlock.getLocations();
+    DFSClient.LOG.warn("No live nodes contain block " + lostBlock.getBlock() +
+        " after checking nodes = " + Arrays.toString(nodes) +
+        ", ignoredNodes = " + ignoredNodes);
+  }
+
   private static String getBestNodeDNAddrPairErrorString(
       DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
       DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {
@@ -1136,41 +1144,25 @@
   }
 
   /**
-   * Used when reading contiguous blocks
-   */
-  private void actualGetFromOneDataNode(final DNAddrPair datanode,
-      LocatedBlock block, final long start, final long end, byte[] buf,
-      int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
-      throws IOException {
-    final int length = (int) (end - start + 1);
-    actualGetFromOneDataNode(datanode, block, start, end, buf,
-        new int[]{offset}, new int[]{length}, corruptedBlockMap);
-  }
-
-  /**
    * Read data from one DataNode.
-   * @param datanode the datanode from which to read data
-   * @param block the located block containing the requested data
-   * @param startInBlk the startInBlk offset of the block
-   * @param endInBlk the endInBlk offset of the block
-   * @param buf the given byte array into which the data is read
-   * @param offsets the data may be read into multiple segments of the buf
-   *                (when reading a striped block). this array indicates the
-   *                offset of each buf segment.
-   * @param lengths the length of each buf segment
+   *
+   * @param datanode          the datanode from which to read data
+   * @param block             the located block containing the requested data
+   * @param startInBlk        the startInBlk offset of the block
+   * @param endInBlk          the endInBlk offset of the block
+   * @param buf               the given byte array into which the data is read
+   * @param offset            the offset in buf
    * @param corruptedBlockMap map recording list of datanodes with corrupted
    *                          block replica
    */
-  void actualGetFromOneDataNode(final DNAddrPair datanode,
-      LocatedBlock block, final long startInBlk, final long endInBlk,
-      byte[] buf, int[] offsets, int[] lengths,
+  void actualGetFromOneDataNode(final DNAddrPair datanode, LocatedBlock block,
+      final long startInBlk, final long endInBlk, byte[] buf, int offset,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
     DFSClientFaultInjector.get().startFetchFromDatanode();
     int refetchToken = 1; // only need to get a new access token once
     int refetchEncryptionKey = 1; // only need to get a new encryption key once
     final int len = (int) (endInBlk - startInBlk + 1);
-    checkReadPortions(offsets, lengths, len);
 
     while (true) {
       // cached block locations may have been updated by chooseDataNode()
@@ -1182,13 +1174,11 @@
         DFSClientFaultInjector.get().fetchFromDatanodeException();
         reader = getBlockReader(block, startInBlk, len, datanode.addr,
             datanode.storageType, datanode.info);
-        for (int i = 0; i < offsets.length; i++) {
-          int nread = reader.readAll(buf, offsets[i], lengths[i]);
-          updateReadStatistics(readStatistics, nread, reader);
-          if (nread != lengths[i]) {
-            throw new IOException("truncated return from reader.read(): " +
-                "excpected " + lengths[i] + ", got " + nread);
-          }
+        int nread = reader.readAll(buf, offset, len);
+        updateReadStatistics(readStatistics, nread, reader);
+        if (nread != len) {
+          throw new IOException("truncated return from reader.read(): " +
+              "excpected " + len + ", got " + nread);
         }
         DFSClientFaultInjector.get().readFromDatanodeDelay();
         return;
@@ -1204,7 +1194,7 @@
         throw new IOException(msg);
       } catch (IOException e) {
         if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
-          DFSClient.LOG.info("Will fetch a new encryption key and retry, " 
+          DFSClient.LOG.info("Will fetch a new encryption key and retry, "
               + "encryption key was invalid when connecting to " + datanode.addr
               + " : " + e);
           // The encryption key used is invalid.
@@ -1244,24 +1234,6 @@
   }
 
   /**
-   * This method verifies that the read portions are valid and do not overlap
-   * with each other.
-   */
-  private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) {
-    Preconditions.checkArgument(offsets.length == lengths.length && offsets.length > 0);
-    int sum = 0;
-    for (int i = 0; i < lengths.length; i++) {
-      if (i > 0) {
-        int gap = offsets[i] - offsets[i - 1];
-        // make sure read portions do not overlap with each other
-        Preconditions.checkArgument(gap >= lengths[i - 1]);
-      }
-      sum += lengths[i];
-    }
-    Preconditions.checkArgument(sum == totalLen);
-  }
-
-  /**
    * Like {@link #fetchBlockByteRange}except we start up a second, parallel,
    * 'hedged' read if the first read is taking longer than configured amount of
    * time. We then wait on which ever read returns first.
@@ -1476,7 +1448,7 @@
       long targetStart = position - blk.getStartOffset();
       long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
       try {
-        if (dfsClient.isHedgedReadsEnabled()) {
+        if (dfsClient.isHedgedReadsEnabled() && !blk.isStriped()) {
           hedgedFetchBlockByteRange(blk, targetStart,
               targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
         } else {
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 47c4b7e..6039177 100755
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -52,6 +52,7 @@
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
@@ -110,7 +111,7 @@
   protected final int bytesPerChecksum;
 
   protected DFSPacket currentPacket = null;
-  private DataStreamer streamer;
+  protected DataStreamer streamer;
   protected int packetSize = 0; // write packet size, not including the header.
   protected int chunksPerPacket = 0;
   protected long lastFlushOffset = 0; // offset when flush was invoked
@@ -136,7 +137,7 @@
     }
 
     return new DFSPacket(buf, chunksPerPkt, offsetInBlock, seqno,
-                         getChecksumSize(), lastPacketInBlock);
+        getChecksumSize(), lastPacketInBlock);
   }
 
   @Override
@@ -165,7 +166,7 @@
     return value;
   }
 
-  /** 
+  /**
    * @return the object for computing checksum.
    *         The type is NULL if checksum is not computed.
    */
@@ -178,7 +179,7 @@
     }
     return checksum;
   }
- 
+
   private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
       HdfsFileStatus stat, DataChecksum checksum) throws IOException {
     super(getChecksum4Compute(checksum, stat));
@@ -194,7 +195,7 @@
       DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
           +"{}", src);
     }
-    
+
     this.bytesPerChecksum = checksum.getBytesPerChecksum();
     if (bytesPerChecksum <= 0) {
       throw new HadoopIllegalArgumentException(
@@ -211,14 +212,17 @@
   /** Construct a new output stream for creating a file. */
   protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
       EnumSet<CreateFlag> flag, Progressable progress,
-      DataChecksum checksum, String[] favoredNodes) throws IOException {
+      DataChecksum checksum, String[] favoredNodes, boolean createStreamer)
+      throws IOException {
     this(dfsClient, src, progress, stat, checksum);
     this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
 
     computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
 
-    streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
-        cachingStrategy, byteArrayManager, favoredNodes);
+    if (createStreamer) {
+      streamer = new DataStreamer(stat, null, dfsClient, src, progress,
+          checksum, cachingStrategy, byteArrayManager, favoredNodes);
+    }
   }
 
   static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
@@ -269,8 +273,14 @@
         }
       }
       Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!");
-      final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
-          flag, progress, checksum, favoredNodes);
+      final DFSOutputStream out;
+      if(stat.getErasureCodingPolicy() != null) {
+        out = new DFSStripedOutputStream(dfsClient, src, stat,
+            flag, progress, checksum, favoredNodes);
+      } else {
+        out = new DFSOutputStream(dfsClient, src, stat,
+            flag, progress, checksum, favoredNodes, true);
+      }
       out.start();
       return out;
     } finally {
@@ -282,7 +292,7 @@
   private DFSOutputStream(DFSClient dfsClient, String src,
       EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
       HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
-          throws IOException {
+      throws IOException {
     this(dfsClient, src, progress, stat, checksum);
     initialFileSize = stat.getLen(); // length of file when opened
     this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK);
@@ -350,6 +360,9 @@
       String[] favoredNodes) throws IOException {
     TraceScope scope =
         dfsClient.newPathTraceScope("newStreamForAppend", src);
+    if(stat.getErasureCodingPolicy() != null) {
+      throw new IOException("Not support appending to a striping layout file yet.");
+    }
     try {
       final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
           progress, lastBlock, stat, checksum, favoredNodes);
@@ -394,10 +407,10 @@
     if (currentPacket == null) {
       currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
           .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
-      DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno={},"
+      DFSClient.LOG.debug("WriteChunk allocating new packet seqno={},"
               + " src={}, packetSize={}, chunksPerPacket={}, bytesCurBlock={}",
           currentPacket.getSeqno(), src, packetSize, chunksPerPacket,
-          getStreamer().getBytesCurBlock());
+          getStreamer().getBytesCurBlock() + ", " + this);
     }
 
     currentPacket.writeChecksum(checksum, ckoff, cklen);
@@ -406,7 +419,6 @@
     getStreamer().incBytesCurBlock(len);
 
     // If packet is full, enqueue it for transmission
-    //
     if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
         getStreamer().getBytesCurBlock() == blockSize) {
       enqueueCurrentPacketFull();
@@ -420,8 +432,8 @@
 
   void enqueueCurrentPacketFull() throws IOException {
     LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
-        + " appendChunk={}, {}", currentPacket, src, getStreamer()
-        .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
+            + " appendChunk={}, {}", currentPacket, src, getStreamer()
+            .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
         getStreamer());
     enqueueCurrentPacket();
     adjustChunkBoundary();
@@ -460,7 +472,7 @@
    *
    * @throws IOException
    */
-  protected void endBlock() throws IOException {
+  void endBlock() throws IOException {
     if (getStreamer().getBytesCurBlock() == blockSize) {
       setCurrentPacketToEmpty();
       enqueueCurrentPacket();
@@ -468,7 +480,7 @@
       lastFlushOffset = 0;
     }
   }
-  
+
   /**
    * Flushes out to all replicas of the block. The data is in the buffers
    * of the DNs but not necessarily in the DN's OS buffers.
@@ -500,16 +512,16 @@
       scope.close();
     }
   }
-  
+
   /**
    * The expected semantics is all data have flushed out to all replicas 
    * and all replicas have done posix fsync equivalent - ie the OS has 
    * flushed it to the disk device (but the disk may have it in its cache).
-   * 
+   *
    * Note that only the current block is flushed to the disk device.
    * To guarantee durable sync across block boundaries the stream should
    * be created with {@link CreateFlag#SYNC_BLOCK}.
-   * 
+   *
    * @param syncFlags
    *          Indicate the semantic of the sync. Currently used to specify
    *          whether or not to update the block length in NameNode.
@@ -526,7 +538,7 @@
 
   /**
    * Flush/Sync buffered data to DataNodes.
-   * 
+   *
    * @param isSync
    *          Whether or not to require all replicas to flush data to the disk
    *          device
@@ -669,7 +681,7 @@
   /**
    * Note that this is not a public API;
    * use {@link HdfsDataOutputStream#getCurrentBlockReplication()} instead.
-   * 
+   *
    * @return the number of valid replicas of the current block
    */
   public synchronized int getCurrentBlockReplication() throws IOException {
@@ -684,7 +696,7 @@
     }
     return currentNodes.length;
   }
-  
+
   /**
    * Waits till all existing data is flushed and confirmations 
    * received from datanodes. 
@@ -708,7 +720,7 @@
   protected synchronized void start() {
     getStreamer().start();
   }
-  
+
   /**
    * Aborts this output stream and releases any system 
    * resources associated with this stream.
@@ -746,7 +758,7 @@
       setClosed();
     }
   }
-  
+
   /**
    * Closes this output stream and releases any system 
    * resources associated with this stream.
@@ -877,7 +889,7 @@
     do {
       prevStrategy = this.cachingStrategy.get();
       nextStrategy = new CachingStrategy.Builder(prevStrategy).
-                        setDropBehind(dropBehind).build();
+          setDropBehind(dropBehind).build();
     } while (!this.cachingStrategy.compareAndSet(prevStrategy, nextStrategy));
   }
 
@@ -904,4 +916,57 @@
   protected DataStreamer getStreamer() {
     return streamer;
   }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + ":" + streamer;
+  }
+
+  static LocatedBlock addBlock(DatanodeInfo[] excludedNodes, DFSClient dfsClient,
+      String src, ExtendedBlock prevBlock, long fileId, String[] favoredNodes)
+      throws IOException {
+    final DfsClientConf conf = dfsClient.getConf();
+    int retries = conf.getNumBlockWriteLocateFollowingRetry();
+    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
+    long localstart = Time.monotonicNow();
+    while (true) {
+      try {
+        return dfsClient.namenode.addBlock(src, dfsClient.clientName, prevBlock,
+            excludedNodes, fileId, favoredNodes);
+      } catch (RemoteException e) {
+        IOException ue = e.unwrapRemoteException(FileNotFoundException.class,
+            AccessControlException.class,
+            NSQuotaExceededException.class,
+            DSQuotaExceededException.class,
+            QuotaByStorageTypeExceededException.class,
+            UnresolvedPathException.class);
+        if (ue != e) {
+          throw ue; // no need to retry these exceptions
+        }
+        if (NotReplicatedYetException.class.getName().equals(e.getClassName())) {
+          if (retries == 0) {
+            throw e;
+          } else {
+            --retries;
+            LOG.info("Exception while adding a block", e);
+            long elapsed = Time.monotonicNow() - localstart;
+            if (elapsed > 5000) {
+              LOG.info("Waiting for replication for " + (elapsed / 1000)
+                  + " seconds");
+            }
+            try {
+              LOG.warn("NotReplicatedYetException sleeping " + src
+                  + " retries left " + retries);
+              Thread.sleep(sleeptime);
+              sleeptime *= 2;
+            } catch (InterruptedException ie) {
+              LOG.warn("Caught exception", ie);
+            }
+          }
+        } else {
+          throw e;
+        }
+      }
+    }
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
index 9a8ca6f..191691b 100755
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
@@ -20,6 +20,7 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
 import java.nio.channels.ClosedChannelException;
 import java.util.Arrays;
 
@@ -38,7 +39,7 @@
  ****************************************************************/
 
 @InterfaceAudience.Private
-class DFSPacket {
+public class DFSPacket {
   public static final long HEART_BEAT_SEQNO = -1L;
   private static SpanId[] EMPTY = new SpanId[0];
   private final long seqno; // sequence number of buffer in block
@@ -81,7 +82,7 @@
    * @param checksumSize the size of checksum
    * @param lastPacketInBlock if this is the last packet
    */
-  DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno,
+  public DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno,
                    int checksumSize, boolean lastPacketInBlock) {
     this.lastPacketInBlock = lastPacketInBlock;
     this.numChunks = 0;
@@ -115,6 +116,19 @@
     dataPos += len;
   }
 
+  public synchronized void writeData(ByteBuffer inBuffer, int len)
+      throws ClosedChannelException {
+    checkBuffer();
+    len =  len > inBuffer.remaining() ? inBuffer.remaining() : len;
+    if (dataPos + len > buf.length) {
+      throw new BufferOverflowException();
+    }
+    for (int i = 0; i < len; i++) {
+      buf[dataPos + i] = inBuffer.get();
+    }
+    dataPos += len;
+  }
+
   /**
    * Write checksums to this packet
    *
@@ -123,7 +137,7 @@
    * @param len the length of checksums to write
    * @throws ClosedChannelException
    */
-  synchronized void writeChecksum(byte[] inarray, int off, int len)
+  public synchronized void writeChecksum(byte[] inarray, int off, int len)
       throws ClosedChannelException {
     checkBuffer();
     if (len == 0) {
@@ -142,7 +156,7 @@
    * @param stm
    * @throws IOException
    */
-  synchronized void writeTo(DataOutputStream stm) throws IOException {
+  public synchronized void writeTo(DataOutputStream stm) throws IOException {
     checkBuffer();
 
     final int dataLen = dataPos - dataStart;
@@ -224,7 +238,7 @@
    *
    * @return true if the packet is the last packet
    */
-  boolean isLastPacketInBlock(){
+  boolean isLastPacketInBlock() {
     return lastPacketInBlock;
   }
 
@@ -233,7 +247,7 @@
    *
    * @return the sequence number of this packet
    */
-  long getSeqno(){
+  long getSeqno() {
     return seqno;
   }
 
@@ -242,14 +256,14 @@
    *
    * @return the number of chunks in this packet
    */
-  synchronized int getNumChunks(){
+  synchronized int getNumChunks() {
     return numChunks;
   }
 
   /**
    * increase the number of chunks by one
    */
-  synchronized void incNumChunks(){
+  synchronized void incNumChunks() {
     numChunks++;
   }
 
@@ -258,7 +272,7 @@
    *
    * @return the maximum number of packets
    */
-  int getMaxChunks(){
+  int getMaxChunks() {
     return maxChunks;
   }
 
@@ -267,7 +281,7 @@
    *
    * @param syncBlock if to sync block
    */
-  synchronized void setSyncBlock(boolean syncBlock){
+  synchronized void setSyncBlock(boolean syncBlock) {
     this.syncBlock = syncBlock;
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
new file mode 100644
index 0000000..69105a0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@ -0,0 +1,972 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.ReadOption;
+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.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.io.ByteBufferPool;
+
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
+
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.util.DirectBufferPool;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import java.util.Collection;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+
+/**
+ * DFSStripedInputStream reads from striped block groups
+ */
+public class DFSStripedInputStream extends DFSInputStream {
+
+  private static class ReaderRetryPolicy {
+    private int fetchEncryptionKeyTimes = 1;
+    private int fetchTokenTimes = 1;
+
+    void refetchEncryptionKey() {
+      fetchEncryptionKeyTimes--;
+    }
+
+    void refetchToken() {
+      fetchTokenTimes--;
+    }
+
+    boolean shouldRefetchEncryptionKey() {
+      return fetchEncryptionKeyTimes > 0;
+    }
+
+    boolean shouldRefetchToken() {
+      return fetchTokenTimes > 0;
+    }
+  }
+
+  /** Used to indicate the buffered data's range in the block group */
+  private static class StripeRange {
+    /** start offset in the block group (inclusive) */
+    final long offsetInBlock;
+    /** length of the stripe range */
+    final long length;
+
+    StripeRange(long offsetInBlock, long length) {
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      this.offsetInBlock = offsetInBlock;
+      this.length = length;
+    }
+
+    boolean include(long pos) {
+      return pos >= offsetInBlock && pos < offsetInBlock + length;
+    }
+  }
+
+  private static class BlockReaderInfo {
+    final BlockReader reader;
+    final DatanodeInfo datanode;
+    /**
+     * when initializing block readers, their starting offsets are set to the same
+     * number: the smallest internal block offsets among all the readers. This is
+     * because it is possible that for some internal blocks we have to read
+     * "backwards" for decoding purpose. We thus use this offset array to track
+     * offsets for all the block readers so that we can skip data if necessary.
+     */
+    long blockReaderOffset;
+    /**
+     * We use this field to indicate whether we should use this reader. In case
+     * we hit any issue with this reader, we set this field to true and avoid
+     * using it for the next stripe.
+     */
+    boolean shouldSkip = false;
+
+    BlockReaderInfo(BlockReader reader, DatanodeInfo dn, long offset) {
+      this.reader = reader;
+      this.datanode = dn;
+      this.blockReaderOffset = offset;
+    }
+
+    void setOffset(long offset) {
+      this.blockReaderOffset = offset;
+    }
+
+    void skip() {
+      this.shouldSkip = true;
+    }
+  }
+
+  private static final DirectBufferPool bufferPool = new DirectBufferPool();
+
+  private final BlockReaderInfo[] blockReaders;
+  private final int cellSize;
+  private final short dataBlkNum;
+  private final short parityBlkNum;
+  private final int groupSize;
+  /** the buffer for a complete stripe */
+  private ByteBuffer curStripeBuf;
+  private ByteBuffer parityBuf;
+  private final ErasureCodingPolicy ecPolicy;
+  private final RawErasureDecoder decoder;
+
+  /**
+   * indicate the start/end offset of the current buffered stripe in the
+   * block group
+   */
+  private StripeRange curStripeRange;
+  private final CompletionService<Void> readingService;
+
+  /**
+   * When warning the user of a lost block in striping mode, we remember the
+   * dead nodes we've logged. All other striping blocks on these nodes can be
+   * considered lost too, and we don't want to log a warning for each of them.
+   * This is to prevent the log from being too verbose. Refer to HDFS-8920.
+   *
+   * To minimize the overhead, we only store the datanodeUuid in this set
+   */
+  private final Set<String> warnedNodes = Collections.newSetFromMap(
+      new ConcurrentHashMap<String, Boolean>());
+
+  DFSStripedInputStream(DFSClient dfsClient, String src,
+      boolean verifyChecksum, ErasureCodingPolicy ecPolicy,
+      LocatedBlocks locatedBlocks) throws IOException {
+    super(dfsClient, src, verifyChecksum, locatedBlocks);
+
+    assert ecPolicy != null;
+    this.ecPolicy = ecPolicy;
+    this.cellSize = ecPolicy.getCellSize();
+    dataBlkNum = (short) ecPolicy.getNumDataUnits();
+    parityBlkNum = (short) ecPolicy.getNumParityUnits();
+    groupSize = dataBlkNum + parityBlkNum;
+    blockReaders = new BlockReaderInfo[groupSize];
+    curStripeRange = new StripeRange(0, 0);
+    readingService =
+        new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool());
+    decoder = CodecUtil.createRSRawDecoder(dfsClient.getConfiguration(),
+        dataBlkNum, parityBlkNum);
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("Creating an striped input stream for file " + src);
+    }
+  }
+
+  private void resetCurStripeBuffer() {
+    if (curStripeBuf == null) {
+      curStripeBuf = bufferPool.getBuffer(cellSize * dataBlkNum);
+    }
+    curStripeBuf.clear();
+    curStripeRange = new StripeRange(0, 0);
+  }
+
+  private ByteBuffer getParityBuffer() {
+    if (parityBuf == null) {
+      parityBuf = bufferPool.getBuffer(cellSize * parityBlkNum);
+    }
+    parityBuf.clear();
+    return parityBuf;
+  }
+
+  /**
+   * When seeking into a new block group, create blockReader for each internal
+   * block in the group.
+   */
+  private synchronized void blockSeekTo(long target) throws IOException {
+    if (target >= getFileLength()) {
+      throw new IOException("Attempted to read past end of file");
+    }
+
+    // Will be getting a new BlockReader.
+    closeCurrentBlockReaders();
+
+    // Compute desired striped block group
+    LocatedStripedBlock targetBlockGroup = getBlockGroupAt(target);
+    // Update current position
+    this.pos = target;
+    this.blockEnd = targetBlockGroup.getStartOffset() +
+        targetBlockGroup.getBlockSize() - 1;
+    currentLocatedBlock = targetBlockGroup;
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    super.close();
+    if (curStripeBuf != null) {
+      bufferPool.returnBuffer(curStripeBuf);
+      curStripeBuf = null;
+    }
+    if (parityBuf != null) {
+      bufferPool.returnBuffer(parityBuf);
+      parityBuf = null;
+    }
+  }
+
+  /**
+   * Extend the super method with the logic of switching between cells.
+   * When reaching the end of a cell, proceed to the next cell and read it
+   * with the next blockReader.
+   */
+  @Override
+  protected void closeCurrentBlockReaders() {
+    resetCurStripeBuffer();
+    if (blockReaders ==  null || blockReaders.length == 0) {
+      return;
+    }
+    for (int i = 0; i < groupSize; i++) {
+      closeReader(blockReaders[i]);
+      blockReaders[i] = null;
+    }
+    blockEnd = -1;
+  }
+
+  private void closeReader(BlockReaderInfo readerInfo) {
+    if (readerInfo != null) {
+//      IOUtils.cleanup(null, readerInfo.reader);
+      readerInfo.skip();
+    }
+  }
+
+  private long getOffsetInBlockGroup() {
+    return getOffsetInBlockGroup(pos);
+  }
+
+  private long getOffsetInBlockGroup(long pos) {
+    return pos - currentLocatedBlock.getStartOffset();
+  }
+
+  /**
+   * Read a new stripe covering the current position, and store the data in the
+   * {@link #curStripeBuf}.
+   */
+  private void readOneStripe(
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    resetCurStripeBuffer();
+
+    // compute stripe range based on pos
+    final long offsetInBlockGroup = getOffsetInBlockGroup();
+    final long stripeLen = cellSize * dataBlkNum;
+    final int stripeIndex = (int) (offsetInBlockGroup / stripeLen);
+    final int stripeBufOffset = (int) (offsetInBlockGroup % stripeLen);
+    final int stripeLimit = (int) Math.min(currentLocatedBlock.getBlockSize()
+        - (stripeIndex * stripeLen), stripeLen);
+    StripeRange stripeRange = new StripeRange(offsetInBlockGroup,
+        stripeLimit - stripeBufOffset);
+
+    LocatedStripedBlock blockGroup = (LocatedStripedBlock) currentLocatedBlock;
+    AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(ecPolicy, cellSize,
+        blockGroup, offsetInBlockGroup,
+        offsetInBlockGroup + stripeRange.length - 1, curStripeBuf);
+    final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(
+        blockGroup, cellSize, dataBlkNum, parityBlkNum);
+    // read the whole stripe
+    for (AlignedStripe stripe : stripes) {
+      // Parse group to get chosen DN location
+      StripeReader sreader = new StatefulStripeReader(readingService, stripe,
+          blks, blockReaders, corruptedBlockMap);
+      sreader.readStripe();
+    }
+    curStripeBuf.position(stripeBufOffset);
+    curStripeBuf.limit(stripeLimit);
+    curStripeRange = stripeRange;
+  }
+
+  private Callable<Void> readCells(final BlockReader reader,
+      final DatanodeInfo datanode, final long currentReaderOffset,
+      final long targetReaderOffset, final ByteBufferStrategy[] strategies,
+      final ExtendedBlock currentBlock,
+      final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
+    return new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        // reader can be null if getBlockReaderWithRetry failed or
+        // the reader hit exception before
+        if (reader == null) {
+          throw new IOException("The BlockReader is null. " +
+              "The BlockReader creation failed or the reader hit exception.");
+        }
+        Preconditions.checkState(currentReaderOffset <= targetReaderOffset);
+        if (currentReaderOffset < targetReaderOffset) {
+          long skipped = reader.skip(targetReaderOffset - currentReaderOffset);
+          Preconditions.checkState(
+              skipped == targetReaderOffset - currentReaderOffset);
+        }
+        int result = 0;
+        for (ByteBufferStrategy strategy : strategies) {
+          result += readToBuffer(reader, datanode, strategy, currentBlock,
+              corruptedBlockMap);
+        }
+        return null;
+      }
+    };
+  }
+
+  private int readToBuffer(BlockReader blockReader,
+      DatanodeInfo currentNode, ByteBufferStrategy strategy,
+      ExtendedBlock currentBlock,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    final int targetLength = strategy.buf.remaining();
+    int length = 0;
+    try {
+      while (length < targetLength) {
+        int ret = strategy.doRead(blockReader, 0, 0);
+        if (ret < 0) {
+          throw new IOException("Unexpected EOS from the reader");
+        }
+        length += ret;
+      }
+      return length;
+    } catch (ChecksumException ce) {
+      DFSClient.LOG.warn("Found Checksum error for "
+          + currentBlock + " from " + currentNode
+          + " at " + ce.getPos());
+      // we want to remember which block replicas we have tried
+      addIntoCorruptedBlockMap(currentBlock, currentNode,
+          corruptedBlockMap);
+      throw ce;
+    } catch (IOException e) {
+      DFSClient.LOG.warn("Exception while reading from "
+          + currentBlock + " of " + src + " from "
+          + currentNode, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Seek to a new arbitrary location
+   */
+  @Override
+  public synchronized void seek(long targetPos) throws IOException {
+    if (targetPos > getFileLength()) {
+      throw new EOFException("Cannot seek after EOF");
+    }
+    if (targetPos < 0) {
+      throw new EOFException("Cannot seek to negative offset");
+    }
+    if (closed.get()) {
+      throw new IOException("Stream is closed!");
+    }
+    if (targetPos <= blockEnd) {
+      final long targetOffsetInBlk = getOffsetInBlockGroup(targetPos);
+      if (curStripeRange.include(targetOffsetInBlk)) {
+        int bufOffset = getStripedBufOffset(targetOffsetInBlk);
+        curStripeBuf.position(bufOffset);
+        pos = targetPos;
+        return;
+      }
+    }
+    pos = targetPos;
+    blockEnd = -1;
+  }
+
+  private int getStripedBufOffset(long offsetInBlockGroup) {
+    final long stripeLen = cellSize * dataBlkNum;
+    // compute the position in the curStripeBuf based on "pos"
+    return (int) (offsetInBlockGroup % stripeLen);
+  }
+
+  @Override
+  public synchronized boolean seekToNewSource(long targetPos)
+      throws IOException {
+    return false;
+  }
+
+  @Override
+  protected synchronized int readWithStrategy(ReaderStrategy strategy,
+      int off, int len) throws IOException {
+    dfsClient.checkOpen();
+    if (closed.get()) {
+      throw new IOException("Stream closed");
+    }
+    Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap =
+        new ConcurrentHashMap<>();
+    if (pos < getFileLength()) {
+      try {
+        if (pos > blockEnd) {
+          blockSeekTo(pos);
+        }
+        int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
+        synchronized (infoLock) {
+          if (locatedBlocks.isLastBlockComplete()) {
+            realLen = (int) Math.min(realLen,
+                locatedBlocks.getFileLength() - pos);
+          }
+        }
+
+        /** Number of bytes already read into buffer */
+        int result = 0;
+        while (result < realLen) {
+          if (!curStripeRange.include(getOffsetInBlockGroup())) {
+            readOneStripe(corruptedBlockMap);
+          }
+          int ret = copyToTargetBuf(strategy, off + result, realLen - result);
+          result += ret;
+          pos += ret;
+        }
+        if (dfsClient.stats != null) {
+          dfsClient.stats.incrementBytesRead(result);
+        }
+        return result;
+      } finally {
+        // Check if need to report block replicas corruption either read
+        // was successful or ChecksumException occured.
+        reportCheckSumFailure(corruptedBlockMap,
+            currentLocatedBlock.getLocations().length);
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Copy the data from {@link #curStripeBuf} into the given buffer
+   * @param strategy the ReaderStrategy containing the given buffer
+   * @param offset the offset of the given buffer. Used only when strategy is
+   *               a ByteArrayStrategy
+   * @param length target length
+   * @return number of bytes copied
+   */
+  private int copyToTargetBuf(ReaderStrategy strategy, int offset, int length) {
+    final long offsetInBlk = getOffsetInBlockGroup();
+    int bufOffset = getStripedBufOffset(offsetInBlk);
+    curStripeBuf.position(bufOffset);
+    return strategy.copyFrom(curStripeBuf, offset,
+        Math.min(length, curStripeBuf.remaining()));
+  }
+
+  /**
+   * The super method {@link DFSInputStream#refreshLocatedBlock} refreshes
+   * cached LocatedBlock by executing {@link DFSInputStream#getBlockAt} again.
+   * This method extends the logic by first remembering the index of the
+   * internal block, and re-parsing the refreshed block group with the same
+   * index.
+   */
+  @Override
+  protected LocatedBlock refreshLocatedBlock(LocatedBlock block)
+      throws IOException {
+    int idx = StripedBlockUtil.getBlockIndex(block.getBlock().getLocalBlock());
+    LocatedBlock lb = getBlockGroupAt(block.getStartOffset());
+    // If indexing information is returned, iterate through the index array
+    // to find the entry for position idx in the group
+    LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
+    int i = 0;
+    for (; i < lsb.getBlockIndices().length; i++) {
+      if (lsb.getBlockIndices()[i] == idx) {
+        break;
+      }
+    }
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("refreshLocatedBlock for striped blocks, offset="
+          + block.getStartOffset() + ". Obtained block " + lb + ", idx=" + idx);
+    }
+    return StripedBlockUtil.constructInternalBlock(
+        lsb, i, cellSize, dataBlkNum, idx);
+  }
+
+  private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException {
+    LocatedBlock lb = super.getBlockAt(offset);
+    assert lb instanceof LocatedStripedBlock : "NameNode" +
+        " should return a LocatedStripedBlock for a striped file";
+    return (LocatedStripedBlock)lb;
+  }
+
+  /**
+   * Real implementation of pread.
+   */
+  @Override
+  protected void fetchBlockByteRange(LocatedBlock block, long start,
+      long end, byte[] buf, int offset,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    // Refresh the striped block group
+    LocatedStripedBlock blockGroup = getBlockGroupAt(block.getStartOffset());
+
+    AlignedStripe[] stripes = StripedBlockUtil.divideByteRangeIntoStripes(
+        ecPolicy, cellSize, blockGroup, start, end, buf, offset);
+    CompletionService<Void> readService = new ExecutorCompletionService<>(
+        dfsClient.getStripedReadsThreadPool());
+    final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(
+        blockGroup, cellSize, dataBlkNum, parityBlkNum);
+    final BlockReaderInfo[] preaderInfos = new BlockReaderInfo[groupSize];
+    try {
+      for (AlignedStripe stripe : stripes) {
+        // Parse group to get chosen DN location
+        StripeReader preader = new PositionStripeReader(readService, stripe,
+            blks, preaderInfos, corruptedBlockMap);
+        preader.readStripe();
+      }
+    } finally {
+      for (BlockReaderInfo preaderInfo : preaderInfos) {
+        closeReader(preaderInfo);
+      }
+    }
+  }
+
+  @Override
+  protected void reportLostBlock(LocatedBlock lostBlock,
+      Collection<DatanodeInfo> ignoredNodes) {
+    DatanodeInfo[] nodes = lostBlock.getLocations();
+    if (nodes != null && nodes.length > 0) {
+      List<String> dnUUIDs = new ArrayList<>();
+      for (DatanodeInfo node : nodes) {
+        dnUUIDs.add(node.getDatanodeUuid());
+      }
+      if (!warnedNodes.containsAll(dnUUIDs)) {
+        DFSClient.LOG.warn(Arrays.toString(nodes) + " are unavailable and " +
+            "all striping blocks on them are lost. " +
+            "IgnoredNodes = " + ignoredNodes);
+        warnedNodes.addAll(dnUUIDs);
+      }
+    } else {
+      super.reportLostBlock(lostBlock, ignoredNodes);
+    }
+  }
+
+  /**
+   * The reader for reading a complete {@link AlignedStripe}. Note that an
+   * {@link AlignedStripe} may cross multiple stripes with cellSize width.
+   */
+  private abstract class StripeReader {
+    final Map<Future<Void>, Integer> futures = new HashMap<>();
+    final AlignedStripe alignedStripe;
+    final CompletionService<Void> service;
+    final LocatedBlock[] targetBlocks;
+    final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap;
+    final BlockReaderInfo[] readerInfos;
+
+    StripeReader(CompletionService<Void> service, AlignedStripe alignedStripe,
+        LocatedBlock[] targetBlocks, BlockReaderInfo[] readerInfos,
+        Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
+      this.service = service;
+      this.alignedStripe = alignedStripe;
+      this.targetBlocks = targetBlocks;
+      this.readerInfos = readerInfos;
+      this.corruptedBlockMap = corruptedBlockMap;
+    }
+
+    /** prepare all the data chunks */
+    abstract void prepareDecodeInputs();
+
+    /** prepare the parity chunk and block reader if necessary */
+    abstract boolean prepareParityChunk(int index) throws IOException;
+
+    abstract void decode();
+
+    void updateState4SuccessRead(StripingChunkReadResult result) {
+      Preconditions.checkArgument(
+          result.state == StripingChunkReadResult.SUCCESSFUL);
+      readerInfos[result.index].setOffset(alignedStripe.getOffsetInBlock()
+          + alignedStripe.getSpanInBlock());
+    }
+
+    private void checkMissingBlocks() throws IOException {
+      if (alignedStripe.missingChunksNum > parityBlkNum) {
+        clearFutures(futures.keySet());
+        throw new IOException(alignedStripe.missingChunksNum
+            + " missing blocks, the stripe is: " + alignedStripe);
+      }
+    }
+
+    /**
+     * We need decoding. Thus go through all the data chunks and make sure we
+     * submit read requests for all of them.
+     */
+    private void readDataForDecoding() throws IOException {
+      prepareDecodeInputs();
+      for (int i = 0; i < dataBlkNum; i++) {
+        Preconditions.checkNotNull(alignedStripe.chunks[i]);
+        if (alignedStripe.chunks[i].state == StripingChunk.REQUESTED) {
+          if (!readChunk(targetBlocks[i], i)) {
+            alignedStripe.missingChunksNum++;
+          }
+        }
+      }
+      checkMissingBlocks();
+    }
+
+    void readParityChunks(int num) throws IOException {
+      for (int i = dataBlkNum, j = 0; i < dataBlkNum + parityBlkNum && j < num;
+           i++) {
+        if (alignedStripe.chunks[i] == null) {
+          if (prepareParityChunk(i) && readChunk(targetBlocks[i], i)) {
+            j++;
+          } else {
+            alignedStripe.missingChunksNum++;
+          }
+        }
+      }
+      checkMissingBlocks();
+    }
+
+    boolean createBlockReader(LocatedBlock block, int chunkIndex)
+        throws IOException {
+      BlockReader reader = null;
+      final ReaderRetryPolicy retry = new ReaderRetryPolicy();
+      DNAddrPair dnInfo = new DNAddrPair(null, null, null);
+
+      while(true) {
+        try {
+          // the cached block location might have been re-fetched, so always
+          // get it from cache.
+          block = refreshLocatedBlock(block);
+          targetBlocks[chunkIndex] = block;
+
+          // internal block has one location, just rule out the deadNodes
+          dnInfo = getBestNodeDNAddrPair(block, null);
+          if (dnInfo == null) {
+            break;
+          }
+          reader = getBlockReader(block, alignedStripe.getOffsetInBlock(),
+              block.getBlockSize() - alignedStripe.getOffsetInBlock(),
+              dnInfo.addr, dnInfo.storageType, dnInfo.info);
+        } catch (IOException e) {
+          if (e instanceof InvalidEncryptionKeyException &&
+              retry.shouldRefetchEncryptionKey()) {
+            DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+                + "encryption key was invalid when connecting to " + dnInfo.addr
+                + " : " + e);
+            dfsClient.clearDataEncryptionKey();
+            retry.refetchEncryptionKey();
+          } else if (retry.shouldRefetchToken() &&
+              tokenRefetchNeeded(e, dnInfo.addr)) {
+            fetchBlockAt(block.getStartOffset());
+            retry.refetchToken();
+          } else {
+            //TODO: handles connection issues
+            DFSClient.LOG.warn("Failed to connect to " + dnInfo.addr + " for " +
+                "block" + block.getBlock(), e);
+            // re-fetch the block in case the block has been moved
+            fetchBlockAt(block.getStartOffset());
+            addToDeadNodes(dnInfo.info);
+          }
+        }
+        if (reader != null) {
+          readerInfos[chunkIndex] = new BlockReaderInfo(reader, dnInfo.info,
+              alignedStripe.getOffsetInBlock());
+          return true;
+        }
+      }
+      return false;
+    }
+
+    private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) {
+      if (chunk.byteBuffer != null) {
+        ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer);
+        return new ByteBufferStrategy[]{strategy};
+      } else {
+        ByteBufferStrategy[] strategies =
+            new ByteBufferStrategy[chunk.byteArray.getOffsets().length];
+        for (int i = 0; i < strategies.length; i++) {
+          ByteBuffer buffer = ByteBuffer.wrap(chunk.byteArray.buf(),
+              chunk.byteArray.getOffsets()[i], chunk.byteArray.getLengths()[i]);
+          strategies[i] = new ByteBufferStrategy(buffer);
+        }
+        return strategies;
+      }
+    }
+
+    boolean readChunk(final LocatedBlock block, int chunkIndex)
+        throws IOException {
+      final StripingChunk chunk = alignedStripe.chunks[chunkIndex];
+      if (block == null) {
+        chunk.state = StripingChunk.MISSING;
+        return false;
+      }
+      if (readerInfos[chunkIndex] == null) {
+        if (!createBlockReader(block, chunkIndex)) {
+          chunk.state = StripingChunk.MISSING;
+          return false;
+        }
+      } else if (readerInfos[chunkIndex].shouldSkip) {
+        chunk.state = StripingChunk.MISSING;
+        return false;
+      }
+
+      chunk.state = StripingChunk.PENDING;
+      Callable<Void> readCallable = readCells(readerInfos[chunkIndex].reader,
+          readerInfos[chunkIndex].datanode,
+          readerInfos[chunkIndex].blockReaderOffset,
+          alignedStripe.getOffsetInBlock(), getReadStrategies(chunk),
+          block.getBlock(), corruptedBlockMap);
+
+      Future<Void> request = service.submit(readCallable);
+      futures.put(request, chunkIndex);
+      return true;
+    }
+
+    /** read the whole stripe. do decoding if necessary */
+    void readStripe() throws IOException {
+      for (int i = 0; i < dataBlkNum; i++) {
+        if (alignedStripe.chunks[i] != null &&
+            alignedStripe.chunks[i].state != StripingChunk.ALLZERO) {
+          if (!readChunk(targetBlocks[i], i)) {
+            alignedStripe.missingChunksNum++;
+          }
+        }
+      }
+      // There are missing block locations at this stage. Thus we need to read
+      // the full stripe and one more parity block.
+      if (alignedStripe.missingChunksNum > 0) {
+        checkMissingBlocks();
+        readDataForDecoding();
+        // read parity chunks
+        readParityChunks(alignedStripe.missingChunksNum);
+      }
+      // TODO: for a full stripe we can start reading (dataBlkNum + 1) chunks
+
+      // Input buffers for potential decode operation, which remains null until
+      // first read failure
+      while (!futures.isEmpty()) {
+        try {
+          StripingChunkReadResult r = StripedBlockUtil
+              .getNextCompletedStripedRead(service, futures, 0);
+          if (DFSClient.LOG.isDebugEnabled()) {
+            DFSClient.LOG.debug("Read task returned: " + r + ", for stripe "
+                + alignedStripe);
+          }
+          StripingChunk returnedChunk = alignedStripe.chunks[r.index];
+          Preconditions.checkNotNull(returnedChunk);
+          Preconditions.checkState(returnedChunk.state == StripingChunk.PENDING);
+
+          if (r.state == StripingChunkReadResult.SUCCESSFUL) {
+            returnedChunk.state = StripingChunk.FETCHED;
+            alignedStripe.fetchedChunksNum++;
+            updateState4SuccessRead(r);
+            if (alignedStripe.fetchedChunksNum == dataBlkNum) {
+              clearFutures(futures.keySet());
+              break;
+            }
+          } else {
+            returnedChunk.state = StripingChunk.MISSING;
+            // close the corresponding reader
+            closeReader(readerInfos[r.index]);
+
+            final int missing = alignedStripe.missingChunksNum;
+            alignedStripe.missingChunksNum++;
+            checkMissingBlocks();
+
+            readDataForDecoding();
+            readParityChunks(alignedStripe.missingChunksNum - missing);
+          }
+        } catch (InterruptedException ie) {
+          String err = "Read request interrupted";
+          DFSClient.LOG.error(err);
+          clearFutures(futures.keySet());
+          // Don't decode if read interrupted
+          throw new InterruptedIOException(err);
+        }
+      }
+
+      if (alignedStripe.missingChunksNum > 0) {
+        decode();
+      }
+    }
+  }
+
+  class PositionStripeReader extends StripeReader {
+    private byte[][] decodeInputs = null;
+
+    PositionStripeReader(CompletionService<Void> service,
+        AlignedStripe alignedStripe, LocatedBlock[] targetBlocks,
+        BlockReaderInfo[] readerInfos,
+        Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
+      super(service, alignedStripe, targetBlocks, readerInfos,
+          corruptedBlockMap);
+    }
+
+    @Override
+    void prepareDecodeInputs() {
+      if (decodeInputs == null) {
+        decodeInputs = StripedBlockUtil.initDecodeInputs(alignedStripe,
+            dataBlkNum, parityBlkNum);
+      }
+    }
+
+    @Override
+    boolean prepareParityChunk(int index) {
+      Preconditions.checkState(index >= dataBlkNum &&
+          alignedStripe.chunks[index] == null);
+      final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index,
+          dataBlkNum, parityBlkNum);
+      alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]);
+      alignedStripe.chunks[index].addByteArraySlice(0,
+          (int) alignedStripe.getSpanInBlock());
+      return true;
+    }
+
+    @Override
+    void decode() {
+      StripedBlockUtil.finalizeDecodeInputs(decodeInputs, dataBlkNum,
+          parityBlkNum, alignedStripe);
+      StripedBlockUtil.decodeAndFillBuffer(decodeInputs, alignedStripe,
+          dataBlkNum, parityBlkNum, decoder);
+    }
+  }
+
+  class StatefulStripeReader extends StripeReader {
+    ByteBuffer[] decodeInputs;
+
+    StatefulStripeReader(CompletionService<Void> service,
+        AlignedStripe alignedStripe, LocatedBlock[] targetBlocks,
+        BlockReaderInfo[] readerInfos,
+        Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
+      super(service, alignedStripe, targetBlocks, readerInfos,
+          corruptedBlockMap);
+    }
+
+    @Override
+    void prepareDecodeInputs() {
+      if (decodeInputs == null) {
+        decodeInputs = new ByteBuffer[dataBlkNum + parityBlkNum];
+        final ByteBuffer cur;
+        synchronized (DFSStripedInputStream.this) {
+          cur = curStripeBuf.duplicate();
+        }
+        StripedBlockUtil.VerticalRange range = alignedStripe.range;
+        for (int i = 0; i < dataBlkNum; i++) {
+          cur.limit(cur.capacity());
+          int pos = (int) (range.offsetInBlock % cellSize + cellSize * i);
+          cur.position(pos);
+          cur.limit((int) (pos + range.spanInBlock));
+          final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
+              dataBlkNum, parityBlkNum);
+          decodeInputs[decodeIndex] = cur.slice();
+          if (alignedStripe.chunks[i] == null) {
+            alignedStripe.chunks[i] = new StripingChunk(
+                decodeInputs[decodeIndex]);
+          }
+        }
+      }
+    }
+
+    @Override
+    boolean prepareParityChunk(int index) throws IOException {
+      Preconditions.checkState(index >= dataBlkNum
+          && alignedStripe.chunks[index] == null);
+      if (blockReaders[index] != null && blockReaders[index].shouldSkip) {
+        alignedStripe.chunks[index] = new StripingChunk(StripingChunk.MISSING);
+        // we have failed the block reader before
+        return false;
+      }
+      final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index,
+          dataBlkNum, parityBlkNum);
+      ByteBuffer buf = getParityBuffer().duplicate();
+      buf.position(cellSize * decodeIndex);
+      buf.limit(cellSize * decodeIndex + (int) alignedStripe.range.spanInBlock);
+      decodeInputs[decodeIndex] = buf.slice();
+      alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]);
+      return true;
+    }
+
+    @Override
+    void decode() {
+      // TODO no copy for data chunks. this depends on HADOOP-12047
+      final int span = (int) alignedStripe.getSpanInBlock();
+      for (int i = 0; i < alignedStripe.chunks.length; i++) {
+        final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
+            dataBlkNum, parityBlkNum);
+        if (alignedStripe.chunks[i] != null &&
+            alignedStripe.chunks[i].state == StripingChunk.ALLZERO) {
+          for (int j = 0; j < span; j++) {
+            decodeInputs[decodeIndex].put((byte) 0);
+          }
+          decodeInputs[decodeIndex].flip();
+        } else if (alignedStripe.chunks[i] != null &&
+            alignedStripe.chunks[i].state == StripingChunk.FETCHED) {
+          decodeInputs[decodeIndex].position(0);
+          decodeInputs[decodeIndex].limit(span);
+        }
+      }
+      int[] decodeIndices = new int[parityBlkNum];
+      int pos = 0;
+      for (int i = 0; i < alignedStripe.chunks.length; i++) {
+        if (alignedStripe.chunks[i] != null &&
+            alignedStripe.chunks[i].state == StripingChunk.MISSING) {
+          int  decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
+              dataBlkNum, parityBlkNum);
+          if (i < dataBlkNum) {
+            decodeIndices[pos++] = decodeIndex;
+          } else {
+            decodeInputs[decodeIndex] = null;
+          }
+        }
+      }
+      decodeIndices = Arrays.copyOf(decodeIndices, pos);
+
+      final int decodeChunkNum = decodeIndices.length;
+      ByteBuffer[] outputs = new ByteBuffer[decodeChunkNum];
+      for (int i = 0; i < decodeChunkNum; i++) {
+        outputs[i] = decodeInputs[decodeIndices[i]];
+        outputs[i].position(0);
+        outputs[i].limit((int) alignedStripe.range.spanInBlock);
+        decodeInputs[decodeIndices[i]] = null;
+      }
+
+      decoder.decode(decodeInputs, decodeIndices, outputs);
+    }
+  }
+
+  /**
+   * May need online read recovery, zero-copy read doesn't make
+   * sense, so don't support it.
+   */
+  @Override
+  public synchronized ByteBuffer read(ByteBufferPool bufferPool,
+      int maxLength, EnumSet<ReadOption> opts)
+          throws IOException, UnsupportedOperationException {
+    throw new UnsupportedOperationException(
+        "Not support enhanced byte buffer access.");
+  }
+
+  @Override
+  public synchronized void releaseBuffer(ByteBuffer buffer) {
+    throw new UnsupportedOperationException(
+        "Not support enhanced byte buffer access.");
+  }
+
+  /** A variation to {@link DFSInputStream#cancelAll} */
+  private void clearFutures(Collection<Future<Void>> futures) {
+    for (Future<Void> future : futures) {
+      future.cancel(false);
+    }
+    futures.clear();
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
new file mode 100644
index 0000000..bf4e10e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -0,0 +1,953 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
+
+import com.google.common.base.Preconditions;
+import org.apache.htrace.core.TraceScope;
+
+
+/**
+ * This class supports writing files in striped layout and erasure coded format.
+ * Each stripe contains a sequence of cells.
+ */
+@InterfaceAudience.Private
+public class DFSStripedOutputStream extends DFSOutputStream {
+  static class MultipleBlockingQueue<T> {
+    private final List<BlockingQueue<T>> queues;
+
+    MultipleBlockingQueue(int numQueue, int queueSize) {
+      List<BlockingQueue<T>> list = new ArrayList<>(numQueue);
+      for (int i = 0; i < numQueue; i++) {
+        list.add(new LinkedBlockingQueue<T>(queueSize));
+      }
+      queues = Collections.synchronizedList(list);
+    }
+
+    void offer(int i, T object) {
+      final boolean b = queues.get(i).offer(object);
+      Preconditions.checkState(b, "Failed to offer " + object
+          + " to queue, i=" + i);
+    }
+
+    T take(int i) throws InterruptedIOException {
+      try {
+        return queues.get(i).take();
+      } catch(InterruptedException ie) {
+        throw DFSUtilClient.toInterruptedIOException("take interrupted, i=" + i, ie);
+      }
+    }
+
+    T takeWithTimeout(int i) throws InterruptedIOException {
+      try {
+        return queues.get(i).poll(100, TimeUnit.MILLISECONDS);
+      } catch (InterruptedException e) {
+        throw DFSUtilClient.toInterruptedIOException("take interrupted, i=" + i, e);
+      }
+    }
+
+    T poll(int i) {
+      return queues.get(i).poll();
+    }
+
+    T peek(int i) {
+      return queues.get(i).peek();
+    }
+
+    void clear() {
+      for (BlockingQueue<T> q : queues) {
+        q.clear();
+      }
+    }
+  }
+
+  /** Coordinate the communication between the streamers. */
+  static class Coordinator {
+    /**
+     * The next internal block to write to for each streamers. The
+     * DFSStripedOutputStream makes the {@link ClientProtocol#addBlock} RPC to
+     * get a new block group. The block group is split to internal blocks, which
+     * are then distributed into the queue for streamers to retrieve.
+     */
+    private final MultipleBlockingQueue<LocatedBlock> followingBlocks;
+    /**
+     * Used to sync among all the streamers before allocating a new block. The
+     * DFSStripedOutputStream uses this to make sure every streamer has finished
+     * writing the previous block.
+     */
+    private final MultipleBlockingQueue<ExtendedBlock> endBlocks;
+
+    /**
+     * The following data structures are used for syncing while handling errors
+     */
+    private final MultipleBlockingQueue<LocatedBlock> newBlocks;
+    private final Map<StripedDataStreamer, Boolean> updateStreamerMap;
+    private final MultipleBlockingQueue<Boolean> streamerUpdateResult;
+
+    Coordinator(final int numAllBlocks) {
+      followingBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
+      endBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
+      newBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
+      updateStreamerMap = Collections.synchronizedMap(
+          new HashMap<StripedDataStreamer, Boolean>(numAllBlocks));
+      streamerUpdateResult = new MultipleBlockingQueue<>(numAllBlocks, 1);
+    }
+
+    MultipleBlockingQueue<LocatedBlock> getFollowingBlocks() {
+      return followingBlocks;
+    }
+
+    MultipleBlockingQueue<LocatedBlock> getNewBlocks() {
+      return newBlocks;
+    }
+
+    void offerEndBlock(int i, ExtendedBlock block) {
+      endBlocks.offer(i, block);
+    }
+
+    void offerStreamerUpdateResult(int i, boolean success) {
+      streamerUpdateResult.offer(i, success);
+    }
+
+    boolean takeStreamerUpdateResult(int i) throws InterruptedIOException {
+      return streamerUpdateResult.take(i);
+    }
+
+    void updateStreamer(StripedDataStreamer streamer,
+        boolean success) {
+      assert !updateStreamerMap.containsKey(streamer);
+      updateStreamerMap.put(streamer, success);
+    }
+
+    void clearFailureStates() {
+      newBlocks.clear();
+      updateStreamerMap.clear();
+      streamerUpdateResult.clear();
+    }
+  }
+
+  /** Buffers for writing the data and parity cells of a stripe. */
+  class CellBuffers {
+    private final ByteBuffer[] buffers;
+    private final byte[][] checksumArrays;
+
+    CellBuffers(int numParityBlocks) throws InterruptedException{
+      if (cellSize % bytesPerChecksum != 0) {
+        throw new HadoopIllegalArgumentException("Invalid values: "
+            + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (="
+            + bytesPerChecksum + ") must divide cell size (=" + cellSize + ").");
+      }
+
+      checksumArrays = new byte[numParityBlocks][];
+      final int size = getChecksumSize() * (cellSize / bytesPerChecksum);
+      for (int i = 0; i < checksumArrays.length; i++) {
+        checksumArrays[i] = new byte[size];
+      }
+
+      buffers = new ByteBuffer[numAllBlocks];
+      for (int i = 0; i < buffers.length; i++) {
+        buffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
+      }
+    }
+
+    private ByteBuffer[] getBuffers() {
+      return buffers;
+    }
+
+    byte[] getChecksumArray(int i) {
+      return checksumArrays[i - numDataBlocks];
+    }
+
+    private int addTo(int i, byte[] b, int off, int len) {
+      final ByteBuffer buf = buffers[i];
+      final int pos = buf.position() + len;
+      Preconditions.checkState(pos <= cellSize);
+      buf.put(b, off, len);
+      return pos;
+    }
+
+    private void clear() {
+      for (int i = 0; i< numAllBlocks; i++) {
+        buffers[i].clear();
+        if (i >= numDataBlocks) {
+          Arrays.fill(buffers[i].array(), (byte) 0);
+        }
+      }
+    }
+
+    private void release() {
+      for (int i = 0; i < numAllBlocks; i++) {
+        byteArrayManager.release(buffers[i].array());
+      }
+    }
+
+    private void flipDataBuffers() {
+      for (int i = 0; i < numDataBlocks; i++) {
+        buffers[i].flip();
+      }
+    }
+  }
+
+  private final Coordinator coordinator;
+  private final CellBuffers cellBuffers;
+  private final RawErasureEncoder encoder;
+  private final List<StripedDataStreamer> streamers;
+  private final DFSPacket[] currentPackets; // current Packet of each streamer
+
+  /** Size of each striping cell, must be a multiple of bytesPerChecksum */
+  private final int cellSize;
+  private final int numAllBlocks;
+  private final int numDataBlocks;
+  private ExtendedBlock currentBlockGroup;
+  private final String[] favoredNodes;
+  private final List<StripedDataStreamer> failedStreamers;
+
+  /** Construct a new output stream for creating a file. */
+  DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
+                         EnumSet<CreateFlag> flag, Progressable progress,
+                         DataChecksum checksum, String[] favoredNodes)
+                         throws IOException {
+    super(dfsClient, src, stat, flag, progress, checksum, favoredNodes, false);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating DFSStripedOutputStream for " + src);
+    }
+
+    final ErasureCodingPolicy ecPolicy = stat.getErasureCodingPolicy();
+    final int numParityBlocks = ecPolicy.getNumParityUnits();
+    cellSize = ecPolicy.getCellSize();
+    numDataBlocks = ecPolicy.getNumDataUnits();
+    numAllBlocks = numDataBlocks + numParityBlocks;
+    this.favoredNodes = favoredNodes;
+    failedStreamers = new ArrayList<>();
+
+    encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(),
+        numDataBlocks, numParityBlocks);
+
+    coordinator = new Coordinator(numAllBlocks);
+    try {
+      cellBuffers = new CellBuffers(numParityBlocks);
+    } catch (InterruptedException ie) {
+      throw DFSUtilClient.toInterruptedIOException(
+          "Failed to create cell buffers", ie);
+    }
+
+    streamers = new ArrayList<>(numAllBlocks);
+    for (short i = 0; i < numAllBlocks; i++) {
+      StripedDataStreamer streamer = new StripedDataStreamer(stat,
+          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
+          favoredNodes, i, coordinator);
+      streamers.add(streamer);
+    }
+    currentPackets = new DFSPacket[streamers.size()];
+    setCurrentStreamer(0);
+  }
+
+  StripedDataStreamer getStripedDataStreamer(int i) {
+    return streamers.get(i);
+  }
+
+  int getCurrentIndex() {
+    return getCurrentStreamer().getIndex();
+  }
+
+  private synchronized StripedDataStreamer getCurrentStreamer() {
+    return (StripedDataStreamer) streamer;
+  }
+
+  private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) {
+    // backup currentPacket for current streamer
+    if (streamer != null) {
+      int oldIdx = streamers.indexOf(getCurrentStreamer());
+      if (oldIdx >= 0) {
+        currentPackets[oldIdx] = currentPacket;
+      }
+    }
+
+    streamer = getStripedDataStreamer(newIdx);
+    currentPacket = currentPackets[newIdx];
+    adjustChunkBoundary();
+
+    return getCurrentStreamer();
+  }
+
+  /**
+   * Encode the buffers, i.e. compute parities.
+   *
+   * @param buffers data buffers + parity buffers
+   */
+  private static void encode(RawErasureEncoder encoder, int numData,
+      ByteBuffer[] buffers) {
+    final ByteBuffer[] dataBuffers = new ByteBuffer[numData];
+    final ByteBuffer[] parityBuffers = new ByteBuffer[buffers.length - numData];
+    System.arraycopy(buffers, 0, dataBuffers, 0, dataBuffers.length);
+    System.arraycopy(buffers, numData, parityBuffers, 0, parityBuffers.length);
+
+    encoder.encode(dataBuffers, parityBuffers);
+  }
+
+  /**
+   * check all the existing StripedDataStreamer and find newly failed streamers.
+   * @return The newly failed streamers.
+   * @throws IOException if less than {@link #numDataBlocks} streamers are still
+   *                     healthy.
+   */
+  private Set<StripedDataStreamer> checkStreamers() throws IOException {
+    Set<StripedDataStreamer> newFailed = new HashSet<>();
+    for(StripedDataStreamer s : streamers) {
+      if (!s.isHealthy() && !failedStreamers.contains(s)) {
+        newFailed.add(s);
+      }
+    }
+
+    final int failCount = failedStreamers.size() + newFailed.size();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("checkStreamers: " + streamers);
+      LOG.debug("healthy streamer count=" + (numAllBlocks - failCount));
+      LOG.debug("original failed streamers: " + failedStreamers);
+      LOG.debug("newly failed streamers: " + newFailed);
+    }
+    if (failCount > (numAllBlocks - numDataBlocks)) {
+      throw new IOException("Failed: the number of failed blocks = "
+          + failCount + " > the number of data blocks = "
+          + (numAllBlocks - numDataBlocks));
+    }
+    return newFailed;
+  }
+
+  private void handleStreamerFailure(String err, Exception e)
+      throws IOException {
+    LOG.warn("Failed: " + err + ", " + this, e);
+    getCurrentStreamer().getErrorState().setInternalError();
+    getCurrentStreamer().close(true);
+    checkStreamers();
+    currentPacket = null;
+  }
+
+  private void replaceFailedStreamers() {
+    assert streamers.size() == numAllBlocks;
+    for (short i = 0; i < numAllBlocks; i++) {
+      final StripedDataStreamer oldStreamer = getStripedDataStreamer(i);
+      if (!oldStreamer.isHealthy()) {
+        StripedDataStreamer streamer = new StripedDataStreamer(oldStreamer.stat,
+            dfsClient, src, oldStreamer.progress,
+            oldStreamer.checksum4WriteBlock, cachingStrategy, byteArrayManager,
+            favoredNodes, i, coordinator);
+        streamers.set(i, streamer);
+        currentPackets[i] = null;
+        if (i == 0) {
+          this.streamer = streamer;
+        }
+        streamer.start();
+      }
+    }
+  }
+
+  private void waitEndBlocks(int i) throws IOException {
+    while (getStripedDataStreamer(i).isHealthy()) {
+      final ExtendedBlock b = coordinator.endBlocks.takeWithTimeout(i);
+      if (b != null) {
+        StripedBlockUtil.checkBlocks(currentBlockGroup, i, b);
+        return;
+      }
+    }
+  }
+
+  private void allocateNewBlock() throws IOException {
+    if (currentBlockGroup != null) {
+      for (int i = 0; i < numAllBlocks; i++) {
+        // sync all the healthy streamers before writing to the new block
+        waitEndBlocks(i);
+      }
+    }
+    failedStreamers.clear();
+    // replace failed streamers
+    replaceFailedStreamers();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Allocating new block group. The previous block group: "
+          + currentBlockGroup);
+    }
+
+    // TODO collect excludedNodes from all the data streamers
+    final LocatedBlock lb = addBlock(null, dfsClient, src, currentBlockGroup,
+        fileId, favoredNodes);
+    assert lb.isStriped();
+    if (lb.getLocations().length < numDataBlocks) {
+      throw new IOException("Failed to get " + numDataBlocks
+          + " nodes from namenode: blockGroupSize= " + numAllBlocks
+          + ", blocks.length= " + lb.getLocations().length);
+    }
+    // assign the new block to the current block group
+    currentBlockGroup = lb.getBlock();
+
+    final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
+        (LocatedStripedBlock) lb, cellSize, numDataBlocks,
+        numAllBlocks - numDataBlocks);
+    for (int i = 0; i < blocks.length; i++) {
+      StripedDataStreamer si = getStripedDataStreamer(i);
+      if (si.isHealthy()) { // skipping failed data streamer
+        if (blocks[i] == null) {
+          // Set exception and close streamer as there is no block locations
+          // found for the parity block.
+          LOG.warn("Failed to get block location for parity block, index=" + i);
+          si.getLastException().set(
+              new IOException("Failed to get following block, i=" + i));
+          si.getErrorState().setInternalError();
+          si.close(true);
+        } else {
+          coordinator.getFollowingBlocks().offer(i, blocks[i]);
+        }
+      }
+    }
+  }
+
+  private boolean shouldEndBlockGroup() {
+    return currentBlockGroup != null &&
+        currentBlockGroup.getNumBytes() == blockSize * numDataBlocks;
+  }
+
+  @Override
+  protected synchronized void writeChunk(byte[] bytes, int offset, int len,
+      byte[] checksum, int ckoff, int cklen) throws IOException {
+    final int index = getCurrentIndex();
+    final StripedDataStreamer current = getCurrentStreamer();
+    final int pos = cellBuffers.addTo(index, bytes, offset, len);
+    final boolean cellFull = pos == cellSize;
+
+    if (currentBlockGroup == null || shouldEndBlockGroup()) {
+      // the incoming data should belong to a new block. Allocate a new block.
+      allocateNewBlock();
+    }
+
+    currentBlockGroup.setNumBytes(currentBlockGroup.getNumBytes() + len);
+    if (current.isHealthy()) {
+      try {
+        super.writeChunk(bytes, offset, len, checksum, ckoff, cklen);
+      } catch(Exception e) {
+        handleStreamerFailure("offset=" + offset + ", length=" + len, e);
+      }
+    }
+
+    // Two extra steps are needed when a striping cell is full:
+    // 1. Forward the current index pointer
+    // 2. Generate parity packets if a full stripe of data cells are present
+    if (cellFull) {
+      int next = index + 1;
+      //When all data cells in a stripe are ready, we need to encode
+      //them and generate some parity cells. These cells will be
+      //converted to packets and put to their DataStreamer's queue.
+      if (next == numDataBlocks) {
+        cellBuffers.flipDataBuffers();
+        writeParityCells();
+        next = 0;
+        // check failure state for all the streamers. Bump GS if necessary
+        checkStreamerFailures();
+
+        // if this is the end of the block group, end each internal block
+        if (shouldEndBlockGroup()) {
+          for (int i = 0; i < numAllBlocks; i++) {
+            final StripedDataStreamer s = setCurrentStreamer(i);
+            if (s.isHealthy()) {
+              try {
+                endBlock();
+              } catch (IOException ignored) {}
+            }
+          }
+        }
+      }
+      setCurrentStreamer(next);
+    }
+  }
+
+  @Override
+  void enqueueCurrentPacketFull() throws IOException {
+    LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
+            + " appendChunk={}, {}", currentPacket, src, getStreamer()
+            .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
+        getStreamer());
+    enqueueCurrentPacket();
+    adjustChunkBoundary();
+    // no need to end block here
+  }
+
+  private Set<StripedDataStreamer> markExternalErrorOnStreamers() {
+    Set<StripedDataStreamer> healthySet = new HashSet<>();
+    for (StripedDataStreamer streamer : streamers) {
+      if (streamer.isHealthy() &&
+          streamer.getStage() == BlockConstructionStage.DATA_STREAMING) {
+        streamer.setExternalError();
+        healthySet.add(streamer);
+      }
+    }
+    return healthySet;
+  }
+
+  /**
+   * Check and handle data streamer failures. This is called only when we have
+   * written a full stripe (i.e., enqueue all packets for a full stripe), or
+   * when we're closing the outputstream.
+   */
+  private void checkStreamerFailures() throws IOException {
+    Set<StripedDataStreamer> newFailed = checkStreamers();
+    if (newFailed.size() > 0) {
+      // for healthy streamers, wait till all of them have fetched the new block
+      // and flushed out all the enqueued packets.
+      flushAllInternals();
+    }
+    // get all the current failed streamers after the flush
+    newFailed = checkStreamers();
+    while (newFailed.size() > 0) {
+      failedStreamers.addAll(newFailed);
+      coordinator.clearFailureStates();
+
+      // mark all the healthy streamers as external error
+      Set<StripedDataStreamer> healthySet = markExternalErrorOnStreamers();
+
+      // we have newly failed streamers, update block for pipeline
+      final ExtendedBlock newBG = updateBlockForPipeline(healthySet);
+
+      // wait till all the healthy streamers to
+      // 1) get the updated block info
+      // 2) create new block outputstream
+      newFailed = waitCreatingNewStreams(healthySet);
+      if (newFailed.size() + failedStreamers.size() >
+          numAllBlocks - numDataBlocks) {
+        throw new IOException(
+            "Data streamers failed while creating new block streams: "
+                + newFailed + ". There are not enough healthy streamers.");
+      }
+      for (StripedDataStreamer failedStreamer : newFailed) {
+        assert !failedStreamer.isHealthy();
+      }
+
+      // TODO we can also succeed if all the failed streamers have not taken
+      // the updated block
+      if (newFailed.size() == 0) {
+        // reset external error state of all the streamers
+        for (StripedDataStreamer streamer : healthySet) {
+          assert streamer.isHealthy();
+          streamer.getErrorState().reset();
+        }
+        updatePipeline(newBG);
+      }
+      for (int i = 0; i < numAllBlocks; i++) {
+        coordinator.offerStreamerUpdateResult(i, newFailed.size() == 0);
+      }
+    }
+  }
+
+  private int checkStreamerUpdates(Set<StripedDataStreamer> failed,
+      Set<StripedDataStreamer> streamers) {
+    for (StripedDataStreamer streamer : streamers) {
+      if (!coordinator.updateStreamerMap.containsKey(streamer)) {
+        if (!streamer.isHealthy() &&
+            coordinator.getNewBlocks().peek(streamer.getIndex()) != null) {
+          // this streamer had internal error before getting updated block
+          failed.add(streamer);
+        }
+      }
+    }
+    return coordinator.updateStreamerMap.size() + failed.size();
+  }
+
+  private Set<StripedDataStreamer> waitCreatingNewStreams(
+      Set<StripedDataStreamer> healthyStreamers) throws IOException {
+    Set<StripedDataStreamer> failed = new HashSet<>();
+    final int expectedNum = healthyStreamers.size();
+    final long socketTimeout = dfsClient.getConf().getSocketTimeout();
+    // the total wait time should be less than the socket timeout, otherwise
+    // a slow streamer may cause other streamers to timeout. here we wait for
+    // half of the socket timeout
+    long remaingTime = socketTimeout > 0 ? socketTimeout/2 : Long.MAX_VALUE;
+    final long waitInterval = 1000;
+    synchronized (coordinator) {
+      while (checkStreamerUpdates(failed, healthyStreamers) < expectedNum
+          && remaingTime > 0) {
+        try {
+          long start = Time.monotonicNow();
+          coordinator.wait(waitInterval);
+          remaingTime -= Time.monotonicNow() - start;
+        } catch (InterruptedException e) {
+          throw DFSUtilClient.toInterruptedIOException("Interrupted when waiting" +
+              " for results of updating striped streamers", e);
+        }
+      }
+    }
+    synchronized (coordinator) {
+      for (StripedDataStreamer streamer : healthyStreamers) {
+        if (!coordinator.updateStreamerMap.containsKey(streamer)) {
+          // close the streamer if it is too slow to create new connection
+          streamer.setStreamerAsClosed();
+          failed.add(streamer);
+        }
+      }
+    }
+    for (Map.Entry<StripedDataStreamer, Boolean> entry :
+        coordinator.updateStreamerMap.entrySet()) {
+      if (!entry.getValue()) {
+        failed.add(entry.getKey());
+      }
+    }
+    for (StripedDataStreamer failedStreamer : failed) {
+      healthyStreamers.remove(failedStreamer);
+    }
+    return failed;
+  }
+
+  /**
+   * Call {@link ClientProtocol#updateBlockForPipeline} and assign updated block
+   * to healthy streamers.
+   * @param healthyStreamers The healthy data streamers. These streamers join
+   *                         the failure handling.
+   */
+  private ExtendedBlock updateBlockForPipeline(
+      Set<StripedDataStreamer> healthyStreamers) throws IOException {
+    final LocatedBlock updated = dfsClient.namenode.updateBlockForPipeline(
+        currentBlockGroup, dfsClient.clientName);
+    final long newGS = updated.getBlock().getGenerationStamp();
+    ExtendedBlock newBlock = new ExtendedBlock(currentBlockGroup);
+    newBlock.setGenerationStamp(newGS);
+    final LocatedBlock[] updatedBlks = StripedBlockUtil.parseStripedBlockGroup(
+        (LocatedStripedBlock) updated, cellSize, numDataBlocks,
+        numAllBlocks - numDataBlocks);
+
+    for (int i = 0; i < numAllBlocks; i++) {
+      StripedDataStreamer si = getStripedDataStreamer(i);
+      if (healthyStreamers.contains(si)) {
+        final LocatedBlock lb = new LocatedBlock(new ExtendedBlock(newBlock),
+            null, null, null, -1, updated.isCorrupt(), null);
+        lb.setBlockToken(updatedBlks[i].getBlockToken());
+        coordinator.getNewBlocks().offer(i, lb);
+      }
+    }
+    return newBlock;
+  }
+
+  private void updatePipeline(ExtendedBlock newBG) throws IOException {
+    final DatanodeInfo[] newNodes = new DatanodeInfo[numAllBlocks];
+    final String[] newStorageIDs = new String[numAllBlocks];
+    for (int i = 0; i < numAllBlocks; i++) {
+      final StripedDataStreamer streamer = getStripedDataStreamer(i);
+      final DatanodeInfo[] nodes = streamer.getNodes();
+      final String[] storageIDs = streamer.getStorageIDs();
+      if (streamer.isHealthy() && nodes != null && storageIDs != null) {
+        newNodes[i] = nodes[0];
+        newStorageIDs[i] = storageIDs[0];
+      } else {
+        newNodes[i] = new DatanodeInfo(DatanodeID.EMPTY_DATANODE_ID);
+        newStorageIDs[i] = "";
+      }
+    }
+    dfsClient.namenode.updatePipeline(dfsClient.clientName, currentBlockGroup,
+        newBG, newNodes, newStorageIDs);
+    currentBlockGroup = newBG;
+  }
+
+  private int stripeDataSize() {
+    return numDataBlocks * cellSize;
+  }
+
+  @Override
+  public void hflush() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void hsync() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected synchronized void start() {
+    for (StripedDataStreamer streamer : streamers) {
+      streamer.start();
+    }
+  }
+
+  @Override
+  synchronized void abort() throws IOException {
+    if (isClosed()) {
+      return;
+    }
+    for (StripedDataStreamer streamer : streamers) {
+      streamer.getLastException().set(new IOException("Lease timeout of "
+          + (dfsClient.getConf().getHdfsTimeout()/1000) +
+          " seconds expired."));
+    }
+    closeThreads(true);
+    dfsClient.endFileLease(fileId);
+  }
+
+  @Override
+  boolean isClosed() {
+    if (closed) {
+      return true;
+    }
+    for(StripedDataStreamer s : streamers) {
+      if (!s.streamerClosed()) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  protected void closeThreads(boolean force) throws IOException {
+    final MultipleIOException.Builder b = new MultipleIOException.Builder();
+    try {
+      for (StripedDataStreamer streamer : streamers) {
+        try {
+          streamer.close(force);
+          streamer.join();
+          streamer.closeSocket();
+        } catch (Exception e) {
+          try {
+            handleStreamerFailure("force=" + force, e);
+          } catch (IOException ioe) {
+            b.add(ioe);
+          }
+        } finally {
+          streamer.setSocketToNull();
+        }
+      }
+    } finally {
+      setClosed();
+    }
+    final IOException ioe = b.build();
+    if (ioe != null) {
+      throw ioe;
+    }
+  }
+
+  private boolean generateParityCellsForLastStripe() {
+    final long currentBlockGroupBytes = currentBlockGroup == null ?
+        0 : currentBlockGroup.getNumBytes();
+    final long lastStripeSize = currentBlockGroupBytes % stripeDataSize();
+    if (lastStripeSize == 0) {
+      return false;
+    }
+
+    final long parityCellSize = lastStripeSize < cellSize?
+        lastStripeSize : cellSize;
+    final ByteBuffer[] buffers = cellBuffers.getBuffers();
+
+    for (int i = 0; i < numAllBlocks; i++) {
+      // Pad zero bytes to make all cells exactly the size of parityCellSize
+      // If internal block is smaller than parity block, pad zero bytes.
+      // Also pad zero bytes to all parity cells
+      final int position = buffers[i].position();
+      assert position <= parityCellSize : "If an internal block is smaller" +
+          " than parity block, then its last cell should be small than last" +
+          " parity cell";
+      for (int j = 0; j < parityCellSize - position; j++) {
+        buffers[i].put((byte) 0);
+      }
+      buffers[i].flip();
+    }
+    return true;
+  }
+
+  void writeParityCells() throws IOException {
+    final ByteBuffer[] buffers = cellBuffers.getBuffers();
+    //encode the data cells
+    encode(encoder, numDataBlocks, buffers);
+    for (int i = numDataBlocks; i < numAllBlocks; i++) {
+      writeParity(i, buffers[i], cellBuffers.getChecksumArray(i));
+    }
+    cellBuffers.clear();
+  }
+
+  void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf)
+      throws IOException {
+    final StripedDataStreamer current = setCurrentStreamer(index);
+    final int len = buffer.limit();
+
+    final long oldBytes = current.getBytesCurBlock();
+    if (current.isHealthy()) {
+      try {
+        DataChecksum sum = getDataChecksum();
+        sum.calculateChunkedSums(buffer.array(), 0, len, checksumBuf, 0);
+        for (int i = 0; i < len; i += sum.getBytesPerChecksum()) {
+          int chunkLen = Math.min(sum.getBytesPerChecksum(), len - i);
+          int ckOffset = i / sum.getBytesPerChecksum() * getChecksumSize();
+          super.writeChunk(buffer.array(), i, chunkLen, checksumBuf, ckOffset,
+              getChecksumSize());
+        }
+      } catch(Exception e) {
+        handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e);
+      }
+    }
+  }
+
+  @Override
+  void setClosed() {
+    super.setClosed();
+    for (int i = 0; i < numAllBlocks; i++) {
+      getStripedDataStreamer(i).release();
+    }
+    cellBuffers.release();
+  }
+
+  @Override
+  protected synchronized void closeImpl() throws IOException {
+    if (isClosed()) {
+      final MultipleIOException.Builder b = new MultipleIOException.Builder();
+      for(int i = 0; i < streamers.size(); i++) {
+        final StripedDataStreamer si = getStripedDataStreamer(i);
+        try {
+          si.getLastException().check(true);
+        } catch (IOException e) {
+          b.add(e);
+        }
+      }
+      final IOException ioe = b.build();
+      if (ioe != null) {
+        throw ioe;
+      }
+      return;
+    }
+
+    try {
+      // flush from all upper layers
+      flushBuffer();
+      // if the last stripe is incomplete, generate and write parity cells
+      if (generateParityCellsForLastStripe()) {
+        writeParityCells();
+      }
+      enqueueAllCurrentPackets();
+
+      // flush all the data packets
+      flushAllInternals();
+      // check failures
+      checkStreamerFailures();
+
+      for (int i = 0; i < numAllBlocks; i++) {
+        final StripedDataStreamer s = setCurrentStreamer(i);
+        if (s.isHealthy()) {
+          try {
+            if (s.getBytesCurBlock() > 0) {
+              setCurrentPacketToEmpty();
+            }
+            // flush the last "close" packet to Datanode
+            flushInternal();
+          } catch(Exception e) {
+            // TODO for both close and endBlock, we currently do not handle
+            // failures when sending the last packet. We actually do not need to
+            // bump GS for this kind of failure. Thus counting the total number
+            // of failures may be good enough.
+          }
+        }
+      }
+
+      closeThreads(false);
+      TraceScope scope = dfsClient.getTracer().newScope("completeFile");
+      try {
+        completeFile(currentBlockGroup);
+      } finally {
+        scope.close();
+      }
+      dfsClient.endFileLease(fileId);
+    } catch (ClosedChannelException ignored) {
+    } finally {
+      setClosed();
+    }
+  }
+
+  private void enqueueAllCurrentPackets() throws IOException {
+    int idx = streamers.indexOf(getCurrentStreamer());
+    for(int i = 0; i < streamers.size(); i++) {
+      final StripedDataStreamer si = setCurrentStreamer(i);
+      if (si.isHealthy() && currentPacket != null) {
+        try {
+          enqueueCurrentPacket();
+        } catch (IOException e) {
+          handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e);
+        }
+      }
+    }
+    setCurrentStreamer(idx);
+  }
+
+  void flushAllInternals() throws IOException {
+    int current = getCurrentIndex();
+
+    for (int i = 0; i < numAllBlocks; i++) {
+      final StripedDataStreamer s = setCurrentStreamer(i);
+      if (s.isHealthy()) {
+        try {
+          // flush all data to Datanode
+          flushInternal();
+        } catch(Exception e) {
+          handleStreamerFailure("flushInternal " + s, e);
+        }
+      }
+    }
+    setCurrentStreamer(current);
+  }
+
+  static void sleep(long ms, String op) throws InterruptedIOException {
+    try {
+      Thread.sleep(ms);
+    } catch(InterruptedException ie) {
+      throw DFSUtilClient.toInterruptedIOException(
+          "Sleep interrupted during " + op, ie);
+    }
+  }
+
+  @Override
+  ExtendedBlock getBlock() {
+    return currentBlockGroup;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index 71ce7cc..d1829d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -54,6 +54,7 @@
 
 import javax.net.SocketFactory;
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.io.UnsupportedEncodingException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -646,4 +647,11 @@
     return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
         + namenode.getHostName() + portString);
   }
+
+  public static InterruptedIOException toInterruptedIOException(String message,
+      InterruptedException e) {
+    final InterruptedIOException iioe = new InterruptedIOException(message);
+    iioe.initCause(e);
+    return iioe;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 4ea1f415..683d98d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -22,7 +22,6 @@
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InterruptedIOException;
@@ -45,16 +44,12 @@
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
@@ -68,13 +63,10 @@
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MultipleIOException;
-import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
@@ -166,7 +158,7 @@
     packets.clear();
   }
   
-  static class LastExceptionInStreamer {
+  class LastExceptionInStreamer {
     private IOException thrown;
 
     synchronized void set(Throwable t) {
@@ -184,7 +176,8 @@
       if (thrown != null) {
         if (LOG.isTraceEnabled()) {
           // wrap and print the exception to know when the check is called
-          LOG.trace("Got Exception while checking", new Throwable(thrown));
+          LOG.trace("Got Exception while checking, " + DataStreamer.this,
+              new Throwable(thrown));
         }
         final IOException e = thrown;
         if (resetToNull) {
@@ -200,8 +193,12 @@
     }
   }
 
+  enum ErrorType {
+    NONE, INTERNAL, EXTERNAL
+  }
+
   static class ErrorState {
-    private boolean error = false;
+    ErrorType error = ErrorType.NONE;
     private int badNodeIndex = -1;
     private int restartingNodeIndex = -1;
     private long restartingNodeDeadline = 0;
@@ -211,23 +208,46 @@
       this.datanodeRestartTimeout = datanodeRestartTimeout;
     }
 
-    synchronized void reset() {
-      error = false;
+    synchronized void resetInternalError() {
+      if (hasInternalError()) {
+        error = ErrorType.NONE;
+      }
       badNodeIndex = -1;
       restartingNodeIndex = -1;
       restartingNodeDeadline = 0;
     }
 
+    synchronized void reset() {
+      error = ErrorType.NONE;
+      badNodeIndex = -1;
+      restartingNodeIndex = -1;
+      restartingNodeDeadline = 0;
+    }
+
+    synchronized boolean hasInternalError() {
+      return error == ErrorType.INTERNAL;
+    }
+
+    synchronized boolean hasExternalError() {
+      return error == ErrorType.EXTERNAL;
+    }
+
     synchronized boolean hasError() {
-      return error;
+      return error != ErrorType.NONE;
     }
 
     synchronized boolean hasDatanodeError() {
-      return error && isNodeMarked();
+      return error == ErrorType.INTERNAL && isNodeMarked();
     }
 
-    synchronized void setError(boolean err) {
-      this.error = err;
+    synchronized void setInternalError() {
+      this.error = ErrorType.INTERNAL;
+    }
+
+    synchronized void setExternalError() {
+      if (!hasInternalError()) {
+        this.error = ErrorType.EXTERNAL;
+      }
     }
 
     synchronized void setBadNodeIndex(int index) {
@@ -290,14 +310,14 @@
       }
 
       if (!isRestartingNode()) {
-        error = false;
+        error = ErrorType.NONE;
       }
       badNodeIndex = -1;
     }
 
     synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) {
       if (restartingNodeIndex >= 0) {
-        if (!error) {
+        if (error == ErrorType.NONE) {
           throw new IllegalStateException("error=false while checking" +
               " restarting node deadline");
         }
@@ -328,8 +348,8 @@
   }
 
   private volatile boolean streamerClosed = false;
-  private ExtendedBlock block; // its length is number of bytes acked
-  private Token<BlockTokenIdentifier> accessToken;
+  protected ExtendedBlock block; // its length is number of bytes acked
+  protected Token<BlockTokenIdentifier> accessToken;
   private DataOutputStream blockStream;
   private DataInputStream blockReplyStream;
   private ResponseProcessor response = null;
@@ -339,7 +359,7 @@
   private final ErrorState errorState;
 
   private BlockConstructionStage stage;  // block construction stage
-  private long bytesSent = 0; // number of bytes that've been sent
+  protected long bytesSent = 0; // number of bytes that've been sent
   private final boolean isLazyPersistFile;
 
   /** Nodes have been used in the pipeline before and have failed. */
@@ -359,16 +379,16 @@
   private final LastExceptionInStreamer lastException = new LastExceptionInStreamer();
   private Socket s;
 
-  private final DFSClient dfsClient;
-  private final String src;
+  protected final DFSClient dfsClient;
+  protected final String src;
   /** Only for DataTransferProtocol.writeBlock(..) */
-  private final DataChecksum checksum4WriteBlock;
-  private final Progressable progress;
-  private final HdfsFileStatus stat;
+  final DataChecksum checksum4WriteBlock;
+  final Progressable progress;
+  protected final HdfsFileStatus stat;
   // appending to existing partial block
   private volatile boolean appendChunk = false;
   // both dataQueue and ackQueue are protected by dataQueue lock
-  private final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
+  protected final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
   private final LinkedList<DFSPacket> ackQueue = new LinkedList<>();
   private final AtomicReference<CachingStrategy> cachingStrategy;
   private final ByteArrayManager byteArrayManager;
@@ -385,14 +405,16 @@
       CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10;
   private int lastCongestionBackoffTime;
 
-  private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
+  protected final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
   private final String[] favoredNodes;
 
-  private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src,
+  private DataStreamer(HdfsFileStatus stat, ExtendedBlock block,
+                       DFSClient dfsClient, String src,
                        Progressable progress, DataChecksum checksum,
                        AtomicReference<CachingStrategy> cachingStrategy,
                        ByteArrayManager byteArrayManage,
                        boolean isAppend, String[] favoredNodes) {
+    this.block = block;
     this.dfsClient = dfsClient;
     this.src = src;
     this.progress = progress;
@@ -417,9 +439,8 @@
                String src, Progressable progress, DataChecksum checksum,
                AtomicReference<CachingStrategy> cachingStrategy,
                ByteArrayManager byteArrayManage, String[] favoredNodes) {
-    this(stat, dfsClient, src, progress, checksum, cachingStrategy,
+    this(stat, block, dfsClient, src, progress, checksum, cachingStrategy,
         byteArrayManage, false, favoredNodes);
-    this.block = block;
     stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
   }
 
@@ -433,10 +454,9 @@
                String src, Progressable progress, DataChecksum checksum,
                AtomicReference<CachingStrategy> cachingStrategy,
                ByteArrayManager byteArrayManage) throws IOException {
-    this(stat, dfsClient, src, progress, checksum, cachingStrategy,
+    this(stat, lastBlock.getBlock(), dfsClient, src, progress, checksum, cachingStrategy,
         byteArrayManage, true, null);
     stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
-    block = lastBlock.getBlock();
     bytesSent = block.getNumBytes();
     accessToken = lastBlock.getBlockToken();
   }
@@ -457,6 +477,10 @@
     }
   }
 
+  void setAccessToken(Token<BlockTokenIdentifier> t) {
+    this.accessToken = t;
+  }
+
   private void setPipeline(LocatedBlock lb) {
     setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
   }
@@ -479,8 +503,8 @@
     stage = BlockConstructionStage.DATA_STREAMING;
   }
 
-  private void endBlock() {
-    LOG.debug("Closing old block {}", block);
+  protected void endBlock() {
+    LOG.debug("Closing old block " + block);
     this.setName("DataStreamer for file " + src);
     closeResponder();
     closeStream();
@@ -515,7 +539,7 @@
       DFSPacket one;
       try {
         // process datanode IO errors if any
-        boolean doSleep = processDatanodeError();
+        boolean doSleep = processDatanodeOrExternalError();
 
         final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2; 
         synchronized (dataQueue) {
@@ -560,6 +584,9 @@
         }
 
         // get new block from namenode.
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("stage=" + stage + ", " + this);
+        }
         if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
           LOG.debug("Allocating new block");
           setPipeline(nextBlockOutputStream());
@@ -576,10 +603,7 @@
         long lastByteOffsetInBlock = one.getLastByteOffsetBlock();
         if (lastByteOffsetInBlock > stat.getBlockSize()) {
           throw new IOException("BlockSize " + stat.getBlockSize() +
-              " is smaller than data size. " +
-              " Offset of packet in block " +
-              lastByteOffsetInBlock +
-              " Aborting file " + src);
+              " < lastByteOffsetInBlock, " + this + ", " + one);
         }
 
         if (one.isLastPacketInBlock()) {
@@ -617,7 +641,7 @@
           }
         }
 
-        LOG.debug("DataStreamer block {} sending packet {}", block, one);
+        LOG.debug(this + " sending " + one);
 
         // write out data to remote datanode
         TraceScope writeScope = dfsClient.getTracer().
@@ -682,7 +706,7 @@
         }
         lastException.set(e);
         assert !(e instanceof NullPointerException);
-        errorState.setError(true);
+        errorState.setInternalError();
         if (!errorState.isNodeMarked()) {
           // Not a datanode issue
           streamerClosed = true;
@@ -825,6 +849,9 @@
     }
   }
 
+  void setStreamerAsClosed() {
+    streamerClosed = true;
+  }
 
   private void checkClosed() throws IOException {
     if (streamerClosed) {
@@ -845,7 +872,7 @@
     }
   }
 
-  private void closeStream() {
+  void closeStream() {
     final MultipleIOException.Builder b = new MultipleIOException.Builder();
 
     if (blockStream != null) {
@@ -1028,7 +1055,7 @@
         } catch (Exception e) {
           if (!responderClosed) {
             lastException.set(e);
-            errorState.setError(true);
+            errorState.setInternalError();
             errorState.markFirstNodeIfNotMarked();
             synchronized (dataQueue) {
               dataQueue.notifyAll();
@@ -1053,14 +1080,18 @@
     }
   }
 
+  private boolean shouldHandleExternalError(){
+    return errorState.hasExternalError() && blockStream != null;
+  }
+
   /**
    * If this stream has encountered any errors, shutdown threads
    * and mark the stream as closed.
    *
    * @return true if it should sleep for a while after returning.
    */
-  private boolean processDatanodeError() throws IOException {
-    if (!errorState.hasDatanodeError()) {
+  private boolean processDatanodeOrExternalError() throws IOException {
+    if (!errorState.hasDatanodeError() && !shouldHandleExternalError()) {
       return false;
     }
     if (response != null) {
@@ -1093,7 +1124,8 @@
         return false;
       }
     }
-    boolean doSleep = setupPipelineForAppendOrRecovery();
+
+    setupPipelineForAppendOrRecovery();
 
     if (!streamerClosed && dfsClient.clientRunning) {
       if (stage == BlockConstructionStage.PIPELINE_CLOSE) {
@@ -1126,7 +1158,7 @@
       }
     }
 
-    return doSleep;
+    return false;
   }
 
   void setHflush() {
@@ -1283,7 +1315,7 @@
    * This happens when a file is appended or data streaming fails
    * It keeps on trying until a pipeline is setup
    */
-  private boolean setupPipelineForAppendOrRecovery() throws IOException {
+  private void setupPipelineForAppendOrRecovery() throws IOException {
     // check number of datanodes
     if (nodes == null || nodes.length == 0) {
       String msg = "Could not get block locations. " + "Source file \""
@@ -1291,19 +1323,23 @@
       LOG.warn(msg);
       lastException.set(new IOException(msg));
       streamerClosed = true;
-      return false;
+      return;
     }
+    setupPipelineInternal(nodes, storageTypes);
+  }
 
+  protected void setupPipelineInternal(DatanodeInfo[] datanodes,
+      StorageType[] nodeStorageTypes) throws IOException {
     boolean success = false;
     long newGS = 0L;
     while (!success && !streamerClosed && dfsClient.clientRunning) {
       if (!handleRestartingDatanode()) {
-        return false;
+        return;
       }
 
-      final boolean isRecovery = errorState.hasError();
+      final boolean isRecovery = errorState.hasInternalError();
       if (!handleBadDatanode()) {
-        return false;
+        return;
       }
 
       handleDatanodeReplacement();
@@ -1324,7 +1360,6 @@
     if (success) {
       block = updatePipeline(newGS);
     }
-    return false; // do not sleep, continue processing
   }
 
   /**
@@ -1332,7 +1367,7 @@
    * This process is repeated until the deadline or the node starts back up.
    * @return true if it should continue.
    */
-  private boolean handleRestartingDatanode() {
+  boolean handleRestartingDatanode() {
     if (errorState.isRestartingNode()) {
       // 4 seconds or the configured deadline period, whichever is shorter.
       // This is the retry interval and recovery will be retried in this
@@ -1355,7 +1390,7 @@
    * Remove bad node from list of nodes if badNodeIndex was set.
    * @return true if it should continue.
    */
-  private boolean handleBadDatanode() {
+  boolean handleBadDatanode() {
     final int badNodeIndex = errorState.getBadNodeIndex();
     if (badNodeIndex >= 0) {
       if (nodes.length <= 1) {
@@ -1405,7 +1440,7 @@
     }
   }
 
-  private void failPacket4Testing() {
+  void failPacket4Testing() {
     if (failPacket) { // for testing
       failPacket = false;
       try {
@@ -1417,37 +1452,43 @@
     }
   }
 
-  LocatedBlock updateBlockForPipeline() throws IOException {
-    return dfsClient.namenode.updateBlockForPipeline(
-        block, dfsClient.clientName);
+  private LocatedBlock updateBlockForPipeline() throws IOException {
+    return dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName);
+  }
+
+  static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) {
+    return new ExtendedBlock(b.getBlockPoolId(), b.getBlockId(),
+        b.getNumBytes(), newGS);
   }
 
   /** update pipeline at the namenode */
   ExtendedBlock updatePipeline(long newGS) throws IOException {
-    final ExtendedBlock newBlock = new ExtendedBlock(
-        block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
+    final ExtendedBlock newBlock = newBlock(block, newGS);
     dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
         nodes, storageIDs);
     return newBlock;
   }
 
+  private int getNumBlockWriteRetry() {
+    return dfsClient.getConf().getNumBlockWriteRetry();
+  }
+
   /**
    * Open a DataStreamer to a DataNode so that it can be written to.
    * This happens when a file is created and each time a new block is allocated.
    * Must get block ID and the IDs of the destinations from the namenode.
    * Returns the list of target datanodes.
    */
-  private LocatedBlock nextBlockOutputStream() throws IOException {
+  protected LocatedBlock nextBlockOutputStream() throws IOException {
     LocatedBlock lb = null;
     DatanodeInfo[] nodes = null;
     StorageType[] storageTypes = null;
-    int count = dfsClient.getConf().getNumBlockWriteRetry();
+    int count = getNumBlockWriteRetry();
     boolean success = false;
     ExtendedBlock oldBlock = block;
     do {
-      errorState.reset();
+      errorState.resetInternalError();
       lastException.clear();
-      success = false;
 
       DatanodeInfo[] excluded =
           excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
@@ -1487,7 +1528,7 @@
   // connects to the first datanode in the pipeline
   // Returns true if success, otherwise return failure.
   //
-  private boolean createBlockOutputStream(DatanodeInfo[] nodes,
+  boolean createBlockOutputStream(DatanodeInfo[] nodes,
       StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
     if (nodes.length == 0) {
       LOG.info("nodes are empty for write pipeline of " + block);
@@ -1497,7 +1538,7 @@
     String firstBadLink = "";
     boolean checkRestart = false;
     if (LOG.isDebugEnabled()) {
-      LOG.debug("pipeline = " + Arrays.asList(nodes));
+      LOG.debug("pipeline = " + Arrays.toString(nodes) + ", " + this);
     }
 
     // persist blocks on namenode on next flush
@@ -1566,10 +1607,10 @@
         assert null == blockStream : "Previous blockStream unclosed";
         blockStream = out;
         result =  true; // success
-        errorState.reset();
+        errorState.resetInternalError();
       } catch (IOException ie) {
         if (!errorState.isRestartingNode()) {
-          LOG.info("Exception in createBlockOutputStream", ie);
+          LOG.info("Exception in createBlockOutputStream " + this, ie);
         }
         if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
           LOG.info("Will fetch a new encryption key and retry, "
@@ -1602,7 +1643,7 @@
         if (checkRestart && shouldWaitForRestart(i)) {
           errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
         }
-        errorState.setError(true);
+        errorState.setInternalError();
         lastException.set(ie);
         result =  false;  // error
       } finally {
@@ -1642,58 +1683,10 @@
     }
   }
 
-  protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
+  private LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
       throws IOException {
-    final DfsClientConf conf = dfsClient.getConf(); 
-    int retries = conf.getNumBlockWriteLocateFollowingRetry();
-    long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
-    while (true) {
-      long localstart = Time.monotonicNow();
-      while (true) {
-        try {
-          return dfsClient.namenode.addBlock(src, dfsClient.clientName,
-              block, excludedNodes, stat.getFileId(), favoredNodes);
-        } catch (RemoteException e) {
-          IOException ue =
-              e.unwrapRemoteException(FileNotFoundException.class,
-                  AccessControlException.class,
-                  NSQuotaExceededException.class,
-                  DSQuotaExceededException.class,
-                  QuotaByStorageTypeExceededException.class,
-                  UnresolvedPathException.class);
-          if (ue != e) {
-            throw ue; // no need to retry these exceptions
-          }
-
-
-          if (NotReplicatedYetException.class.getName().
-              equals(e.getClassName())) {
-            if (retries == 0) {
-              throw e;
-            } else {
-              --retries;
-              LOG.info("Exception while adding a block", e);
-              long elapsed = Time.monotonicNow() - localstart;
-              if (elapsed > 5000) {
-                LOG.info("Waiting for replication for "
-                    + (elapsed / 1000) + " seconds");
-              }
-              try {
-                LOG.warn("NotReplicatedYetException sleeping " + src
-                    + " retries left " + retries);
-                Thread.sleep(sleeptime);
-                sleeptime *= 2;
-              } catch (InterruptedException ie) {
-                LOG.warn("Caught exception", ie);
-              }
-            }
-          } else {
-            throw e;
-          }
-
-        }
-      }
-    }
+    return DFSOutputStream.addBlock(excludedNodes, dfsClient, src, block,
+        stat.getFileId(), favoredNodes);
   }
 
   /**
@@ -1748,6 +1741,14 @@
     return nodes;
   }
 
+  String[] getStorageIDs() {
+    return storageIDs;
+  }
+
+  BlockConstructionStage getStage() {
+    return stage;
+  }
+
   /**
    * return the token of the block
    *
@@ -1757,6 +1758,10 @@
     return accessToken;
   }
 
+  ErrorState getErrorState() {
+    return errorState;
+  }
+
   /**
    * Put a packet to the data queue
    *
@@ -1768,7 +1773,7 @@
       packet.addTraceParent(Tracer.getCurrentSpanId());
       dataQueue.addLast(packet);
       lastQueuedSeqno = packet.getSeqno();
-      LOG.debug("Queued packet {}", packet.getSeqno());
+      LOG.debug("Queued " + packet + ", " + this);
       dataQueue.notifyAll();
     }
   }
@@ -1920,7 +1925,6 @@
 
   @Override
   public String toString() {
-    return  (block == null? null: block.getLocalBlock())
-        + "@" + Arrays.toString(getNodes());
+    return block == null? "block==null": "" + block.getLocalBlock();
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index ea86d2d..18cc124 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -85,6 +85,7 @@
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Credentials;
@@ -2256,4 +2257,79 @@
       throws IOException {
     return dfs.getInotifyEventStream(lastReadTxid);
   }
+
+  /**
+   * Set the source path to the specified erasure coding policy.
+   *
+   * @param path     The directory to set the policy
+   * @param ecPolicy The erasure coding policy. If not specified default will be used.
+   * @throws IOException
+   */
+  public void setErasureCodingPolicy(final Path path, final ErasureCodingPolicy ecPolicy)
+      throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        dfs.setErasureCodingPolicy(getPathName(p), ecPolicy);
+        return null;
+      }
+
+      @Override
+      public Void next(final FileSystem fs, final Path p) throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          myDfs.setErasureCodingPolicy(p, ecPolicy);
+          return null;
+        }
+        throw new UnsupportedOperationException(
+            "Cannot setErasureCodingPolicy through a symlink to a "
+                + "non-DistributedFileSystem: " + path + " -> " + p);
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * Get erasure coding policy information for the specified path
+   *
+   * @param path The path of the file or directory
+   * @return Returns the policy information if file or directory on the path
+   * is erasure coded, null otherwise
+   * @throws IOException
+   */
+  public ErasureCodingPolicy getErasureCodingPolicy(final Path path)
+      throws IOException {
+    Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<ErasureCodingPolicy>() {
+      @Override
+      public ErasureCodingPolicy doCall(final Path p) throws IOException,
+          UnresolvedLinkException {
+        return dfs.getErasureCodingPolicy(getPathName(p));
+      }
+
+      @Override
+      public ErasureCodingPolicy next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem) fs;
+          return myDfs.getErasureCodingPolicy(p);
+        }
+        throw new UnsupportedOperationException(
+            "Cannot getErasureCodingPolicy through a symlink to a "
+                + "non-DistributedFileSystem: " + path + " -> " + p);
+      }
+    }.resolve(this, absF);
+  }
+
+  /**
+   * Retrieve all the erasure coding policies supported by this file system.
+   *
+   * @return all erasure coding policies supported by this file system.
+   * @throws IOException
+   */
+  public Collection<ErasureCodingPolicy> getAllErasureCodingPolicies()
+      throws IOException {
+    return Arrays.asList(dfs.getErasureCodingPolicies());
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
index 2eb9d52..015e5cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
@@ -24,6 +24,7 @@
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+import org.apache.hadoop.util.DataChecksum;
 
 /**
  * An ExternalBlockReader uses pluggable ReplicaAccessor objects to read from
@@ -123,4 +124,9 @@
     // For now, pluggable ReplicaAccessors do not support zero-copy.
     return null;
   }
+
+  @Override
+  public DataChecksum getDataChecksum() {
+    return null;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
index 017be9f..81ae829 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
@@ -509,4 +509,9 @@
   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
     return null;
   }
+
+  @Override
+  public DataChecksum getDataChecksum() {
+    return checksum;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index ca31e67..942c37c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@ -469,4 +469,9 @@
   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
     return null;
   }
+
+  @Override
+  public DataChecksum getDataChecksum() {
+    return checksum;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
new file mode 100644
index 0000000..a313ecb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -0,0 +1,187 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Progressable;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class extends {@link DataStreamer} to support writing striped blocks
+ * to datanodes.
+ * A {@link DFSStripedOutputStream} has multiple {@link StripedDataStreamer}s.
+ * Whenever the streamers need to talk the namenode, only the fastest streamer
+ * sends an rpc call to the namenode and then populates the result for the
+ * other streamers.
+ */
+public class StripedDataStreamer extends DataStreamer {
+  private final Coordinator coordinator;
+  private final int index;
+
+  StripedDataStreamer(HdfsFileStatus stat,
+                      DFSClient dfsClient, String src,
+                      Progressable progress, DataChecksum checksum,
+                      AtomicReference<CachingStrategy> cachingStrategy,
+                      ByteArrayManager byteArrayManage, String[] favoredNodes,
+                      short index, Coordinator coordinator) {
+    super(stat, null, dfsClient, src, progress, checksum, cachingStrategy,
+        byteArrayManage, favoredNodes);
+    this.index = index;
+    this.coordinator = coordinator;
+  }
+
+  int getIndex() {
+    return index;
+  }
+
+  boolean isHealthy() {
+    return !streamerClosed() && !getErrorState().hasInternalError();
+  }
+
+  @Override
+  protected void endBlock() {
+    coordinator.offerEndBlock(index, block);
+    super.endBlock();
+  }
+
+  /**
+   * The upper level DFSStripedOutputStream will allocate the new block group.
+   * All the striped data streamer only needs to fetch from the queue, which
+   * should be already be ready.
+   */
+  private LocatedBlock getFollowingBlock() throws IOException {
+    if (!this.isHealthy()) {
+      // No internal block for this streamer, maybe no enough healthy DN.
+      // Throw the exception which has been set by the StripedOutputStream.
+      this.getLastException().check(false);
+    }
+    return coordinator.getFollowingBlocks().poll(index);
+  }
+
+  @Override
+  protected LocatedBlock nextBlockOutputStream() throws IOException {
+    boolean success;
+    LocatedBlock lb = getFollowingBlock();
+    block = lb.getBlock();
+    block.setNumBytes(0);
+    bytesSent = 0;
+    accessToken = lb.getBlockToken();
+
+    DatanodeInfo[] nodes = lb.getLocations();
+    StorageType[] storageTypes = lb.getStorageTypes();
+
+    // Connect to the DataNode. If fail the internal error state will be set.
+    success = createBlockOutputStream(nodes, storageTypes, 0L, false);
+
+    if (!success) {
+      block = null;
+      final DatanodeInfo badNode = nodes[getErrorState().getBadNodeIndex()];
+      LOG.info("Excluding datanode " + badNode);
+      excludedNodes.put(badNode, badNode);
+      throw new IOException("Unable to create new block.");
+    }
+    return lb;
+  }
+
+  @VisibleForTesting
+  LocatedBlock peekFollowingBlock() {
+    return coordinator.getFollowingBlocks().peek(index);
+  }
+
+  @Override
+  protected void setupPipelineInternal(DatanodeInfo[] nodes,
+      StorageType[] nodeStorageTypes) throws IOException {
+    boolean success = false;
+    while (!success && !streamerClosed() && dfsClient.clientRunning) {
+      if (!handleRestartingDatanode()) {
+        return;
+      }
+      if (!handleBadDatanode()) {
+        // for striped streamer if it is datanode error then close the stream
+        // and return. no need to replace datanode
+        return;
+      }
+
+      // get a new generation stamp and an access token
+      final LocatedBlock lb = coordinator.getNewBlocks().take(index);
+      long newGS = lb.getBlock().getGenerationStamp();
+      setAccessToken(lb.getBlockToken());
+
+      // set up the pipeline again with the remaining nodes. when a striped
+      // data streamer comes here, it must be in external error state.
+      assert getErrorState().hasExternalError();
+      success = createBlockOutputStream(nodes, nodeStorageTypes, newGS, true);
+
+      failPacket4Testing();
+      getErrorState().checkRestartingNodeDeadline(nodes);
+
+      // notify coordinator the result of createBlockOutputStream
+      synchronized (coordinator) {
+        if (!streamerClosed()) {
+          coordinator.updateStreamer(this, success);
+          coordinator.notify();
+        } else {
+          success = false;
+        }
+      }
+
+      if (success) {
+        // wait for results of other streamers
+        success = coordinator.takeStreamerUpdateResult(index);
+        if (success) {
+          // if all succeeded, update its block using the new GS
+          block = newBlock(block, newGS);
+        } else {
+          // otherwise close the block stream and restart the recovery process
+          closeStream();
+        }
+      } else {
+        // if fail, close the stream. The internal error state and last
+        // exception have already been set in createBlockOutputStream
+        // TODO: wait for restarting DataNodes during RollingUpgrade
+        closeStream();
+        setStreamerAsClosed();
+      }
+    } // while
+  }
+
+  void setExternalError() {
+    getErrorState().setExternalError();
+    synchronized (dataQueue) {
+      dataQueue.notifyAll();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "#" + index + ": " + (!isHealthy() ? "failed, ": "") + super.toString();
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 992c374..416384e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -289,6 +289,18 @@
     int     THREADPOOL_SIZE_DEFAULT = 0;
   }
 
+  /** dfs.client.read.striped configuration properties */
+  interface StripedRead {
+    String PREFIX = Read.PREFIX + "striped.";
+
+    String  THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size";
+    /**
+     * With default RS-6-3-64k erasure coding policy, each normal read could span
+     * 6 DNs, so this default value accommodates 3 read streams
+     */
+    int     THREADPOOL_SIZE_DEFAULT = 18;
+  }
+
   /** dfs.http.client configuration properties */
   interface HttpClient {
     String  PREFIX = "dfs.http.client.";
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
index d46ab47..b99e3ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.client.impl;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -130,6 +131,9 @@
   private final List<Class<? extends ReplicaAccessorBuilder>>
       replicaAccessorBuilderClasses;
 
+  private final int stripedReadThreadpoolSize;
+
+
   public DfsClientConf(Configuration conf) {
     // The hdfsTimeout is currently the same as the ipc timeout
     hdfsTimeout = Client.getTimeout(conf);
@@ -237,6 +241,12 @@
         HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY,
         HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_DEFAULT);
 
+    stripedReadThreadpoolSize = conf.getInt(
+        HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY,
+        HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_DEFAULT);
+    Preconditions.checkArgument(stripedReadThreadpoolSize > 0, "The value of " +
+        HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY +
+        " must be greater than 0.");
     replicaAccessorBuilderClasses = loadReplicaAccessorBuilderClasses(conf);
   }
 
@@ -519,6 +529,13 @@
   }
 
   /**
+   * @return the stripedReadThreadpoolSize
+   */
+  public int getStripedReadThreadpoolSize() {
+    return stripedReadThreadpoolSize;
+  }
+
+  /**
    * @return the replicaAccessorBuilderClasses
    */
   public List<Class<? extends ReplicaAccessorBuilder>>
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 8528999..a0504a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1483,4 +1483,31 @@
    */
   @Idempotent
   EventBatchList getEditsFromTxid(long txid) throws IOException;
+
+  /**
+   * Set an erasure coding policy on a specified path.
+   * @param src The path to set policy on.
+   * @param ecPolicy The erasure coding policy. If null, default policy will
+   *                 be used
+   */
+  @AtMostOnce
+  void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
+      throws IOException;
+
+  /**
+   * Get the erasure coding policies loaded in Namenode
+   *
+   * @throws IOException
+   */
+  @Idempotent
+  ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException;
+
+  /**
+   * Get the information about the EC policy for the path
+   *
+   * @param src path to get the info for
+   * @throws IOException
+   */
+  @Idempotent
+  ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException;
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
index 6d72285..c709cbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
@@ -38,6 +38,8 @@
 @InterfaceStability.Evolving
 public class DatanodeID implements Comparable<DatanodeID> {
   public static final DatanodeID[] EMPTY_ARRAY = {};
+  public static final DatanodeID EMPTY_DATANODE_ID = new DatanodeID("null",
+      "null", "null", 0, 0, 0, 0);
 
   private String ipAddr;     // IP address
   private String hostName;   // hostname claimed by datanode
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
new file mode 100644
index 0000000..e5dfdff
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.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.hdfs.protocol;
+
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+import java.util.Map;
+
+/**
+ * A policy about how to write/read/code an erasure coding file.
+ */
+public final class ErasureCodingPolicy {
+
+  private final String name;
+  private final ECSchema schema;
+  private final int cellSize;
+
+  public ErasureCodingPolicy(String name, ECSchema schema, int cellSize){
+    this.name = name;
+    this.schema = schema;
+    this.cellSize = cellSize;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public ECSchema getSchema() {
+    return schema;
+  }
+
+  public int getCellSize() {
+    return cellSize;
+  }
+
+  public int getNumDataUnits() {
+    return schema.getNumDataUnits();
+  }
+
+  public int getNumParityUnits() {
+    return schema.getNumParityUnits();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ErasureCodingPolicy that = (ErasureCodingPolicy) o;
+
+    if (that.getName().equals(name) && that.getCellSize() == cellSize
+        && that.getSchema().equals(schema)) {
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = name.hashCode();
+    result = 31 * result + schema.hashCode();
+    result = 31 * result + cellSize;
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ErasureCodingPolicy=[");
+    sb.append("Name=" + name + ", ");
+    sb.append("Schema=[" + schema.toString() + "], ");
+    sb.append("CellSize=" + cellSize + " ");
+    sb.append("]");
+    return sb.toString();
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
index 34f429a..6e05ce0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
@@ -48,6 +48,8 @@
 
   private final FileEncryptionInfo feInfo;
 
+  private final ErasureCodingPolicy ecPolicy;
+  
   // Used by dir, not including dot and dotdot. Always zero for a regular file.
   private final int childrenNum;
   private final byte storagePolicy;
@@ -73,7 +75,7 @@
       long blocksize, long modification_time, long access_time,
       FsPermission permission, String owner, String group, byte[] symlink,
       byte[] path, long fileId, int childrenNum, FileEncryptionInfo feInfo,
-      byte storagePolicy) {
+      byte storagePolicy, ErasureCodingPolicy ecPolicy) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
@@ -93,6 +95,7 @@
     this.childrenNum = childrenNum;
     this.feInfo = feInfo;
     this.storagePolicy = storagePolicy;
+    this.ecPolicy = ecPolicy;
   }
 
   /**
@@ -250,6 +253,10 @@
     return feInfo;
   }
 
+  public ErasureCodingPolicy getErasureCodingPolicy() {
+    return ecPolicy;
+  }
+
   public final int getChildrenNum() {
     return childrenNum;
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
index 23e8f57..2121dcf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
@@ -58,10 +58,11 @@
       int block_replication, long blocksize, long modification_time,
       long access_time, FsPermission permission, String owner, String group,
       byte[] symlink, byte[] path, long fileId, LocatedBlocks locations,
-      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy) {
+      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy,
+      ErasureCodingPolicy ecPolicy) {
     super(length, isdir, block_replication, blocksize, modification_time,
         access_time, permission, owner, group, symlink, path, fileId,
-        childrenNum, feInfo, storagePolicy);
+        childrenNum, feInfo, storagePolicy, ecPolicy);
     this.locations = locations;
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
index cc13f10..a9596bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
@@ -49,14 +49,14 @@
   // 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) {
@@ -203,4 +203,8 @@
         + "; locs=" + Arrays.asList(locs)
         + "}";
   }
+
+  public boolean isStriped() {
+    return false;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
index e4896977..6e01bbe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
@@ -37,6 +37,7 @@
   private final LocatedBlock lastLocatedBlock;
   private final boolean isLastBlockComplete;
   private final FileEncryptionInfo fileEncryptionInfo;
+  private final ErasureCodingPolicy ecPolicy;
 
   public LocatedBlocks() {
     fileLength = 0;
@@ -45,17 +46,20 @@
     lastLocatedBlock = null;
     isLastBlockComplete = false;
     fileEncryptionInfo = null;
+    ecPolicy = null;
   }
 
   public LocatedBlocks(long flength, boolean isUnderConstuction,
-    List<LocatedBlock> blks, LocatedBlock lastBlock,
-    boolean isLastBlockCompleted, FileEncryptionInfo feInfo) {
+      List<LocatedBlock> blks, LocatedBlock lastBlock,
+      boolean isLastBlockCompleted, FileEncryptionInfo feInfo,
+      ErasureCodingPolicy ecPolicy) {
     fileLength = flength;
     blocks = blks;
     underConstruction = isUnderConstuction;
     this.lastLocatedBlock = lastBlock;
     this.isLastBlockComplete = isLastBlockCompleted;
     this.fileEncryptionInfo = feInfo;
+    this.ecPolicy = ecPolicy;
   }
 
   /**
@@ -112,6 +116,13 @@
   }
 
   /**
+   * @return The ECPolicy for ErasureCoded file, null otherwise.
+   */
+  public ErasureCodingPolicy getErasureCodingPolicy() {
+    return ecPolicy;
+  }
+
+  /**
    * Find block containing specified offset.
    *
    * @return block if found, or null otherwise.
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
new file mode 100644
index 0000000..a9a80c2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
@@ -0,0 +1,86 @@
+/**
+ * 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.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+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 static final int[] EMPTY_INDICES = {};
+  private static final Token<BlockTokenIdentifier> EMPTY_TOKEN = new Token<>();
+
+  private int[] blockIndices;
+  private Token<BlockTokenIdentifier>[] blockTokens;
+
+  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);
+
+    if (indices == null) {
+      this.blockIndices = EMPTY_INDICES;
+    } else {
+      this.blockIndices = new int[indices.length];
+      System.arraycopy(indices, 0, blockIndices, 0, indices.length);
+    }
+    blockTokens = new Token[blockIndices.length];
+    for (int i = 0; i < blockIndices.length; i++) {
+      blockTokens[i] = EMPTY_TOKEN;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "{" + getBlock()
+        + "; getBlockSize()=" + getBlockSize()
+        + "; corrupt=" + isCorrupt()
+        + "; offset=" + getStartOffset()
+        + "; locs=" + Arrays.asList(getLocations())
+        + "; indices=" + Arrays.toString(blockIndices)
+        + "}";
+  }
+
+  public int[] getBlockIndices() {
+    return this.blockIndices;
+  }
+
+  @Override
+  public boolean isStriped() {
+    return true;
+  }
+
+  public Token<BlockTokenIdentifier>[] getBlockTokens() {
+    return blockTokens;
+  }
+
+  public void setBlockTokens(Token<BlockTokenIdentifier>[] tokens) {
+    this.blockTokens = tokens;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
index ac19d44..813ea26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
@@ -61,7 +61,7 @@
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
         access_time, permission, owner, group, null, localName, inodeId,
-        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
     this.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index f4ce46d..7b02691 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -161,10 +161,16 @@
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
+import org.apache.hadoop.hdfs.protocol.proto.*;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
@@ -176,6 +182,7 @@
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolTranslator;
@@ -233,6 +240,10 @@
   VOID_GET_STORAGE_POLICIES_REQUEST =
       GetStoragePoliciesRequestProto.newBuilder().build();
 
+  private final static GetErasureCodingPoliciesRequestProto
+  VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto
+      .newBuilder().build();
+
   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
     rpcProxy = proxy;
   }
@@ -328,7 +339,7 @@
     try {
       AppendResponseProto res = rpcProxy.append(null, req);
       LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
-          .convert(res.getBlock()) : null;
+          .convertLocatedBlockProto(res.getBlock()) : null;
       HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
           : null;
       return new LastBlockWithStatus(lastBlock, stat);
@@ -416,7 +427,8 @@
       req.addAllFavoredNodes(Arrays.asList(favoredNodes));
     }
     try {
-      return PBHelperClient.convert(rpcProxy.addBlock(null, req.build()).getBlock());
+      return PBHelperClient.convertLocatedBlockProto(
+          rpcProxy.addBlock(null, req.build()).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -441,8 +453,8 @@
         .setClientName(clientName)
         .build();
     try {
-      return PBHelperClient.convert(rpcProxy.getAdditionalDatanode(null, req)
-          .getBlock());
+      return PBHelperClient.convertLocatedBlockProto(
+          rpcProxy.getAdditionalDatanode(null, req).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -469,7 +481,7 @@
   @Override
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
     ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
-        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
+        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlocks(blocks)))
         .build();
     try {
       rpcProxy.reportBadBlocks(null, req);
@@ -901,7 +913,7 @@
         .setClientName(clientName)
         .build();
     try {
-      return PBHelperClient.convert(
+      return PBHelperClient.convertLocatedBlockProto(
           rpcProxy.updateBlockForPipeline(null, req).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -1407,6 +1419,23 @@
   }
 
   @Override
+  public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
+      throws IOException {
+    final SetErasureCodingPolicyRequestProto.Builder builder =
+        SetErasureCodingPolicyRequestProto.newBuilder();
+    builder.setSrc(src);
+    if (ecPolicy != null) {
+      builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
+    }
+    SetErasureCodingPolicyRequestProto req = builder.build();
+    try {
+      rpcProxy.setErasureCodingPolicy(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
     SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()
@@ -1528,4 +1557,37 @@
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
+    try {
+      GetErasureCodingPoliciesResponseProto response = rpcProxy
+          .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST);
+      ErasureCodingPolicy[] ecPolicies =
+          new ErasureCodingPolicy[response.getEcPoliciesCount()];
+      int i = 0;
+      for (ErasureCodingPolicyProto ecPolicyProto : response.getEcPoliciesList()) {
+        ecPolicies[i++] = PBHelperClient.convertErasureCodingPolicy(ecPolicyProto);
+      }
+      return ecPolicies;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
+    GetErasureCodingPolicyRequestProto req = GetErasureCodingPolicyRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      GetErasureCodingPolicyResponseProto response = rpcProxy.getErasureCodingPolicy(
+          null, req);
+      if (response.hasEcPolicy()) {
+        return PBHelperClient.convertErasureCodingPolicy(response.getEcPolicy());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index ae0a3f6..a7cacc7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -23,7 +23,10 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -31,8 +34,6 @@
 import com.google.protobuf.ByteString;
 import com.google.protobuf.CodedInputStream;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-
 import org.apache.hadoop.crypto.CipherOption;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
@@ -71,6 +72,7 @@
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -81,6 +83,7 @@
 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;
@@ -112,6 +115,8 @@
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
@@ -128,6 +133,7 @@
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
@@ -161,6 +167,7 @@
 import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
@@ -497,7 +504,7 @@
     return null;
   }
 
-  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()];
@@ -517,8 +524,17 @@
       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<>(locs.size());
+    List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
     List<Boolean> isCachedList = proto.getIsCachedList();
     for (int i=0; i<isCachedList.size(); i++) {
       if (isCachedList.get(i)) {
@@ -526,9 +542,23 @@
       }
     }
 
-    LocatedBlock lb = new LocatedBlock(convert(proto.getB()), targets,
-        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
-        cachedLocs.toArray(new DatanodeInfo[0]));
+    final LocatedBlock lb;
+    if (indices == null) {
+      lb = new LocatedBlock(PBHelperClient.convert(proto.getB()), targets,
+          storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
+          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
+    } else {
+      lb = new LocatedStripedBlock(PBHelperClient.convert(proto.getB()), targets,
+          storageIDs, storageTypes, indices, proto.getOffset(),
+          proto.getCorrupt(),
+          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
+      List<TokenProto> tokenProtos = proto.getBlockTokensList();
+      Token<BlockTokenIdentifier>[] blockTokens = new Token[indices.length];
+      for (int i = 0; i < indices.length; i++) {
+        blockTokens[i] = convert(tokenProtos.get(i));
+      }
+      ((LocatedStripedBlock) lb).setBlockTokens(blockTokens);
+    }
     lb.setBlockToken(convert(proto.getBlockToken()));
 
     return lb;
@@ -591,11 +621,12 @@
   public static LocatedBlocks convert(LocatedBlocksProto lb) {
     return new LocatedBlocks(
         lb.getFileLength(), lb.getUnderConstruction(),
-        convertLocatedBlock(lb.getBlocksList()),
-        lb.hasLastBlock() ? convert(lb.getLastBlock()) : null,
+        convertLocatedBlocks(lb.getBlocksList()),
+        lb.hasLastBlock() ?
+            convertLocatedBlockProto(lb.getLastBlock()) : null,
         lb.getIsLastBlockComplete(),
-        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) :
-            null);
+        lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null,
+        lb.hasEcPolicy() ? convertErasureCodingPolicy(lb.getEcPolicy()) : null);
   }
 
   public static BlockStoragePolicy[] convertStoragePolicies(
@@ -716,23 +747,41 @@
   }
 
   // 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 List<LocatedBlockProto> convertLocatedBlock2(List<LocatedBlock> lb) {
+  public static LocatedBlock[] convertLocatedBlocks(LocatedBlockProto[] lb) {
+    if (lb == null) return null;
+    return convertLocatedBlocks(Arrays.asList(lb))
+        .toArray(new LocatedBlock[lb.length]);
+  }
+
+  public static List<LocatedBlock> convertLocatedBlocks(
+      List<LocatedBlockProto> lb) {
     if (lb == null) return null;
     final int len = lb.size();
-    List<LocatedBlockProto> result = new ArrayList<>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(convert(lb.get(i)));
+    List<LocatedBlock> result = new ArrayList<>(len);
+    for (LocatedBlockProto aLb : lb) {
+      result.add(convertLocatedBlockProto(aLb));
     }
     return result;
   }
 
-  public static LocatedBlockProto convert(LocatedBlock b) {
+  public static List<LocatedBlockProto> convertLocatedBlocks2(
+      List<LocatedBlock> lb) {
+    if (lb == null) return null;
+    final int len = lb.size();
+    List<LocatedBlockProto> result = new ArrayList<>(len);
+    for (LocatedBlock aLb : lb) {
+      result.add(convertLocatedBlock(aLb));
+    }
+    return result;
+  }
+
+  public static LocatedBlockProto convertLocatedBlock(LocatedBlock b) {
     if (b == null) return null;
     Builder builder = LocatedBlockProto.newBuilder();
     DatanodeInfo[] locs = b.getLocations();
@@ -740,7 +789,7 @@
         Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
     for (int i = 0; i < locs.length; i++) {
       DatanodeInfo loc = locs[i];
-      builder.addLocs(i, convert(loc));
+      builder.addLocs(i, PBHelperClient.convert(loc));
       boolean locIsCached = cachedLocs.contains(loc);
       builder.addIsCached(locIsCached);
       if (locIsCached) {
@@ -749,21 +798,30 @@
     }
     Preconditions.checkArgument(cachedLocs.size() == 0,
         "Found additional cached replica locations that are not in the set of"
-        + " storage-backed locations!");
+            + " storage-backed locations!");
 
     StorageType[] storageTypes = b.getStorageTypes();
     if (storageTypes != null) {
-      for (int i = 0; i < storageTypes.length; ++i) {
-        builder.addStorageTypes(convertStorageType(storageTypes[i]));
+      for (StorageType storageType : storageTypes) {
+        builder.addStorageTypes(convertStorageType(storageType));
       }
     }
     final String[] storageIDs = b.getStorageIDs();
     if (storageIDs != null) {
       builder.addAllStorageIDs(Arrays.asList(storageIDs));
     }
+    if (b instanceof LocatedStripedBlock) {
+      LocatedStripedBlock sb = (LocatedStripedBlock) b;
+      int[] indices = sb.getBlockIndices();
+      Token<BlockTokenIdentifier>[] blockTokens = sb.getBlockTokens();
+      for (int i = 0; i < indices.length; i++) {
+        builder.addBlockIndex(indices[i]);
+        builder.addBlockTokens(PBHelperClient.convert(blockTokens[i]));
+      }
+    }
 
-    return builder.setB(convert(b.getBlock()))
-        .setBlockToken(convert(b.getBlockToken()))
+    return builder.setB(PBHelperClient.convert(b.getBlock()))
+        .setBlockToken(PBHelperClient.convert(b.getBlockToken()))
         .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
   }
 
@@ -897,7 +955,7 @@
     final int len = lb.size();
     List<LocatedBlock> result = new ArrayList<>(len);
     for (int i = 0; i < len; ++i) {
-      result.add(convert(lb.get(i)));
+      result.add(convertLocatedBlockProto(lb.get(i)));
     }
     return result;
   }
@@ -1087,7 +1145,7 @@
 
   public static CachePoolInfo convert (CachePoolInfoProto proto) {
     // Pool name is a required field, the rest are optional
-    String poolName = checkNotNull(proto.getPoolName());
+    String poolName = Preconditions.checkNotNull(proto.getPoolName());
     CachePoolInfo info = new CachePoolInfo(poolName);
     if (proto.hasOwnerName()) {
         info.setOwnerName(proto.getOwnerName());
@@ -1341,7 +1399,8 @@
         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
-            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
+    fs.hasEcPolicy() ? convertErasureCodingPolicy(fs.getEcPolicy()) : null);
   }
 
   public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
@@ -1655,14 +1714,18 @@
     LocatedBlocksProto.Builder builder =
         LocatedBlocksProto.newBuilder();
     if (lb.getLastLocatedBlock() != null) {
-      builder.setLastBlock(convert(lb.getLastLocatedBlock()));
+      builder.setLastBlock(
+          convertLocatedBlock(lb.getLastLocatedBlock()));
     }
     if (lb.getFileEncryptionInfo() != null) {
       builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
     }
+    if (lb.getErasureCodingPolicy() != null) {
+      builder.setEcPolicy(convertErasureCodingPolicy(lb.getErasureCodingPolicy()));
+    }
     return builder.setFileLength(lb.getFileLength())
         .setUnderConstruction(lb.isUnderConstruction())
-        .addAllBlocks(convertLocatedBlock2(lb.getLocatedBlocks()))
+        .addAllBlocks(convertLocatedBlocks2(lb.getLocatedBlocks()))
         .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
   }
 
@@ -1763,6 +1826,9 @@
         builder.setLocations(convert(locations));
       }
     }
+    if(fs.getErasureCodingPolicy() != null) {
+      builder.setEcPolicy(convertErasureCodingPolicy(fs.getErasureCodingPolicy()));
+    }
     return builder.build();
   }
 
@@ -2326,4 +2392,44 @@
     }
     return Arrays.asList(ret);
   }
+
+  public static ECSchema convertECSchema(HdfsProtos.ECSchemaProto schema) {
+    List<HdfsProtos.ECSchemaOptionEntryProto> optionsList = schema.getOptionsList();
+    Map<String, String> options = new HashMap<>(optionsList.size());
+    for (HdfsProtos.ECSchemaOptionEntryProto option : optionsList) {
+      options.put(option.getKey(), option.getValue());
+    }
+    return new ECSchema(schema.getCodecName(), schema.getDataUnits(),
+        schema.getParityUnits(), options);
+  }
+
+  public static HdfsProtos.ECSchemaProto convertECSchema(ECSchema schema) {
+    HdfsProtos.ECSchemaProto.Builder builder = HdfsProtos.ECSchemaProto.newBuilder()
+        .setCodecName(schema.getCodecName())
+        .setDataUnits(schema.getNumDataUnits())
+        .setParityUnits(schema.getNumParityUnits());
+    Set<Map.Entry<String, String>> entrySet = schema.getExtraOptions().entrySet();
+    for (Map.Entry<String, String> entry : entrySet) {
+      builder.addOptions(HdfsProtos.ECSchemaOptionEntryProto.newBuilder()
+          .setKey(entry.getKey()).setValue(entry.getValue()).build());
+    }
+    return builder.build();
+  }
+
+  public static ErasureCodingPolicy convertErasureCodingPolicy(
+      ErasureCodingPolicyProto policy) {
+    return new ErasureCodingPolicy(policy.getName(),
+        convertECSchema(policy.getSchema()),
+        policy.getCellSize());
+  }
+
+  public static ErasureCodingPolicyProto convertErasureCodingPolicy(
+      ErasureCodingPolicy policy) {
+    ErasureCodingPolicyProto.Builder builder = ErasureCodingPolicyProto
+        .newBuilder()
+        .setName(policy.getName())
+        .setSchema(convertECSchema(policy.getSchema()))
+        .setCellSize(policy.getCellSize());
+    return builder.build();
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
new file mode 100644
index 0000000..264c532
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
@@ -0,0 +1,952 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.util;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSStripedOutputStream;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.security.token.Token;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.io.IOException;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * When accessing a file in striped layout, operations on logical byte ranges
+ * in the file need to be mapped to physical byte ranges on block files stored
+ * on DataNodes. This utility class facilities this mapping by defining and
+ * exposing a number of striping-related concepts. The most basic ones are
+ * illustrated in the following diagram. Unless otherwise specified, all
+ * range-related calculations are inclusive (the end offset of the previous
+ * range should be 1 byte lower than the start offset of the next one).
+ *
+ *  | <----  Block Group ----> |   <- Block Group: logical unit composing
+ *  |                          |        striped HDFS files.
+ *  blk_0      blk_1       blk_2   <- Internal Blocks: each internal block
+ *    |          |           |          represents a physically stored local
+ *    v          v           v          block file
+ * +------+   +------+   +------+
+ * |cell_0|   |cell_1|   |cell_2|  <- {@link StripingCell} represents the
+ * +------+   +------+   +------+       logical order that a Block Group should
+ * |cell_3|   |cell_4|   |cell_5|       be accessed: cell_0, cell_1, ...
+ * +------+   +------+   +------+
+ * |cell_6|   |cell_7|   |cell_8|
+ * +------+   +------+   +------+
+ * |cell_9|
+ * +------+  <- A cell contains cellSize bytes of data
+ */
+@InterfaceAudience.Private
+public class StripedBlockUtil {
+
+  /**
+   * This method parses a striped block group into individual blocks.
+   *
+   * @param bg The striped block group
+   * @param cellSize The size of a striping cell
+   * @param dataBlkNum The number of data blocks
+   * @return An array containing the blocks in the group
+   */
+  public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg,
+      int cellSize, int dataBlkNum, int parityBlkNum) {
+    int locatedBGSize = bg.getBlockIndices().length;
+    LocatedBlock[] lbs = new LocatedBlock[dataBlkNum + parityBlkNum];
+    for (short i = 0; i < locatedBGSize; i++) {
+      final int idx = bg.getBlockIndices()[i];
+      // for now we do not use redundant replica of an internal block
+      if (idx < (dataBlkNum + parityBlkNum) && lbs[idx] == null) {
+        lbs[idx] = constructInternalBlock(bg, i, cellSize,
+            dataBlkNum, idx);
+      }
+    }
+    return lbs;
+  }
+
+  /**
+   * This method creates an internal block at the given index of a block group
+   *
+   * @param idxInReturnedLocs The index in the stored locations in the
+   *                          {@link LocatedStripedBlock} object
+   * @param idxInBlockGroup The logical index in the striped block group
+   * @return The constructed internal block
+   */
+  public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg,
+      int idxInReturnedLocs, int cellSize, int dataBlkNum,
+      int idxInBlockGroup) {
+    final ExtendedBlock blk = constructInternalBlock(
+        bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup);
+    final LocatedBlock locatedBlock;
+    if (idxInReturnedLocs < bg.getLocations().length) {
+      locatedBlock = new LocatedBlock(blk,
+          new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
+          new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
+          new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
+          bg.getStartOffset(), bg.isCorrupt(), null);
+    } else {
+      locatedBlock = new LocatedBlock(blk, null, null, null,
+          bg.getStartOffset(), bg.isCorrupt(), null);
+    }
+    Token<BlockTokenIdentifier>[] blockTokens = bg.getBlockTokens();
+    if (idxInReturnedLocs < blockTokens.length) {
+      locatedBlock.setBlockToken(blockTokens[idxInReturnedLocs]);
+    }
+    return locatedBlock;
+  }
+
+  /**
+   * This method creates an internal {@link ExtendedBlock} at the given index
+   * of a block group.
+   */
+  public static ExtendedBlock constructInternalBlock(ExtendedBlock blockGroup,
+      int cellSize, int dataBlkNum, int idxInBlockGroup) {
+    ExtendedBlock block = new ExtendedBlock(blockGroup);
+    block.setBlockId(blockGroup.getBlockId() + idxInBlockGroup);
+    block.setNumBytes(getInternalBlockLength(blockGroup.getNumBytes(),
+        cellSize, dataBlkNum, idxInBlockGroup));
+    return block;
+  }
+
+  /**
+   * Get the size of an internal block at the given index of a block group
+   *
+   * @param dataSize Size of the block group only counting data blocks
+   * @param cellSize The size of a striping cell
+   * @param numDataBlocks The number of data blocks
+   * @param i The logical index in the striped block group
+   * @return The size of the internal block at the specified index
+   */
+  public static long getInternalBlockLength(long dataSize,
+      int cellSize, int numDataBlocks, int i) {
+    Preconditions.checkArgument(dataSize >= 0);
+    Preconditions.checkArgument(cellSize > 0);
+    Preconditions.checkArgument(numDataBlocks > 0);
+    Preconditions.checkArgument(i >= 0);
+    // Size of each stripe (only counting data blocks)
+    final int stripeSize = cellSize * numDataBlocks;
+    // If block group ends at stripe boundary, each internal block has an equal
+    // share of the group
+    final int lastStripeDataLen = (int)(dataSize % stripeSize);
+    if (lastStripeDataLen == 0) {
+      return dataSize / numDataBlocks;
+    }
+
+    final int numStripes = (int) ((dataSize - 1) / stripeSize + 1);
+    return (numStripes - 1L)*cellSize
+        + lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i);
+  }
+
+  private static int lastCellSize(int size, int cellSize, int numDataBlocks,
+      int i) {
+    if (i < numDataBlocks) {
+      // parity block size (i.e. i >= numDataBlocks) is the same as 
+      // the first data block size (i.e. i = 0).
+      size -= i*cellSize;
+      if (size < 0) {
+        size = 0;
+      }
+    }
+    return size > cellSize? cellSize: size;
+  }
+
+  /**
+   * Given a byte's offset in an internal block, calculate the offset in
+   * the block group
+   */
+  public static long offsetInBlkToOffsetInBG(int cellSize, int dataBlkNum,
+      long offsetInBlk, int idxInBlockGroup) {
+    int cellIdxInBlk = (int) (offsetInBlk / cellSize);
+    return cellIdxInBlk * cellSize * dataBlkNum // n full stripes before offset
+        + idxInBlockGroup * cellSize // m full cells before offset
+        + offsetInBlk % cellSize; // partial cell
+  }
+
+  /**
+   * Get the next completed striped read task
+   *
+   * @return {@link StripingChunkReadResult} indicating the status of the read task
+   *          succeeded, and the block index of the task. If the method times
+   *          out without getting any completed read tasks, -1 is returned as
+   *          block index.
+   * @throws InterruptedException
+   */
+  public static StripingChunkReadResult getNextCompletedStripedRead(
+      CompletionService<Void> readService, Map<Future<Void>, Integer> futures,
+      final long timeoutMillis) throws InterruptedException {
+    Preconditions.checkArgument(!futures.isEmpty());
+    Future<Void> future = null;
+    try {
+      if (timeoutMillis > 0) {
+        future = readService.poll(timeoutMillis, TimeUnit.MILLISECONDS);
+      } else {
+        future = readService.take();
+      }
+      if (future != null) {
+        future.get();
+        return new StripingChunkReadResult(futures.remove(future),
+            StripingChunkReadResult.SUCCESSFUL);
+      } else {
+        return new StripingChunkReadResult(StripingChunkReadResult.TIMEOUT);
+      }
+    } catch (ExecutionException e) {
+      if (DFSClient.LOG.isDebugEnabled()) {
+        DFSClient.LOG.debug("ExecutionException " + e);
+      }
+      return new StripingChunkReadResult(futures.remove(future),
+          StripingChunkReadResult.FAILED);
+    } catch (CancellationException e) {
+      return new StripingChunkReadResult(futures.remove(future),
+          StripingChunkReadResult.CANCELLED);
+    }
+  }
+
+  /**
+   * Get the total usage of the striped blocks, which is the total of data
+   * blocks and parity blocks
+   *
+   * @param numDataBlkBytes
+   *          Size of the block group only counting data blocks
+   * @param dataBlkNum
+   *          The number of data blocks
+   * @param parityBlkNum
+   *          The number of parity blocks
+   * @param cellSize
+   *          The size of a striping cell
+   * @return The total usage of data blocks and parity blocks
+   */
+  public static long spaceConsumedByStripedBlock(long numDataBlkBytes,
+      int dataBlkNum, int parityBlkNum, int cellSize) {
+    int parityIndex = dataBlkNum + 1;
+    long numParityBlkBytes = getInternalBlockLength(numDataBlkBytes, cellSize,
+        dataBlkNum, parityIndex) * parityBlkNum;
+    return numDataBlkBytes + numParityBlkBytes;
+  }
+
+  /**
+   * Initialize the decoding input buffers based on the chunk states in an
+   * {@link AlignedStripe}. For each chunk that was not initially requested,
+   * schedule a new fetch request with the decoding input buffer as transfer
+   * destination.
+   */
+  public static byte[][] initDecodeInputs(AlignedStripe alignedStripe,
+      int dataBlkNum, int parityBlkNum) {
+    byte[][] decodeInputs =
+        new byte[dataBlkNum + parityBlkNum][(int) alignedStripe.getSpanInBlock()];
+    // read the full data aligned stripe
+    for (int i = 0; i < dataBlkNum; i++) {
+      if (alignedStripe.chunks[i] == null) {
+        final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum);
+        alignedStripe.chunks[i] = new StripingChunk(decodeInputs[decodeIndex]);
+        alignedStripe.chunks[i].addByteArraySlice(0,
+            (int) alignedStripe.getSpanInBlock());
+      }
+    }
+    return decodeInputs;
+  }
+
+  /**
+   * Some fetched {@link StripingChunk} might be stored in original application
+   * buffer instead of prepared decode input buffers. Some others are beyond
+   * the range of the internal blocks and should correspond to all zero bytes.
+   * When all pending requests have returned, this method should be called to
+   * finalize decode input buffers.
+   */
+  public static void finalizeDecodeInputs(final byte[][] decodeInputs,
+      int dataBlkNum, int parityBlkNum, AlignedStripe alignedStripe) {
+    for (int i = 0; i < alignedStripe.chunks.length; i++) {
+      final StripingChunk chunk = alignedStripe.chunks[i];
+      final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum);
+      if (chunk != null && chunk.state == StripingChunk.FETCHED) {
+        chunk.copyTo(decodeInputs[decodeIndex]);
+      } else if (chunk != null && chunk.state == StripingChunk.ALLZERO) {
+        Arrays.fill(decodeInputs[decodeIndex], (byte) 0);
+      } else {
+        decodeInputs[decodeIndex] = null;
+      }
+    }
+  }
+
+  /**
+   * Currently decoding requires parity chunks are before data chunks.
+   * The indices are opposite to what we store in NN. In future we may
+   * improve the decoding to make the indices order the same as in NN.
+   *
+   * @param index The index to convert
+   * @param dataBlkNum The number of data blocks
+   * @param parityBlkNum The number of parity blocks
+   * @return converted index
+   */
+  public static int convertIndex4Decode(int index, int dataBlkNum,
+      int parityBlkNum) {
+    return index < dataBlkNum ? index + parityBlkNum : index - dataBlkNum;
+  }
+
+  public static int convertDecodeIndexBack(int index, int dataBlkNum,
+      int parityBlkNum) {
+    return index < parityBlkNum ? index + dataBlkNum : index - parityBlkNum;
+  }
+
+  /**
+   * Decode based on the given input buffers and erasure coding policy.
+   */
+  public static void decodeAndFillBuffer(final byte[][] decodeInputs,
+      AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum,
+      RawErasureDecoder decoder) {
+    // Step 1: prepare indices and output buffers for missing data units
+    int[] decodeIndices = new int[parityBlkNum];
+    int pos = 0;
+    for (int i = 0; i < dataBlkNum; i++) {
+      if (alignedStripe.chunks[i] != null &&
+          alignedStripe.chunks[i].state == StripingChunk.MISSING){
+        decodeIndices[pos++] = convertIndex4Decode(i, dataBlkNum, parityBlkNum);
+      }
+    }
+    decodeIndices = Arrays.copyOf(decodeIndices, pos);
+    byte[][] decodeOutputs =
+        new byte[decodeIndices.length][(int) alignedStripe.getSpanInBlock()];
+
+    // Step 2: decode into prepared output buffers
+    decoder.decode(decodeInputs, decodeIndices, decodeOutputs);
+
+    // Step 3: fill original application buffer with decoded data
+    for (int i = 0; i < decodeIndices.length; i++) {
+      int missingBlkIdx = convertDecodeIndexBack(decodeIndices[i],
+          dataBlkNum, parityBlkNum);
+      StripingChunk chunk = alignedStripe.chunks[missingBlkIdx];
+      if (chunk.state == StripingChunk.MISSING) {
+        chunk.copyFrom(decodeOutputs[i]);
+      }
+    }
+  }
+
+  /**
+   * Similar functionality with {@link #divideByteRangeIntoStripes}, but is used
+   * by stateful read and uses ByteBuffer as reading target buffer. Besides the
+   * read range is within a single stripe thus the calculation logic is simpler.
+   */
+  public static AlignedStripe[] divideOneStripe(ErasureCodingPolicy ecPolicy,
+      int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup,
+      long rangeEndInBlockGroup, ByteBuffer buf) {
+    final int dataBlkNum = ecPolicy.getNumDataUnits();
+    // Step 1: map the byte range to StripingCells
+    StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize,
+        blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup);
+
+    // Step 2: get the unmerged ranges on each internal block
+    VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize,
+        cells);
+
+    // Step 3: merge into stripes
+    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges);
+
+    // Step 4: calculate each chunk's position in destination buffer. Since the
+    // whole read range is within a single stripe, the logic is simpler here.
+    int bufOffset = (int) (rangeStartInBlockGroup % ((long) cellSize * dataBlkNum));
+    for (StripingCell cell : cells) {
+      long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
+      long cellEnd = cellStart + cell.size - 1;
+      for (AlignedStripe s : stripes) {
+        long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1;
+        long overlapStart = Math.max(cellStart, s.getOffsetInBlock());
+        long overlapEnd = Math.min(cellEnd, stripeEnd);
+        int overLapLen = (int) (overlapEnd - overlapStart + 1);
+        if (overLapLen > 0) {
+          Preconditions.checkState(s.chunks[cell.idxInStripe] == null);
+          final int pos = (int) (bufOffset + overlapStart - cellStart);
+          buf.position(pos);
+          buf.limit(pos + overLapLen);
+          s.chunks[cell.idxInStripe] = new StripingChunk(buf.slice());
+        }
+      }
+      bufOffset += cell.size;
+    }
+
+    // Step 5: prepare ALLZERO blocks
+    prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum);
+    return stripes;
+  }
+
+  /**
+   * This method divides a requested byte range into an array of inclusive
+   * {@link AlignedStripe}.
+   * @param ecPolicy The codec policy for the file, which carries the numbers
+   *                 of data / parity blocks
+   * @param cellSize Cell size of stripe
+   * @param blockGroup The striped block group
+   * @param rangeStartInBlockGroup The byte range's start offset in block group
+   * @param rangeEndInBlockGroup The byte range's end offset in block group
+   * @param buf Destination buffer of the read operation for the byte range
+   * @param offsetInBuf Start offset into the destination buffer
+   *
+   * At most 5 stripes will be generated from each logical range, as
+   * demonstrated in the header of {@link AlignedStripe}.
+   */
+  public static AlignedStripe[] divideByteRangeIntoStripes(ErasureCodingPolicy ecPolicy,
+      int cellSize, LocatedStripedBlock blockGroup,
+      long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf,
+      int offsetInBuf) {
+
+    // Step 0: analyze range and calculate basic parameters
+    final int dataBlkNum = ecPolicy.getNumDataUnits();
+
+    // Step 1: map the byte range to StripingCells
+    StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize,
+        blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup);
+
+    // Step 2: get the unmerged ranges on each internal block
+    VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize,
+        cells);
+
+    // Step 3: merge into at most 5 stripes
+    AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges);
+
+    // Step 4: calculate each chunk's position in destination buffer
+    calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf, offsetInBuf);
+
+    // Step 5: prepare ALLZERO blocks
+    prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum);
+
+    return stripes;
+  }
+
+  /**
+   * Map the logical byte range to a set of inclusive {@link StripingCell}
+   * instances, each representing the overlap of the byte range to a cell
+   * used by {@link DFSStripedOutputStream} in encoding
+   */
+  @VisibleForTesting
+  private static StripingCell[] getStripingCellsOfByteRange(ErasureCodingPolicy ecPolicy,
+      int cellSize, LocatedStripedBlock blockGroup,
+      long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
+    Preconditions.checkArgument(
+        rangeStartInBlockGroup <= rangeEndInBlockGroup &&
+            rangeEndInBlockGroup < blockGroup.getBlockSize());
+    long len = rangeEndInBlockGroup - rangeStartInBlockGroup + 1;
+    int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
+    int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize);
+    int numCells = lastCellIdxInBG - firstCellIdxInBG + 1;
+    StripingCell[] cells = new StripingCell[numCells];
+
+    final int firstCellOffset = (int) (rangeStartInBlockGroup % cellSize);
+    final int firstCellSize =
+        (int) Math.min(cellSize - (rangeStartInBlockGroup % cellSize), len);
+    cells[0] = new StripingCell(ecPolicy, firstCellSize, firstCellIdxInBG,
+        firstCellOffset);
+    if (lastCellIdxInBG != firstCellIdxInBG) {
+      final int lastCellSize = (int) (rangeEndInBlockGroup % cellSize) + 1;
+      cells[numCells - 1] = new StripingCell(ecPolicy, lastCellSize,
+          lastCellIdxInBG, 0);
+    }
+
+    for (int i = 1; i < numCells - 1; i++) {
+      cells[i] = new StripingCell(ecPolicy, cellSize, i + firstCellIdxInBG, 0);
+    }
+
+    return cells;
+  }
+
+  /**
+   * Given a logical byte range, mapped to each {@link StripingCell}, calculate
+   * the physical byte range (inclusive) on each stored internal block.
+   */
+  @VisibleForTesting
+  private static VerticalRange[] getRangesForInternalBlocks(ErasureCodingPolicy ecPolicy,
+      int cellSize, StripingCell[] cells) {
+    int dataBlkNum = ecPolicy.getNumDataUnits();
+    int parityBlkNum = ecPolicy.getNumParityUnits();
+
+    VerticalRange ranges[] = new VerticalRange[dataBlkNum + parityBlkNum];
+
+    long earliestStart = Long.MAX_VALUE;
+    long latestEnd = -1;
+    for (StripingCell cell : cells) {
+      // iterate through all cells and update the list of StripeRanges
+      if (ranges[cell.idxInStripe] == null) {
+        ranges[cell.idxInStripe] = new VerticalRange(
+            cell.idxInInternalBlk * cellSize + cell.offset, cell.size);
+      } else {
+        ranges[cell.idxInStripe].spanInBlock += cell.size;
+      }
+      VerticalRange range = ranges[cell.idxInStripe];
+      if (range.offsetInBlock < earliestStart) {
+        earliestStart = range.offsetInBlock;
+      }
+      if (range.offsetInBlock + range.spanInBlock - 1 > latestEnd) {
+        latestEnd = range.offsetInBlock + range.spanInBlock - 1;
+      }
+    }
+
+    // Each parity block should be fetched at maximum range of all data blocks
+    for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) {
+      ranges[i] = new VerticalRange(earliestStart,
+          latestEnd - earliestStart + 1);
+    }
+
+    return ranges;
+  }
+
+  /**
+   * Merge byte ranges on each internal block into a set of inclusive
+   * {@link AlignedStripe} instances.
+   */
+  private static AlignedStripe[] mergeRangesForInternalBlocks(
+      ErasureCodingPolicy ecPolicy, VerticalRange[] ranges) {
+    int dataBlkNum = ecPolicy.getNumDataUnits();
+    int parityBlkNum = ecPolicy.getNumParityUnits();
+    List<AlignedStripe> stripes = new ArrayList<>();
+    SortedSet<Long> stripePoints = new TreeSet<>();
+    for (VerticalRange r : ranges) {
+      if (r != null) {
+        stripePoints.add(r.offsetInBlock);
+        stripePoints.add(r.offsetInBlock + r.spanInBlock);
+      }
+    }
+
+    long prev = -1;
+    for (long point : stripePoints) {
+      if (prev >= 0) {
+        stripes.add(new AlignedStripe(prev, point - prev,
+            dataBlkNum + parityBlkNum));
+      }
+      prev = point;
+    }
+    return stripes.toArray(new AlignedStripe[stripes.size()]);
+  }
+
+  private static void calcualteChunkPositionsInBuf(int cellSize,
+      AlignedStripe[] stripes, StripingCell[] cells, byte[] buf,
+      int offsetInBuf) {
+    /**
+     *     | <--------------- AlignedStripe --------------->|
+     *
+     *     |<- length_0 ->|<--  length_1  -->|<- length_2 ->|
+     * +------------------+------------------+----------------+
+     * |    cell_0_0_0    |    cell_3_1_0    |   cell_6_2_0   |  <- blk_0
+     * +------------------+------------------+----------------+
+     *   _/                \_______________________
+     *  |                                          |
+     *  v offset_0                                 v offset_1
+     * +----------------------------------------------------------+
+     * |  cell_0_0_0 |  cell_1_0_1 and cell_2_0_2  |cell_3_1_0 ...|   <- buf
+     * |  (partial)  |    (from blk_1 and blk_2)   |              |
+     * +----------------------------------------------------------+
+     *
+     * Cell indexing convention defined in {@link StripingCell}
+     */
+    int done = 0;
+    for (StripingCell cell : cells) {
+      long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
+      long cellEnd = cellStart + cell.size - 1;
+      for (AlignedStripe s : stripes) {
+        long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1;
+        long overlapStart = Math.max(cellStart, s.getOffsetInBlock());
+        long overlapEnd = Math.min(cellEnd, stripeEnd);
+        int overLapLen = (int) (overlapEnd - overlapStart + 1);
+        if (overLapLen <= 0) {
+          continue;
+        }
+        if (s.chunks[cell.idxInStripe] == null) {
+          s.chunks[cell.idxInStripe] = new StripingChunk(buf);
+        }
+        s.chunks[cell.idxInStripe].addByteArraySlice(
+            (int)(offsetInBuf + done + overlapStart - cellStart), overLapLen);
+      }
+      done += cell.size;
+    }
+  }
+
+  /**
+   * If a {@link StripingChunk} maps to a byte range beyond an internal block's
+   * size, the chunk should be treated as zero bytes in decoding.
+   */
+  private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup,
+      AlignedStripe[] stripes, int cellSize, int dataBlkNum) {
+    for (AlignedStripe s : stripes) {
+      for (int i = 0; i < dataBlkNum; i++) {
+        long internalBlkLen = getInternalBlockLength(blockGroup.getBlockSize(),
+            cellSize, dataBlkNum, i);
+        if (internalBlkLen <= s.getOffsetInBlock()) {
+          Preconditions.checkState(s.chunks[i] == null);
+          s.chunks[i] = new StripingChunk(StripingChunk.ALLZERO);
+        }
+      }
+    }
+  }
+
+  /**
+   * Cell is the unit of encoding used in {@link DFSStripedOutputStream}. This
+   * size impacts how a logical offset in the file or block group translates
+   * to physical byte offset in a stored internal block. The StripingCell util
+   * class facilitates this calculation. Each StripingCell is inclusive with
+   * its start and end offsets -- e.g., the end logical offset of cell_0_0_0
+   * should be 1 byte lower than the start logical offset of cell_1_0_1.
+   *
+   *  | <------- Striped Block Group -------> |
+   *    blk_0          blk_1          blk_2
+   *      |              |              |
+   *      v              v              v
+   * +----------+   +----------+   +----------+
+   * |cell_0_0_0|   |cell_1_0_1|   |cell_2_0_2|
+   * +----------+   +----------+   +----------+
+   * |cell_3_1_0|   |cell_4_1_1|   |cell_5_1_2| <- {@link #idxInBlkGroup} = 5
+   * +----------+   +----------+   +----------+    {@link #idxInInternalBlk} = 1
+   *                                               {@link #idxInStripe} = 2
+   * A StripingCell is a special instance of {@link StripingChunk} whose offset
+   * and size align with the cell used when writing data.
+   * TODO: consider parity cells
+   */
+  @VisibleForTesting
+  static class StripingCell {
+    final ErasureCodingPolicy ecPolicy;
+    /** Logical order in a block group, used when doing I/O to a block group */
+    final int idxInBlkGroup;
+    final int idxInInternalBlk;
+    final int idxInStripe;
+    /**
+     * When a logical byte range is mapped to a set of cells, it might
+     * partially overlap with the first and last cells. This field and the
+     * {@link #size} variable represent the start offset and size of the
+     * overlap.
+     */
+    final int offset;
+    final int size;
+
+    StripingCell(ErasureCodingPolicy ecPolicy, int cellSize, int idxInBlkGroup,
+        int offset) {
+      this.ecPolicy = ecPolicy;
+      this.idxInBlkGroup = idxInBlkGroup;
+      this.idxInInternalBlk = idxInBlkGroup / ecPolicy.getNumDataUnits();
+      this.idxInStripe = idxInBlkGroup -
+          this.idxInInternalBlk * ecPolicy.getNumDataUnits();
+      this.offset = offset;
+      this.size = cellSize;
+    }
+  }
+
+  /**
+   * Given a requested byte range on a striped block group, an AlignedStripe
+   * represents an inclusive {@link VerticalRange} that is aligned with both
+   * the byte range and boundaries of all internal blocks. As illustrated in
+   * the diagram, any given byte range on a block group leads to 1~5
+   * AlignedStripe's.
+   *
+   * |<-------- Striped Block Group -------->|
+   * blk_0   blk_1   blk_2      blk_3   blk_4
+   *                 +----+  |  +----+  +----+
+   *                 |full|  |  |    |  |    | <- AlignedStripe0:
+   *         +----+  |~~~~|  |  |~~~~|  |~~~~|      1st cell is partial
+   *         |part|  |    |  |  |    |  |    | <- AlignedStripe1: byte range
+   * +----+  +----+  +----+  |  |~~~~|  |~~~~|      doesn't start at 1st block
+   * |full|  |full|  |full|  |  |    |  |    |
+   * |cell|  |cell|  |cell|  |  |    |  |    | <- AlignedStripe2 (full stripe)
+   * |    |  |    |  |    |  |  |    |  |    |
+   * +----+  +----+  +----+  |  |~~~~|  |~~~~|
+   * |full|  |part|          |  |    |  |    | <- AlignedStripe3: byte range
+   * |~~~~|  +----+          |  |~~~~|  |~~~~|      doesn't end at last block
+   * |    |                  |  |    |  |    | <- AlignedStripe4:
+   * +----+                  |  +----+  +----+      last cell is partial
+   *                         |
+   * <---- data blocks ----> | <--- parity --->
+   *
+   * An AlignedStripe is the basic unit of reading from a striped block group,
+   * because within the AlignedStripe, all internal blocks can be processed in
+   * a uniform manner.
+   *
+   * The coverage of an AlignedStripe on an internal block is represented as a
+   * {@link StripingChunk}.
+   *
+   * To simplify the logic of reading a logical byte range from a block group,
+   * a StripingChunk is either completely in the requested byte range or
+   * completely outside the requested byte range.
+   */
+  public static class AlignedStripe {
+    public VerticalRange range;
+    /** status of each chunk in the stripe */
+    public final StripingChunk[] chunks;
+    public int fetchedChunksNum = 0;
+    public int missingChunksNum = 0;
+
+    public AlignedStripe(long offsetInBlock, long length, int width) {
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      this.range = new VerticalRange(offsetInBlock, length);
+      this.chunks = new StripingChunk[width];
+    }
+
+    public boolean include(long pos) {
+      return range.include(pos);
+    }
+
+    public long getOffsetInBlock() {
+      return range.offsetInBlock;
+    }
+
+    public long getSpanInBlock() {
+      return range.spanInBlock;
+    }
+
+    @Override
+    public String toString() {
+      return "Offset=" + range.offsetInBlock + ", length=" + range.spanInBlock +
+          ", fetchedChunksNum=" + fetchedChunksNum +
+          ", missingChunksNum=" + missingChunksNum;
+    }
+  }
+
+  /**
+   * A simple utility class representing an arbitrary vertical inclusive range
+   * starting at {@link #offsetInBlock} and lasting for {@link #spanInBlock}
+   * bytes in an internal block. Note that VerticalRange doesn't necessarily
+   * align with {@link StripingCell}.
+   *
+   * |<- Striped Block Group ->|
+   *  blk_0
+   *    |
+   *    v
+   * +-----+
+   * |~~~~~| <-- {@link #offsetInBlock}
+   * |     |  ^
+   * |     |  |
+   * |     |  | {@link #spanInBlock}
+   * |     |  v
+   * |~~~~~| ---
+   * |     |
+   * +-----+
+   */
+  public static class VerticalRange {
+    /** start offset in the block group (inclusive) */
+    public long offsetInBlock;
+    /** length of the stripe range */
+    public long spanInBlock;
+
+    public VerticalRange(long offsetInBlock, long length) {
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      this.offsetInBlock = offsetInBlock;
+      this.spanInBlock = length;
+    }
+
+    /** whether a position is in the range */
+    public boolean include(long pos) {
+      return pos >= offsetInBlock && pos < offsetInBlock + spanInBlock;
+    }
+  }
+
+  /**
+   * Indicates the coverage of an {@link AlignedStripe} on an internal block,
+   * and the state of the chunk in the context of the read request.
+   *
+   * |<---------------- Striped Block Group --------------->|
+   *   blk_0        blk_1        blk_2          blk_3   blk_4
+   *                           +---------+  |  +----+  +----+
+   *     null         null     |REQUESTED|  |  |null|  |null| <- AlignedStripe0
+   *              +---------+  |---------|  |  |----|  |----|
+   *     null     |REQUESTED|  |REQUESTED|  |  |null|  |null| <- AlignedStripe1
+   * +---------+  +---------+  +---------+  |  +----+  +----+
+   * |REQUESTED|  |REQUESTED|    ALLZERO    |  |null|  |null| <- AlignedStripe2
+   * +---------+  +---------+               |  +----+  +----+
+   * <----------- data blocks ------------> | <--- parity --->
+   */
+  public static class StripingChunk {
+    /** Chunk has been successfully fetched */
+    public static final int FETCHED = 0x01;
+    /** Chunk has encountered failed when being fetched */
+    public static final int MISSING = 0x02;
+    /** Chunk being fetched (fetching task is in-flight) */
+    public static final int PENDING = 0x04;
+    /**
+     * Chunk is requested either by application or for decoding, need to
+     * schedule read task
+     */
+    public static final int REQUESTED = 0X08;
+    /**
+     * Internal block is short and has no overlap with chunk. Chunk considered
+     * all-zero bytes in codec calculations.
+     */
+    public static final int ALLZERO = 0X0f;
+
+    /**
+     * If a chunk is completely in requested range, the state transition is:
+     * REQUESTED (when AlignedStripe created) -> PENDING -> {FETCHED | MISSING}
+     * If a chunk is completely outside requested range (including parity
+     * chunks), state transition is:
+     * null (AlignedStripe created) -> REQUESTED (upon failure) -> PENDING ...
+     */
+    public int state = REQUESTED;
+
+    public final ChunkByteArray byteArray;
+    public final ByteBuffer byteBuffer;
+
+    public StripingChunk(byte[] buf) {
+      this.byteArray = new ChunkByteArray(buf);
+      byteBuffer = null;
+    }
+
+    public StripingChunk(ByteBuffer buf) {
+      this.byteArray = null;
+      this.byteBuffer = buf;
+    }
+
+    public StripingChunk(int state) {
+      this.byteArray = null;
+      this.byteBuffer = null;
+      this.state = state;
+    }
+
+    public void addByteArraySlice(int offset, int length) {
+      assert byteArray != null;
+      byteArray.offsetsInBuf.add(offset);
+      byteArray.lengthsInBuf.add(length);
+    }
+
+    void copyTo(byte[] target) {
+      assert byteArray != null;
+      byteArray.copyTo(target);
+    }
+
+    void copyFrom(byte[] src) {
+      assert byteArray != null;
+      byteArray.copyFrom(src);
+    }
+  }
+
+  public static class ChunkByteArray {
+    private final byte[] buf;
+    private final List<Integer> offsetsInBuf;
+    private final List<Integer> lengthsInBuf;
+
+    ChunkByteArray(byte[] buf) {
+      this.buf = buf;
+      this.offsetsInBuf = new ArrayList<>();
+      this.lengthsInBuf = new ArrayList<>();
+    }
+
+    public int[] getOffsets() {
+      int[] offsets = new int[offsetsInBuf.size()];
+      for (int i = 0; i < offsets.length; i++) {
+        offsets[i] = offsetsInBuf.get(i);
+      }
+      return offsets;
+    }
+
+    public int[] getLengths() {
+      int[] lens = new int[this.lengthsInBuf.size()];
+      for (int i = 0; i < lens.length; i++) {
+        lens[i] = this.lengthsInBuf.get(i);
+      }
+      return lens;
+    }
+
+    public byte[] buf() {
+      return buf;
+    }
+
+    void copyTo(byte[] target) {
+      int posInBuf = 0;
+      for (int i = 0; i < offsetsInBuf.size(); i++) {
+        System.arraycopy(buf, offsetsInBuf.get(i),
+            target, posInBuf, lengthsInBuf.get(i));
+        posInBuf += lengthsInBuf.get(i);
+      }
+    }
+
+    void copyFrom(byte[] src) {
+      int srcPos = 0;
+      for (int j = 0; j < offsetsInBuf.size(); j++) {
+        System.arraycopy(src, srcPos, buf, offsetsInBuf.get(j),
+            lengthsInBuf.get(j));
+        srcPos += lengthsInBuf.get(j);
+      }
+    }
+  }
+
+  /**
+   * This class represents result from a striped read request.
+   * If the task was successful or the internal computation failed,
+   * an index is also returned.
+   */
+  public static class StripingChunkReadResult {
+    public static final int SUCCESSFUL = 0x01;
+    public static final int FAILED = 0x02;
+    public static final int TIMEOUT = 0x04;
+    public static final int CANCELLED = 0x08;
+
+    public final int index;
+    public final int state;
+
+    public StripingChunkReadResult(int state) {
+      Preconditions.checkArgument(state == TIMEOUT,
+          "Only timeout result should return negative index.");
+      this.index = -1;
+      this.state = state;
+    }
+
+    public StripingChunkReadResult(int index, int state) {
+      Preconditions.checkArgument(state != TIMEOUT,
+          "Timeout result should return negative index.");
+      this.index = index;
+      this.state = state;
+    }
+
+    @Override
+    public String toString() {
+      return "(index=" + index + ", state =" + state + ")";
+    }
+  }
+
+  /**
+   * Check if the information such as IDs and generation stamps in block-i
+   * match the block group.
+   */
+  public static void checkBlocks(ExtendedBlock blockGroup,
+      int i, ExtendedBlock blocki) throws IOException {
+    if (!blocki.getBlockPoolId().equals(blockGroup.getBlockPoolId())) {
+      throw new IOException("Block pool IDs mismatched: block" + i + "="
+          + blocki + ", expected block group=" + blockGroup);
+    }
+    if (blocki.getBlockId() - i != blockGroup.getBlockId()) {
+      throw new IOException("Block IDs mismatched: block" + i + "="
+          + blocki + ", expected block group=" + blockGroup);
+    }
+    if (blocki.getGenerationStamp() != blockGroup.getGenerationStamp()) {
+      throw new IOException("Generation stamps mismatched: block" + i + "="
+          + blocki + ", expected block group=" + blockGroup);
+    }
+  }
+
+  public static int getBlockIndex(Block reportedBlock) {
+    long BLOCK_GROUP_INDEX_MASK = 15;
+    return (int) (reportedBlock.getBlockId() &
+        BLOCK_GROUP_INDEX_MASK);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index 3f85814..d938997 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -132,7 +132,7 @@
         blockSize, mTime, aTime, permission, owner, group,
         symlink, DFSUtilClient.string2Bytes(localName),
         fileId, childrenNum, null,
-        storagePolicy);
+        storagePolicy, null);
   }
 
   /** Convert a Json map to an ExtendedBlock object. */
@@ -504,7 +504,7 @@
         (Map<?, ?>) m.get("lastLocatedBlock"));
     final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete");
     return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks,
-        lastLocatedBlock, isLastBlockComplete, null);
+        lastLocatedBlock, isLastBlockComplete, null, null);
   }
 
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 7d32568..6a140eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -34,6 +34,7 @@
 import "xattr.proto";
 import "encryption.proto";
 import "inotify.proto";
+import "erasurecoding.proto";
 
 /**
  * The ClientNamenodeProtocol Service defines the interface between a client 
@@ -866,8 +867,14 @@
       returns(ListEncryptionZonesResponseProto);
   rpc getEZForPath(GetEZForPathRequestProto)
       returns(GetEZForPathResponseProto);
+  rpc setErasureCodingPolicy(SetErasureCodingPolicyRequestProto)
+      returns(SetErasureCodingPolicyResponseProto);
   rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto)
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
       returns(GetEditsFromTxidResponseProto);
+  rpc getErasureCodingPolicies(GetErasureCodingPoliciesRequestProto)
+      returns(GetErasureCodingPoliciesResponseProto);
+  rpc getErasureCodingPolicy(GetErasureCodingPolicyRequestProto)
+      returns(GetErasureCodingPolicyResponseProto);
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
new file mode 100644
index 0000000..fa24aef
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ 
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "ErasureCodingProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "hdfs.proto";
+
+message SetErasureCodingPolicyRequestProto {
+  required string src = 1;
+  optional ErasureCodingPolicyProto ecPolicy = 2;
+}
+
+message SetErasureCodingPolicyResponseProto {
+}
+
+message GetErasureCodingPoliciesRequestProto { // void request
+}
+
+message GetErasureCodingPoliciesResponseProto {
+  repeated ErasureCodingPolicyProto ecPolicies = 1;
+}
+
+message GetErasureCodingPolicyRequestProto {
+  required string src = 1; // path to get the policy info
+}
+
+message GetErasureCodingPolicyResponseProto {
+  optional ErasureCodingPolicyProto ecPolicy = 1;
+}
+
+/**
+ * Block erasure coding recovery info
+ */
+message BlockECRecoveryInfoProto {
+  required ExtendedBlockProto block = 1;
+  required DatanodeInfosProto sourceDnInfos = 2;
+  required DatanodeInfosProto targetDnInfos = 3;
+  required StorageUuidsProto targetStorageUuids = 4;
+  required StorageTypesProto targetStorageTypes = 5;
+  repeated uint32 liveBlockIndices = 6;
+  required ErasureCodingPolicyProto ecPolicy = 7;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index 0e2d541..d35fb57 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -215,6 +215,10 @@
   repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
   repeated StorageTypeProto storageTypes = 7;
   repeated string storageIDs = 8;
+
+  // striped block related fields
+  repeated uint32 blockIndex = 9; // used for striped block to indicate block index for each storage
+  repeated hadoop.common.TokenProto blockTokens = 10; // each internal block has a block token
 }
 
 message DataEncryptionKeyProto {
@@ -295,6 +299,33 @@
   optional LocatedBlockProto lastBlock = 4;
   required bool isLastBlockComplete = 5;
   optional FileEncryptionInfoProto fileEncryptionInfo = 6;
+
+  // Optional field for erasure coding
+  optional ErasureCodingPolicyProto ecPolicy = 7;
+}
+
+/**
+ * ECSchema options entry
+ */
+message ECSchemaOptionEntryProto {
+  required string key = 1;
+  required string value = 2;
+}
+
+/**
+ * ECSchema for erasurecoding
+ */
+message ECSchemaProto {
+  required string codecName = 1;
+  required uint32 dataUnits = 2;
+  required uint32 parityUnits = 3;
+  repeated ECSchemaOptionEntryProto options = 4;
+}
+
+message ErasureCodingPolicyProto {
+  required string name = 1;
+  required ECSchemaProto schema = 2;
+  required uint32 cellSize = 3;
 }
 
 /**
@@ -331,7 +362,10 @@
   optional FileEncryptionInfoProto fileEncryptionInfo = 15;
 
   optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
-} 
+
+  // Optional field for erasure coding
+  optional ErasureCodingPolicyProto ecPolicy = 17;
+}
 
 /**
  * Checksum algorithms/types used in HDFS
@@ -444,3 +478,11 @@
   required string blockPoolId = 1;
   optional bool finalized = 2 [default = false];
 }
+
+
+/**
+ * A list of storage IDs.
+ */
+message StorageUuidsProto {
+  repeated string storageUuids = 1;
+}
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 100755
index 0000000..6a01d61
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -0,0 +1,455 @@
+  BREAKDOWN OF HDFS-7285 SUBTASKS AND RELATED JIRAS
+
+    HDFS-7347. Configurable erasure coding policy for individual files and
+    directories ( Zhe Zhang via vinayakumarb )
+
+    HDFS-7339. Representing striped block groups in NameNode with hierarchical
+    naming protocol ( Zhe Zhang )
+
+    HDFS-7652. Process block reports for erasure coded blocks (Zhe Zhang)
+
+    HDFS-7716. Erasure Coding: extend BlockInfo to handle EC info (Jing Zhao)
+
+    HDFS-7749. Erasure Coding: Add striped block support in INodeFile (Jing Zhao)
+
+    HDFS-7837. Erasure Coding: allocate and persist striped blocks in NameNode
+    (Jing Zhao via Zhe Zhang)
+
+    HDFS-7872. Erasure Coding: INodeFile.dumpTreeRecursively() supports to print
+    striped blocks (Takuya Fukudome via jing9)
+
+    HDFS-7853. Erasure coding: extend LocatedBlocks to support reading from
+    striped files (Jing Zhao)
+
+    HDFS-7826. Erasure Coding: Update INodeFile quota computation for striped
+    blocks ( Kai Sasaki via jing9 )
+
+    HDFS-7912. Erasure Coding: track BlockInfo instead of Block in
+    UnderReplicatedBlocks and PendingReplicationBlocks (Jing Zhao)
+
+    HDFS-7369. Erasure coding: distribute recovery work for striped blocks to
+    DataNode (Zhe Zhang)
+
+    HDFS-7864. Erasure Coding: Update safemode calculation for striped blocks
+    (GAO Rui via jing9)
+
+    HDFS-7827. Erasure Coding: support striped blocks in non-protobuf fsimage
+    ( Hui Zheng via jing9 )
+
+    HDFS-7616. Add a test for BlockGroup support in FSImage.
+    (Takuya Fukudome via szetszwo)
+
+    HDFS-7907. Erasure Coding: track invalid, corrupt, and under-recovery striped
+    blocks in NameNode (Jing Zhao)
+
+    HDFS-8005. Erasure Coding: simplify striped block recovery work computation
+    and add tests (Jing Zhao)
+
+    HDFS-7617. Add unit tests for editlog transactions for EC 
+    (Hui Zheng via Zhe Zhang)
+
+    HDFS-7839. Erasure coding: implement facilities in NameNode to create and
+    manage EC zones (Zhe Zhang)
+
+    HDFS-7969. Erasure coding: NameNode support for lease recovery of striped
+    block groups. (Zhe Zhang)
+
+    HDFS-7782. Erasure coding: pread from files in striped layout.
+    (Zhe Zhang and Jing Zhao via Zhe Zhang)
+
+    HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from
+    NameNode (vinayakumarb)
+
+    HDFS-8074. Define a system-wide default EC schema. (Kai Zheng)
+
+    HDFS-8077. Erasure coding: fix bugs in EC zone and symlinks.
+    (Jing Zhao and Zhe Zhang via Jing Zhao)
+
+    HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng)
+
+    HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng)
+
+    HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all
+    ECSchemas loaded in Namenode. (vinayakumarb)
+
+    HDFS-8122. Erasure Coding: Support specifying ECSchema during creation of ECZone.
+    (Vinayakumar B via Zhe Zhang)
+
+    HDFS-8114. Erasure coding: Add auditlog FSNamesystem#createErasureCodingZone if this
+    operation fails. (Rakesh R via Zhe Zhang)
+
+    HDFS-8123. Erasure Coding: Better to move EC related proto messages to a
+    separate erasurecoding proto file (Rakesh R via vinayakumarb)
+
+    HDFS-7349. Support DFS command for the EC encoding (vinayakumarb)
+
+    HDFS-8120. Erasure coding: created util class to analyze striped block groups.
+    (Contributed by Zhe Zhang and Li Bo via Jing Zhao)
+
+    HDFS-7994. Detect if resevered EC Block ID is already used during namenode
+    startup. (Hui Zheng via szetszwo)
+
+    HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. (Hui Zheng via zhz).
+
+    HDFS-8166. DFSStripedOutputStream should not create empty blocks. (Jing Zhao)
+
+    HDFS-7937. Erasure Coding: INodeFile quota computation unit tests.
+    (Kai Sasaki via Jing Zhao)
+
+    HDFS-8145. Fix the editlog corruption exposed by failed TestAddStripedBlocks.
+    (Jing Zhao)
+
+    HDFS-8146. Protobuf changes for BlockECRecoveryCommand and its fields for
+    making it ready for transfer to DN (Uma Maheswara Rao G via vinayakumarb)
+
+    HDFS-8181. createErasureCodingZone sets retryCache state as false always
+    (Uma Maheswara Rao G via vinayakumarb)
+
+    HDFS-8190. StripedBlockUtil.getInternalBlockLength may have overflow error.
+    (szetszwo)
+
+    HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to 
+    create BlockReader. (szetszwo via Zhe Zhang)
+
+    HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema
+    in FileSystemLinkResolver. (szetszwo via Zhe Zhang)
+
+    HDFS-8024. Erasure Coding: ECworker frame, basics, bootstraping and configuration.
+    (umamahesh)
+
+    HDFS-8156. Add/implement necessary APIs even we just have the system default 
+    schema. (Kai Zheng via Zhe Zhang)
+
+    HDFS-8136. Client gets and uses EC schema when reads and writes a stripping
+    file. (Kai Sasaki via Kai Zheng)
+
+    HDFS-8233. Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last
+    stripe is at the block group boundary. (jing9)
+
+    HDFS-8223. Should calculate checksum for parity blocks in DFSStripedOutputStream.
+    (Yi Liu via jing9)
+
+    HDFS-8228. Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause 
+    block id conflicts (Jing Zhao via Zhe Zhang)
+
+    HDFS-8033. Erasure coding: stateful (non-positional) read from files in 
+    striped layout (Zhe Zhang)
+
+    HDFS-8230. Erasure Coding: Ignore DatanodeProtocol#DNA_ERASURE_CODING_RECOVERY 
+    commands from standbynode if any (vinayakumarb)
+
+    HDFS-8189. ClientProtocol#createErasureCodingZone API was wrongly annotated
+    as Idempotent (vinayakumarb)
+
+    HDFS-8235. Erasure Coding: Create DFSStripedInputStream in DFSClient#open.
+    (Kai Sasaki via jing9)
+
+    HDFS-8272. Erasure Coding: simplify the retry logic in DFSStripedInputStream 
+    (stateful read). (Jing Zhao via Zhe Zhang)
+
+    HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil.
+    (Zhe Zhang)
+
+    HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of 
+    datastreamer threads. (Rakesh R via Zhe Zhang)
+
+    HDFS-8308. Erasure Coding: NameNode may get blocked in waitForLoadingFSImage()
+    when loading editlog. (jing9)
+
+    HDFS-7949. WebImageViewer need support file size calculation with striped 
+    blocks. (Rakesh R via Zhe Zhang)
+
+    HDFS-8316. Erasure coding: refactor EC constants to be consistent with HDFS-8249.
+    (Zhe Zhang via jing9)
+
+    HDFS-8281. Erasure Coding: implement parallel stateful reading for striped layout.
+    (jing9)
+
+    HDFS-8137. Send the EC schema to DataNode via EC encoding/recovering command(umamahesh)
+
+    HDFS-8242. Erasure Coding: XML based end-to-end test for ECCli commands
+    (Rakesh R via vinayakumarb)
+
+    HDFS-8324. Add trace info to DFSClient#getErasureCodingZoneInfo(..) (vinayakumarb via 
+    umamahesh)
+
+    HDFS-7672. Handle write failure for stripping blocks and refactor the
+    existing code in DFSStripedOutputStream and StripedDataStreamer.  (szetszwo)
+
+    HDFS-7348. Erasure Coding: DataNode reconstruct striped blocks. 
+    (Yi Liu via Zhe Zhang)
+
+    HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng)
+
+    HDFS-8334. Erasure coding: rename DFSStripedInputStream related test 
+    classes. (Zhe Zhang)
+
+    HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding
+    (umamahesh)
+
+    HDFS-8203. Erasure Coding: Seek and other Ops in DFSStripedInputStream.
+    (Yi Liu via jing9)
+
+    HDFS-8289. Erasure Coding: add ECSchema to HdfsFileStatus. (Yong Zhang via
+    jing9)
+
+    HDFS-8355. Erasure Coding: Refactor BlockInfo and BlockInfoUnderConstruction.
+    (Tsz Wo Nicholas Sze via jing9)
+
+    HDFS-7678. Erasure coding: DFSInputStream with decode functionality (pread).
+    (Zhe Zhang)
+
+    HDFS-8372. Erasure coding: compute storage type quotas for striped files,
+    to be consistent with HDFS-8327. (Zhe Zhang via jing9)
+
+    HDFS-8368. Erasure Coding: DFS opening a non-existent file need to be 
+    handled properly (Rakesh R via zhz)
+
+    HDFS-8363. Erasure Coding: DFSStripedInputStream#seekToNewSource. (yliu)
+
+    HDFS-8195. Erasure coding: Fix file quota change when we complete/commit 
+    the striped blocks. (Takuya Fukudome via zhz)
+
+    HDFS-8364. Erasure coding: fix some minor bugs in EC CLI
+    (Walter Su via vinayakumarb)
+
+    HDFS-8391. NN should consider current EC tasks handling count from DN while 
+    assigning new tasks. (umamahesh)
+
+    HDFS-8367. BlockInfoStriped uses EC schema. (Kai Sasaki via Kai Zheng)
+
+    HDFS-8352. Erasure Coding: test webhdfs read write stripe file. (waltersu4549)
+
+    HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe.
+    (Walter Su via jing9)
+
+    HDFS-8418. Fix the isNeededReplication calculation for Striped block in NN.
+    (Yi Liu via jing9)
+
+    HDFS-8320. Erasure coding: consolidate striping-related terminologies. (zhz)
+
+    HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue 
+    configurable in DFSStripedOutputStream. (Li Bo)
+
+    HDFS-8378. Erasure Coding: Few improvements for the erasure coding worker.
+    (Rakesh R via waltersu4549)
+
+    HDFS-8375. Add cellSize as an XAttr to ECZone. ( Vinayakumar B via zhz).
+
+    HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file.
+    (Yi Liu via zhz).
+
+    HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream.
+    (Tsz Wo Nicholas Sze via jing9)
+
+    HDFS-8427. Remove dataBlockNum and parityBlockNum from BlockInfoStriped.
+    (Kai Sasaki via jing9)
+
+    HDFS-8186. Erasure coding: Make block placement policy for EC file configurable.
+    (Walter Su via zhz)
+
+    HDFS-8294. Erasure Coding: Fix Findbug warnings present in erasure coding.
+    (Rakesh R via zhz)
+
+    HDFS-8441. Erasure Coding: make condition check earlier for setReplication.
+    (waltersu4549)
+
+    HDFS-7768. Change fsck to support EC files.  (Takanobu Asanuma via szetszwo)
+
+    HDFS-8382. Remove chunkSize and initialize from erasure coder. (Kai Zheng)
+
+    HDFS-8408. Revisit and refactor ErasureCodingInfo (vinayakumarb)
+
+    HDFS-8479. Erasure coding: fix striping related logic in FSDirWriteFileOp to
+    sync with HDFS-8421. (Zhe Zhang via jing9)
+
+    HDFS-8481. Erasure coding: remove workarounds in client side stripped blocks 
+    recovering. (zhz)
+
+    HDFS-8336. Expose some administrative erasure coding operations to HdfsAdmin
+    (Uma Maheswara Rao G via vinayakumarb)
+
+    HDFS-8444. Erasure Coding: fix cannot rename a zone dir
+    (Walter Su via vinayakumarb)
+
+    HDFS-8517. Fix a decoding issue in stripped block recovering in client side.
+    (Kai Zheng via jing9)
+
+    HDFS-8453. Erasure coding: properly handle start offset for internal blocks
+    in a block group. (Zhe Zhang via jing9)
+
+    HDFS-7621. Erasure Coding: update the Balancer/Mover data migration logic.
+    (Walter Su via zhz)
+
+    HDFS-8328. Follow-on to update decode for DataNode striped blocks
+    reconstruction. (yliu)
+
+    HDFS-8319. Erasure Coding: support decoding for stateful read.
+    (Jing Zhao via zhz)
+
+    HDFS-8460. Erasure Coding: stateful read result doesn't match data 
+    occasionally because of flawed test. (Walter Su via zhz)
+
+    HDFS-8556. Erasure Coding: Fix usage of 'createZone' (vinayakumarb)
+
+    HDFS-8571. Fix TestErasureCodingCli test (Vinayakumar B via waltersu4549)
+
+    HDFS-8450. Erasure Coding: Consolidate erasure coding zone related
+    implementation into a single class (Rakesh R via vinayakumarb)
+
+    HDFS-8585. Erasure Coding: Remove dataBlockNum and parityBlockNum from
+    StripedBlockProto. (Yi Liu via jing9)
+
+    HDFS-8559. Erasure Coding: fix non-protobuf fsimage for striped blocks.
+    (Jing Zhao via yliu)
+
+    HDFS-8580. Erasure coding: Persist cellSize in BlockInfoStriped and
+    StripedBlocksFeature. (Walter Su via jing9)
+
+    HDFS-8466. Refactor BlockInfoContiguous and fix NPE in
+    TestBlockInfo#testCopyConstructor() (vinayakumarb)
+
+    HDFS-8254. Avoid assigning a leading streamer in StripedDataStreamer to
+    tolerate datanode failure. (Tsz Wo Nicholas Sze via jing9)
+
+    HDFS-8543. Erasure Coding: processOverReplicatedBlock() handles striped block.
+    (Walter Su via jing9)
+
+    HDFS-8602. Erasure Coding: Client can't read(decode) the EC files which have
+    corrupt blocks. (jing9 and Kai Sasaki)
+
+    HDFS-8567. Erasure Coding: SafeMode handles file smaller than a full stripe.
+    (Walter Su via jing9)
+
+    HDFS-8253. DFSStripedOutputStream.closeThreads releases cellBuffers
+    multiple times.  (Kai Sasaki via szetszwo)
+
+    HDFS-8468. 2 RPC calls for every file read in DFSClient#open(..) resulting in
+    double Audit log entries (vinayakumarb)
+
+    HDFS-8684. Erasure Coding: fix some block number calculation for striped
+    block. (yliu)
+
+    HDFS-8461. Erasure coding: fix priority level of UnderReplicatedBlocks for
+    striped block. (Walter Su via jing9)
+
+    HDFS-8719. Erasure Coding: client generates too many small packets when
+    writing parity data. (Li Bo via waltersu4549)
+
+    HDFS-8563. Erasure Coding: fsck handles file smaller than a full stripe.
+    (Walter Su via jing9)
+
+    HDFS-8484. Erasure coding: Two contiguous blocks occupy IDs belong to same
+    striped group. (Walter Su via jing9)
+
+    HDFS-8744. Erasure Coding: the number of chunks in packet is not updated
+    when writing parity data. (Li Bo)
+
+    HDFS-8669. Erasure Coding: handle missing internal block locations in
+    DFSStripedInputStream. (jing9)
+
+    HDFS-8702. Erasure coding: update BlockManager.blockHasEnoughRacks(..) logic
+    for striped block. (Kai Sasaki via jing9)
+
+    HDFS-8734. Erasure Coding: fix one cell need two packets. (Walter Su via
+    jing9)
+
+    HDFS-8619. Erasure Coding: revisit replica counting for striped blocks.
+    (Jing Zhao via yliu)
+
+    HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous
+    blocks in INodeFile. (Zhe Zhang and Yi Liu via zhz)
+
+    HDFS-8787. Erasure coding: rename BlockInfoContiguousUC and BlockInfoStripedUC
+    to be consistent with trunk. (zhz)
+
+    HDFS-8433. Erasure coding: set blockToken in LocatedStripedBlock.(waltersu4549)
+
+    HDFS-8760. Erasure Coding: reuse BlockReader when reading the same block in pread.
+    (jing9)
+
+    HDFS-8781. Erasure Coding: Correctly handle BlockManager#InvalidateBlocks for
+    striped block. (Yi Liu via jing9)
+
+    HDFS-8813. Erasure Coding: Client no need to decode missing parity blocks.
+    (Walter Su via jing9)
+
+    HDFS-8798. Erasure Coding: fix DFSStripedInputStream/DFSStripedOutputStream
+    re-fetch token when expired. (Walter Su via jing9)
+
+    HDFS-8769. Erasure coding: unit test for SequentialBlockGroupIdGenerator.
+    (Rakesh R via waltersu4549)
+
+    HDFS-8202. Improve end to end stirpping file test to add erasure recovering
+    test. (Xinwei Qin via zhz)
+
+    HDFS-8804. Erasure Coding: use DirectBufferPool in DFSStripedInputStream for
+    buffer allocation. (jing9)
+
+    HDFS-8399. Erasure Coding: unit test the behaviour of BlockManager recovery
+    work for the deleted blocks. (Rakesh R via zhz)
+
+    HDFS-8857. Erasure Coding: Fix ArrayIndexOutOfBoundsException in
+    TestWriteStripedFileWithFailure. (Li Bo)
+
+    HDFS-8827. Erasure Coding: Fix NPE when NameNode processes over-replicated
+    striped blocks. (Walter Su and Takuya Fukudome via jing9)
+
+    HDFS-8854. Erasure coding: add ECPolicy to replace schema+cellSize in
+    hadoop-hdfs. (Walter Su via zhz)
+
+    HDFS-8220. Erasure Coding: StripedDataStreamer fails to handle the
+    blocklocations which doesn't satisfy BlockGroupSize. (Rakesh R via zhz)
+
+    HDFS-8838. Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream
+    when the data length is small. (szetszwo via waltersu4549)
+
+    HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC
+    to use BlockUnderConstructionFeature. (Jing Zhao via waltersu4549)
+
+    HDFS-8937. Erasure coding: do not throw exception when setting replication on
+    EC file. (Gao Rui via jing9)
+
+    HDFS-8978. Erasure coding: fix 2 failed tests of DFSStripedOutputStream.
+    (Walter Su via jing9)
+
+    HDFS-8833. Erasure coding: store EC schema and cell size in INodeFile and 
+    eliminate notion of EC zones. (zhz)
+
+    HDFS-8853. Erasure Coding: Provide ECSchema validation when setting EC
+    policy. (andreina via zhz)
+
+    HDFS-8975. Erasure coding : Fix random failure in TestSafeModeWithStripedFile
+    (J.Andreina via vinayakumarb)
+
+    HDFS-7351. Document the HDFS Erasure Coding feature.
+    (umamahesh and Zhe Zhang via wang)
+
+    HDFS-8899. Erasure Coding: use threadpool for EC recovery tasks on DataNode.
+    (Rakesh R via zhz)
+
+    HDFS-9088. Cleanup erasure coding documentation. (wang via zhz)
+
+    HDFS-9097. Erasure coding: update EC command "-s" flag to "-p" when 
+    specifying policy. (zhz)
+
+    HDFS-9086. Rename dfs.datanode.stripedread.threshold.millis to 
+    dfs.datanode.stripedread.timeout.millis. (wang via zhz)
+
+    HDFS-8550. Erasure Coding: Fix FindBugs Multithreaded correctness Warning.
+    (Rakesh R via zhz)
+
+    HDFS-9113. ErasureCodingWorker#processErasureCodingTasks should not fail to process
+    remaining tasks due to one invalid ECTask (umamahesh)
+
+    HDFS-9091. Erasure Coding: Provide DistributedFilesystem API to 
+    getAllErasureCodingPolicies. (Rakesh R via zhz)
+
+    HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too
+    verbose and hurt performance. (Rui Li via Kai Zheng)
+
+    HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from
+    ErasureCodingPolicy (Vinayakumar B via zhz)
+
+    HDFS-9040. Erasure coding: coordinate data streamers in
+    DFSStripedOutputStream. (jing9 and Walter Su)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index 852b040..8b1ede8 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -34,6 +34,7 @@
   hadoop_add_subcommand "debug" "run a Debug Admin to execute HDFS debug commands"
   hadoop_add_subcommand "dfs" "run a filesystem command on the file system"
   hadoop_add_subcommand "dfsadmin" "run a DFS admin client"
+  hadoop_add_subcommand "erasurecode" "run a HDFS ErasureCoding CLI"
   hadoop_add_subcommand "fetchdt" "fetch a delegation token from the NameNode"
   hadoop_add_subcommand "fsck" "run a DFS filesystem checking utility"
   hadoop_add_subcommand "getconf" "get config values from configuration"
@@ -133,6 +134,11 @@
     hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS"
     HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
   ;;
+  erasurecode)
+    CLASS=org.apache.hadoop.hdfs.tools.erasurecode.ECCli
+    hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS"
+    HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
+  ;;
   fetchdt)
     CLASS=org.apache.hadoop.hdfs.tools.DelegationTokenFetcher
   ;;
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 37d74e3..0d24c8f 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
@@ -23,6 +23,7 @@
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
 import org.apache.hadoop.http.HttpConfig;
 
@@ -172,6 +173,8 @@
   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_SAFEMODE_REPLICATION_MIN_KEY =
       "dfs.namenode.safemode.replication.min";
   public static final String  DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY = "dfs.namenode.replication.pending.timeout-sec";
@@ -400,6 +403,14 @@
   public static final int     DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT = 21600;
   public static final String  DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY = "dfs.datanode.directoryscan.threads";
   public static final int     DFS_DATANODE_DIRECTORYSCAN_THREADS_DEFAULT = 1;
+  public static final String  DFS_DATANODE_STRIPED_READ_THREADS_KEY = "dfs.datanode.stripedread.threads";
+  public static final int     DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT = 20;
+  public static final String  DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.stripedread.buffer.size";
+  public static final int     DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 64 * 1024;
+  public static final String  DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY = "dfs.datanode.stripedread.timeout.millis";
+  public static final int     DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT = 5000; //5s
+  public static final String  DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY = "dfs.datanode.striped.blockrecovery.threads.size";
+  public static final int     DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT = 8;
   public static final String
       DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY =
       "dfs.datanode.directoryscan.throttle.limit.ms.per.sec";
@@ -464,6 +475,8 @@
   public static final Class<BlockPlacementPolicyDefault> DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT = BlockPlacementPolicyDefault.class;
   public static final String  DFS_REPLICATION_MAX_KEY = "dfs.replication.max";
   public static final int     DFS_REPLICATION_MAX_DEFAULT = 512;
+  public static final String DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY = "dfs.block.placement.ec.classname";
+  public static final Class<BlockPlacementPolicyRackFaultTolerant> DFS_BLOCK_PLACEMENT_EC_CLASSNAME_DEFAULT = BlockPlacementPolicyRackFaultTolerant.class;
 
   public static final String  DFS_DF_INTERVAL_KEY = "dfs.df.interval";
   public static final int     DFS_DF_INTERVAL_DEFAULT = 60000;
@@ -738,7 +751,7 @@
       "dfs.namenode.reject-unresolved-dn-topology-mapping";
   public static final boolean DFS_REJECT_UNRESOLVED_DN_TOPOLOGY_MAPPING_DEFAULT =
       false;
-  
+
   // Slow io warning log threshold settings for dfsclient and datanode.
   public static final String DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY =
     "dfs.datanode.slow.io.warning.threshold.ms";
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 5d405ab..2309843 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -36,6 +36,7 @@
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.io.PrintStream;
 import java.io.UnsupportedEncodingException;
 import java.net.InetAddress;
@@ -88,6 +89,7 @@
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 import com.google.protobuf.BlockingService;
 
 @InterfaceAudience.Private
@@ -1448,5 +1450,4 @@
         .createKeyProviderCryptoExtension(keyProvider);
     return cryptoProvider;
   }
-
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
index 84499bb..7b055fc8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
@@ -40,6 +40,7 @@
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 
 /**
  * The public API for performing administrative functions on HDFS. Those writing
@@ -363,4 +364,39 @@
       throws IOException {
     dfs.setStoragePolicy(src, policyName);
   }
+
+  /**
+   * Set the source path to the specified erasure coding policy.
+   *
+   * @param path The source path referring to a directory.
+   * @param ecPolicy The erasure coding policy for the directory.
+   *                 If null, the default will be used.
+   * @throws IOException
+   */
+  public void setErasureCodingPolicy(final Path path,
+      final ErasureCodingPolicy ecPolicy) throws IOException {
+    dfs.setErasureCodingPolicy(path, ecPolicy);
+  }
+
+  /**
+   * Get the erasure coding policy information for the specified path
+   *
+   * @param path
+   * @return Returns the policy information if file or directory on the path is
+   *          erasure coded. Null otherwise.
+   * @throws IOException
+   */
+  public ErasureCodingPolicy getErasureCodingPolicy(final Path path)
+      throws IOException {
+    return dfs.getErasureCodingPolicy(path);
+  }
+
+  /**
+   * Get the Erasure coding policies supported.
+   *
+   * @throws IOException
+   */
+  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
+    return dfs.getClient().getErasureCodingPolicies();
+  }
 }
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 d93277c..1e4b8995 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
@@ -200,6 +200,12 @@
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
@@ -216,6 +222,7 @@
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
@@ -422,7 +429,8 @@
           req.getClientName(), flags);
       AppendResponseProto.Builder builder = AppendResponseProto.newBuilder();
       if (result.getLastBlock() != null) {
-        builder.setBlock(PBHelperClient.convert(result.getLastBlock()));
+        builder.setBlock(PBHelperClient.convertLocatedBlock(
+            result.getLastBlock()));
       }
       if (result.getFileStatus() != null) {
         builder.setStat(PBHelperClient.convert(result.getFileStatus()));
@@ -498,7 +506,7 @@
           (favor == null || favor.size() == 0) ? null : favor
               .toArray(new String[favor.size()]));
       return AddBlockResponseProto.newBuilder()
-          .setBlock(PBHelperClient.convert(result)).build();
+          .setBlock(PBHelperClient.convertLocatedBlock(result)).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -522,7 +530,7 @@
               new DatanodeInfoProto[excludesList.size()])),
           req.getNumAdditionalNodes(), req.getClientName());
       return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
-          PBHelperClient.convert(result))
+      PBHelperClient.convertLocatedBlock(result))
           .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -548,7 +556,7 @@
       ReportBadBlocksRequestProto req) throws ServiceException {
     try {
       List<LocatedBlockProto> bl = req.getBlocksList();
-      server.reportBadBlocks(PBHelperClient.convertLocatedBlock(
+      server.reportBadBlocks(PBHelperClient.convertLocatedBlocks(
           bl.toArray(new LocatedBlockProto[bl.size()])));
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -953,8 +961,8 @@
       RpcController controller, UpdateBlockForPipelineRequestProto req)
       throws ServiceException {
     try {
-      LocatedBlockProto result = PBHelperClient.convert(server
-          .updateBlockForPipeline(PBHelperClient.convert(req.getBlock()),
+      LocatedBlockProto result = PBHelperClient.convertLocatedBlock(
+          server.updateBlockForPipeline(PBHelperClient.convert(req.getBlock()),
               req.getClientName()));
       return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result)
           .build();
@@ -1394,6 +1402,20 @@
   }
 
   @Override
+  public SetErasureCodingPolicyResponseProto setErasureCodingPolicy(
+      RpcController controller, SetErasureCodingPolicyRequestProto req)
+      throws ServiceException {
+    try {
+      ErasureCodingPolicy ecPolicy = req.hasEcPolicy() ?
+          PBHelperClient.convertErasureCodingPolicy(req.getEcPolicy()) : null;
+      server.setErasureCodingPolicy(req.getSrc(), ecPolicy);
+      return SetErasureCodingPolicyResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public SetXAttrResponseProto setXAttr(RpcController controller,
       SetXAttrRequestProto req) throws ServiceException {
     try {
@@ -1514,4 +1536,35 @@
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetErasureCodingPoliciesResponseProto getErasureCodingPolicies(RpcController controller,
+      GetErasureCodingPoliciesRequestProto request) throws ServiceException {
+    try {
+      ErasureCodingPolicy[] ecPolicies = server.getErasureCodingPolicies();
+      GetErasureCodingPoliciesResponseProto.Builder resBuilder = GetErasureCodingPoliciesResponseProto
+          .newBuilder();
+      for (ErasureCodingPolicy ecPolicy : ecPolicies) {
+        resBuilder.addEcPolicies(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
+      }
+      return resBuilder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetErasureCodingPolicyResponseProto getErasureCodingPolicy(RpcController controller,
+      GetErasureCodingPolicyRequestProto request) throws ServiceException {
+    try {
+      ErasureCodingPolicy ecPolicy = server.getErasureCodingPolicy(request.getSrc());
+      GetErasureCodingPolicyResponseProto.Builder builder = GetErasureCodingPolicyResponseProto.newBuilder();
+      if (ecPolicy != null) {
+        builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy));
+      }
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }
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 705d573..6b20af5 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
@@ -281,7 +281,7 @@
     ReportBadBlocksRequestProto.Builder builder = ReportBadBlocksRequestProto
         .newBuilder();
     for (int i = 0; i < blocks.length; i++) {
-      builder.addBlocks(i, PBHelperClient.convert(blocks[i]));
+      builder.addBlocks(i, PBHelperClient.convertLocatedBlock(blocks[i]));
     }
     ReportBadBlocksRequestProto req = builder.build();
     try {
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 4b9f7c4..93c11ba 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
@@ -259,7 +259,7 @@
     List<LocatedBlockProto> lbps = request.getBlocksList();
     LocatedBlock [] blocks = new LocatedBlock [lbps.size()];
     for(int i=0; i<lbps.size(); i++) {
-      blocks[i] = PBHelperClient.convert(lbps.get(i));
+      blocks[i] = PBHelperClient.convertLocatedBlockProto(lbps.get(i));
     }
     try {
       impl.reportBadBlocks(blocks);
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 75b3811..05c498f 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
@@ -19,6 +19,7 @@
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 
 import com.google.protobuf.ByteString;
@@ -30,10 +31,11 @@
 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.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
@@ -45,7 +47,16 @@
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
+import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
@@ -65,7 +76,6 @@
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.ReplicaStateProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
@@ -76,12 +86,15 @@
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -169,22 +182,34 @@
   }
 
   public static BlockWithLocationsProto convert(BlockWithLocations blk) {
-    return BlockWithLocationsProto.newBuilder()
-        .setBlock(PBHelperClient.convert(blk.getBlock()))
+    BlockWithLocationsProto.Builder builder = BlockWithLocationsProto
+        .newBuilder().setBlock(PBHelperClient.convert(blk.getBlock()))
         .addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids()))
         .addAllStorageUuids(Arrays.asList(blk.getStorageIDs()))
-        .addAllStorageTypes(PBHelperClient.convertStorageTypes(blk.getStorageTypes()))
-        .build();
+        .addAllStorageTypes(PBHelperClient.convertStorageTypes(blk.getStorageTypes()));
+    if (blk instanceof StripedBlockWithLocations) {
+      StripedBlockWithLocations sblk = (StripedBlockWithLocations) blk;
+      builder.setIndices(PBHelperClient.getByteString(sblk.getIndices()));
+      builder.setDataBlockNum(sblk.getDataBlockNum());
+      builder.setCellSize(sblk.getCellSize());
+    }
+    return builder.build();
   }
 
   public static BlockWithLocations convert(BlockWithLocationsProto b) {
     final List<String> datanodeUuids = b.getDatanodeUuidsList();
     final List<String> storageUuids = b.getStorageUuidsList();
     final List<StorageTypeProto> storageTypes = b.getStorageTypesList();
-    return new BlockWithLocations(PBHelperClient.convert(b.getBlock()),
+    BlockWithLocations blk = new BlockWithLocations(PBHelperClient.
+        convert(b.getBlock()),
         datanodeUuids.toArray(new String[datanodeUuids.size()]),
         storageUuids.toArray(new String[storageUuids.size()]),
         PBHelperClient.convertStorageTypes(storageTypes, storageUuids.size()));
+    if (b.hasIndices()) {
+      blk = new StripedBlockWithLocations(blk, b.getIndices().toByteArray(),
+          (short) b.getDataBlockNum(), b.getCellSize());
+    }
+    return blk;
   }
 
   public static BlocksWithLocationsProto convert(BlocksWithLocations blks) {
@@ -333,7 +358,7 @@
     if (b == null) {
       return null;
     }
-    LocatedBlockProto lb = PBHelperClient.convert((LocatedBlock) b);
+    LocatedBlockProto lb = PBHelperClient.convertLocatedBlock(b);
     RecoveringBlockProto.Builder builder = RecoveringBlockProto.newBuilder();
     builder.setBlock(lb).setNewGenStamp(b.getNewGenerationStamp());
     if(b.getNewBlock() != null)
@@ -413,6 +438,8 @@
       return REG_CMD;
     case BlockIdCommand:
       return PBHelper.convert(proto.getBlkIdCmd());
+    case BlockECRecoveryCommand:
+      return PBHelper.convert(proto.getBlkECRecoveryCmd());
     default:
       return null;
     }
@@ -542,6 +569,11 @@
       builder.setCmdType(DatanodeCommandProto.Type.BlockIdCommand).
         setBlkIdCmd(PBHelper.convert((BlockIdCommand) datanodeCommand));
       break;
+    case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY:
+      builder.setCmdType(DatanodeCommandProto.Type.BlockECRecoveryCommand)
+          .setBlkECRecoveryCmd(
+              convert((BlockECRecoveryCommand) datanodeCommand));
+      break;
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     default:
       builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand);
@@ -705,7 +737,6 @@
         .build();
   }
 
-
   public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
     if (s == null) return null;
     switch (s.getState()) {
@@ -790,4 +821,140 @@
         setLeaseId(context.getLeaseId()).
         build();
   }
+
+  private static List<Integer> convertIntArray(short[] liveBlockIndices) {
+    List<Integer> liveBlockIndicesList = new ArrayList<>();
+    for (short s : liveBlockIndices) {
+      liveBlockIndicesList.add((int) s);
+    }
+    return liveBlockIndicesList;
+  }
+
+  private static StorageTypesProto convertStorageTypesProto(
+      StorageType[] targetStorageTypes) {
+    StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
+    for (StorageType storageType : targetStorageTypes) {
+      builder.addStorageTypes(PBHelperClient.convertStorageType(storageType));
+    }
+    return builder.build();
+  }
+
+  private static HdfsProtos.StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) {
+    HdfsProtos.StorageUuidsProto.Builder builder = HdfsProtos.StorageUuidsProto.newBuilder();
+    for (String storageUuid : targetStorageIDs) {
+      builder.addStorageUuids(storageUuid);
+    }
+    return builder.build();
+  }
+
+  private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) {
+    DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder();
+    for (DatanodeInfo datanodeInfo : dnInfos) {
+      builder.addDatanodes(PBHelperClient.convert(datanodeInfo));
+    }
+    return builder.build();
+  }
+
+  private static String[] convert(HdfsProtos.StorageUuidsProto targetStorageUuidsProto) {
+    List<String> storageUuidsList = targetStorageUuidsProto
+        .getStorageUuidsList();
+    String[] storageUuids = new String[storageUuidsList.size()];
+    for (int i = 0; i < storageUuidsList.size(); i++) {
+      storageUuids[i] = storageUuidsList.get(i);
+    }
+    return storageUuids;
+  }
+
+  public static BlockECRecoveryInfo convertBlockECRecoveryInfo(
+      BlockECRecoveryInfoProto blockEcRecoveryInfoProto) {
+    ExtendedBlockProto blockProto = blockEcRecoveryInfoProto.getBlock();
+    ExtendedBlock block = PBHelperClient.convert(blockProto);
+
+    DatanodeInfosProto sourceDnInfosProto = blockEcRecoveryInfoProto
+        .getSourceDnInfos();
+    DatanodeInfo[] sourceDnInfos = PBHelperClient.convert(sourceDnInfosProto);
+
+    DatanodeInfosProto targetDnInfosProto = blockEcRecoveryInfoProto
+        .getTargetDnInfos();
+    DatanodeInfo[] targetDnInfos = PBHelperClient.convert(targetDnInfosProto);
+
+    HdfsProtos.StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto
+        .getTargetStorageUuids();
+    String[] targetStorageUuids = convert(targetStorageUuidsProto);
+
+    StorageTypesProto targetStorageTypesProto = blockEcRecoveryInfoProto
+        .getTargetStorageTypes();
+    StorageType[] convertStorageTypes = PBHelperClient.convertStorageTypes(
+        targetStorageTypesProto.getStorageTypesList(), targetStorageTypesProto
+            .getStorageTypesList().size());
+
+    List<Integer> liveBlockIndicesList = blockEcRecoveryInfoProto
+        .getLiveBlockIndicesList();
+    short[] liveBlkIndices = new short[liveBlockIndicesList.size()];
+    for (int i = 0; i < liveBlockIndicesList.size(); i++) {
+      liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue();
+    }
+
+    ErasureCodingPolicy ecPolicy =
+        PBHelperClient.convertErasureCodingPolicy(
+            blockEcRecoveryInfoProto.getEcPolicy());
+
+    return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos,
+        targetStorageUuids, convertStorageTypes, liveBlkIndices, ecPolicy);
+  }
+
+  public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo(
+      BlockECRecoveryInfo blockEcRecoveryInfo) {
+    BlockECRecoveryInfoProto.Builder builder = BlockECRecoveryInfoProto
+        .newBuilder();
+    builder.setBlock(PBHelperClient.convert(
+        blockEcRecoveryInfo.getExtendedBlock()));
+
+    DatanodeInfo[] sourceDnInfos = blockEcRecoveryInfo.getSourceDnInfos();
+    builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos));
+
+    DatanodeInfo[] targetDnInfos = blockEcRecoveryInfo.getTargetDnInfos();
+    builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos));
+
+    String[] targetStorageIDs = blockEcRecoveryInfo.getTargetStorageIDs();
+    builder.setTargetStorageUuids(convertStorageIDs(targetStorageIDs));
+
+    StorageType[] targetStorageTypes = blockEcRecoveryInfo
+        .getTargetStorageTypes();
+    builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes));
+
+    short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices();
+    builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices));
+
+    builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(
+        blockEcRecoveryInfo.getErasureCodingPolicy()));
+
+    return builder.build();
+  }
+
+  public static BlockECRecoveryCommandProto convert(
+      BlockECRecoveryCommand blkECRecoveryCmd) {
+    BlockECRecoveryCommandProto.Builder builder = BlockECRecoveryCommandProto
+        .newBuilder();
+    Collection<BlockECRecoveryInfo> blockECRecoveryInfos = blkECRecoveryCmd
+        .getECTasks();
+    for (BlockECRecoveryInfo blkECRecoveryInfo : blockECRecoveryInfos) {
+      builder
+          .addBlockECRecoveryinfo(convertBlockECRecoveryInfo(blkECRecoveryInfo));
+    }
+    return builder.build();
+  }
+
+  public static BlockECRecoveryCommand convert(
+      BlockECRecoveryCommandProto blkECRecoveryCmdProto) {
+    Collection<BlockECRecoveryInfo> blkECRecoveryInfos = new ArrayList<>();
+    List<BlockECRecoveryInfoProto> blockECRecoveryinfoList = blkECRecoveryCmdProto
+        .getBlockECRecoveryinfoList();
+    for (BlockECRecoveryInfoProto blockECRecoveryInfoProto : blockECRecoveryinfoList) {
+      blkECRecoveryInfos
+          .add(convertBlockECRecoveryInfo(blockECRecoveryInfoProto));
+    }
+    return new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY,
+        blkECRecoveryInfos);
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index f3f3d6f..2bd28ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -49,8 +49,8 @@
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Source;
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Task;
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Util;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -211,7 +211,9 @@
    */
   private static void checkReplicationPolicyCompatibility(Configuration conf
       ) throws UnsupportedActionException {
-    if (!(BlockPlacementPolicy.getInstance(conf, null, null, null) instanceof 
+    BlockPlacementPolicies placementPolicies =
+        new BlockPlacementPolicies(conf, null, null, null);
+    if (!(placementPolicies.getPolicy(false) instanceof
         BlockPlacementPolicyDefault)) {
       throw new UnsupportedActionException(
           "Balancer without BlockPlacementPolicyDefault");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index ff2d762..3b36555 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.balancer;
 
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
 import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
 
 import java.io.BufferedInputStream;
@@ -65,6 +66,7 @@
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
@@ -148,18 +150,17 @@
     private final Map<Block, DBlock> map = new HashMap<Block, DBlock>();
 
     /**
-     * Get the block from the map;
-     * if the block is not found, create a new block and put it in the map.
+     * Put block in the map if it's not found
+     * @return the block which be put in the map the first time
      */
-    private DBlock get(Block b) {
-      DBlock block = map.get(b);
-      if (block == null) {
-        block = new DBlock(b);
-        map.put(b, block);
+    private DBlock putIfAbsent(Block blk, DBlock dblk) {
+      if (!map.containsKey(blk)) {
+        map.put(blk, dblk);
+        return dblk;
       }
-      return block;
+      return map.get(blk);
     }
-    
+
     /** Remove all blocks except for the moved blocks. */
     private void removeAllButRetain(MovedBlocks<StorageGroup> movedBlocks) {
       for (Iterator<Block> i = map.keySet().iterator(); i.hasNext();) {
@@ -200,9 +201,9 @@
     }
   }
 
-  /** This class keeps track of a scheduled block move */
+  /** This class keeps track of a scheduled reportedBlock move */
   public class PendingMove {
-    private DBlock block;
+    private DBlock reportedBlock;
     private Source source;
     private DDatanode proxySource;
     private StorageGroup target;
@@ -214,7 +215,7 @@
 
     @Override
     public String toString() {
-      final Block b = block != null ? block.getBlock() : null;
+      final Block b = reportedBlock != null ? reportedBlock.getBlock() : null;
       String bStr = b != null ? (b + " with size=" + b.getNumBytes() + " ")
           : " ";
       return bStr + "from " + source.getDisplayName() + " to " + target
@@ -223,8 +224,8 @@
     }
 
     /**
-     * Choose a block & a proxy source for this pendingMove whose source &
-     * target have already been chosen.
+     * Choose a good block/blockGroup from source & Get reportedBlock from
+     * the block & Choose a proxy source for the reportedBlock.
      * 
      * @return true if a block and its proxy are chosen; false otherwise
      */
@@ -248,7 +249,11 @@
       synchronized (block) {
         synchronized (movedBlocks) {
           if (isGoodBlockCandidate(source, target, targetStorageType, block)) {
-            this.block = block;
+            if (block instanceof DBlockStriped) {
+              reportedBlock = ((DBlockStriped) block).getInternalBlock(source);
+            } else {
+              reportedBlock = block;
+            }
             if (chooseProxySource()) {
               movedBlocks.put(block);
               if (LOG.isDebugEnabled()) {
@@ -275,7 +280,7 @@
       }
       // if node group is supported, first try add nodes in the same node group
       if (cluster.isNodeGroupAware()) {
-        for (StorageGroup loc : block.getLocations()) {
+        for (StorageGroup loc : reportedBlock.getLocations()) {
           if (cluster.isOnSameNodeGroup(loc.getDatanodeInfo(), targetDN)
               && addTo(loc)) {
             return true;
@@ -283,13 +288,13 @@
         }
       }
       // check if there is replica which is on the same rack with the target
-      for (StorageGroup loc : block.getLocations()) {
+      for (StorageGroup loc : reportedBlock.getLocations()) {
         if (cluster.isOnSameRack(loc.getDatanodeInfo(), targetDN) && addTo(loc)) {
           return true;
         }
       }
       // find out a non-busy replica
-      for (StorageGroup loc : block.getLocations()) {
+      for (StorageGroup loc : reportedBlock.getLocations()) {
         if (addTo(loc)) {
           return true;
         }
@@ -297,7 +302,7 @@
       return false;
     }
 
-    /** add to a proxy source for specific block movement */
+    /** add to a proxy source for specific reportedBlock movement */
     private boolean addTo(StorageGroup g) {
       final DDatanode dn = g.getDDatanode();
       if (dn.addPendingBlock(this)) {
@@ -310,6 +315,7 @@
     /** Dispatch the move to the proxy source & wait for the response. */
     private void dispatch() {
       LOG.info("Start moving " + this);
+      assert !(reportedBlock instanceof DBlockStriped);
 
       Socket sock = new Socket();
       DataOutputStream out = null;
@@ -324,7 +330,7 @@
         OutputStream unbufOut = sock.getOutputStream();
         InputStream unbufIn = sock.getInputStream();
         ExtendedBlock eb = new ExtendedBlock(nnc.getBlockpoolID(),
-            block.getBlock());
+            reportedBlock.getBlock());
         final KeyManager km = nnc.getKeyManager(); 
         Token<BlockTokenIdentifier> accessToken = km.getAccessToken(eb);
         IOStreamPair saslStreams = saslClient.socketSend(sock, unbufOut,
@@ -338,7 +344,7 @@
 
         sendRequest(out, eb, accessToken);
         receiveResponse(in);
-        nnc.getBytesMoved().addAndGet(block.getNumBytes());
+        nnc.getBytesMoved().addAndGet(reportedBlock.getNumBytes());
         target.getDDatanode().setHasSuccess();
         LOG.info("Successfully moved " + this);
       } catch (IOException e) {
@@ -367,14 +373,14 @@
       }
     }
 
-    /** Send a block replace request to the output stream */
+    /** Send a reportedBlock replace request to the output stream */
     private void sendRequest(DataOutputStream out, ExtendedBlock eb,
         Token<BlockTokenIdentifier> accessToken) throws IOException {
       new Sender(out).replaceBlock(eb, target.storageType, accessToken,
           source.getDatanodeInfo().getDatanodeUuid(), proxySource.datanode);
     }
 
-    /** Receive a block copy response from the input stream */
+    /** Receive a reportedBlock copy response from the input stream */
     private void receiveResponse(DataInputStream in) throws IOException {
       BlockOpResponseProto response =
           BlockOpResponseProto.parseFrom(vintPrefixed(in));
@@ -382,13 +388,13 @@
         // read intermediate responses
         response = BlockOpResponseProto.parseFrom(vintPrefixed(in));
       }
-      String logInfo = "block move is failed";
+      String logInfo = "reportedBlock move is failed";
       DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
     }
 
     /** reset the object */
     private void reset() {
-      block = null;
+      reportedBlock = null;
       source = null;
       proxySource = null;
       target = null;
@@ -400,6 +406,47 @@
     public DBlock(Block block) {
       super(block);
     }
+
+    public long getNumBytes(StorageGroup storage) {
+      return super.getNumBytes();
+    }
+  }
+
+  public static class DBlockStriped extends DBlock {
+
+    final byte[] indices;
+    final short dataBlockNum;
+    final int cellSize;
+
+    public DBlockStriped(Block block, byte[] indices, short dataBlockNum,
+        int cellSize) {
+      super(block);
+      this.indices = indices;
+      this.dataBlockNum = dataBlockNum;
+      this.cellSize = cellSize;
+    }
+
+    public DBlock getInternalBlock(StorageGroup storage) {
+      int idxInLocs = locations.indexOf(storage);
+      if (idxInLocs == -1) {
+        return null;
+      }
+      byte idxInGroup = indices[idxInLocs];
+      long blkId = getBlock().getBlockId() + idxInGroup;
+      long numBytes = getInternalBlockLength(getNumBytes(), cellSize,
+          dataBlockNum, idxInGroup);
+      Block blk = new Block(getBlock());
+      blk.setBlockId(blkId);
+      blk.setNumBytes(numBytes);
+      DBlock dblk = new DBlock(blk);
+      dblk.addLocation(storage);
+      return dblk;
+    }
+
+    @Override
+    public long getNumBytes(StorageGroup storage) {
+      return getInternalBlock(storage).getNumBytes();
+    }
   }
 
   /** The class represents a desired move. */
@@ -475,7 +522,7 @@
       private PendingMove addPendingMove(DBlock block, final PendingMove pm) {
         if (getDDatanode().addPendingBlock(pm)) {
           if (pm.markMovedIfGoodBlock(block, getStorageType())) {
-            incScheduledSize(pm.block.getNumBytes());
+            incScheduledSize(pm.reportedBlock.getNumBytes());
             return pm;
           } else {
             getDDatanode().removePendingBlock(pm);
@@ -650,30 +697,44 @@
      */
     private long getBlockList() throws IOException {
       final long size = Math.min(getBlocksSize, blocksToReceive);
-      final BlocksWithLocations newBlocks = nnc.getBlocks(getDatanodeInfo(), size);
+      final BlocksWithLocations newBlksLocs =
+          nnc.getBlocks(getDatanodeInfo(), size);
 
       if (LOG.isTraceEnabled()) {
         LOG.trace("getBlocks(" + getDatanodeInfo() + ", "
             + StringUtils.TraditionalBinaryPrefix.long2String(size, "B", 2)
-            + ") returns " + newBlocks.getBlocks().length + " blocks.");
+            + ") returns " + newBlksLocs.getBlocks().length + " blocks.");
       }
 
       long bytesReceived = 0;
-      for (BlockWithLocations blk : newBlocks.getBlocks()) {
+      for (BlockWithLocations blkLocs : newBlksLocs.getBlocks()) {
         // Skip small blocks.
-        if (blk.getBlock().getNumBytes() < getBlocksMinBlockSize) {
+        if (blkLocs.getBlock().getNumBytes() < getBlocksMinBlockSize) {
           continue;
         }
 
-        bytesReceived += blk.getBlock().getNumBytes();
+        DBlock block;
+        if (blkLocs instanceof StripedBlockWithLocations) {
+          StripedBlockWithLocations sblkLocs =
+              (StripedBlockWithLocations) blkLocs;
+          // approximate size
+          bytesReceived += sblkLocs.getBlock().getNumBytes() /
+              sblkLocs.getDataBlockNum();
+          block = new DBlockStriped(sblkLocs.getBlock(), sblkLocs.getIndices(),
+              sblkLocs.getDataBlockNum(), sblkLocs.getCellSize());
+        } else {
+          bytesReceived += blkLocs.getBlock().getNumBytes();
+          block = new DBlock(blkLocs.getBlock());
+        }
+
         synchronized (globalBlocks) {
-          final DBlock block = globalBlocks.get(blk.getBlock());
+          block = globalBlocks.putIfAbsent(blkLocs.getBlock(), block);
           synchronized (block) {
             block.clearLocations();
 
             // update locations
-            final String[] datanodeUuids = blk.getDatanodeUuids();
-            final StorageType[] storageTypes = blk.getStorageTypes();
+            final String[] datanodeUuids = blkLocs.getDatanodeUuids();
+            final StorageType[] storageTypes = blkLocs.getStorageTypes();
             for (int i = 0; i < datanodeUuids.length; i++) {
               final StorageGroup g = storageGroupMap.get(
                   datanodeUuids[i], storageTypes[i]);
@@ -712,6 +773,8 @@
      * target throttling has been considered. They are chosen only when they
      * have the capacity to support this block move. The block should be
      * dispatched immediately after this method is returned.
+     * If the block is a block group. Only the internal block on this source
+     * will be dispatched.
      * 
      * @return a move that's good for the source to dispatch immediately.
      */
@@ -723,7 +786,7 @@
         if (target.addPendingBlock(pendingBlock)) {
           // target is not busy, so do a tentative block allocation
           if (pendingBlock.chooseBlockAndProxy()) {
-            long blockSize = pendingBlock.block.getNumBytes();
+            long blockSize = pendingBlock.reportedBlock.getNumBytes(this);
             incScheduledSize(-blockSize);
             task.size -= blockSize;
             if (task.size == 0) {
@@ -802,7 +865,7 @@
             blocksToReceive -= received;
             continue;
           } catch (IOException e) {
-            LOG.warn("Exception while getting block list", e);
+            LOG.warn("Exception while getting reportedBlock list", e);
             return;
           }
         } else {
@@ -962,7 +1025,7 @@
 
 
   public void executePendingMove(final PendingMove p) {
-    // move the block
+    // move the reportedBlock
     final DDatanode targetDn = p.target.getDDatanode();
     ExecutorService moveExecutor = targetDn.getMoveExecutor();
     if (moveExecutor == null) {
@@ -975,7 +1038,6 @@
       LOG.warn("No mover threads available: skip moving " + p);
       return;
     }
-
     moveExecutor.execute(new Runnable() {
       @Override
       public void run() {
@@ -1020,14 +1082,14 @@
       }
     }
 
-    // wait for all block moving to be done
+    // wait for all reportedBlock moving to be done
     waitForMoveCompletion(targets);
 
     return getBytesMoved() - bytesLastMoved;
   }
 
   /**
-   * Wait for all block move confirmations.
+   * Wait for all reportedBlock move confirmations.
    * @return true if there is failed move execution
    */
   public static boolean waitForMoveCompletion(
@@ -1066,10 +1128,10 @@
   }
 
   /**
-   * Decide if the block is a good candidate to be moved from source to target.
-   * A block is a good candidate if
+   * Decide if the block/blockGroup is a good candidate to be moved from source
+   * to target. A block is a good candidate if
    * 1. the block is not in the process of being moved/has not been moved;
-   * 2. the block does not have a replica on the target;
+   * 2. the block does not have a replica/internalBlock on the target;
    * 3. doing the move does not reduce the number of racks that the block has
    */
   private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target,
@@ -1086,7 +1148,7 @@
     }
     final DatanodeInfo targetDatanode = target.getDatanodeInfo();
     if (source.getDatanodeInfo().equals(targetDatanode)) {
-      // the block is moved inside same DN
+      // the reportedBlock is moved inside same DN
       return true;
     }
 
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 95d9983..2f214be 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 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 (striped or contiguous).
    */
   public BlockInfo[] getBlocks();
 
@@ -55,6 +55,12 @@
   public long getPreferredBlockSize();
 
   /**
+   * Get block replication for the collection.
+   * @return block replication value. Return 0 if the file is erasure coded.
+   */
+  public short getPreferredBlockReplication();
+
+  /**
    * @return the storage policy ID.
    */
   public byte getStoragePolicyID();
@@ -65,7 +71,7 @@
   public String getName();
 
   /**
-   * Set the block at the given index.
+   * Set the block (contiguous or striped) at the given index.
    */
   public void setBlock(int index, BlockInfo blk);
 
@@ -73,7 +79,8 @@
    * Convert the last block of the collection to an under-construction block
    * and set the locations.
    */
-  public void convertLastBlockToUC(BlockInfo lastBlock, DatanodeStorageInfo[] targets) throws IOException;
+  public void convertLastBlockToUC(BlockInfo lastBlock,
+      DatanodeStorageInfo[] targets) throws IOException;
 
   /**
    * @return whether the block collection is under construction.
@@ -81,6 +88,11 @@
   public boolean isUnderConstruction();
 
   /**
+   * @return whether the block collection is in striping format
+   */
+  boolean isStriped();
+
+  /**
    * @return the id for the block collection
    */
   long getId();
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 feebd87..685cfcb 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
@@ -54,10 +54,12 @@
    * The global block ID space for this file system.
    */
   private final SequentialBlockIdGenerator blockIdGenerator;
+  private final SequentialBlockGroupIdGenerator blockGroupIdGenerator;
 
   public BlockIdManager(BlockManager blockManager) {
     this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager);
+    this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager);
   }
 
   /**
@@ -102,21 +104,38 @@
   }
 
   /**
-   * 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) {
@@ -187,10 +206,14 @@
   /**
    * Increments, logs and then returns the block ID
    */
-  public long nextBlockId() {
+  public long nextContiguousBlockId() {
     return blockIdGenerator.nextValue();
   }
 
+  public long nextStripedBlockId() {
+    return blockGroupIdGenerator.nextValue();
+  }
+
   public boolean isGenStampInFuture(Block block) {
     if (isLegacyBlock(block)) {
       return block.getGenerationStamp() > getGenerationStampV1();
@@ -206,4 +229,27 @@
       .LAST_RESERVED_BLOCK_ID);
     generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP;
   }
-}
\ No newline at end of file
+
+  public static boolean isStripedBlockID(long id) {
+    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 & (~HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
+  }
+
+  public static int getBlockIndex(Block reportedBlock) {
+    return (int) (reportedBlock.getBlockId() &
+        HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
+  }
+
+  SequentialBlockGroupIdGenerator getBlockGroupIdGenerator() {
+    return blockGroupIdGenerator;
+  }
+}
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 810784d..92a1135 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
@@ -31,9 +31,10 @@
 import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID;
 
 /**
- * BlockInfo class maintains for a given block
- * the {@link BlockCollection} it is part of and datanodes where the replicas of
- * the block are stored.
+ * 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.
  */
 @InterfaceAudience.Private
 public abstract class BlockInfo extends Block
@@ -72,28 +73,20 @@
 
   /**
    * Construct an entry for blocksmap
-   * @param replication the block's replication factor
+   * @param size the block's replication factor, or the total number of blocks
+   *             in the block group
    */
-  public BlockInfo(short replication) {
-    this.triplets = new Object[3*replication];
+  public BlockInfo(short size) {
+    this.triplets = new Object[3 * size];
     this.bcId = INVALID_INODE_ID;
-    this.replication = replication;
+    this.replication = isStriped() ? 0 : size;
   }
 
-  public BlockInfo(Block blk, short replication) {
+  public BlockInfo(Block blk, short size) {
     super(blk);
-    this.triplets = new Object[3*replication];
+    this.triplets = new Object[3*size];
     this.bcId = INVALID_INODE_ID;
-    this.replication = replication;
-  }
-
-  /**
-   * Copy construction.
-   * @param from BlockInfo to copy from.
-   */
-  protected BlockInfo(BlockInfo from) {
-    this(from, from.getReplication());
-    this.bcId = from.bcId;
+    this.replication = isStriped() ? 0 : size;
   }
 
   public short getReplication() {
@@ -133,7 +126,7 @@
     BlockInfo info = (BlockInfo)triplets[index*3+1];
     assert info == null ||
         info.getClass().getName().startsWith(BlockInfo.class.getName()) :
-              "BlockInfo is expected at " + index*3;
+        "BlockInfo is expected at " + index*3;
     return info;
   }
 
@@ -164,7 +157,7 @@
   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];
+    BlockInfo info = (BlockInfo) triplets[index*3+1];
     triplets[index*3+1] = to;
     return info;
   }
@@ -175,12 +168,12 @@
    *
    * @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
+   * @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];
+    BlockInfo info = (BlockInfo) triplets[index*3+2];
     triplets[index*3+2] = to;
     return info;
   }
@@ -192,20 +185,31 @@
   }
 
   /**
-   * Count the number of data-nodes the block belongs to.
+   * 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.
+   * 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);
+  abstract boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock);
 
   /**
    * Remove {@link DatanodeStorageInfo} location for a block
    */
   abstract boolean removeStorage(DatanodeStorageInfo storage);
 
+  public abstract boolean isStriped();
+
+  /** @return true if there is no datanode storage associated with the block */
+  abstract boolean hasNoStorage();
+
   /**
    * Find specified DatanodeStorageInfo.
    * @return DatanodeStorageInfo or null if not found.
@@ -214,10 +218,9 @@
     int len = getCapacity();
     for(int idx = 0; idx < len; idx++) {
       DatanodeStorageInfo cur = getStorageInfo(idx);
-      if(cur == null)
-        break;
-      if(cur.getDatanodeDescriptor() == dn)
+      if(cur != null && cur.getDatanodeDescriptor() == dn) {
         return cur;
+      }
     }
     return null;
   }
@@ -233,9 +236,6 @@
       if (cur == storageInfo) {
         return idx;
       }
-      if (cur == null) {
-        break;
-      }
     }
     return -1;
   }
@@ -246,16 +246,16 @@
    * 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) {
+  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.";
+        "Block is already in the list and cannot be inserted.";
     this.setPrevious(dnIndex, null);
     this.setNext(dnIndex, head);
-    if(head != null)
+    if (head != null) {
       head.setPrevious(head.findStorageInfo(storage), this);
+    }
     return this;
   }
 
@@ -267,24 +267,28 @@
    * @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)
+  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
+    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)
+    if (prev != null) {
       prev.setNext(prev.findStorageInfo(storage), next);
-    if(next != null)
+    }
+    if (next != null) {
       next.setPrevious(next.findStorageInfo(storage), prev);
-    if(this == head)  // removing the head
+    }
+    if (this == head) { // removing the head
       head = next;
+    }
     return head;
   }
 
@@ -294,8 +298,8 @@
    *
    * @return the new head of the list.
    */
-  public BlockInfo moveBlockToHead(BlockInfo head,
-      DatanodeStorageInfo storage, int curIndex, int headIndex) {
+  public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage,
+      int curIndex, int headIndex) {
     if (head == this) {
       return this;
     }
@@ -357,11 +361,12 @@
   public void convertToBlockUnderConstruction(BlockUCState s,
       DatanodeStorageInfo[] targets) {
     if (isComplete()) {
-      uc = new BlockUnderConstructionFeature(this, s, targets);
+      uc = new BlockUnderConstructionFeature(this, s, targets,
+          this.isStriped());
     } else {
       // the block is already under construction
       uc.setBlockUCState(s);
-      uc.setExpectedLocations(this, targets);
+      uc.setExpectedLocations(this, targets, this.isStriped());
     }
   }
 
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 94fb222..746e298 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
@@ -25,7 +25,6 @@
  */
 @InterfaceAudience.Private
 public class BlockInfoContiguous extends BlockInfo {
-  public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
 
   public BlockInfoContiguous(short size) {
     super(size);
@@ -36,14 +35,6 @@
   }
 
   /**
-   * Copy construction.
-   * @param from BlockInfoContiguous to copy from.
-   */
-  protected BlockInfoContiguous(BlockInfoContiguous from) {
-    super(from);
-  }
-
-  /**
    * Ensure that there is enough  space to include num more triplets.
    * @return first free triplet index.
    */
@@ -62,7 +53,7 @@
   }
 
   @Override
-  boolean addStorage(DatanodeStorageInfo storage) {
+  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
     // find the last null node
     int lastNode = ensureCapacity(1);
     setStorageInfo(lastNode, storage);
@@ -104,4 +95,14 @@
     }
     return 0;
   }
+
+  @Override
+  public final boolean isStriped() {
+    return false;
+  }
+
+  @Override
+  final boolean hasNoStorage() {
+    return getStorageInfo(0) == null;
+  }
 }
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..d1067b3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -0,0 +1,236 @@
+/**
+ * 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.util.StripedBlockUtil;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+
+/**
+ * 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 ErasureCodingPolicy ecPolicy;
+  /**
+   * 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, ErasureCodingPolicy ecPolicy) {
+    super(blk, (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()));
+    indices = new byte[ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()];
+    initIndices();
+    this.ecPolicy = ecPolicy;
+  }
+
+  public short getTotalBlockNum() {
+    return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits());
+  }
+
+  public short getDataBlockNum() {
+    return (short) ecPolicy.getNumDataUnits();
+  }
+
+  public short getParityBlockNum() {
+    return (short) ecPolicy.getNumParityUnits();
+  }
+
+  public int getCellSize() {
+    return ecPolicy.getCellSize();
+  }
+
+  /**
+   * If the block is committed/completed and its length is less than a full
+   * stripe, it returns the the number of actual data blocks.
+   * Otherwise it returns the number of data units specified by erasure coding policy.
+   */
+  public short getRealDataBlockNum() {
+    if (isComplete() || getBlockUCState() == BlockUCState.COMMITTED) {
+      return (short) Math.min(getDataBlockNum(),
+          (getNumBytes() - 1) / ecPolicy.getCellSize() + 1);
+    } else {
+      return getDataBlockNum();
+    }
+  }
+
+  public short getRealTotalBlockNum() {
+    return (short) (getRealDataBlockNum() + getParityBlockNum());
+  }
+
+  public ErasureCodingPolicy getErasureCodingPolicy() {
+    return ecPolicy;
+  }
+
+  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;
+  }
+
+  int getStorageBlockIndex(DatanodeStorageInfo storage) {
+    int i = this.findStorageInfo(storage);
+    return i == -1 ? -1 : indices[i];
+  }
+
+  /**
+   * Identify the block stored in the given datanode storage. Note that
+   * the returned block has the same block Id with the one seen/reported by the
+   * DataNode.
+   */
+  Block getBlockOnStorage(DatanodeStorageInfo storage) {
+    int index = getStorageBlockIndex(storage);
+    if (index < 0) {
+      return null;
+    } else {
+      Block block = new Block(this);
+      block.setBlockId(this.getBlockId() + index);
+      return block;
+    }
+  }
+
+  @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);
+      }
+    }
+  }
+
+  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 StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(),
+        ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits(),
+        ecPolicy.getCellSize());
+    }
+
+  @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";
+    int num = 0;
+    for (int idx = getCapacity()-1; idx >= 0; idx--) {
+      if (getStorageInfo(idx) != null) {
+        num++;
+      }
+    }
+    return num;
+  }
+
+  @Override
+  final boolean hasNoStorage() {
+    final int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      if (getStorageInfo(idx) != null) {
+        return false;
+      }
+    }
+    return true;
+  }
+}
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 b0a11fe..18bfc41 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
@@ -22,6 +22,7 @@
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
@@ -42,6 +43,7 @@
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -54,9 +56,9 @@
 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.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.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
@@ -78,6 +80,7 @@
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -87,9 +90,14 @@
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
+
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.LightWeightGSet;
 import org.apache.hadoop.util.Time;
@@ -190,7 +198,11 @@
   /** Store blocks -> datanodedescriptor(s) map of corrupt replicas */
   final CorruptReplicasMap corruptReplicas = new CorruptReplicasMap();
 
-  /** Blocks to be invalidated. */
+  /**
+   * Blocks to be invalidated.
+   * For a striped block to invalidate, we should track its individual internal
+   * blocks.
+   */
   private final InvalidateBlocks invalidateBlocks;
   
   /**
@@ -207,7 +219,7 @@
    * Maps a StorageID to the set of blocks that are "extra" for this
    * DataNode. We'll eventually remove these extras.
    */
-  public final Map<String, LightWeightLinkedSet<Block>> excessReplicateMap =
+  public final Map<String, LightWeightLinkedSet<BlockInfo>> excessReplicateMap =
     new HashMap<>();
 
   /**
@@ -275,12 +287,15 @@
   private double replicationQueuesInitProgress = 0.0;
 
   /** for block replicas placement */
-  private BlockPlacementPolicy blockplacement;
+  private BlockPlacementPolicies placementPolicies;
   private final BlockStoragePolicySuite storagePolicySuite;
 
   /** Check whether name system is running before terminating */
   private boolean checkNSRunning = true;
 
+  /** Check whether there are any non-EC blocks using StripedID */
+  private boolean hasNonEcBlockUsingStripedID = false;
+
   public BlockManager(final Namesystem namesystem, final Configuration conf)
     throws IOException {
     this.namesystem = namesystem;
@@ -296,7 +311,7 @@
     // Compute the map capacity by allocating 2% of total memory
     blocksMap = new BlocksMap(
         LightWeightGSet.computeCapacity(2.0, "BlocksMap"));
-    blockplacement = BlockPlacementPolicy.getInstance(
+    placementPolicies = new BlockPlacementPolicies(
       conf, datanodeManager.getFSClusterStats(),
       datanodeManager.getNetworkTopology(),
       datanodeManager.getHost2DatanodeMap());
@@ -498,15 +513,7 @@
 
   @VisibleForTesting
   public BlockPlacementPolicy getBlockPlacementPolicy() {
-    return blockplacement;
-  }
-
-  /** Set BlockPlacementPolicy */
-  public void setBlockPlacementPolicy(BlockPlacementPolicy newpolicy) {
-    if (newpolicy == null) {
-      throw new HadoopIllegalArgumentException("newpolicy == null");
-    }
-    this.blockplacement = newpolicy;
+    return placementPolicies.getPolicy(false);
   }
 
   /** Dump meta data to out. */
@@ -556,9 +563,9 @@
     
     NumberReplicas numReplicas = new NumberReplicas();
     // source node returned is not used
-    chooseSourceDatanode(block, containingNodes,
+    chooseSourceDatanodes(getStoredBlock(block), containingNodes,
         containingLiveReplicasNodes, numReplicas,
-        UnderReplicatedBlocks.LEVEL);
+        new LinkedList<Short>(), UnderReplicatedBlocks.LEVEL);
     
     // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are 
     // not included in the numReplicas.liveReplicas() count
@@ -605,11 +612,28 @@
     return maxReplicationStreams;
   }
 
-  /**
-   * @return true if the block has minimum replicas
-   */
-  public boolean checkMinReplication(BlockInfo block) {
-    return (countNodes(block).liveReplicas() >= minReplication);
+  public int getDefaultStorageNum(BlockInfo block) {
+    if (block.isStriped()) {
+      return ((BlockInfoStriped) block).getRealTotalBlockNum();
+    } else {
+      return defaultReplication;
+    }
+  }
+
+  public short getMinStorageNum(BlockInfo block) {
+    if (block.isStriped()) {
+      return ((BlockInfoStriped) block).getRealDataBlockNum();
+    } else {
+      return minReplication;
+    }
+  }
+
+  public boolean hasMinStorage(BlockInfo block) {
+    return countNodes(block).liveReplicas() >= getMinStorageNum(block);
+  }
+
+  public boolean hasMinStorage(BlockInfo block, int liveNum) {
+    return liveNum >= getMinStorageNum(block);
   }
 
   /**
@@ -626,8 +650,8 @@
     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();
+        "commitBlock length is less than the stored one "
+            + commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
     block.commitBlock(commitBlock);
     return true;
   }
@@ -653,7 +677,7 @@
       return false; // already completed (e.g. by syncBlock)
     
     final boolean b = commitBlock(lastBlock, commitBlock);
-    if (countNodes(lastBlock).liveReplicas() >= minReplication) {
+      if (hasMinStorage(lastBlock)) {
       completeBlock(lastBlock, false);
     }
     return b;
@@ -671,7 +695,7 @@
     }
 
     int numNodes = curBlock.numNodes();
-    if (!force && numNodes < minReplication) {
+    if (!force && !hasMinStorage(curBlock, numNodes)) {
       throw new IOException("Cannot complete block: "
           + "block does not satisfy minimal replication requirement.");
     }
@@ -688,8 +712,10 @@
     // 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).getRealDataBlockNum() : minReplication;
     namesystem.incrementSafeBlockCount(
-        Math.min(numNodes, minReplication));
+        Math.min(numNodes, minStorage), curBlock);
   }
 
   /**
@@ -740,14 +766,17 @@
 
     // remove this block from the list of pending blocks to be deleted. 
     for (DatanodeStorageInfo storage : targets) {
-      invalidateBlocks.remove(storage.getDatanodeDescriptor(), lastBlock);
+      final Block b = getBlockOnStorage(lastBlock, storage);
+      if (b != null) {
+        invalidateBlocks.remove(storage.getDatanodeDescriptor(), b);
+      }
     }
     
     // Adjust safe-mode totals, since under-construction blocks don't
     // count in safe-mode.
     namesystem.adjustSafeModeBlockTotals(
         // decrement safe if we had enough
-        targets.length >= minReplication ? -1 : 0,
+        hasMinStorage(lastBlock, targets.length) ? -1 : 0,
         // always decrement total blocks
         -1);
 
@@ -761,23 +790,24 @@
   /**
    * Get all valid locations of the block
    */
-  private List<DatanodeStorageInfo> getValidLocations(Block block) {
+  private List<DatanodeStorageInfo> getValidLocations(BlockInfo block) {
     final List<DatanodeStorageInfo> locations
         = new ArrayList<DatanodeStorageInfo>(blocksMap.numNodes(block));
     for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
       // filter invalidate replicas
-      if(!invalidateBlocks.contains(storage.getDatanodeDescriptor(), block)) {
+      Block b = getBlockOnStorage(block, storage);
+      if(b != null && 
+          !invalidateBlocks.contains(storage.getDatanodeDescriptor(), b)) {
         locations.add(storage);
       }
     }
     return locations;
   }
-  
-  private List<LocatedBlock> createLocatedBlockList(
-      final BlockInfo[] 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++) {
@@ -790,10 +820,10 @@
     }
 
     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();
@@ -806,7 +836,7 @@
 
   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++) {
@@ -819,7 +849,7 @@
     
     return createLocatedBlock(blocks[curBlk], curPos, mode);
   }
-  
+
   private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos,
     final AccessMode mode) throws IOException {
     final LocatedBlock lb = createLocatedBlock(blk, pos);
@@ -830,13 +860,22 @@
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
-      ) throws IOException {
+  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos)
+      throws IOException {
     if (!blk.isComplete()) {
-      final DatanodeStorageInfo[] storages = blk.getUnderConstructionFeature()
-          .getExpectedStorageLocations();
-      final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
-      return newLocatedBlock(eb, storages, pos, false);
+      final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature();
+      if (blk.isStriped()) {
+        final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
+        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
+            blk);
+        return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
+            false);
+      } else {
+        final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
+        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
+            blk);
+        return newLocatedBlock(eb, storages, pos, false);
+      }
     }
 
     // get block locations
@@ -853,13 +892,21 @@
         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 :
@@ -869,7 +916,9 @@
       " numCorrupt: " + numCorruptNodes +
       " numCorruptRepls: " + numCorruptReplicas;
     final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
-    return newLocatedBlock(eb, machines, pos, isCorrupt);
+    return blockIndices == null ?
+        newLocatedBlock(eb, machines, pos, isCorrupt) :
+        newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
   }
 
   /** Create a LocatedBlocks. */
@@ -877,14 +926,15 @@
       final long fileSizeExcludeBlocksUnderConstruction,
       final boolean isFileUnderConstruction, final long offset,
       final long length, final boolean needBlockToken,
-      final boolean inSnapshot, FileEncryptionInfo feInfo)
+      final boolean inSnapshot, FileEncryptionInfo feInfo,
+      ErasureCodingPolicy ecPolicy)
       throws IOException {
     assert namesystem.hasReadLock();
     if (blocks == null) {
       return null;
     } else if (blocks.length == 0) {
       return new LocatedBlocks(0, isFileUnderConstruction,
-          Collections.<LocatedBlock>emptyList(), null, false, feInfo);
+          Collections.<LocatedBlock> emptyList(), null, false, feInfo, ecPolicy);
     } else {
       if (LOG.isDebugEnabled()) {
         LOG.debug("blocks = " + java.util.Arrays.asList(blocks));
@@ -907,9 +957,9 @@
             fileSizeExcludeBlocksUnderConstruction, mode);
         isComplete = true;
       }
-      return new LocatedBlocks(
-          fileSizeExcludeBlocksUnderConstruction, isFileUnderConstruction,
-          locatedblocks, lastlb, isComplete, feInfo);
+      return new LocatedBlocks(fileSizeExcludeBlocksUnderConstruction,
+          isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo,
+          ecPolicy);
     }
   }
 
@@ -924,9 +974,24 @@
       final AccessMode mode) throws IOException {
     if (isBlockTokenEnabled()) {
       // Use cached UGI if serving RPC calls.
-      b.setBlockToken(blockTokenSecretManager.generateToken(
-          NameNode.getRemoteUser().getShortUserName(),
-          b.getBlock(), EnumSet.of(mode)));
+      if (b.isStriped()) {
+        Preconditions.checkState(b instanceof LocatedStripedBlock);
+        LocatedStripedBlock sb = (LocatedStripedBlock) b;
+        int[] indices = sb.getBlockIndices();
+        Token<BlockTokenIdentifier>[] blockTokens = new Token[indices.length];
+        ExtendedBlock internalBlock = new ExtendedBlock(b.getBlock());
+        for (int i = 0; i < indices.length; i++) {
+          internalBlock.setBlockId(b.getBlock().getBlockId() + indices[i]);
+          blockTokens[i] = blockTokenSecretManager.generateToken(
+              NameNode.getRemoteUser().getShortUserName(),
+              internalBlock, EnumSet.of(mode));
+        }
+        sb.setBlockTokens(blockTokens);
+      } else {
+        b.setBlockToken(blockTokenSecretManager.generateToken(
+            NameNode.getRemoteUser().getShortUserName(),
+            b.getBlock(), EnumSet.of(mode)));
+      }
     }    
   }
 
@@ -1068,7 +1133,7 @@
    
   /** Remove the blocks associated to the given datanode. */
   void removeBlocksAssociatedTo(final DatanodeDescriptor node) {
-    final Iterator<? extends Block> it = node.getBlockIterator();
+    final Iterator<BlockInfo> it = node.getBlockIterator();
     while(it.hasNext()) {
       removeStoredBlock(it.next(), node);
     }
@@ -1082,12 +1147,15 @@
   /** Remove the blocks associated to the given DatanodeStorageInfo. */
   void removeBlocksAssociatedTo(final DatanodeStorageInfo storageInfo) {
     assert namesystem.hasWriteLock();
-    final Iterator<? extends Block> it = storageInfo.getBlockIterator();
+    final Iterator<BlockInfo> it = storageInfo.getBlockIterator();
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     while(it.hasNext()) {
-      Block block = it.next();
+      BlockInfo block = it.next();
       removeStoredBlock(block, node);
-      invalidateBlocks.remove(node, block);
+      final Block b = getBlockOnStorage(block, storageInfo);
+      if (b != null) {
+        invalidateBlocks.remove(node, b);
+      }
     }
     namesystem.checkSafeMode();
   }
@@ -1107,22 +1175,32 @@
    * Adds block to list of blocks which will be invalidated on all its
    * datanodes.
    */
-  private void addToInvalidates(Block b) {
+  private void addToInvalidates(BlockInfo storedBlock) {
     if (!isPopulatingReplQueues()) {
       return;
     }
     StringBuilder datanodes = new StringBuilder();
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock,
+        State.NORMAL)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
-      invalidateBlocks.add(b, node, false);
-      datanodes.append(node).append(" ");
+      final Block b = getBlockOnStorage(storedBlock, storage);
+      if (b != null) {
+        invalidateBlocks.add(b, node, false);
+        datanodes.append(node).append(" ");
+      }
     }
     if (datanodes.length() != 0) {
-      blockLog.debug("BLOCK* addToInvalidates: {} {}", b,
+      blockLog.debug("BLOCK* addToInvalidates: {} {}", storedBlock,
           datanodes.toString());
     }
   }
 
+  private Block getBlockOnStorage(BlockInfo storedBlock,
+      DatanodeStorageInfo storage) {
+    return storedBlock.isStriped() ?
+        ((BlockInfoStriped) storedBlock).getBlockOnStorage(storage) : storedBlock;
+  }
+
   /**
    * Remove all block invalidation tasks under this datanode UUID;
    * used when a datanode registers with a new UUID and the old one
@@ -1146,7 +1224,8 @@
   public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
       final DatanodeInfo dn, String storageID, String reason) throws IOException {
     assert namesystem.hasWriteLock();
-    final BlockInfo 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
@@ -1163,44 +1242,52 @@
           + ") 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);
   }
 
   /**
-   * 
-   * @param b
+   * Mark a replica (of a contiguous block) or an internal block (of a striped
+   * block group) as corrupt.
+   * @param b Indicating the reported bad block and the corresponding BlockInfo
+   *          stored in blocksMap.
    * @param storageInfo storage that contains the block, if known. null otherwise.
-   * @throws IOException
    */
   private void markBlockAsCorrupt(BlockToMarkCorrupt b,
       DatanodeStorageInfo storageInfo,
       DatanodeDescriptor node) throws IOException {
-
-    if (b.getCorrupted().isDeleted()) {
+    if (b.getStored().isDeleted()) {
       blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
           " corrupt as it does not belong to any file", b);
       addToInvalidates(b.getCorrupted(), node);
       return;
-    } 
-    short expectedReplicas = b.getCorrupted().getReplication();
+    }
+    short expectedReplicas =
+        getExpectedReplicaNum(b.getStored());
 
     // Add replica to the data-node if it is not already there
     if (storageInfo != null) {
-      storageInfo.addBlock(b.getStored());
+      storageInfo.addBlock(b.getStored(), b.getCorrupted());
     }
 
-    // Add this replica to corruptReplicas Map
-    corruptReplicas.addToCorruptReplicasMap(b.getCorrupted(), node,
-        b.getReason(), b.getReasonCode());
+    // Add this replica to corruptReplicas Map. For striped blocks, we always
+    // use the id of whole striped block group when adding to corruptReplicas
+    Block corrupted = new Block(b.getCorrupted());
+    if (b.getStored().isStriped()) {
+      corrupted.setBlockId(b.getStored().getBlockId());
+    }
+    corruptReplicas.addToCorruptReplicasMap(corrupted, node, b.getReason(),
+        b.getReasonCode());
 
     NumberReplicas numberOfReplicas = countNodes(b.getStored());
     boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >=
         expectedReplicas;
-    boolean minReplicationSatisfied =
-        numberOfReplicas.liveReplicas() >= minReplication;
+
+    boolean minReplicationSatisfied = hasMinStorage(b.getStored(),
+        numberOfReplicas.liveReplicas());
+
     boolean hasMoreCorruptReplicas = minReplicationSatisfied &&
         (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) >
         expectedReplicas;
@@ -1215,7 +1302,7 @@
     if (hasEnoughLiveReplicas || hasMoreCorruptReplicas
         || corruptedDuringWrite) {
       // the block is over-replicated so invalidate the replicas immediately
-      invalidateBlock(b, node);
+      invalidateBlock(b, node, numberOfReplicas);
     } else if (isPopulatingReplQueues()) {
       // add the block to neededReplication
       updateNeededReplications(b.getStored(), -1, 0);
@@ -1227,8 +1314,8 @@
    * @return true if the block was successfully invalidated and no longer
    * present in the BlocksMap
    */
-  private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn
-      ) throws IOException {
+  private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn,
+      NumberReplicas nr) throws IOException {
     blockLog.debug("BLOCK* invalidateBlock: {} on {}", b, dn);
     DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
     if (node == null) {
@@ -1237,7 +1324,6 @@
     }
 
     // Check how many copies we have of the block
-    NumberReplicas nr = countNodes(b.getStored());
     if (nr.replicasOnStaleNodes() > 0) {
       blockLog.debug("BLOCK* invalidateBlocks: postponing " +
           "invalidation of {} on {} because {} replica(s) are located on " +
@@ -1245,17 +1331,14 @@
           nr.replicasOnStaleNodes());
       postponeBlock(b.getCorrupted());
       return false;
-    } else if (nr.liveReplicas() >= 1) {
-      // If we have at least one copy on a live node, then we can delete it.
+    } else {
+      // we already checked the number of replicas in the caller of this
+      // function and know there are enough live replicas, so we can delete it.
       addToInvalidates(b.getCorrupted(), dn);
       removeStoredBlock(b.getStored(), node);
       blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.",
           b, dn);
       return true;
-    } else {
-      blockLog.debug("BLOCK* invalidateBlocks: {} on {} is the only copy and" +
-          " was not deleted", b, dn);
-      return false;
     }
   }
 
@@ -1308,15 +1391,15 @@
   }
 
   /**
-   * 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 {
@@ -1326,27 +1409,35 @@
     } 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 scheduledWork = 0;
-    final List<ReplicationWork> work = new LinkedList<>();
+  int computeRecoveryWorkForBlocks(List<List<BlockInfo>> blocksToRecover) {
+    int requiredReplication, numEffectiveReplicas;
+    List<DatanodeDescriptor> containingNodes;
+    BlockCollection bc;
+    int additionalReplRequired;
 
+    int scheduledWork = 0;
+    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)) {
-            ReplicationWork rw = scheduleReplication(block, priority);
+        for (int priority = 0; priority < blocksToRecover.size(); priority++) {
+          for (BlockInfo block : blocksToRecover.get(priority)) {
+            BlockRecoveryWork rw = scheduleRecovery(block, priority);
             if (rw != null) {
-              work.add(rw);
+              recovWork.add(rw);
             }
           }
         }
@@ -1355,8 +1446,9 @@
       namesystem.writeUnlock();
     }
 
+    // Step 2: choose target nodes for each recovery task
     final Set<Node> excludedNodes = new HashSet<>();
-    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();
@@ -1367,12 +1459,15 @@
       // choose replication targets: NOT HOLDING THE GLOBAL LOCK
       // It is costly to extract the filename for which chooseTargets is called,
       // so for now we pass in the block collection itself.
-      rw.chooseTargets(blockplacement, storagePolicySuite, excludedNodes);
+      final BlockPlacementPolicy placementPolicy =
+          placementPolicies.getPolicy(rw.getBlock().isStriped());
+      rw.chooseTargets(placementPolicy, storagePolicySuite, excludedNodes);
     }
 
+    // Step 3: add tasks to the DN
     namesystem.writeLock();
     try {
-      for(ReplicationWork rw : work){
+      for(BlockRecoveryWork rw : recovWork){
         final DatanodeStorageInfo[] targets = rw.getTargets();
         if(targets == null || targets.length == 0){
           rw.resetTargets();
@@ -1380,7 +1475,7 @@
         }
 
         synchronized (neededReplications) {
-          if (validateReplicationWork(rw)) {
+          if (validateRecoveryWork(rw)) {
             scheduledWork++;
           }
         }
@@ -1391,7 +1486,7 @@
 
     if (blockLog.isInfoEnabled()) {
       // log which blocks have been scheduled for replication
-      for(ReplicationWork rw : work){
+      for(BlockRecoveryWork rw : recovWork){
         DatanodeStorageInfo[] targets = rw.getTargets();
         if (targets != null && targets.length != 0) {
           StringBuilder targetList = new StringBuilder("datanode(s)");
@@ -1399,7 +1494,7 @@
             targetList.append(' ');
             targetList.append(target.getDatanodeDescriptor());
           }
-          blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.getSrcNode(),
+          blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.getSrcNodes(),
               rw.getBlock(), targetList);
         }
       }
@@ -1416,10 +1511,10 @@
       NumberReplicas numReplicas, int pendingReplicaNum, int required) {
     int numEffectiveReplicas = numReplicas.liveReplicas() + pendingReplicaNum;
     return (numEffectiveReplicas >= required) &&
-        (pendingReplicaNum > 0 || blockHasEnoughRacks(block));
+        (pendingReplicaNum > 0 || blockHasEnoughRacks(block, required));
   }
 
-  private ReplicationWork scheduleReplication(BlockInfo block, int priority) {
+  private BlockRecoveryWork scheduleRecovery(BlockInfo block, int priority) {
     // block should belong to a file
     BlockCollection bc = getBlockCollection(block);
     // abandoned block or block reopened for append
@@ -1436,10 +1531,14 @@
     List<DatanodeDescriptor> containingNodes = new ArrayList<>();
     List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
     NumberReplicas numReplicas = new NumberReplicas();
-    DatanodeDescriptor srcNode = chooseSourceDatanode(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");
+    List<Short> liveBlockIndices = new ArrayList<>();
+    final DatanodeDescriptor[] srcNodes = chooseSourceDatanodes(block,
+        containingNodes, liveReplicaNodes, numReplicas,
+        liveBlockIndices, priority);
+    if(srcNodes == null || srcNodes.length == 0) {
+      // block can not be recovered from any node
+      LOG.debug("Block " + block + " cannot be recovered " +
+          "from any node");
       return null;
     }
 
@@ -1463,11 +1562,23 @@
     } else {
       additionalReplRequired = 1; // Needed on a new rack
     }
-    return new ReplicationWork(block, bc, srcNode, containingNodes,
-        liveReplicaNodes, additionalReplRequired, priority);
+
+    if (block.isStriped()) {
+      short[] indices = new short[liveBlockIndices.size()];
+      for (int i = 0 ; i < liveBlockIndices.size(); i++) {
+        indices[i] = liveBlockIndices.get(i);
+      }
+      return new ErasureCodingWork(block, bc, srcNodes,
+          containingNodes, liveReplicaNodes, additionalReplRequired,
+          priority, indices);
+    } else {
+      return new ReplicationWork(block, bc, srcNodes,
+          containingNodes, liveReplicaNodes, additionalReplRequired,
+          priority);
+    }
   }
 
-  private boolean validateReplicationWork(ReplicationWork rw) {
+  private boolean validateRecoveryWork(BlockRecoveryWork rw) {
     BlockInfo block = rw.getBlock();
     int priority = rw.getPriority();
     // Recheck since global lock was released
@@ -1496,16 +1607,41 @@
 
     DatanodeStorageInfo[] targets = rw.getTargets();
     if ( (numReplicas.liveReplicas() >= requiredReplication) &&
-        (!blockHasEnoughRacks(block)) ) {
-      if (rw.getSrcNode().getNetworkLocation().equals(
+        (!blockHasEnoughRacks(block, requiredReplication)) ) {
+      if (rw.getSrcNodes()[0].getNetworkLocation().equals(
           targets[0].getDatanodeDescriptor().getNetworkLocation())) {
         //No use continuing, unless a new rack in this case
         return false;
       }
     }
 
-    // Add block to the to be replicated list
-    rw.getSrcNode().addBlockToBeReplicated(block, targets);
+    // Add block to the to be recovered list
+    if (block.isStriped()) {
+      assert rw instanceof ErasureCodingWork;
+      assert rw.getTargets().length > 0;
+      String src = getBlockCollection(block).getName();
+      ErasureCodingPolicy ecPolicy = null;
+      try {
+        ecPolicy = namesystem.getErasureCodingPolicyForPath(src);
+      } catch (IOException e) {
+        blockLog
+            .warn("Failed to get EC policy for the file {} ", src);
+      }
+      if (ecPolicy == null) {
+        blockLog.warn("No erasure coding policy found for the file {}. "
+            + "So cannot proceed for recovery", src);
+        // TODO: we may have to revisit later for what we can do better to
+        // handle this case.
+        return false;
+      }
+      rw.getTargets()[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
+          new ExtendedBlock(namesystem.getBlockPoolId(), block),
+          rw.getSrcNodes(), rw.getTargets(),
+          ((ErasureCodingWork) rw).getLiveBlockIndicies(), ecPolicy);
+    } else {
+      rw.getSrcNodes()[0].addBlockToBeReplicated(block, targets);
+    }
+
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
 
     // Move the block-replication into a "pending" state.
@@ -1527,7 +1663,7 @@
   /** Choose target for WebHDFS redirection. */
   public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src,
       DatanodeDescriptor clientnode, Set<Node> excludes, long blocksize) {
-    return blockplacement.chooseTarget(src, 1, clientnode,
+    return placementPolicies.getPolicy(false).chooseTarget(src, 1, clientnode,
         Collections.<DatanodeStorageInfo>emptyList(), false, excludes,
         blocksize, storagePolicySuite.getDefaultPolicy());
   }
@@ -1539,9 +1675,10 @@
       List<DatanodeStorageInfo> chosen,
       Set<Node> excludes,
       long blocksize,
-      byte storagePolicyID) {
-    
+      byte storagePolicyID,
+      boolean isStriped) {
     final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
+    final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped);
     return blockplacement.chooseTarget(src, numAdditionalNodes, clientnode,
         chosen, true, excludes, blocksize, storagePolicy);
   }
@@ -1559,10 +1696,12 @@
       final Set<Node> excludedNodes,
       final long blocksize,
       final List<String> favoredNodes,
-      final byte storagePolicyID) throws IOException {
+      final byte storagePolicyID,
+      final boolean isStriped) throws IOException {
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
         getDatanodeDescriptors(favoredNodes);
     final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
+    final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped);
     final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
         numOfReplicas, client, excludedNodes, blocksize, 
         favoredDatanodeDescriptors, storagePolicy);
@@ -1597,55 +1736,59 @@
   }
 
   /**
-   * 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 liveBlockIndices List to be populated with indices of healthy
+   *                         blocks in a striped block group
    * @param priority integer representing replication priority of the given
    *                 block
-   * @return the DatanodeDescriptor of the chosen node from which to replicate
-   *         the given block
+   * @return the array of DatanodeDescriptor of the chosen nodes from which to
+   *         recover the given block
    */
-   @VisibleForTesting
-   DatanodeDescriptor chooseSourceDatanode(Block block,
-       List<DatanodeDescriptor> containingNodes,
-       List<DatanodeStorageInfo>  nodesContainingLiveReplicas,
-       NumberReplicas numReplicas,
-       int priority) {
+  @VisibleForTesting
+  DatanodeDescriptor[] chooseSourceDatanodes(BlockInfo block,
+      List<DatanodeDescriptor> containingNodes,
+      List<DatanodeStorageInfo> nodesContainingLiveReplicas,
+      NumberReplicas numReplicas,
+      List<Short> liveBlockIndices, int priority) {
     containingNodes.clear();
     nodesContainingLiveReplicas.clear();
-    DatanodeDescriptor srcNode = null;
+    List<DatanodeDescriptor> srcNodes = new ArrayList<>();
     int live = 0;
     int decommissioned = 0;
     int decommissioning = 0;
     int corrupt = 0;
     int excess = 0;
-    
+    liveBlockIndices.clear();
+    final boolean isStriped = block.isStriped();
+
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+    for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
-      LightWeightLinkedSet<Block> excessBlocks =
+      LightWeightLinkedSet<BlockInfo> 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()) {
@@ -1680,21 +1823,25 @@
       if(node.isDecommissioned())
         continue;
 
-      // We got this far, current node is a reasonable choice
-      if (srcNode == null) {
-        srcNode = node;
+      if(isStriped || srcNodes.isEmpty()) {
+        srcNodes.add(node);
+        if (isStriped) {
+          liveBlockIndices.add((short) ((BlockInfoStriped) block).
+              getStorageBlockIndex(storage));
+        }
         continue;
       }
-      // switch to a different node randomly
+      // for replicated block, 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(ThreadLocalRandom.current().nextBoolean())
-        srcNode = node;
+      if (!isStriped && ThreadLocalRandom.current().nextBoolean()) {
+        srcNodes.set(0, node);
+      }
     }
     if(numReplicas != null)
       numReplicas.initialize(live, decommissioned, decommissioning, corrupt,
           excess, 0);
-    return srcNode;
+    return srcNodes.toArray(new DatanodeDescriptor[srcNodes.size()]);
   }
 
   /**
@@ -1758,7 +1905,7 @@
    * reported by the datanode in the block report. 
    */
   static class StatefulBlockInfo {
-    final BlockInfo storedBlock;
+    final BlockInfo storedBlock; // should be UC block
     final Block reportedBlock;
     final ReplicaState reportedState;
     
@@ -1771,6 +1918,16 @@
     }
   }
 
+  private static class BlockInfoToAdd {
+    final BlockInfo stored;
+    final Block reported;
+
+    BlockInfoToAdd(BlockInfo stored, Block reported) {
+      this.stored = stored;
+      this.reported = reported;
+    }
+  }
+
   /**
    * The given storage is reporting all its blocks.
    * Update the (storage-->block list) and (block-->storage list) maps.
@@ -1874,8 +2031,8 @@
       metrics.addBlockReport((int) (endTime - startTime));
     }
     blockLog.info("BLOCK* processReport: from storage {} node {}, " +
-        "blocks: {}, hasStaleStorage: {}, processing time: {} msecs", storage
-        .getStorageID(), nodeID, newReport.getNumberOfBlocks(),
+            "blocks: {}, hasStaleStorage: {}, processing time: {} msecs", storage
+            .getStorageID(), nodeID, newReport.getNumberOfBlocks(),
         node.hasStaleStorages(), (endTime - startTime));
     return !node.hasStaleStorages();
   }
@@ -1883,8 +2040,8 @@
   private void removeZombieReplicas(BlockReportContext context,
       DatanodeStorageInfo zombie) {
     LOG.warn("processReport 0x{}: removing zombie storage {}, which no " +
-             "longer exists on the DataNode.",
-              Long.toHexString(context.getReportId()), zombie.getStorageID());
+            "longer exists on the DataNode.",
+        Long.toHexString(context.getReportId()), zombie.getStorageID());
     assert(namesystem.hasWriteLock());
     Iterator<BlockInfo> iter = zombie.getBlockIterator();
     int prevBlocks = zombie.numBlocks();
@@ -1897,13 +2054,16 @@
       // more than one storage on a datanode (and because it's a difficult
       // assumption to really enforce)
       removeStoredBlock(block, zombie.getDatanodeDescriptor());
-      invalidateBlocks.remove(zombie.getDatanodeDescriptor(), block);
+      Block b = getBlockOnStorage(block, zombie);
+      if (b != null) {
+        invalidateBlocks.remove(zombie.getDatanodeDescriptor(), b);
+      }
     }
     assert(zombie.numBlocks() == 0);
     LOG.warn("processReport 0x{}: removed {} replicas from storage {}, " +
             "which no longer exists on the DataNode.",
-            Long.toHexString(context.getReportId()), prevBlocks,
-            zombie.getStorageID());
+        Long.toHexString(context.getReportId()), prevBlocks,
+        zombie.getStorageID());
   }
 
   /**
@@ -1947,7 +2107,7 @@
           break;
         }
 
-        BlockInfo bi = blocksMap.getStoredBlock(b);
+        BlockInfo bi = getStoredBlock(b);
         if (bi == null) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
@@ -1987,11 +2147,11 @@
     // Modify the (block-->datanode) map, according to the difference
     // between the old and new block report.
     //
-    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
-    Collection<Block> toRemove = new TreeSet<Block>();
-    Collection<Block> toInvalidate = new LinkedList<Block>();
-    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
-    Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
+    Collection<BlockInfoToAdd> toAdd = new LinkedList<>();
+    Collection<BlockInfo> toRemove = new TreeSet<>();
+    Collection<Block> toInvalidate = new LinkedList<>();
+    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<>();
+    Collection<StatefulBlockInfo> toUC = new LinkedList<>();
     reportDiff(storageInfo, report,
         toAdd, toRemove, toInvalidate, toCorrupt, toUC);
    
@@ -2000,12 +2160,13 @@
     for (StatefulBlockInfo b : toUC) { 
       addStoredBlockUnderConstruction(b, storageInfo);
     }
-    for (Block b : toRemove) {
+    for (BlockInfo b : toRemove) {
       removeStoredBlock(b, node);
     }
     int numBlocksLogged = 0;
-    for (BlockInfo 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) {
@@ -2026,17 +2187,18 @@
    * Mark block replicas as corrupt except those on the storages in 
    * newStorages list.
    */
-  public void markBlockReplicasAsCorrupt(BlockInfo 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 {
@@ -2094,8 +2256,8 @@
             QUEUE_REASON_FUTURE_GENSTAMP);
         continue;
       }
-      
-      BlockInfo storedBlock = blocksMap.getStoredBlock(iblk);
+
+      BlockInfo storedBlock = getStoredBlock(iblk);
       // If block does not belong to any file, we are done.
       if (storedBlock == null) continue;
       
@@ -2126,29 +2288,31 @@
         if (namesystem.isInSnapshot(storedBlock)) {
           int numOfReplicas = storedBlock.getUnderConstructionFeature()
               .getNumExpectedLocations();
-          namesystem.incrementSafeBlockCount(numOfReplicas);
+          namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock);
         }
         //and fall through to next clause
       }      
       //add replica if appropriate
       if (reportedState == ReplicaState.FINALIZED) {
-        addStoredBlockImmediate(storedBlock, storageInfo);
+        addStoredBlockImmediate(storedBlock, iblk, storageInfo);
       }
     }
   }
 
   private void reportDiff(DatanodeStorageInfo storageInfo, 
-      BlockListAsLongs newReport, 
-      Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
-      Collection<Block> toRemove,           // remove from DatanodeDescriptor
+      BlockListAsLongs newReport,
+      Collection<BlockInfoToAdd> toAdd,     // add to DatanodeDescriptor
+      Collection<BlockInfo> toRemove,       // remove from DatanodeDescriptor
       Collection<Block> toInvalidate,       // should be removed from DN
       Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
       Collection<StatefulBlockInfo> toUC) { // add to under-construction list
 
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
-    BlockInfo delimiter = new BlockInfoContiguous(new Block(), (short) 1);
-    AddBlockResult result = storageInfo.addBlock(delimiter);
+    Block delimiterBlock = new Block();
+    BlockInfo 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
@@ -2174,8 +2338,9 @@
     // all of them are next to the delimiter
     Iterator<BlockInfo> it =
         storageInfo.new BlockIterator(delimiter.getNext(0));
-    while(it.hasNext())
+    while (it.hasNext()) {
       toRemove.add(it.next());
+    }
     storageInfo.removeBlock(delimiter);
   }
 
@@ -2212,8 +2377,8 @@
    */
   private BlockInfo processReportedBlock(
       final DatanodeStorageInfo storageInfo,
-      final Block block, final ReplicaState reportedState, 
-      final Collection<BlockInfo> toAdd,
+      final Block block, final ReplicaState reportedState,
+      final Collection<BlockInfoToAdd> toAdd,
       final Collection<Block> toInvalidate, 
       final Collection<BlockToMarkCorrupt> toCorrupt,
       final Collection<StatefulBlockInfo> toUC) {
@@ -2234,7 +2399,7 @@
     }
     
     // find block by blockId
-    BlockInfo storedBlock = blocksMap.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.
@@ -2282,7 +2447,7 @@
     if (reportedState == ReplicaState.FINALIZED
         && (storedBlock.findStorageInfo(storageInfo) == -1 ||
             corruptReplicas.isReplicaCorrupt(storedBlock, dn))) {
-      toAdd.add(storedBlock);
+      toAdd.add(new BlockInfoToAdd(storedBlock, block));
     }
     return storedBlock;
   }
@@ -2328,7 +2493,7 @@
       if (rbi.getReportedState() == null) {
         // This is a DELETE_BLOCK request
         DatanodeStorageInfo storageInfo = rbi.getStorageInfo();
-        removeStoredBlock(rbi.getBlock(),
+        removeStoredBlock(getStoredBlock(rbi.getBlock()),
             storageInfo.getDatanodeDescriptor());
       } else {
         processAndHandleReportedBlock(rbi.getStorageInfo(),
@@ -2376,12 +2541,26 @@
       case COMMITTED:
         if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
           final long reportedGS = reported.getGenerationStamp();
-          return new BlockToMarkCorrupt(storedBlock, reportedGS,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS,
               "block is " + ucState + " and reported genstamp " + reportedGS
               + " does not match genstamp in block map "
               + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
-        } else if (storedBlock.getNumBytes() != reported.getNumBytes()) {
-          return new BlockToMarkCorrupt(storedBlock,
+        }
+        boolean wrongSize;
+        if (storedBlock.isStriped()) {
+          assert BlockIdManager.isStripedBlockID(reported.getBlockId());
+          assert storedBlock.getBlockId() ==
+              BlockIdManager.convertToStripedID(reported.getBlockId());
+          BlockInfoStriped stripedBlock = (BlockInfoStriped) storedBlock;
+          int reportedBlkIdx = BlockIdManager.getBlockIndex(reported);
+          wrongSize = reported.getNumBytes() != getInternalBlockLength(
+              stripedBlock.getNumBytes(), stripedBlock.getCellSize(),
+              stripedBlock.getDataBlockNum(), reportedBlkIdx);
+        } else {
+          wrongSize = storedBlock.getNumBytes() != reported.getNumBytes();
+        }
+        if (wrongSize) {
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
               "block is " + ucState + " and reported length " +
               reported.getNumBytes() + " does not match " +
               "length in block map " + storedBlock.getNumBytes(),
@@ -2392,8 +2571,8 @@
       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(new Block(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);
@@ -2408,7 +2587,7 @@
         return null; // not corrupt
       } else if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
         final long reportedGS = reported.getGenerationStamp();
-        return new BlockToMarkCorrupt(storedBlock, reportedGS,
+        return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS,
             "reported " + reportedState + " replica with genstamp " + reportedGS
             + " does not match COMPLETE block's genstamp in block map "
             + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
@@ -2423,7 +2602,7 @@
               "complete with the same genstamp");
           return null;
         } else {
-          return new BlockToMarkCorrupt(storedBlock,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
               "reported replica has invalid state " + reportedState,
               Reason.INVALID_STATE);
         }
@@ -2436,7 +2615,8 @@
       " 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(new Block(reported), storedBlock, msg,
+          Reason.INVALID_STATE);
     }
   }
 
@@ -2469,7 +2649,7 @@
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
         (block.findStorageInfo(storageInfo) < 0)) {
-      addStoredBlock(block, storageInfo, null, true);
+      addStoredBlock(block, ucBlock.reportedBlock, storageInfo, null, true);
     }
   } 
 
@@ -2484,39 +2664,40 @@
    * 
    * @throws IOException
    */
-  private void addStoredBlockImmediate(BlockInfo storedBlock,
+  private void addStoredBlockImmediate(BlockInfo storedBlock, Block reported,
       DatanodeStorageInfo storageInfo)
   throws IOException {
     assert (storedBlock != null && namesystem.hasWriteLock());
-    if (!namesystem.isInStartupSafeMode() 
+    if (!namesystem.isInStartupSafeMode()
         || isPopulatingReplQueues()) {
-      addStoredBlock(storedBlock, storageInfo, null, false);
+      addStoredBlock(storedBlock, reported, storageInfo, null, false);
       return;
     }
 
     // just add it
-    AddBlockResult result = storageInfo.addBlock(storedBlock);
+    AddBlockResult result = storageInfo.addBlock(storedBlock, reported);
 
     // Now check for completion of blocks and safe block count
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
-        && numCurrentReplica >= minReplication) {
+        && hasMinStorage(storedBlock, numCurrentReplica)) {
       completeBlock(storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
       // 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,
                                DatanodeStorageInfo storageInfo,
                                DatanodeDescriptor delNodeHint,
                                boolean logEveryBlock)
@@ -2526,7 +2707,7 @@
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     if (!block.isComplete()) {
       //refresh our copy in case the block got completed in another thread
-      storedBlock = blocksMap.getStoredBlock(block);
+      storedBlock = getStoredBlock(block);
     } else {
       storedBlock = block;
     }
@@ -2543,7 +2724,7 @@
     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) {
@@ -2574,7 +2755,7 @@
       + pendingReplications.getNumReplicas(storedBlock);
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
-        numLiveReplicas >= minReplication) {
+        hasMinStorage(storedBlock, numLiveReplicas)) {
       completeBlock(storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
@@ -2582,7 +2763,7 @@
       // 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
@@ -2612,11 +2793,12 @@
     int numCorruptNodes = num.corruptReplicas();
     if (numCorruptNodes != corruptReplicasCount) {
       LOG.warn("Inconsistent number of corrupt replicas for " +
-          storedBlock + "blockMap has " + numCorruptNodes + 
+          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, num);
+    }
     return storedBlock;
   }
 
@@ -2648,18 +2830,20 @@
    *
    * @param blk Block whose corrupt replicas need to be invalidated
    */
-  private void invalidateCorruptReplicas(BlockInfo blk) {
+  private void invalidateCorruptReplicas(BlockInfo blk, Block reported,
+      NumberReplicas numberReplicas) {
     Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
     boolean removedFromBlocksMap = true;
     if (nodes == null)
       return;
     // make a copy of the array of nodes in order to avoid
     // ConcurrentModificationException, when the block is removed from the node
-    DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
+    DatanodeDescriptor[] nodesCopy =
+        nodes.toArray(new DatanodeDescriptor[nodes.size()]);
     for (DatanodeDescriptor node : nodesCopy) {
       try {
-        if (!invalidateBlock(new BlockToMarkCorrupt(blk, null,
-              Reason.ANY), node)) {
+        if (!invalidateBlock(new BlockToMarkCorrupt(reported, blk, null,
+            Reason.ANY), node, numberReplicas)) {
           removedFromBlocksMap = false;
         }
       } catch (IOException e) {
@@ -2810,6 +2994,15 @@
   }
 
   /**
+   * Get the value of whether there are any non-EC blocks using StripedID.
+   *
+   * @return Returns the value of whether there are any non-EC blocks using StripedID.
+   */
+  public boolean hasNonEcBlockUsingStripedID(){
+    return hasNonEcBlockUsingStripedID;
+  }
+
+  /**
    * Process a single possibly misreplicated block. This adds it to the
    * appropriate queues if necessary, and returns a result code indicating
    * what happened with it.
@@ -2876,14 +3069,14 @@
    * If there are any extras, call chooseExcessReplicates() to
    * mark them in the excessReplicateMap.
    */
-  private void processOverReplicatedBlock(final Block block,
+  private void processOverReplicatedBlock(final BlockInfo block,
       final short replication, final DatanodeDescriptor addedNode,
       DatanodeDescriptor delNodeHint) {
     assert namesystem.hasWriteLock();
     if (addedNode == delNodeHint) {
       delNodeHint = null;
     }
-    Collection<DatanodeStorageInfo> nonExcess = new ArrayList<DatanodeStorageInfo>();
+    Collection<DatanodeStorageInfo> nonExcess = new ArrayList<>();
     Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
         .getNodes(block);
     for(DatanodeStorageInfo storage : blocksMap.getStorages(block, State.NORMAL)) {
@@ -2897,8 +3090,8 @@
         postponeBlock(block);
         return;
       }
-      LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(cur
-          .getDatanodeUuid());
+      LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+          cur.getDatanodeUuid());
       if (excessBlocks == null || !excessBlocks.contains(block)) {
         if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
           // exclude corrupt replicas
@@ -2908,10 +3101,29 @@
         }
       }
     }
-    chooseExcessReplicates(nonExcess, block, replication, 
-        addedNode, delNodeHint, blockplacement);
+    chooseExcessReplicates(nonExcess, block, replication, addedNode,
+        delNodeHint);
   }
 
+  private void chooseExcessReplicates(
+      final Collection<DatanodeStorageInfo> nonExcess,
+      BlockInfo storedBlock, short replication,
+      DatanodeDescriptor addedNode,
+      DatanodeDescriptor delNodeHint) {
+    assert namesystem.hasWriteLock();
+    // first form a rack to datanodes map and
+    BlockCollection bc = getBlockCollection(storedBlock);
+    if (storedBlock.isStriped()) {
+      chooseExcessReplicasStriped(bc, nonExcess, storedBlock, delNodeHint);
+    } else {
+      final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(
+          bc.getStoragePolicyID());
+      final List<StorageType> excessTypes = storagePolicy.chooseExcess(
+          replication, DatanodeStorageInfo.toStorageTypes(nonExcess));
+      chooseExcessReplicasContiguous(bc, nonExcess, storedBlock,
+          replication, addedNode, delNodeHint, excessTypes);
+    }
+  }
 
   /**
    * We want "replication" replicates for the block, but we now have too many.  
@@ -2927,24 +3139,16 @@
    * If no such a node is available,
    * then pick a node with least free space
    */
-  private void chooseExcessReplicates(final Collection<DatanodeStorageInfo> nonExcess, 
-                              Block b, short replication,
-                              DatanodeDescriptor addedNode,
-                              DatanodeDescriptor delNodeHint,
-                              BlockPlacementPolicy replicator) {
-    assert namesystem.hasWriteLock();
-    // first form a rack to datanodes map and
-    BlockInfo bi = getStoredBlock(b);
-    BlockCollection bc = getBlockCollection(bi);
-    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(bc.getStoragePolicyID());
-    final List<StorageType> excessTypes = storagePolicy.chooseExcess(
-        replication, DatanodeStorageInfo.toStorageTypes(nonExcess));
-
-
-    final Map<String, List<DatanodeStorageInfo>> rackMap
-        = new HashMap<String, List<DatanodeStorageInfo>>();
-    final List<DatanodeStorageInfo> moreThanOne = new ArrayList<DatanodeStorageInfo>();
-    final List<DatanodeStorageInfo> exactlyOne = new ArrayList<DatanodeStorageInfo>();
+  private void chooseExcessReplicasContiguous(BlockCollection bc,
+      final Collection<DatanodeStorageInfo> nonExcess,
+      BlockInfo storedBlock, short replication,
+      DatanodeDescriptor addedNode,
+      DatanodeDescriptor delNodeHint,
+      List<StorageType> excessTypes) {
+    BlockPlacementPolicy replicator = placementPolicies.getPolicy(false);
+    final Map<String, List<DatanodeStorageInfo>> rackMap = new HashMap<>();
+    final List<DatanodeStorageInfo> moreThanOne = new ArrayList<>();
+    final List<DatanodeStorageInfo> exactlyOne = new ArrayList<>();
     
     // split nodes into two sets
     // moreThanOne contains nodes on rack with more than one replica
@@ -2965,33 +3169,111 @@
           moreThanOne, excessTypes)) {
         cur = delNodeHintStorage;
       } else { // regular excessive replica removal
-        cur = replicator.chooseReplicaToDelete(bc, b, replication,
+        cur = replicator.chooseReplicaToDelete(bc, storedBlock, replication,
             moreThanOne, exactlyOne, excessTypes);
       }
       firstOne = false;
-
       // adjust rackmap, moreThanOne, and exactlyOne
       replicator.adjustSetsWithChosenReplica(rackMap, moreThanOne,
           exactlyOne, cur);
 
-      nonExcess.remove(cur);
-      addToExcessReplicate(cur.getDatanodeDescriptor(), b);
-
-      //
-      // The 'excessblocks' tracks blocks until we get confirmation
-      // that the datanode has deleted them; the only way we remove them
-      // is when we get a "removeBlock" message.  
-      //
-      // The 'invalidate' list is used to inform the datanode the block 
-      // should be deleted.  Items are removed from the invalidate list
-      // upon giving instructions to the namenode.
-      //
-      addToInvalidates(b, cur.getDatanodeDescriptor());
-      blockLog.debug("BLOCK* chooseExcessReplicates: "
-                +"({}, {}) is added to invalidated blocks set", cur, b);
+      processChosenExcessReplica(nonExcess, cur, storedBlock);
     }
   }
 
+  /**
+   * We want block group has every internal block, but we have redundant
+   * internal blocks (which have the same index).
+   * In this method, we delete the redundant internal blocks until only one
+   * left for each index.
+   *
+   * The block placement policy will make sure that the left internal blocks are
+   * spread across racks and also try hard to pick one with least free space.
+   */
+  private void chooseExcessReplicasStriped(BlockCollection bc,
+      final Collection<DatanodeStorageInfo> nonExcess,
+      BlockInfo storedBlock,
+      DatanodeDescriptor delNodeHint) {
+    assert storedBlock instanceof BlockInfoStriped;
+    BlockInfoStriped sblk = (BlockInfoStriped) storedBlock;
+    short groupSize = sblk.getTotalBlockNum();
+    BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(true);
+    List<DatanodeStorageInfo> empty = new ArrayList<>(0);
+
+    // find all duplicated indices
+    BitSet found = new BitSet(groupSize); //indices found
+    BitSet duplicated = new BitSet(groupSize); //indices found more than once
+    HashMap<DatanodeStorageInfo, Integer> storage2index = new HashMap<>();
+    for (DatanodeStorageInfo storage : nonExcess) {
+      int index = sblk.getStorageBlockIndex(storage);
+      assert index >= 0;
+      if (found.get(index)) {
+        duplicated.set(index);
+      }
+      found.set(index);
+      storage2index.put(storage, index);
+    }
+    // the number of target left replicas equals to the of number of the found
+    // indices.
+    int numOfTarget = found.cardinality();
+
+    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(
+        bc.getStoragePolicyID());
+    final List<StorageType> excessTypes = storagePolicy.chooseExcess(
+        (short)numOfTarget, DatanodeStorageInfo.toStorageTypes(nonExcess));
+
+    // use delHint only if delHint is duplicated
+    final DatanodeStorageInfo delStorageHint =
+        DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, delNodeHint);
+    if (delStorageHint != null) {
+      Integer index = storage2index.get(delStorageHint);
+      if (index != null && duplicated.get(index)) {
+        processChosenExcessReplica(nonExcess, delStorageHint, storedBlock);
+      }
+    }
+
+    // for each duplicated index, delete some replicas until only one left
+    for (int targetIndex = duplicated.nextSetBit(0); targetIndex >= 0;
+         targetIndex = duplicated.nextSetBit(targetIndex + 1)) {
+      List<DatanodeStorageInfo> candidates = new ArrayList<>();
+      for (DatanodeStorageInfo storage : nonExcess) {
+        int index = storage2index.get(storage);
+        if (index == targetIndex) {
+          candidates.add(storage);
+        }
+      }
+      Block internalBlock = new Block(storedBlock);
+      internalBlock.setBlockId(storedBlock.getBlockId() + targetIndex);
+      while (candidates.size() > 1) {
+        DatanodeStorageInfo target = placementPolicy.chooseReplicaToDelete(bc,
+            internalBlock, (short)1, candidates, empty, excessTypes);
+        processChosenExcessReplica(nonExcess, target, storedBlock);
+        candidates.remove(target);
+      }
+      duplicated.clear(targetIndex);
+    }
+  }
+
+  private void processChosenExcessReplica(
+      final Collection<DatanodeStorageInfo> nonExcess,
+      final DatanodeStorageInfo chosen, BlockInfo storedBlock) {
+    nonExcess.remove(chosen);
+    addToExcessReplicate(chosen.getDatanodeDescriptor(), storedBlock);
+    //
+    // The 'excessblocks' tracks blocks until we get confirmation
+    // that the datanode has deleted them; the only way we remove them
+    // is when we get a "removeBlock" message.
+    //
+    // The 'invalidate' list is used to inform the datanode the block
+    // should be deleted.  Items are removed from the invalidate list
+    // upon giving instructions to the datanodes.
+    //
+    final Block blockToInvalidate = getBlockOnStorage(storedBlock, chosen);
+    addToInvalidates(blockToInvalidate, chosen.getDatanodeDescriptor());
+    blockLog.debug("BLOCK* chooseExcessReplicates: "
+        + "({}, {}) is added to invalidated blocks set", chosen, storedBlock);
+  }
+
   /** Check if we can use delHint */
   static boolean useDelHint(boolean isFirst, DatanodeStorageInfo delHint,
       DatanodeStorageInfo added, List<DatanodeStorageInfo> moreThan1Racks,
@@ -3013,17 +3295,18 @@
     }
   }
 
-  private void addToExcessReplicate(DatanodeInfo dn, Block block) {
+  private void addToExcessReplicate(DatanodeInfo dn, BlockInfo storedBlock) {
     assert namesystem.hasWriteLock();
-    LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid());
+    LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+        dn.getDatanodeUuid());
     if (excessBlocks == null) {
-      excessBlocks = new LightWeightLinkedSet<Block>();
+      excessBlocks = new LightWeightLinkedSet<>();
       excessReplicateMap.put(dn.getDatanodeUuid(), excessBlocks);
     }
-    if (excessBlocks.add(block)) {
+    if (excessBlocks.add(storedBlock)) {
       excessBlocksCount.incrementAndGet();
       blockLog.debug("BLOCK* addToExcessReplicate: ({}, {}) is added to"
-          + " excessReplicateMap", dn, block);
+          + " excessReplicateMap", dn, storedBlock);
     }
   }
 
@@ -3035,26 +3318,25 @@
           QUEUE_REASON_FUTURE_GENSTAMP);
       return;
     }
-    removeStoredBlock(block, node);
+    removeStoredBlock(getStoredBlock(block), node);
   }
 
   /**
    * Modify (block-->datanode) map. Possibly generate replication tasks, if the
    * removed block is still valid.
    */
-  public void removeStoredBlock(Block block, DatanodeDescriptor node) {
-    blockLog.debug("BLOCK* removeStoredBlock: {} from {}", block, node);
+  public void removeStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) {
+    blockLog.debug("BLOCK* removeStoredBlock: {} from {}", storedBlock, node);
     assert (namesystem.hasWriteLock());
     {
-      BlockInfo storedBlock = getStoredBlock(block);
       if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
         blockLog.debug("BLOCK* removeStoredBlock: {} has already been" +
-            " removed from node {}", block, node);
+            " removed from node {}", storedBlock, node);
         return;
       }
 
       CachedBlock cblock = namesystem.getCacheManager().getCachedBlocks()
-          .get(new CachedBlock(block.getBlockId(), (short) 0, false));
+          .get(new CachedBlock(storedBlock.getBlockId(), (short) 0, false));
       if (cblock != null) {
         boolean removed = false;
         removed |= node.getPendingCached().remove(cblock);
@@ -3062,7 +3344,7 @@
         removed |= node.getPendingUncached().remove(cblock);
         if (removed) {
           blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
-              + "related lists on node {}", block, node);
+              + "related lists on node {}", storedBlock, node);
         }
       }
 
@@ -3082,13 +3364,13 @@
       // We've removed a block from a node, so it's definitely no longer
       // in "excess" there.
       //
-      LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(node
-          .getDatanodeUuid());
+      LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+          node.getDatanodeUuid());
       if (excessBlocks != null) {
-        if (excessBlocks.remove(block)) {
+        if (excessBlocks.remove(storedBlock)) {
           excessBlocksCount.decrementAndGet();
           blockLog.debug("BLOCK* removeStoredBlock: {} is removed from " +
-              "excessBlocks", block);
+              "excessBlocks", storedBlock);
           if (excessBlocks.size() == 0) {
             excessReplicateMap.remove(node.getDatanodeUuid());
           }
@@ -3096,15 +3378,16 @@
       }
 
       // Remove the replica from corruptReplicas
-      corruptReplicas.removeFromCorruptReplicasMap(block, node);
+      corruptReplicas.removeFromCorruptReplicasMap(storedBlock, node);
     }
   }
 
   /**
    * Get all valid locations of the block & add the block to results
-   * return the length of the added block; 0 if the block is not added
+   * @return the length of the added block; 0 if the block is not added. If the
+   * added block is a block group, return its approximate internal block size
    */
-  private long addBlock(Block block, List<BlockWithLocations> results) {
+  private long addBlock(BlockInfo block, List<BlockWithLocations> results) {
     final List<DatanodeStorageInfo> locations = getValidLocations(block);
     if(locations.size() == 0) {
       return 0;
@@ -3118,9 +3401,23 @@
         storageIDs[i] = s.getStorageID();
         storageTypes[i] = s.getStorageType();
       }
-      results.add(new BlockWithLocations(block, datanodeUuids, storageIDs,
-          storageTypes));
-      return block.getNumBytes();
+      BlockWithLocations blkWithLocs = new BlockWithLocations(block,
+          datanodeUuids, storageIDs, storageTypes);
+      if(block.isStriped()) {
+        BlockInfoStriped blockStriped = (BlockInfoStriped) block;
+        byte[] indices = new byte[locations.size()];
+        for (int i = 0; i < locations.size(); i++) {
+          indices[i] =
+              (byte) blockStriped.getStorageBlockIndex(locations.get(i));
+        }
+        results.add(new StripedBlockWithLocations(blkWithLocs, indices,
+            blockStriped.getDataBlockNum(), blockStriped.getCellSize()));
+        // approximate size
+        return block.getNumBytes() / blockStriped.getDataBlockNum();
+      }else{
+        results.add(blkWithLocs);
+        return block.getNumBytes();
+      }
     }
   }
 
@@ -3151,7 +3448,7 @@
     //
     BlockInfo storedBlock = getStoredBlock(block);
     if (storedBlock != null) {
-      pendingReplications.decrement(getStoredBlock(block), node);
+      pendingReplications.decrement(storedBlock, node);
     }
     processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
         delHintNode);
@@ -3162,14 +3459,14 @@
       ReplicaState reportedState, DatanodeDescriptor delHintNode)
       throws IOException {
     // blockReceived reports a finalized block
-    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
+    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
@@ -3179,8 +3476,9 @@
       addStoredBlockUnderConstruction(b, storageInfo);
     }
     long numBlocksLogged = 0;
-    for (BlockInfo 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) {
@@ -3263,6 +3561,8 @@
   /**
    * 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(BlockInfo b) {
     int decommissioned = 0;
@@ -3281,8 +3581,8 @@
       } else if (node.isDecommissioned()) {
         decommissioned++;
       } else {
-        LightWeightLinkedSet<Block> blocksExcess = excessReplicateMap.get(node
-            .getDatanodeUuid());
+        LightWeightLinkedSet<BlockInfo> blocksExcess = excessReplicateMap.get(
+            node.getDatanodeUuid());
         if (blocksExcess != null && blocksExcess.contains(b)) {
           excess++;
         } else {
@@ -3335,12 +3635,13 @@
     int numOverReplicated = 0;
     while(it.hasNext()) {
       final BlockInfo block = it.next();
-      short expectedReplication = block.getReplication();
+      int expectedReplication = this.getReplication(block);
       NumberReplicas num = countNodes(block);
       int numCurrentReplica = num.liveReplicas();
       if (numCurrentReplica > expectedReplication) {
         // over-replicated block 
-        processOverReplicatedBlock(block, expectedReplication, null, null);
+        processOverReplicatedBlock(block, (short) expectedReplication, null,
+            null);
         numOverReplicated++;
       }
     }
@@ -3366,7 +3667,7 @@
     if (pendingReplicationBlocksCount == 0 &&
         underReplicatedBlocksCount == 0) {
       LOG.info("Node {} is dead and there are no under-replicated" +
-          " blocks or blocks pending replication. Safe to decommission.", 
+          " blocks or blocks pending replication. Safe to decommission.",
           node);
       return true;
     }
@@ -3393,6 +3694,11 @@
     return storages;
   }
 
+  /** @return an iterator of the datanodes. */
+  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
+    return blocksMap.getStorages(block);
+  }
+
   public int getTotalBlocks() {
     return blocksMap.size();
   }
@@ -3414,7 +3720,19 @@
   }
 
   public BlockInfo getStoredBlock(Block block) {
-    return blocksMap.getStoredBlock(block);
+    if (!BlockIdManager.isStripedBlockID(block.getBlockId())) {
+      return blocksMap.getStoredBlock(block);
+    }
+    if (!hasNonEcBlockUsingStripedID) {
+      return blocksMap.getStoredBlock(
+          new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
+    }
+    BlockInfo info = blocksMap.getStoredBlock(block);
+    if (info != null) {
+      return info;
+    }
+    return blocksMap.getStoredBlock(
+        new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
   }
 
   /** updates a block in under replication queue */
@@ -3450,7 +3768,7 @@
    */
   public void checkReplication(BlockCollection bc) {
     for (BlockInfo block : bc.getBlocks()) {
-      final short expected = block.getReplication();
+      short expected = getExpectedReplicaNum(block);
       final NumberReplicas n = countNodes(block);
       if (isNeededReplication(block, n.liveReplicas())) {
         neededReplications.add(block, n.liveReplicas(),
@@ -3470,10 +3788,11 @@
     for (BlockInfo b: blocks) {
       if (!b.isComplete()) {
         final int numNodes = b.numNodes();
-        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
-          + b.getBlockUCState() + ", replication# = " + numNodes
-          + (numNodes < minReplication ? " < ": " >= ")
-          + " minimum = " + minReplication + ") in file " + src);
+        final int min = getMinStorageNum(b);
+        final BlockUCState state = b.getBlockUCState();
+        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " + state
+            + ", replication# = " + numNodes + (numNodes < min ? " < " : " >= ")
+            + " minimum = " + min + ") in file " + src);
         return false;
       }
     }
@@ -3528,21 +3847,65 @@
     return toInvalidate.size();
   }
 
-  boolean blockHasEnoughRacks(BlockInfo b) {
+  @VisibleForTesting
+  public boolean containsInvalidateBlock(final DatanodeInfo dn,
+      final Block block) {
+    return invalidateBlocks.contains(dn, block);
+  }
+
+  boolean blockHasEnoughRacks(BlockInfo storedBlock, int expectedStorageNum) {
     if (!this.shouldCheckForEnoughRacks) {
       return true;
     }
-    boolean enoughRacks = false;;
-    Collection<DatanodeDescriptor> corruptNodes = 
-                                  corruptReplicas.getNodes(b);
-    int numExpectedReplicas = getReplication(b);
+    Collection<DatanodeDescriptor> corruptNodes =
+        corruptReplicas.getNodes(storedBlock);
+
+    if (storedBlock.isStriped()) {
+      return blockHasEnoughRacksStriped(storedBlock, corruptNodes);
+    } else {
+      return blockHashEnoughRacksContiguous(storedBlock, expectedStorageNum,
+          corruptNodes);
+    }
+  }
+
+  /**
+   * Verify whether given striped block is distributed through enough racks.
+   * As dicussed in HDFS-7613, ec file requires racks at least as many as
+   * the number of data block number.
+   */
+  boolean blockHasEnoughRacksStriped(BlockInfo storedBlock,
+      Collection<DatanodeDescriptor> corruptNodes) {
+    if (!datanodeManager.hasClusterEverBeenMultiRack()) {
+      return true;
+    }
+    boolean enoughRacks = false;
+    Set<String> rackNameSet = new HashSet<>();
+    int dataBlockNum = ((BlockInfoStriped)storedBlock).getRealDataBlockNum();
+    for (DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) {
+      final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
+      if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
+        if ((corruptNodes == null) || !corruptNodes.contains(cur)) {
+          String rackNameNew = cur.getNetworkLocation();
+          rackNameSet.add(rackNameNew);
+          if (rackNameSet.size() >= dataBlockNum) {
+            enoughRacks = true;
+            break;
+          }
+        }
+      }
+    }
+    return enoughRacks;
+  }
+
+  boolean blockHashEnoughRacksContiguous(BlockInfo storedBlock,
+      int expectedStorageNum, Collection<DatanodeDescriptor> corruptNodes) {
+    boolean enoughRacks = false;
     String rackName = null;
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) {
       final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
         if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
-          if (numExpectedReplicas == 1 ||
-              (numExpectedReplicas > 1 &&
+          if (expectedStorageNum == 1 || (expectedStorageNum > 1 &&
                   !datanodeManager.hasClusterEverBeenMultiRack())) {
             enoughRacks = true;
             break;
@@ -3565,14 +3928,16 @@
    * or if it does not have enough racks.
    */
   boolean isNeededReplication(BlockInfo storedBlock, int current) {
-    int expected = storedBlock.getReplication();
-    return current < expected || !blockHasEnoughRacks(storedBlock);
+    int expected = getExpectedReplicaNum(storedBlock);
+    return current < expected || !blockHasEnoughRacks(storedBlock, expected);
   }
 
   public short getExpectedReplicaNum(BlockInfo block) {
-    return block.getReplication();
+    return block.isStriped() ?
+        ((BlockInfoStriped) block).getRealTotalBlockNum() :
+        block.getReplication();
   }
-  
+
   public long getMissingBlocksCount() {
     // not locking
     return this.neededReplications.getCorruptBlockSize();
@@ -3588,13 +3953,22 @@
     return blocksMap.addBlockCollection(block, bc);
   }
 
-  public BlockCollection getBlockCollection(BlockInfo b) {
-    return namesystem.getBlockCollection(b.getBlockCollectionId());
+  /**
+   * Do some check when adding a block to blocksmap.
+   * For HDFS-7994 to check whether then block is a NonEcBlockUsingStripedID.
+   *
+   */
+  public BlockInfo addBlockCollectionWithCheck(
+      BlockInfo block, BlockCollection bc) {
+    if (!hasNonEcBlockUsingStripedID && !block.isStriped() &&
+        BlockIdManager.isStripedBlockID(block.getBlockId())) {
+      hasNonEcBlockUsingStripedID = true;
+    }
+    return addBlockCollection(block, bc);
   }
 
-  /** @return an iterator of the datanodes. */
-  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
-    return blocksMap.getStorages(block);
+  public BlockCollection getBlockCollection(BlockInfo b) {
+    return namesystem.getBlockCollection(b.getBlockCollectionId());
   }
 
   public int numCorruptReplicas(Block block) {
@@ -3614,7 +3988,7 @@
   private void removeFromExcessReplicateMap(Block block) {
     for (DatanodeStorageInfo info : blocksMap.getStorages(block)) {
       String uuid = info.getDatanodeDescriptor().getDatanodeUuid();
-      LightWeightLinkedSet<Block> excessReplicas = excessReplicateMap.get(uuid);
+      LightWeightLinkedSet<BlockInfo> excessReplicas = excessReplicateMap.get(uuid);
       if (excessReplicas != null) {
         if (excessReplicas.remove(block)) {
           excessBlocksCount.decrementAndGet();
@@ -3658,7 +4032,7 @@
   }
 
   /**
-   * Periodically calls computeReplicationWork().
+   * Periodically calls computeBlockRecoveryWork().
    */
   private class ReplicationMonitor implements Runnable {
 
@@ -3716,7 +4090,7 @@
     final int nodesToProcess = (int) Math.ceil(numlive
         * this.blocksInvalidateWorkPct);
 
-    int workFound = this.computeReplicationWork(blocksToProcess);
+    int workFound = this.computeBlockRecoveryWork(blocksToProcess);
 
     // Update counters
     namesystem.writeLock();
@@ -3756,6 +4130,31 @@
         null);
   }
 
+  public static LocatedStripedBlock newLocatedStripedBlock(
+      ExtendedBlock b, DatanodeStorageInfo[] storages,
+      int[] indices, long startOffset, boolean corrupt) {
+    // startOffset is unknown
+    return new LocatedStripedBlock(
+        b, DatanodeStorageInfo.toDatanodeInfos(storages),
+        DatanodeStorageInfo.toStorageIDs(storages),
+        DatanodeStorageInfo.toStorageTypes(storages),
+        indices, startOffset, corrupt,
+        null);
+  }
+
+  public static LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info,
+      DatanodeStorageInfo[] locs, long offset) throws IOException {
+    final LocatedBlock lb;
+    if (info.isStriped()) {
+      lb = newLocatedStripedBlock(eb, locs,
+          info.getUnderConstructionFeature().getBlockIndices(),
+          offset, false);
+    } else {
+      lb = newLocatedBlock(eb, locs, offset, false);
+    }
+    return lb;
+  }
+
   /**
    * A simple result enum for the result of
    * {@link BlockManager#processMisReplicatedBlock(BlockInfo)}.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java
new file mode 100644
index 0000000..622b258
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.util.ReflectionUtils;
+
+public class BlockPlacementPolicies{
+
+  private final BlockPlacementPolicy replicationPolicy;
+  private final BlockPlacementPolicy ecPolicy;
+
+  public BlockPlacementPolicies(Configuration conf, FSClusterStats stats,
+                                NetworkTopology clusterMap,
+                                Host2NodesMap host2datanodeMap){
+    final Class<? extends BlockPlacementPolicy> replicatorClass = conf
+        .getClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
+            DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT,
+            BlockPlacementPolicy.class);
+    replicationPolicy = ReflectionUtils.newInstance(replicatorClass, conf);
+    replicationPolicy.initialize(conf, stats, clusterMap, host2datanodeMap);
+    final Class<? extends BlockPlacementPolicy> blockPlacementECClass =
+        conf.getClass(DFSConfigKeys.DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY,
+            DFSConfigKeys.DFS_BLOCK_PLACEMENT_EC_CLASSNAME_DEFAULT,
+            BlockPlacementPolicy.class);
+    ecPolicy = ReflectionUtils.newInstance(blockPlacementECClass, conf);
+    ecPolicy.initialize(conf, stats, clusterMap, host2datanodeMap);
+  }
+
+  public BlockPlacementPolicy getPolicy(boolean isStriped){
+    if (isStriped) {
+      return ecPolicy;
+    } else {
+      return replicationPolicy;
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
index 9696179..86aaf79 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
@@ -145,31 +145,7 @@
   abstract protected void initialize(Configuration conf,  FSClusterStats stats, 
                                      NetworkTopology clusterMap, 
                                      Host2NodesMap host2datanodeMap);
-    
-  /**
-   * Get an instance of the configured Block Placement Policy based on the
-   * the configuration property
-   * {@link  DFSConfigKeys#DFS_BLOCK_REPLICATOR_CLASSNAME_KEY}.
-   * 
-   * @param conf the configuration to be used
-   * @param stats an object that is used to retrieve the load on the cluster
-   * @param clusterMap the network topology of the cluster
-   * @return an instance of BlockPlacementPolicy
-   */
-  public static BlockPlacementPolicy getInstance(Configuration conf, 
-                                                 FSClusterStats stats,
-                                                 NetworkTopology clusterMap,
-                                                 Host2NodesMap host2datanodeMap) {
-    final Class<? extends BlockPlacementPolicy> replicatorClass = conf.getClass(
-        DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
-        DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT,
-        BlockPlacementPolicy.class);
-    final BlockPlacementPolicy replicator = ReflectionUtils.newInstance(
-        replicatorClass, conf);
-    replicator.initialize(conf, stats, clusterMap, host2datanodeMap);
-    return replicator;
-  }
-  
+
   /**
    * Adjust rackmap, moreThanOne, and exactlyOne after removing replica on cur.
    *
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockRecoveryWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockRecoveryWork.java
new file mode 100644
index 0000000..ed546df
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockRecoveryWork.java
@@ -0,0 +1,111 @@
+/**
+ * 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.net.Node;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * This class is used internally by
+ * {@link BlockManager#computeRecoveryWorkForBlocks} to represent a task to
+ * recover a block through replication or erasure coding. Recovery is done by
+ * transferring data from srcNodes to targets
+ */
+abstract class BlockRecoveryWork {
+  private final BlockInfo block;
+
+  private final BlockCollection bc;
+
+  /**
+   * An erasure coding recovery task has multiple source nodes.
+   * A replication task only has 1 source node, stored on top of the array
+   */
+  private final DatanodeDescriptor[] srcNodes;
+  /** Nodes containing the block; avoid them in choosing new targets */
+  private final List<DatanodeDescriptor> containingNodes;
+  /** Required by {@link BlockPlacementPolicy#chooseTarget} */
+  private  final List<DatanodeStorageInfo> liveReplicaStorages;
+  private final int additionalReplRequired;
+
+  private DatanodeStorageInfo[] targets;
+  private final int priority;
+
+  public BlockRecoveryWork(BlockInfo block,
+      BlockCollection bc,
+      DatanodeDescriptor[] srcNodes,
+      List<DatanodeDescriptor> containingNodes,
+      List<DatanodeStorageInfo> liveReplicaStorages,
+      int additionalReplRequired,
+      int priority) {
+    this.block = block;
+    this.bc = bc;
+    this.srcNodes = srcNodes;
+    this.containingNodes = containingNodes;
+    this.liveReplicaStorages = liveReplicaStorages;
+    this.additionalReplRequired = additionalReplRequired;
+    this.priority = priority;
+    this.targets = null;
+  }
+
+  DatanodeStorageInfo[] getTargets() {
+    return targets;
+  }
+
+  void resetTargets() {
+    this.targets = null;
+  }
+
+  void setTargets(DatanodeStorageInfo[] targets) {
+    this.targets = targets;
+  }
+
+  List<DatanodeDescriptor> getContainingNodes() {
+    return Collections.unmodifiableList(containingNodes);
+  }
+
+  public int getPriority() {
+    return priority;
+  }
+
+  public BlockInfo getBlock() {
+    return block;
+  }
+
+  public DatanodeDescriptor[] getSrcNodes() {
+    return srcNodes;
+  }
+
+  BlockCollection getBc() {
+    return bc;
+  }
+
+  List<DatanodeStorageInfo> getLiveReplicaStorages() {
+    return liveReplicaStorages;
+  }
+
+  public int getAdditionalReplRequired() {
+    return additionalReplRequired;
+  }
+
+  abstract void chooseTargets(BlockPlacementPolicy blockplacement,
+      BlockStoragePolicySuite storagePolicySuite,
+      Set<Node> excludedNodes);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java
index 3842e56..a871390 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java
@@ -20,6 +20,7 @@
 import static org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.protocol.Block;
 
 /**
  * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a
@@ -27,7 +28,7 @@
  */
 class BlockToMarkCorrupt {
   /** The corrupted block in a datanode. */
-  private final BlockInfo corrupted;
+  private final Block corrupted;
   /** The corresponding block stored in the BlockManager. */
   private final BlockInfo stored;
   /** The reason to mark corrupt. */
@@ -35,7 +36,7 @@
   /** The reason code to be stored */
   private final CorruptReplicasMap.Reason reasonCode;
 
-  BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason,
+  BlockToMarkCorrupt(Block corrupted, BlockInfo stored, String reason,
       CorruptReplicasMap.Reason reasonCode) {
     Preconditions.checkNotNull(corrupted, "corrupted is null");
     Preconditions.checkNotNull(stored, "stored is null");
@@ -46,24 +47,18 @@
     this.reasonCode = reasonCode;
   }
 
-  BlockToMarkCorrupt(BlockInfo stored, String reason,
+  BlockToMarkCorrupt(Block corrupted, BlockInfo stored, long gs, String reason,
       CorruptReplicasMap.Reason reasonCode) {
-    this(stored, stored, reason, reasonCode);
-  }
-
-  BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
-      CorruptReplicasMap.Reason reasonCode) {
-    this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored,
-        reason, reasonCode);
+    this(corrupted, stored, reason, reasonCode);
     //the corrupted block in datanode has a different generation stamp
-    corrupted.setGenerationStamp(gs);
+    this.corrupted.setGenerationStamp(gs);
   }
 
   public boolean isCorruptedDuringWrite() {
     return stored.getGenerationStamp() > corrupted.getGenerationStamp();
   }
 
-  public BlockInfo getCorrupted() {
+  public Block getCorrupted() {
     return corrupted;
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
index 88cf06d..1d4cff3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
@@ -58,20 +58,38 @@
   private Block truncateBlock;
 
   public BlockUnderConstructionFeature(Block blk,
-      BlockUCState state, DatanodeStorageInfo[] targets) {
+      BlockUCState state, DatanodeStorageInfo[] targets, boolean isStriped) {
     assert getBlockUCState() != COMPLETE :
         "BlockUnderConstructionFeature cannot be in COMPLETE state";
     this.blockUCState = state;
-    setExpectedLocations(blk, targets);
+    setExpectedLocations(blk, targets, isStriped);
   }
 
   /** Set expected locations */
-  public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets) {
-    int numLocations = targets == null ? 0 : targets.length;
+  public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets,
+      boolean isStriped) {
+    if (targets == null) {
+      return;
+    }
+    int numLocations = 0;
+    for (DatanodeStorageInfo target : targets) {
+      if (target != null) {
+        numLocations++;
+      }
+    }
+
     this.replicas = new ReplicaUnderConstruction[numLocations];
-    for(int i = 0; i < numLocations; i++) {
-      replicas[i] = new ReplicaUnderConstruction(block, targets[i],
-          ReplicaState.RBW);
+    int offset = 0;
+    for(int i = 0; i < targets.length; i++) {
+      if (targets[i] != null) {
+        // when creating a new striped block we simply sequentially assign block
+        // index to each storage
+        Block replicaBlock = isStriped ?
+            new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
+            block;
+        replicas[offset++] = new ReplicaUnderConstruction(replicaBlock,
+            targets[i], ReplicaState.RBW);
+      }
     }
   }
 
@@ -88,7 +106,19 @@
     return storages;
   }
 
-  /** Get the number of expected locations */
+  /**
+   * @return the index array indicating the block index in each storage. Used
+   * only by striped blocks.
+   */
+  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;
+  }
+
   public int getNumExpectedLocations() {
     return replicas == null ? 0 : replicas.length;
   }
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 9189c32..f41b093 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
@@ -44,8 +44,15 @@
 
     @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
@@ -119,13 +126,17 @@
       return;
 
     blockInfo.setBlockCollectionId(INodeId.INVALID_INODE_ID);
-    for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
+    final int size = blockInfo.isStriped() ?
+        blockInfo.getCapacity() : blockInfo.numNodes();
+    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
+      }
     }
   }
-  
-  /** 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);
   }
@@ -186,8 +197,8 @@
     // remove block from the data-node list and the node from the block info
     boolean removed = node.removeBlock(info);
 
-    if (info.getDatanode(0) == null     // no datanodes left
-              && info.isDeleted()) {  // does not belong to a file
+    if (info.hasNoStorage()    // no datanodes left
+        && info.isDeleted()) { // does not belong to a file
       blocks.remove(b);  // remove block from the map
     }
     return removed;
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 0b398c5..b258f06 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
@@ -39,8 +39,10 @@
 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.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
@@ -48,6 +50,7 @@
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
@@ -223,6 +226,9 @@
   /** A queue of blocks to be replicated by this datanode */
   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<BlockInfo> recoverBlocks = new BlockQueue<>();
   /** A set of blocks to be invalidated by this datanode */
@@ -282,7 +288,8 @@
     }
   }
 
-  DatanodeStorageInfo[] getStorageInfos() {
+  @VisibleForTesting
+  public DatanodeStorageInfo[] getStorageInfos() {
     synchronized (storageMap) {
       final Collection<DatanodeStorageInfo> storages = storageMap.values();
       return storages.toArray(new DatanodeStorageInfo[storages.size()]);
@@ -379,6 +386,7 @@
       this.invalidateBlocks.clear();
       this.recoverBlocks.clear();
       this.replicateBlocks.clear();
+      this.erasurecodeBlocks.clear();
     }
     // pendingCached, cached, and pendingUncached are protected by the
     // FSN lock.
@@ -580,6 +588,7 @@
   Iterator<BlockInfo> getBlockIterator() {
     return new BlockIterator(getStorageInfos());
   }
+
   Iterator<BlockInfo> getBlockIterator(final String storageID) {
     return new BlockIterator(getStorageInfo(storageID));
   }
@@ -601,6 +610,20 @@
   }
 
   /**
+   * Store block erasure coding work.
+   */
+  void addBlockToBeErasureCoded(ExtendedBlock block,
+      DatanodeDescriptor[] sources, DatanodeStorageInfo[] targets,
+      short[] liveBlockIndices, ErasureCodingPolicy ecPolicy) {
+    assert (block != null && sources != null && sources.length > 0);
+    BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets,
+        liveBlockIndices, ecPolicy);
+    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(BlockInfo block) {
@@ -632,6 +655,14 @@
   }
 
   /**
+   * The number of work items that are pending to be replicated
+   */
+  @VisibleForTesting
+  public int getNumberOfBlocksToBeErasureCoded() {
+    return erasurecodeBlocks.size();
+  }
+
+  /**
    * The number of block invalidation items that are pending to 
    * be sent to the datanode
    */
@@ -645,6 +676,10 @@
     return replicateBlocks.poll(maxTransfers);
   }
 
+  public List<BlockECRecoveryInfo> getErasureCodeCommand(int maxTransfers) {
+    return erasurecodeBlocks.poll(maxTransfers);
+  }
+
   public BlockInfo[] getLeaseRecoveryCommand(int maxTransfers) {
     List<BlockInfo> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
@@ -663,6 +698,13 @@
     }
   }
 
+  @VisibleForTesting
+  public boolean containsInvalidateBlock(Block block) {
+    synchronized (invalidateBlocks) {
+      return invalidateBlocks.contains(block);
+    }
+  }
+
   /**
    * Find whether the datanode contains good storage of given type to
    * place block of size <code>blockSize</code>.
@@ -868,6 +910,10 @@
     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;");      
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 a484fcc..28ab716 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
@@ -39,6 +39,7 @@
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.protocol.*;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.*;
@@ -512,6 +513,10 @@
     }
     final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[datanodeID.length];
     for(int i = 0; i < datanodeID.length; i++) {
+      if (datanodeID[i].equals(DatanodeID.EMPTY_DATANODE_ID)) {
+        storages[i] = null;
+        continue;
+      }
       final DatanodeDescriptor dd = getDatanode(datanodeID[i]);
       storages[i] = dd.getStorageInfo(storageIDs[i]);
     }
@@ -1365,9 +1370,9 @@
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =
                 new ArrayList<>(storages.length);
-            for (int i = 0; i < storages.length; i++) {
-              if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) {
-                recoveryLocations.add(storages[i]);
+            for (DatanodeStorageInfo storage : storages) {
+              if (!storage.getDatanodeDescriptor().isStale(staleInterval)) {
+                recoveryLocations.add(storage);
               }
             }
             // If we are performing a truncate recovery than set recovery fields
@@ -1414,6 +1419,13 @@
           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_ERASURE_CODING_RECOVERY,
+              pendingECList));
+        }
         //check block invalidation
         Block[] blks = nodeinfo.getInvalidateBlocks(blockInvalidateLimit);
         if (blks != null) {
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 216d6d2..bb9a706 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@ -21,14 +21,15 @@
 import java.util.Iterator;
 import java.util.List;
 
-import com.google.common.annotations.VisibleForTesting;
-
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * A Datanode has one or more storages. A storage in the Datanode is represented
  * by this class.
@@ -209,6 +210,7 @@
     return getState() == State.FAILED && numBlocks != 0;
   }
 
+  @VisibleForTesting
   public String getStorageID() {
     return storageID;
   }
@@ -233,7 +235,7 @@
     return blockPoolUsed;
   }
 
-  public AddBlockResult addBlock(BlockInfo 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;
@@ -252,10 +254,18 @@
     }
 
     // add to the head of the data-node list
-    b.addStorage(this);
+    b.addStorage(this, reportedBlock);
+    insertToList(b);
+    return result;
+  }
+
+  AddBlockResult addBlock(BlockInfo b) {
+    return addBlock(b, b);
+  }
+
+  public void insertToList(BlockInfo b) {
     blockList = b.listInsert(blockList, this);
     numBlocks++;
-    return result;
   }
 
   public boolean removeBlock(BlockInfo b) {
@@ -274,7 +284,6 @@
   
   Iterator<BlockInfo> getBlockIterator() {
     return new BlockIterator(blockList);
-
   }
 
   /**
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 6d199d7..fb86ff3 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
@@ -234,14 +234,14 @@
   }
 
   /**
-   * Checks whether a block is sufficiently replicated for decommissioning.
-   * Full-strength replication is not always necessary, hence "sufficient".
+   * Checks whether a block is sufficiently replicated/stored for
+   * decommissioning. For replicated blocks or striped blocks, full-strength
+   * replication or storage is not always necessary, hence "sufficient".
    * @return true if sufficient, else false.
    */
-  private boolean isSufficientlyReplicated(BlockInfo block,
-      BlockCollection bc,
+  private boolean isSufficient(BlockInfo block, BlockCollection bc,
       NumberReplicas numberReplicas) {
-    final int numExpected = block.getReplication();
+    final int numExpected = blockManager.getExpectedReplicaNum(block);
     final int numLive = numberReplicas.liveReplicas();
     if (!blockManager.isNeededReplication(block, numLive)) {
       // Block doesn't need replication. Skip.
@@ -255,18 +255,19 @@
     if (numExpected > numLive) {
       if (bc.isUnderConstruction() && block.equals(bc.getLastBlock())) {
         // Can decom a UC block as long as there will still be minReplicas
-        if (numLive >= blockManager.minReplication) {
+        if (blockManager.hasMinStorage(block, numLive)) {
           LOG.trace("UC block {} sufficiently-replicated since numLive ({}) "
-              + ">= minR ({})", block, numLive, blockManager.minReplication);
+              + ">= minR ({})", block, numLive,
+              blockManager.getMinStorageNum(block));
           return true;
         } else {
           LOG.trace("UC block {} insufficiently-replicated since numLive "
               + "({}) < minR ({})", block, numLive,
-              blockManager.minReplication);
+              blockManager.getMinStorageNum(block));
         }
       } else {
         // Can decom a non-UC as long as the default replication is met
-        if (numLive >= blockManager.defaultReplication) {
+        if (numLive >= blockManager.getDefaultStorageNum(block)) {
           return true;
         }
       }
@@ -274,12 +275,12 @@
     return false;
   }
 
-  private static void logBlockReplicationInfo(BlockInfo block,
+  private void logBlockReplicationInfo(BlockInfo block,
       BlockCollection bc,
       DatanodeDescriptor srcNode, NumberReplicas num,
       Iterable<DatanodeStorageInfo> storages) {
     int curReplicas = num.liveReplicas();
-    int curExpectedReplicas = block.getReplication();
+    int curExpectedReplicas = blockManager.getExpectedReplicaNum(block);
     StringBuilder nodeList = new StringBuilder();
     for (DatanodeStorageInfo storage : storages) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
@@ -408,14 +409,14 @@
           // that are insufficiently replicated for further tracking
           LOG.debug("Newly-added node {}, doing full scan to find " +
               "insufficiently-replicated blocks.", dn);
-          blocks = handleInsufficientlyReplicated(dn);
+          blocks = handleInsufficientlyStored(dn);
           decomNodeBlocks.put(dn, blocks);
           fullScan = true;
         } else {
           // This is a known datanode, check if its # of insufficiently 
           // replicated blocks has dropped to zero and if it can be decommed
           LOG.debug("Processing decommission-in-progress node {}", dn);
-          pruneSufficientlyReplicated(dn, blocks);
+          pruneReliableBlocks(dn, blocks);
         }
         if (blocks.size() == 0) {
           if (!fullScan) {
@@ -427,7 +428,7 @@
             // marking the datanode as decommissioned 
             LOG.debug("Node {} has finished replicating current set of "
                 + "blocks, checking with the full block map.", dn);
-            blocks = handleInsufficientlyReplicated(dn);
+            blocks = handleInsufficientlyStored(dn);
             decomNodeBlocks.put(dn, blocks);
           }
           // If the full scan is clean AND the node liveness is okay, 
@@ -468,25 +469,23 @@
     }
 
     /**
-     * Removes sufficiently replicated blocks from the block list of a 
-     * datanode.
+     * Removes reliable blocks from the block list of a datanode.
      */
-    private void pruneSufficientlyReplicated(final DatanodeDescriptor datanode,
+    private void pruneReliableBlocks(final DatanodeDescriptor datanode,
         AbstractList<BlockInfo> blocks) {
       processBlocksForDecomInternal(datanode, blocks.iterator(), null, true);
     }
 
     /**
-     * Returns a list of blocks on a datanode that are insufficiently 
-     * replicated, i.e. are under-replicated enough to prevent decommission.
+     * Returns a list of blocks on a datanode that are insufficiently replicated
+     * or require recovery, i.e. requiring recovery and should prevent
+     * decommission.
      * <p/>
-     * As part of this, it also schedules replication work for 
-     * any under-replicated blocks.
+     * As part of this, it also schedules replication/recovery work.
      *
-     * @param datanode
-     * @return List of insufficiently replicated blocks 
+     * @return List of blocks requiring recovery
      */
-    private AbstractList<BlockInfo> handleInsufficientlyReplicated(
+    private AbstractList<BlockInfo> handleInsufficientlyStored(
         final DatanodeDescriptor datanode) {
       AbstractList<BlockInfo> insufficient = new ChunkedArrayList<>();
       processBlocksForDecomInternal(datanode, datanode.getBlockIterator(),
@@ -497,24 +496,22 @@
     /**
      * Used while checking if decommission-in-progress datanodes can be marked
      * as decommissioned. Combines shared logic of 
-     * pruneSufficientlyReplicated and handleInsufficientlyReplicated.
+     * pruneReliableBlocks and handleInsufficientlyStored.
      *
      * @param datanode                    Datanode
      * @param it                          Iterator over the blocks on the
      *                                    datanode
-     * @param insufficientlyReplicated    Return parameter. If it's not null,
+     * @param insufficientList            Return parameter. If it's not null,
      *                                    will contain the insufficiently
      *                                    replicated-blocks from the list.
-     * @param pruneSufficientlyReplicated whether to remove sufficiently
-     *                                    replicated blocks from the iterator
-     * @return true if there are under-replicated blocks in the provided block
-     * iterator, else false.
+     * @param pruneReliableBlocks         whether to remove blocks reliable
+     *                                    enough from the iterator
      */
     private void processBlocksForDecomInternal(
         final DatanodeDescriptor datanode,
         final Iterator<BlockInfo> it,
-        final List<BlockInfo> insufficientlyReplicated,
-        boolean pruneSufficientlyReplicated) {
+        final List<BlockInfo> insufficientList,
+        boolean pruneReliableBlocks) {
       boolean firstReplicationLog = true;
       int underReplicatedBlocks = 0;
       int decommissionOnlyReplicas = 0;
@@ -539,7 +536,6 @@
         BlockCollection bc = namesystem.getBlockCollection(bcId);
         final NumberReplicas num = blockManager.countNodes(block);
         final int liveReplicas = num.liveReplicas();
-        final int curReplicas = liveReplicas;
 
         // Schedule under-replicated blocks for replication if not already
         // pending
@@ -549,24 +545,24 @@
               blockManager.isPopulatingReplQueues()) {
             // Process these blocks only when active NN is out of safe mode.
             blockManager.neededReplications.add(block,
-                curReplicas,
+                liveReplicas,
                 num.decommissionedAndDecommissioning(),
-                block.getReplication());
+                blockManager.getExpectedReplicaNum(block));
           }
         }
 
         // Even if the block is under-replicated, 
-        // it doesn't block decommission if it's sufficiently replicated 
-        if (isSufficientlyReplicated(block, bc, num)) {
-          if (pruneSufficientlyReplicated) {
+        // it doesn't block decommission if it's sufficiently replicated
+        if (isSufficient(block, bc, num)) {
+          if (pruneReliableBlocks) {
             it.remove();
           }
           continue;
         }
 
         // We've found an insufficiently replicated block.
-        if (insufficientlyReplicated != null) {
-          insufficientlyReplicated.add(block);
+        if (insufficientList != null) {
+          insufficientList.add(block);
         }
         // Log if this is our first time through
         if (firstReplicationLog) {
@@ -579,7 +575,7 @@
         if (bc.isUnderConstruction()) {
           underReplicatedInOpenFiles++;
         }
-        if ((curReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) {
+        if ((liveReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) {
           decommissionOnlyReplicas++;
         }
       }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java
new file mode 100644
index 0000000..761d6d0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java
@@ -0,0 +1,60 @@
+/**
+ * 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.net.Node;
+
+import java.util.List;
+import java.util.Set;
+
+class ErasureCodingWork extends BlockRecoveryWork {
+  private final short[] liveBlockIndicies;
+
+  public ErasureCodingWork(BlockInfo block,
+      BlockCollection bc,
+      DatanodeDescriptor[] srcNodes,
+      List<DatanodeDescriptor> containingNodes,
+      List<DatanodeStorageInfo> liveReplicaStorages,
+      int additionalReplRequired,
+      int priority, short[] liveBlockIndicies) {
+    super(block, bc, srcNodes, containingNodes,
+        liveReplicaStorages, additionalReplRequired, priority);
+    this.liveBlockIndicies = liveBlockIndicies;
+    BlockManager.LOG.debug("Creating an ErasureCodingWork to recover " + block);
+  }
+
+  short[] getLiveBlockIndicies() {
+    return liveBlockIndicies;
+  }
+
+  @Override
+  void chooseTargets(BlockPlacementPolicy blockplacement,
+      BlockStoragePolicySuite storagePolicySuite,
+      Set<Node> excludedNodes) {
+    try {
+      // TODO: new placement policy for EC considering multiple writers
+      DatanodeStorageInfo[] chosenTargets = blockplacement.chooseTarget(
+          getBc().getName(), getAdditionalReplRequired(), getSrcNodes()[0],
+          getLiveReplicaStorages(), false, excludedNodes,
+          getBlock().getNumBytes(),
+          storagePolicySuite.getPolicy(getBc().getStoragePolicyID()));
+      setTargets(chosenTargets);
+    } finally {
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java
index f8a6dad..8266f45 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java
@@ -23,65 +23,31 @@
 import java.util.List;
 import java.util.Set;
 
-class ReplicationWork {
-  private final BlockInfo block;
-  private final BlockCollection bc;
-  private final DatanodeDescriptor srcNode;
-  private final int additionalReplRequired;
-  private final int priority;
-  private final List<DatanodeDescriptor> containingNodes;
-  private final List<DatanodeStorageInfo> liveReplicaStorages;
-  private DatanodeStorageInfo[] targets;
-
+class ReplicationWork extends BlockRecoveryWork {
   public ReplicationWork(BlockInfo block, BlockCollection bc,
-      DatanodeDescriptor srcNode, List<DatanodeDescriptor> containingNodes,
+      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.containingNodes = containingNodes;
-    this.liveReplicaStorages = liveReplicaStorages;
-    this.additionalReplRequired = additionalReplRequired;
-    this.priority = priority;
-    this.targets = null;
+    super(block, bc, srcNodes, containingNodes,
+        liveReplicaStorages, additionalReplRequired, priority);
+    BlockManager.LOG.debug("Creating a ReplicationWork to recover " + block);
   }
 
+  @Override
   void chooseTargets(BlockPlacementPolicy blockplacement,
       BlockStoragePolicySuite storagePolicySuite,
       Set<Node> excludedNodes) {
+    assert getSrcNodes().length > 0
+        : "At least 1 source node should have been selected";
     try {
-      targets = blockplacement.chooseTarget(bc.getName(),
-          additionalReplRequired, srcNode, liveReplicaStorages, false,
-          excludedNodes, block.getNumBytes(),
-          storagePolicySuite.getPolicy(bc.getStoragePolicyID()));
+      DatanodeStorageInfo[] chosenTargets = blockplacement.chooseTarget(
+          getBc().getName(), getAdditionalReplRequired(), getSrcNodes()[0],
+          getLiveReplicaStorages(), false, excludedNodes,
+          getBlock().getNumBytes(),
+          storagePolicySuite.getPolicy(getBc().getStoragePolicyID()));
+      setTargets(chosenTargets);
     } finally {
-      srcNode.decrementPendingReplicationWithoutTargets();
+      getSrcNodes()[0].decrementPendingReplicationWithoutTargets();
     }
   }
-
-  DatanodeStorageInfo[] getTargets() {
-    return targets;
-  }
-
-  void resetTargets() {
-    this.targets = null;
-  }
-
-  List<DatanodeDescriptor> getContainingNodes() {
-    return Collections.unmodifiableList(containingNodes);
-  }
-
-  public int getPriority() {
-    return priority;
-  }
-
-  public BlockInfo getBlock() {
-    return block;
-  }
-
-  public DatanodeDescriptor getSrcNode() {
-    return srcNode;
-  }
 }
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..7a52273
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java
@@ -0,0 +1,86 @@
+/**
+ * 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.util.SequentialNumber;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BLOCK_GROUP_INDEX_MASK;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_BLOCKS_IN_GROUP;
+
+/**
+ * Generate the next valid block group ID by incrementing the maximum block
+ * 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).
+ *
+ * Note that the {@link #nextValue()} methods requires external lock to
+ * guarantee IDs have no conflicts.
+ */
+@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() {
+    skipTo((getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK) + MAX_BLOCKS_IN_GROUP);
+    // Make sure there's no conflict with existing random block IDs
+    final Block b = new Block(getCurrentValue());
+    while (hasValidBlockInRange(b)) {
+      skipTo(getCurrentValue() + MAX_BLOCKS_IN_GROUP);
+      b.setBlockId(getCurrentValue());
+    }
+    if (b.getBlockId() >= 0) {
+      throw new IllegalStateException("All negative block group IDs are used, "
+          + "growing into positive IDs, "
+          + "which might conflict with non-erasure coded blocks.");
+    }
+    return getCurrentValue();
+  }
+
+  /**
+   * @param b A block object whose id is set to the starting point for check
+   * @return true if any ID in the range
+   *      {id, id+HdfsConstants.MAX_BLOCKS_IN_GROUP} is pointed-to by a stored
+   *      block.
+   */
+  private boolean hasValidBlockInRange(Block b) {
+    final long id = b.getBlockId();
+    for (int i = 0; i < MAX_BLOCKS_IN_GROUP; i++) {
+      b.setBlockId(id + i);
+      if (blockManager.getStoredBlock(b) != null) {
+        return true;
+      }
+    }
+    return false;
+  }
+}
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 f053b7b..631b435 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 @@
 
 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.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.util.SequentialNumber;
 
@@ -55,6 +54,11 @@
     while(isValidBlock(b)) {
       b.setBlockId(super.nextValue());
     }
+    if (b.getBlockId() < 0) {
+      throw new IllegalStateException("All positive block IDs are used, " +
+          "wrapping to negative IDs, " +
+          "which might conflict with erasure coded block groups.");
+    }
     return b.getBlockId();
   }
 
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 ebc15b8..7e8f479 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
@@ -34,7 +34,7 @@
  *
  * <p/>
  * The policy for choosing which priority to give added blocks
- * is implemented in {@link #getPriority(int, int, int)}.
+ * is implemented in {@link #getPriority(BlockInfo, int, int, int)}.
  * </p>
  * <p>The queue order is as follows:</p>
  * <ol>
@@ -145,14 +145,28 @@
    * @param expectedReplicas expected number of replicas of the block
    * @return the priority for the blocks, between 0 and ({@link #LEVEL}-1)
    */
-  private int getPriority(int curReplicas,
+  private int getPriority(BlockInfo block,
+                          int curReplicas,
                           int decommissionedReplicas,
                           int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
     if (curReplicas >= expectedReplicas) {
       // Block has enough copies, but not enough racks
       return QUEUE_REPLICAS_BADLY_DISTRIBUTED;
-    } else if (curReplicas == 0) {
+    }
+    if (block.isStriped()) {
+      BlockInfoStriped sblk = (BlockInfoStriped) block;
+      return getPriorityStriped(curReplicas, decommissionedReplicas,
+          sblk.getRealDataBlockNum(), sblk.getParityBlockNum());
+    } else {
+      return getPriorityContiguous(curReplicas, decommissionedReplicas,
+          expectedReplicas);
+    }
+  }
+
+  private int getPriorityContiguous(int curReplicas, int decommissionedReplicas,
+      int expectedReplicas) {
+    if (curReplicas == 0) {
       // If there are zero non-decommissioned replicas but there are
       // some decommissioned replicas, then assign them highest priority
       if (decommissionedReplicas > 0) {
@@ -161,7 +175,7 @@
       //all we have are corrupt blocks
       return QUEUE_WITH_CORRUPT_BLOCKS;
     } else if (curReplicas == 1) {
-      //only on replica -risk of loss
+      // only one replica, highest risk of loss
       // highest priority
       return QUEUE_HIGHEST_PRIORITY;
     } else if ((curReplicas * 3) < expectedReplicas) {
@@ -174,6 +188,27 @@
     }
   }
 
+  private int getPriorityStriped(int curReplicas, int decommissionedReplicas,
+      short dataBlkNum, short parityBlkNum) {
+    if (curReplicas < dataBlkNum) {
+      // There are some replicas on decommissioned nodes so it's not corrupted
+      if (curReplicas + decommissionedReplicas >= dataBlkNum) {
+        return QUEUE_HIGHEST_PRIORITY;
+      }
+      return QUEUE_WITH_CORRUPT_BLOCKS;
+    } else if (curReplicas == dataBlkNum) {
+      // highest risk of loss, highest priority
+      return QUEUE_HIGHEST_PRIORITY;
+    } else if ((curReplicas - dataBlkNum) * 3 < parityBlkNum + 1) {
+      // can only afford one replica loss
+      // this is considered very under-replicated
+      return QUEUE_VERY_UNDER_REPLICATED;
+    } else {
+      // add to the normal queue for under replicated blocks
+      return QUEUE_UNDER_REPLICATED;
+    }
+  }
+
   /** add a block to a under replication queue according to its priority
    * @param block a under replication block
    * @param curReplicas current number of replicas of the block
@@ -186,7 +221,7 @@
                            int decomissionedReplicas,
                            int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
-    int priLevel = getPriority(curReplicas, decomissionedReplicas,
+    int priLevel = getPriority(block, curReplicas, decomissionedReplicas,
                                expectedReplicas);
     if(priorityQueues.get(priLevel).add(block)) {
       if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
@@ -209,7 +244,7 @@
                               int oldReplicas,
                               int decommissionedReplicas,
                               int oldExpectedReplicas) {
-    int priLevel = getPriority(oldReplicas,
+    int priLevel = getPriority(block, oldReplicas,
                                decommissionedReplicas,
                                oldExpectedReplicas);
     boolean removedBlock = remove(block, priLevel);
@@ -283,9 +318,9 @@
                            int curReplicasDelta, int expectedReplicasDelta) {
     int oldReplicas = curReplicas-curReplicasDelta;
     int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
-    int curPri = getPriority(curReplicas, decommissionedReplicas,
+    int curPri = getPriority(block, curReplicas, decommissionedReplicas,
         curExpectedReplicas);
-    int oldPri = getPriority(oldReplicas, decommissionedReplicas,
+    int oldPri = getPriority(block, oldReplicas, decommissionedReplicas,
         oldExpectedReplicas);
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " + 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index c4c3d8a..6208a7d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -368,4 +368,9 @@
       "raw.hdfs.crypto.file.encryption.info";
   String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER =
       "security.hdfs.unreadable.by.superuser";
+  String XATTR_ERASURECODING_POLICY =
+      "raw.hdfs.erasurecoding.policy";
+
+  long BLOCK_GROUP_INDEX_MASK = 15;
+  byte MAX_BLOCKS_IN_GROUP = 16;
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index 7aab4f7..96e74e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -32,11 +32,13 @@
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.protocol.*;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
@@ -721,6 +723,11 @@
         dxcs.balanceThrottler.setBandwidth(bandwidth);
       }
       break;
+    case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY:
+      LOG.info("DatanodeCommand action: DNA_ERASURE_CODING_RECOVERY");
+      Collection<BlockECRecoveryInfo> ecTasks = ((BlockECRecoveryCommand) cmd).getECTasks();
+      dn.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
+      break;
     default:
       LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
     }
@@ -750,6 +757,7 @@
     case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
     case DatanodeProtocol.DNA_CACHE:
     case DatanodeProtocol.DNA_UNCACHE:
+    case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY:
       LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
       break;
     default:
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 bd4943d..b3cb48b 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
@@ -254,6 +254,33 @@
   }
 
   /**
+   * Returns true if connect to datanode via hostname
+   * 
+   * @return boolean true if connect to datanode via hostname
+   */
+  public boolean getConnectToDnViaHostname() {
+    return connectToDnViaHostname;
+  }
+
+  /**
+   * Returns socket timeout
+   * 
+   * @return int socket timeout
+   */
+  public int getSocketTimeout() {
+    return socketTimeout;
+  }
+
+  /**
+   * Returns socket write timeout
+   * 
+   * @return int socket write timeout
+   */
+  public int getSocketWriteTimeout() {
+    return socketWriteTimeout;
+  }
+
+  /**
    * Returns the SaslPropertiesResolver configured for use with
    * DataTransferProtocol, or null if not configured.
    *
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 2fe67fd..10a8a9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -92,6 +92,7 @@
 
 import javax.management.ObjectName;
 
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -156,6 +157,7 @@
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
+import org.apache.hadoop.hdfs.server.datanode.erasurecode.ErasureCodingWorker;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
@@ -366,7 +368,7 @@
   private String supergroup;
   private boolean isPermissionEnabled;
   private String dnUserName = null;
-
+  private ErasureCodingWorker ecWorker;
   final Tracer tracer;
   private final TracerConfigurationManager tracerConfigurationManager;
   private static final int NUM_CORES = Runtime.getRuntime()
@@ -1210,6 +1212,8 @@
     saslClient = new SaslDataTransferClient(dnConf.conf, 
         dnConf.saslPropsResolver, dnConf.trustedChannelResolver);
     saslServer = new SaslDataTransferServer(dnConf, blockPoolTokenSecretManager);
+    // Initialize ErasureCoding worker
+    ecWorker = new ErasureCodingWorker(conf, this);
     startMetricsLogger(conf);
   }
 
@@ -1275,6 +1279,10 @@
     return UUID.randomUUID().toString();
   }
 
+  public SaslDataTransferClient getSaslClient() {
+    return saslClient;
+  }
+
   /**
    * Verify that the DatanodeUuid has been initialized. If this is a new
    * datanode then we generate a new Datanode Uuid and persist it to disk.
@@ -1534,7 +1542,7 @@
   /**
    * Creates either NIO or regular depending on socketWriteTimeout.
    */
-  protected Socket newSocket() throws IOException {
+  public Socket newSocket() throws IOException {
     return (dnConf.socketWriteTimeout > 0) ? 
            SocketChannel.open().socket() : new Socket();                                   
   }
@@ -1919,6 +1927,21 @@
   int getXmitsInProgress() {
     return xmitsInProgress.get();
   }
+  
+  /**
+   * Increments the xmitsInProgress count. xmitsInProgress count represents the
+   * number of data replication/reconstruction tasks running currently.
+   */
+  public void incrementXmitsInProgress() {
+    xmitsInProgress.getAndIncrement();
+  }
+
+  /**
+   * Decrements the xmitsInProgress count
+   */
+  public void decrementXmitsInProgress() {
+    xmitsInProgress.getAndDecrement();
+  }
 
   private void reportBadBlock(final BPOfferService bpos,
       final ExtendedBlock block, final String msg) {
@@ -2138,7 +2161,7 @@
      */
     @Override
     public void run() {
-      xmitsInProgress.getAndIncrement();
+      incrementXmitsInProgress();
       Socket sock = null;
       DataOutputStream out = null;
       DataInputStream in = null;
@@ -2158,11 +2181,8 @@
         //
         // Header info
         //
-        Token<BlockTokenIdentifier> accessToken = BlockTokenSecretManager.DUMMY_TOKEN;
-        if (isBlockTokenEnabled) {
-          accessToken = blockPoolTokenSecretManager.generateToken(b, 
-              EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
-        }
+        Token<BlockTokenIdentifier> accessToken = getBlockAccessToken(b, 
+            EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
 
         long writeTimeout = dnConf.socketWriteTimeout + 
                             HdfsConstants.WRITE_TIMEOUT_EXTENSION * (targets.length-1);
@@ -2220,7 +2240,7 @@
         // check if there are any disk problem
         checkDiskErrorAsync();
       } finally {
-        xmitsInProgress.getAndDecrement();
+        decrementXmitsInProgress();
         IOUtils.closeStream(blockSender);
         IOUtils.closeStream(out);
         IOUtils.closeStream(in);
@@ -2229,6 +2249,19 @@
     }
   }
 
+  /***
+   * Use BlockTokenSecretManager to generate block token for current user.
+   */
+  public Token<BlockTokenIdentifier> getBlockAccessToken(ExtendedBlock b,
+      EnumSet<AccessMode> mode) throws IOException {
+    Token<BlockTokenIdentifier> accessToken = 
+        BlockTokenSecretManager.DUMMY_TOKEN;
+    if (isBlockTokenEnabled) {
+      accessToken = blockPoolTokenSecretManager.generateToken(b, mode);
+    }
+    return accessToken;
+  }
+
   /**
    * Returns a new DataEncryptionKeyFactory that generates a key from the
    * BlockPoolTokenSecretManager, using the block pool ID of the given block.
@@ -2236,7 +2269,7 @@
    * @param block for which the factory needs to create a key
    * @return DataEncryptionKeyFactory for block's block pool ID
    */
-  DataEncryptionKeyFactory getDataEncryptionKeyFactoryForBlock(
+  public DataEncryptionKeyFactory getDataEncryptionKeyFactoryForBlock(
       final ExtendedBlock block) {
     return new DataEncryptionKeyFactory() {
       @Override
@@ -3290,6 +3323,10 @@
     checkSuperuserPrivilege();
     tracerConfigurationManager.removeSpanReceiver(id);
   }
+  
+  public ErasureCodingWorker getErasureCodingWorker(){
+    return ecWorker;
+  }
 
   /**
    * Get timeout value of each OOB type from configuration
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
index 5c8dd85..7873459 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
@@ -54,7 +54,7 @@
       // drop any (illegal) authority in the URI for backwards compatibility
       this.file = new File(uri.getPath());
     } else {
-      throw new IllegalArgumentException("Unsupported URI schema in " + uri);
+      throw new IllegalArgumentException("Unsupported URI ecPolicy in " + uri);
     }
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
new file mode 100644
index 0000000..7c64b37
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@ -0,0 +1,1016 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode.erasurecode;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSPacket;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.RemoteBlockReader2;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.net.TcpPeerServer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DataChecksum;
+
+import com.google.common.base.Preconditions;
+
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.convertIndex4Decode;
+
+/**
+ * ErasureCodingWorker handles the erasure coding recovery work commands. These
+ * commands would be issued from Namenode as part of Datanode's heart beat
+ * response. BPOfferService delegates the work to this class for handling EC
+ * commands.
+ */
+public final class ErasureCodingWorker {
+  private static final Log LOG = DataNode.LOG;
+  
+  private final DataNode datanode; 
+  private final Configuration conf;
+
+  private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL;
+  private ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
+  private final int STRIPED_READ_TIMEOUT_MILLIS;
+  private final int STRIPED_READ_BUFFER_SIZE;
+
+  public ErasureCodingWorker(Configuration conf, DataNode datanode) {
+    this.datanode = datanode;
+    this.conf = conf;
+
+    STRIPED_READ_TIMEOUT_MILLIS = conf.getInt(
+        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY,
+        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT);
+    initializeStripedReadThreadPool(conf.getInt(
+        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_KEY, 
+        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT));
+    STRIPED_READ_BUFFER_SIZE = conf.getInt(
+        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY,
+        DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT);
+
+    initializeStripedBlkRecoveryThreadPool(conf.getInt(
+        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY,
+        DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT));
+  }
+  
+  private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) {
+    return CodecUtil.createRSRawDecoder(conf, numDataUnits, numParityUnits);
+  }
+
+  private void initializeStripedReadThreadPool(int num) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using striped reads; pool threads=" + num);
+    }
+    STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
+        TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+        new Daemon.DaemonFactory() {
+      private final AtomicInteger threadIndex = new AtomicInteger(0);
+
+      @Override
+      public Thread newThread(Runnable r) {
+        Thread t = super.newThread(r);
+        t.setName("stripedRead-" + threadIndex.getAndIncrement());
+        return t;
+      }
+    }, new ThreadPoolExecutor.CallerRunsPolicy() {
+      @Override
+      public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {
+        LOG.info("Execution for striped reading rejected, "
+            + "Executing in current thread");
+        // will run in the current thread
+        super.rejectedExecution(runnable, e);
+      }
+    });
+    STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
+  }
+
+  private void initializeStripedBlkRecoveryThreadPool(int num) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using striped block recovery; pool threads=" + num);
+    }
+    STRIPED_BLK_RECOVERY_THREAD_POOL = new ThreadPoolExecutor(2, num, 60,
+        TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
+        new Daemon.DaemonFactory() {
+          private final AtomicInteger threadIdx = new AtomicInteger(0);
+
+          @Override
+          public Thread newThread(Runnable r) {
+            Thread t = super.newThread(r);
+            t.setName("stripedBlockRecovery-" + threadIdx.getAndIncrement());
+            return t;
+          }
+        });
+    STRIPED_BLK_RECOVERY_THREAD_POOL.allowCoreThreadTimeOut(true);
+  }
+
+  /**
+   * Handles the Erasure Coding recovery work commands.
+   * 
+   * @param ecTasks
+   *          BlockECRecoveryInfo
+   */
+  public void processErasureCodingTasks(Collection<BlockECRecoveryInfo> ecTasks) {
+    for (BlockECRecoveryInfo recoveryInfo : ecTasks) {
+      try {
+        STRIPED_BLK_RECOVERY_THREAD_POOL
+            .submit(new ReconstructAndTransferBlock(recoveryInfo));
+      } catch (Throwable e) {
+        LOG.warn("Failed to recover striped block "
+            + recoveryInfo.getExtendedBlock().getLocalBlock(), e);
+      }
+    }
+  }
+
+  /**
+   * ReconstructAndTransferBlock recover one or more missed striped block in the
+   * striped block group, the minimum number of live striped blocks should be
+   * no less than data block number.
+   * 
+   * | <- Striped Block Group -> |
+   *  blk_0      blk_1       blk_2(*)   blk_3   ...   <- A striped block group
+   *    |          |           |          |  
+   *    v          v           v          v 
+   * +------+   +------+   +------+   +------+
+   * |cell_0|   |cell_1|   |cell_2|   |cell_3|  ...    
+   * +------+   +------+   +------+   +------+     
+   * |cell_4|   |cell_5|   |cell_6|   |cell_7|  ...
+   * +------+   +------+   +------+   +------+
+   * |cell_8|   |cell_9|   |cell10|   |cell11|  ...
+   * +------+   +------+   +------+   +------+
+   *  ...         ...       ...         ...
+   *  
+   * 
+   * We use following steps to recover striped block group, in each round, we
+   * recover <code>bufferSize</code> data until finish, the 
+   * <code>bufferSize</code> is configurable and may be less or larger than 
+   * cell size:
+   * step1: read <code>bufferSize</code> data from minimum number of sources 
+   *        required by recovery.
+   * step2: decode data for targets.
+   * step3: transfer data to targets.
+   * 
+   * In step1, try to read <code>bufferSize</code> data from minimum number
+   * of sources , if there is corrupt or stale sources, read from new source
+   * will be scheduled. The best sources are remembered for next round and 
+   * may be updated in each round.
+   * 
+   * In step2, typically if source blocks we read are all data blocks, we 
+   * need to call encode, and if there is one parity block, we need to call
+   * decode. Notice we only read once and recover all missed striped block 
+   * if they are more than one.
+   * 
+   * In step3, send the recovered data to targets by constructing packet 
+   * and send them directly. Same as continuous block replication, we 
+   * don't check the packet ack. Since the datanode doing the recovery work
+   * are one of the source datanodes, so the recovered data are sent 
+   * remotely.
+   * 
+   * There are some points we can do further improvements in next phase:
+   * 1. we can read the block file directly on the local datanode, 
+   *    currently we use remote block reader. (Notice short-circuit is not
+   *    a good choice, see inline comments).
+   * 2. We need to check the packet ack for EC recovery? Since EC recovery
+   *    is more expensive than continuous block replication, it needs to 
+   *    read from several other datanodes, should we make sure the 
+   *    recovered result received by targets? 
+   */
+  private class ReconstructAndTransferBlock implements Runnable {
+    private final int dataBlkNum;
+    private final int parityBlkNum;
+    private final int cellSize;
+    
+    private RawErasureDecoder decoder;
+
+    // Striped read buffer size
+    private int bufferSize;
+
+    private final ExtendedBlock blockGroup;
+    private final int minRequiredSources;
+    // position in striped internal block
+    private long positionInBlock;
+
+    // sources
+    private final short[] liveIndices;
+    private final DatanodeInfo[] sources;
+
+    private final List<StripedReader> stripedReaders;
+
+    // The buffers and indices for striped blocks whose length is 0
+    private ByteBuffer[] zeroStripeBuffers;
+    private short[] zeroStripeIndices;
+
+    // targets
+    private final DatanodeInfo[] targets;
+    private final StorageType[] targetStorageTypes;
+
+    private final short[] targetIndices;
+    private final ByteBuffer[] targetBuffers;
+
+    private final Socket[] targetSockets;
+    private final DataOutputStream[] targetOutputStreams;
+    private final DataInputStream[] targetInputStreams;
+
+    private final long[] blockOffset4Targets;
+    private final long[] seqNo4Targets;
+
+    private final static int WRITE_PACKET_SIZE = 64 * 1024;
+    private DataChecksum checksum;
+    private int maxChunksPerPacket;
+    private byte[] packetBuf;
+    private byte[] checksumBuf;
+    private int bytesPerChecksum;
+    private int checksumSize;
+
+    private final CachingStrategy cachingStrategy;
+
+    private final Map<Future<Void>, Integer> futures = new HashMap<>();
+    private final CompletionService<Void> readService =
+        new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL);
+
+    ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) {
+      ErasureCodingPolicy ecPolicy = recoveryInfo.getErasureCodingPolicy();
+      dataBlkNum = ecPolicy.getNumDataUnits();
+      parityBlkNum = ecPolicy.getNumParityUnits();
+      cellSize = ecPolicy.getCellSize();
+
+      blockGroup = recoveryInfo.getExtendedBlock();
+      final int cellsNum = (int)((blockGroup.getNumBytes() - 1) / cellSize + 1);
+      minRequiredSources = Math.min(cellsNum, dataBlkNum);
+
+      liveIndices = recoveryInfo.getLiveBlockIndices();
+      sources = recoveryInfo.getSourceDnInfos();
+      stripedReaders = new ArrayList<>(sources.length);
+
+      Preconditions.checkArgument(liveIndices.length >= minRequiredSources,
+          "No enough live striped blocks.");
+      Preconditions.checkArgument(liveIndices.length == sources.length,
+          "liveBlockIndices and source dns should match");
+
+      if (minRequiredSources < dataBlkNum) {
+        zeroStripeBuffers = 
+            new ByteBuffer[dataBlkNum - minRequiredSources];
+        zeroStripeIndices = new short[dataBlkNum - minRequiredSources];
+      }
+
+      targets = recoveryInfo.getTargetDnInfos();
+      targetStorageTypes = recoveryInfo.getTargetStorageTypes();
+      targetIndices = new short[targets.length];
+      targetBuffers = new ByteBuffer[targets.length];
+
+      Preconditions.checkArgument(targetIndices.length <= parityBlkNum,
+          "Too much missed striped blocks.");
+
+      targetSockets = new Socket[targets.length];
+      targetOutputStreams = new DataOutputStream[targets.length];
+      targetInputStreams = new DataInputStream[targets.length];
+
+      blockOffset4Targets = new long[targets.length];
+      seqNo4Targets = new long[targets.length];
+
+      for (int i = 0; i < targets.length; i++) {
+        blockOffset4Targets[i] = 0;
+        seqNo4Targets[i] = 0;
+      }
+
+      getTargetIndices();
+      cachingStrategy = CachingStrategy.newDefaultStrategy();
+    }
+
+    private ByteBuffer allocateBuffer(int length) {
+      return ByteBuffer.allocate(length);
+    }
+
+    private ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) {
+      return StripedBlockUtil.constructInternalBlock(blockGroup, cellSize,
+          dataBlkNum, i);
+    }
+
+    private long getBlockLen(ExtendedBlock blockGroup, int i) { 
+      return StripedBlockUtil.getInternalBlockLength(blockGroup.getNumBytes(),
+          cellSize, dataBlkNum, i);
+    }
+
+    /**
+     * StripedReader is used to read from one source DN, it contains a block
+     * reader, buffer and striped block index.
+     * Only allocate StripedReader once for one source, and the StripedReader
+     * has the same array order with sources. Typically we only need to allocate
+     * minimum number (minRequiredSources) of StripedReader, and allocate
+     * new for new source DN if some existing DN invalid or slow.
+     * If some source DN is corrupt, set the corresponding blockReader to 
+     * null and will never read from it again.
+     *  
+     * @param i the array index of sources
+     * @param offsetInBlock offset for the internal block
+     * @return StripedReader
+     */
+    private StripedReader addStripedReader(int i, long offsetInBlock) {
+      StripedReader reader = new StripedReader(liveIndices[i]);
+      stripedReaders.add(reader);
+
+      BlockReader blockReader = newBlockReader(
+          getBlock(blockGroup, liveIndices[i]), offsetInBlock, sources[i]);
+      if (blockReader != null) {
+        initChecksumAndBufferSizeIfNeeded(blockReader);
+        reader.blockReader = blockReader;
+      }
+      reader.buffer = allocateBuffer(bufferSize);
+      return reader;
+    }
+
+    @Override
+    public void run() {
+      datanode.incrementXmitsInProgress();
+      try {
+        // Store the array indices of source DNs we have read successfully.
+        // In each iteration of read, the success list may be updated if
+        // some source DN is corrupted or slow. And use the updated success
+        // list of DNs for next iteration read.
+        int[] success = new int[minRequiredSources];
+
+        int nsuccess = 0;
+        for (int i = 0; 
+            i < sources.length && nsuccess < minRequiredSources; i++) {
+          StripedReader reader = addStripedReader(i, 0);
+          if (reader.blockReader != null) {
+            success[nsuccess++] = i;
+          }
+        }
+
+        if (nsuccess < minRequiredSources) {
+          String error = "Can't find minimum sources required by "
+              + "recovery, block id: " + blockGroup.getBlockId();
+          throw new IOException(error);
+        }
+
+        if (zeroStripeBuffers != null) {
+          for (int i = 0; i < zeroStripeBuffers.length; i++) {
+            zeroStripeBuffers[i] = allocateBuffer(bufferSize);
+          }
+        }
+
+        for (int i = 0; i < targets.length; i++) {
+          targetBuffers[i] = allocateBuffer(bufferSize);
+        }
+
+        checksumSize = checksum.getChecksumSize();
+        int chunkSize = bytesPerChecksum + checksumSize;
+        maxChunksPerPacket = Math.max(
+            (WRITE_PACKET_SIZE - PacketHeader.PKT_MAX_HEADER_LEN)/chunkSize, 1);
+        int maxPacketSize = chunkSize * maxChunksPerPacket 
+            + PacketHeader.PKT_MAX_HEADER_LEN;
+
+        packetBuf = new byte[maxPacketSize];
+        checksumBuf = new byte[checksumSize * (bufferSize / bytesPerChecksum)];
+
+        // targetsStatus store whether some target is success, it will record
+        // any failed target once, if some target failed (invalid DN or transfer
+        // failed), will not transfer data to it any more.
+        boolean[] targetsStatus = new boolean[targets.length];
+        if (initTargetStreams(targetsStatus) == 0) {
+          String error = "All targets are failed.";
+          throw new IOException(error);
+        }
+
+        long firstStripedBlockLength = getBlockLen(blockGroup, 0);
+        while (positionInBlock < firstStripedBlockLength) {
+          int toRead = Math.min(
+              bufferSize, (int)(firstStripedBlockLength - positionInBlock));
+          // step1: read from minimum source DNs required for reconstruction.
+          //   The returned success list is the source DNs we do real read from
+          success = readMinimumStripedData4Recovery(success);
+
+          // step2: decode to reconstruct targets
+          long remaining = firstStripedBlockLength - positionInBlock;
+          int toRecoverLen = remaining < bufferSize ? 
+              (int)remaining : bufferSize;
+          recoverTargets(success, targetsStatus, toRecoverLen);
+
+          // step3: transfer data
+          if (transferData2Targets(targetsStatus) == 0) {
+            String error = "Transfer failed for all targets.";
+            throw new IOException(error);
+          }
+
+          clearBuffers();
+          positionInBlock += toRead;
+        }
+
+        endTargetBlocks(targetsStatus);
+
+        // Currently we don't check the acks for packets, this is similar as
+        // block replication.
+      } catch (Throwable e) {
+        LOG.warn("Failed to recover striped block: " + blockGroup, e);
+      } finally {
+        datanode.decrementXmitsInProgress();
+        // close block readers
+        for (StripedReader stripedReader : stripedReaders) {
+          closeBlockReader(stripedReader.blockReader);
+        }
+        for (int i = 0; i < targets.length; i++) {
+          IOUtils.closeStream(targetOutputStreams[i]);
+          IOUtils.closeStream(targetInputStreams[i]);
+          IOUtils.closeStream(targetSockets[i]);
+        }
+      }
+    }
+
+    // init checksum from block reader
+    private void initChecksumAndBufferSizeIfNeeded(BlockReader blockReader) {
+      if (checksum == null) {
+        checksum = blockReader.getDataChecksum();
+        bytesPerChecksum = checksum.getBytesPerChecksum();
+        // The bufferSize is flat to divide bytesPerChecksum
+        int readBufferSize = STRIPED_READ_BUFFER_SIZE;
+        bufferSize = readBufferSize < bytesPerChecksum ? bytesPerChecksum :
+          readBufferSize - readBufferSize % bytesPerChecksum;
+      } else {
+        assert blockReader.getDataChecksum().equals(checksum);
+      }
+    }
+
+    private void getTargetIndices() {
+      BitSet bitset = new BitSet(dataBlkNum + parityBlkNum);
+      for (int i = 0; i < sources.length; i++) {
+        bitset.set(liveIndices[i]);
+      }
+      int m = 0;
+      int k = 0;
+      for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {
+        if (!bitset.get(i)) {
+          if (getBlockLen(blockGroup, i) > 0) {
+            if (m < targets.length) {
+              targetIndices[m++] = (short)i;
+            }
+          } else {
+            zeroStripeIndices[k++] = (short)i;
+          }
+        }
+      }
+    }
+
+    private long getReadLength(int index) {
+      long blockLen = getBlockLen(blockGroup, index);
+      long remaining = blockLen - positionInBlock;
+      return remaining > bufferSize ? bufferSize : remaining;
+    }
+
+    /**
+     * Read from minimum source DNs required for reconstruction in the iteration.
+     * First try the success list which we think they are the best DNs
+     * If source DN is corrupt or slow, try to read some other source DN, 
+     * and will update the success list. 
+     * 
+     * Remember the updated success list and return it for following 
+     * operations and next iteration read.
+     * 
+     * @param success the initial success list of source DNs we think best
+     * @return updated success list of source DNs we do real read
+     * @throws IOException
+     */
+    private int[] readMinimumStripedData4Recovery(final int[] success)
+        throws IOException {
+      int nsuccess = 0;
+      int[] newSuccess = new int[minRequiredSources];
+      BitSet used = new BitSet(sources.length);
+      /*
+       * Read from minimum source DNs required, the success list contains
+       * source DNs which we think best.
+       */
+      for (int i = 0; i < minRequiredSources; i++) {
+        StripedReader reader = stripedReaders.get(success[i]);
+        if (getReadLength(liveIndices[success[i]]) > 0) {
+          Callable<Void> readCallable = readFromBlock(
+              reader.blockReader, reader.buffer);
+          Future<Void> f = readService.submit(readCallable);
+          futures.put(f, success[i]);
+        } else {
+          // If the read length is 0, we don't need to do real read
+          reader.buffer.position(0);
+          newSuccess[nsuccess++] = success[i];
+        }
+        used.set(success[i]);
+      }
+
+      while (!futures.isEmpty()) {
+        try {
+          StripingChunkReadResult result =
+              StripedBlockUtil.getNextCompletedStripedRead(
+                  readService, futures, STRIPED_READ_TIMEOUT_MILLIS);
+          int resultIndex = -1;
+          if (result.state == StripingChunkReadResult.SUCCESSFUL) {
+            resultIndex = result.index;
+          } else if (result.state == StripingChunkReadResult.FAILED) {
+            // If read failed for some source DN, we should not use it anymore 
+            // and schedule read from another source DN.
+            StripedReader failedReader = stripedReaders.get(result.index);
+            closeBlockReader(failedReader.blockReader);
+            failedReader.blockReader = null;
+            resultIndex = scheduleNewRead(used);
+          } else if (result.state == StripingChunkReadResult.TIMEOUT) {
+            // If timeout, we also schedule a new read.
+            resultIndex = scheduleNewRead(used);
+          }
+          if (resultIndex >= 0) {
+            newSuccess[nsuccess++] = resultIndex;
+            if (nsuccess >= minRequiredSources) {
+              // cancel remaining reads if we read successfully from minimum
+              // number of source DNs required by reconstruction.
+              cancelReads(futures.keySet());
+              futures.clear();
+              break;
+            }
+          }
+        } catch (InterruptedException e) {
+          LOG.info("Read data interrupted.", e);
+          break;
+        }
+      }
+
+      if (nsuccess < minRequiredSources) {
+        String error = "Can't read data from minimum number of sources "
+            + "required by reconstruction, block id: " + blockGroup.getBlockId();
+        throw new IOException(error);
+      }
+
+      return newSuccess;
+    }
+    
+    private void paddingBufferToLen(ByteBuffer buffer, int len) {
+      int toPadding = len - buffer.position();
+      for (int i = 0; i < toPadding; i++) {
+        buffer.put((byte) 0);
+      }
+    }
+    
+    // Initialize decoder
+    private void initDecoderIfNecessary() {
+      if (decoder == null) {
+        decoder = newDecoder(dataBlkNum, parityBlkNum);
+      }
+    }
+
+    private int[] getErasedIndices(boolean[] targetsStatus) {
+      int[] result = new int[targets.length];
+      int m = 0;
+      for (int i = 0; i < targets.length; i++) {
+        if (targetsStatus[i]) {
+          result[m++] = convertIndex4Decode(targetIndices[i], 
+              dataBlkNum, parityBlkNum);
+        }
+      }
+      return Arrays.copyOf(result, m);
+    }
+
+    private void recoverTargets(int[] success, boolean[] targetsStatus,
+        int toRecoverLen) {
+      initDecoderIfNecessary();
+      ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum];
+      for (int i = 0; i < success.length; i++) {
+        StripedReader reader = stripedReaders.get(success[i]);
+        ByteBuffer buffer = reader.buffer;
+        paddingBufferToLen(buffer, toRecoverLen);
+        inputs[convertIndex4Decode(reader.index, dataBlkNum, parityBlkNum)] = 
+            (ByteBuffer)buffer.flip();
+      }
+      if (success.length < dataBlkNum) {
+        for (int i = 0; i < zeroStripeBuffers.length; i++) {
+          ByteBuffer buffer = zeroStripeBuffers[i];
+          paddingBufferToLen(buffer, toRecoverLen);
+          int index = convertIndex4Decode(zeroStripeIndices[i], dataBlkNum,
+              parityBlkNum);
+          inputs[index] = (ByteBuffer)buffer.flip();
+        }
+      }
+      int[] erasedIndices = getErasedIndices(targetsStatus);
+      ByteBuffer[] outputs = new ByteBuffer[erasedIndices.length];
+      int m = 0;
+      for (int i = 0; i < targetBuffers.length; i++) {
+        if (targetsStatus[i]) {
+          outputs[m++] = targetBuffers[i];
+          outputs[i].limit(toRecoverLen);
+        }
+      }
+      decoder.decode(inputs, erasedIndices, outputs);
+
+      for (int i = 0; i < targets.length; i++) {
+        if (targetsStatus[i]) {
+          long blockLen = getBlockLen(blockGroup, targetIndices[i]);
+          long remaining = blockLen - positionInBlock;
+          if (remaining < 0) {
+            targetBuffers[i].limit(0);
+          } else if (remaining < toRecoverLen) {
+            targetBuffers[i].limit((int)remaining);
+          }
+        }
+      }
+    }
+
+    /**
+     * Schedule a read from some new source DN if some DN is corrupted
+     * or slow, this is called from the read iteration.
+     * Initially we may only have <code>minRequiredSources</code> number of 
+     * StripedReader.
+     * If the position is at the end of target block, don't need to do 
+     * real read, and return the array index of source DN, otherwise -1.
+     * 
+     * @param used the used source DNs in this iteration.
+     * @return the array index of source DN if don't need to do real read.
+     */
+    private int scheduleNewRead(BitSet used) {
+      StripedReader reader = null;
+      // step1: initially we may only have <code>minRequiredSources</code>
+      // number of StripedReader, and there may be some source DNs we never 
+      // read before, so will try to create StripedReader for one new source DN
+      // and try to read from it. If found, go to step 3.
+      int m = stripedReaders.size();
+      while (reader == null && m < sources.length) {
+        reader = addStripedReader(m, positionInBlock);
+        if (getReadLength(liveIndices[m]) > 0) {
+          if (reader.blockReader == null) {
+            reader = null;
+            m++;
+          }
+        } else {
+          used.set(m);
+          return m;
+        }
+      }
+
+      // step2: if there is no new source DN we can use, try to find a source 
+      // DN we ever read from but because some reason, e.g., slow, it
+      // is not in the success DN list at the begin of this iteration, so 
+      // we have not tried it in this iteration. Now we have a chance to 
+      // revisit it again.
+      for (int i = 0; reader == null && i < stripedReaders.size(); i++) {
+        if (!used.get(i)) {
+          StripedReader r = stripedReaders.get(i);
+          if (getReadLength(liveIndices[i]) > 0) {
+            closeBlockReader(r.blockReader);
+            r.blockReader = newBlockReader(
+                getBlock(blockGroup, liveIndices[i]), positionInBlock,
+                sources[i]);
+            if (r.blockReader != null) {
+              m = i;
+              reader = r;
+            }
+          } else {
+            used.set(i);
+            r.buffer.position(0);
+            return i;
+          }
+        }
+      }
+
+      // step3: schedule if find a correct source DN and need to do real read.
+      if (reader != null) {
+        Callable<Void> readCallable = readFromBlock(
+            reader.blockReader, reader.buffer);
+        Future<Void> f = readService.submit(readCallable);
+        futures.put(f, m);
+        used.set(m);
+      }
+
+      return -1;
+    }
+
+    // cancel all reads.
+    private void cancelReads(Collection<Future<Void>> futures) {
+      for (Future<Void> future : futures) {
+        future.cancel(true);
+      }
+    }
+
+    private Callable<Void> readFromBlock(final BlockReader reader,
+        final ByteBuffer buf) {
+      return new Callable<Void>() {
+
+        @Override
+        public Void call() throws Exception {
+          try {
+            actualReadFromBlock(reader, buf);
+            return null;
+          } catch (IOException e) {
+            LOG.info(e.getMessage());
+            throw e;
+          }
+        }
+
+      };
+    }
+
+    /**
+     * Read bytes from block
+     */
+    private void actualReadFromBlock(BlockReader reader, ByteBuffer buf)
+        throws IOException {
+      int len = buf.remaining();
+      int n = 0;
+      while (n < len) {
+        int nread = reader.read(buf);
+        if (nread <= 0) {
+          break;
+        }
+        n += nread;
+      }
+    }
+
+    // close block reader
+    private void closeBlockReader(BlockReader blockReader) {
+      try {
+        if (blockReader != null) {
+          blockReader.close();
+        }
+      } catch (IOException e) {
+        // ignore
+      }
+    }
+
+    private InetSocketAddress getSocketAddress4Transfer(DatanodeInfo dnInfo) {
+      return NetUtils.createSocketAddr(dnInfo.getXferAddr(
+          datanode.getDnConf().getConnectToDnViaHostname()));
+    }
+
+    private BlockReader newBlockReader(final ExtendedBlock block, 
+        long offsetInBlock, DatanodeInfo dnInfo) {
+      if (offsetInBlock >= block.getNumBytes()) {
+        return null;
+      }
+      try {
+        InetSocketAddress dnAddr = getSocketAddress4Transfer(dnInfo);
+        Token<BlockTokenIdentifier> blockToken = datanode.getBlockAccessToken(
+            block, EnumSet.of(BlockTokenIdentifier.AccessMode.READ));
+        /*
+         * This can be further improved if the replica is local, then we can
+         * read directly from DN and need to check the replica is FINALIZED
+         * state, notice we should not use short-circuit local read which
+         * requires config for domain-socket in UNIX or legacy config in Windows.
+         *
+         * TODO: add proper tracer
+         */
+        return RemoteBlockReader2.newBlockReader(
+            "dummy", block, blockToken, offsetInBlock, 
+            block.getNumBytes() - offsetInBlock, true,
+            "", newConnectedPeer(block, dnAddr, blockToken, dnInfo), dnInfo,
+            null, cachingStrategy, null);
+      } catch (IOException e) {
+        return null;
+      }
+    }
+
+    private Peer newConnectedPeer(ExtendedBlock b, InetSocketAddress addr,
+        Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
+        throws IOException {
+      Peer peer = null;
+      boolean success = false;
+      Socket sock = null;
+      final int socketTimeout = datanode.getDnConf().getSocketTimeout(); 
+      try {
+        sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
+        NetUtils.connect(sock, addr, socketTimeout);
+        peer = DFSUtilClient.peerFromSocketAndKey(datanode.getSaslClient(),
+            sock, datanode.getDataEncryptionKeyFactoryForBlock(b),
+            blockToken, datanodeId);
+        peer.setReadTimeout(socketTimeout);
+        success = true;
+        return peer;
+      } finally {
+        if (!success) {
+          IOUtils.cleanup(LOG, peer);
+          IOUtils.closeSocket(sock);
+        }
+      }
+    }
+
+    /**
+     * Send data to targets
+     */
+    private int transferData2Targets(boolean[] targetsStatus) {
+      int nsuccess = 0;
+      for (int i = 0; i < targets.length; i++) {
+        if (targetsStatus[i]) {
+          boolean success = false;
+          try {
+            ByteBuffer buffer = targetBuffers[i];
+            
+            if (buffer.remaining() == 0) {
+              continue;
+            }
+
+            checksum.calculateChunkedSums(
+                buffer.array(), 0, buffer.remaining(), checksumBuf, 0);
+
+            int ckOff = 0;
+            while (buffer.remaining() > 0) {
+              DFSPacket packet = new DFSPacket(packetBuf, maxChunksPerPacket,
+                  blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, false);
+              int maxBytesToPacket = maxChunksPerPacket * bytesPerChecksum;
+              int toWrite = buffer.remaining() > maxBytesToPacket ?
+                  maxBytesToPacket : buffer.remaining();
+              int ckLen = ((toWrite - 1) / bytesPerChecksum + 1) * checksumSize;
+              packet.writeChecksum(checksumBuf, ckOff, ckLen);
+              ckOff += ckLen;
+              packet.writeData(buffer, toWrite);
+
+              // Send packet
+              packet.writeTo(targetOutputStreams[i]);
+
+              blockOffset4Targets[i] += toWrite;
+              nsuccess++;
+              success = true;
+            }
+          } catch (IOException e) {
+            LOG.warn(e.getMessage());
+          }
+          targetsStatus[i] = success;
+        }
+      }
+      return nsuccess;
+    }
+
+    /**
+     * clear all buffers
+     */
+    private void clearBuffers() {
+      for (StripedReader stripedReader : stripedReaders) {
+        if (stripedReader.buffer != null) {
+          stripedReader.buffer.clear();
+        }
+      }
+
+      if (zeroStripeBuffers != null) {
+        for (int i = 0; i < zeroStripeBuffers.length; i++) {
+          zeroStripeBuffers[i].clear();
+        }
+      }
+
+      for (int i = 0; i < targetBuffers.length; i++) {
+        if (targetBuffers[i] != null) {
+          cleanBuffer(targetBuffers[i]);
+        }
+      }
+    }
+    
+    private ByteBuffer cleanBuffer(ByteBuffer buffer) {
+      Arrays.fill(buffer.array(), (byte) 0);
+      return (ByteBuffer)buffer.clear();
+    }
+
+    // send an empty packet to mark the end of the block
+    private void endTargetBlocks(boolean[] targetsStatus) {
+      for (int i = 0; i < targets.length; i++) {
+        if (targetsStatus[i]) {
+          try {
+            DFSPacket packet = new DFSPacket(packetBuf, 0, 
+                blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, true);
+            packet.writeTo(targetOutputStreams[i]);
+            targetOutputStreams[i].flush();
+          } catch (IOException e) {
+            LOG.warn(e.getMessage());
+          }
+        }
+      }
+    }
+
+    /**
+     * Initialize  output/input streams for transferring data to target
+     * and send create block request. 
+     */
+    private int initTargetStreams(boolean[] targetsStatus) {
+      int nsuccess = 0;
+      for (int i = 0; i < targets.length; i++) {
+        Socket socket = null;
+        DataOutputStream out = null;
+        DataInputStream in = null;
+        boolean success = false;
+        try {
+          InetSocketAddress targetAddr = 
+              getSocketAddress4Transfer(targets[i]);
+          socket = datanode.newSocket();
+          NetUtils.connect(socket, targetAddr, 
+              datanode.getDnConf().getSocketTimeout());
+          socket.setSoTimeout(datanode.getDnConf().getSocketTimeout());
+
+          ExtendedBlock block = getBlock(blockGroup, targetIndices[i]);
+          Token<BlockTokenIdentifier> blockToken = 
+              datanode.getBlockAccessToken(block,
+                  EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE));
+
+          long writeTimeout = datanode.getDnConf().getSocketWriteTimeout();
+          OutputStream unbufOut = NetUtils.getOutputStream(socket, writeTimeout);
+          InputStream unbufIn = NetUtils.getInputStream(socket);
+          DataEncryptionKeyFactory keyFactory =
+            datanode.getDataEncryptionKeyFactoryForBlock(block);
+          IOStreamPair saslStreams = datanode.getSaslClient().socketSend(
+              socket, unbufOut, unbufIn, keyFactory, blockToken, targets[i]);
+
+          unbufOut = saslStreams.out;
+          unbufIn = saslStreams.in;
+
+          out = new DataOutputStream(new BufferedOutputStream(unbufOut,
+              DFSUtilClient.getSmallBufferSize(conf)));
+          in = new DataInputStream(unbufIn);
+
+          DatanodeInfo source = new DatanodeInfo(datanode.getDatanodeId());
+          new Sender(out).writeBlock(block, targetStorageTypes[i], 
+              blockToken, "", new DatanodeInfo[]{targets[i]}, 
+              new StorageType[]{targetStorageTypes[i]}, source, 
+              BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, 0, 
+              checksum, cachingStrategy, false, false, null);
+
+          targetSockets[i] = socket;
+          targetOutputStreams[i] = out;
+          targetInputStreams[i] = in;
+          nsuccess++;
+          success = true;
+        } catch (Throwable e) {
+          LOG.warn(e.getMessage());
+        } finally {
+          if (!success) {
+            IOUtils.closeStream(out);
+            IOUtils.closeStream(in);
+            IOUtils.closeStream(socket);
+          }
+        }
+        targetsStatus[i] = success;
+      }
+      return nsuccess;
+    }
+  }
+
+  private static class StripedReader {
+    private final short index; // internal block index
+    private BlockReader blockReader;
+    private ByteBuffer buffer;
+
+    private StripedReader(short index) {
+      this.index = index;
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index afacebb..1e07cad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -45,6 +45,7 @@
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
@@ -174,8 +175,21 @@
     }
   }
 
-  DBlock newDBlock(Block block, List<MLocation> locations) {
-    final DBlock db = new DBlock(block);
+  DBlock newDBlock(LocatedBlock lb, List<MLocation> locations,
+                   ErasureCodingPolicy ecPolicy) {
+    Block blk = lb.getBlock().getLocalBlock();
+    DBlock db;
+    if (lb.isStriped()) {
+      LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
+      byte[] indices = new byte[lsb.getBlockIndices().length];
+      for (int i = 0; i < indices.length; i++) {
+        indices[i] = (byte) lsb.getBlockIndices()[i];
+      }
+      db = new DBlockStriped(blk, indices, (short) ecPolicy.getNumDataUnits(),
+          ecPolicy.getCellSize());
+    } else {
+      db = new DBlock(blk);
+    }
     for(MLocation ml : locations) {
       StorageGroup source = storages.getSource(ml);
       if (source != null) {
@@ -358,9 +372,10 @@
         LOG.warn("Failed to get the storage policy of file " + fullPath);
         return;
       }
-      final List<StorageType> types = policy.chooseStorageTypes(
+      List<StorageType> types = policy.chooseStorageTypes(
           status.getReplication());
 
+      final ErasureCodingPolicy ecPolicy = status.getErasureCodingPolicy();
       final LocatedBlocks locatedBlocks = status.getBlockLocations();
       final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
       List<LocatedBlock> lbs = locatedBlocks.getLocatedBlocks();
@@ -370,10 +385,13 @@
           continue;
         }
         LocatedBlock lb = lbs.get(i);
+        if (lb.isStriped()) {
+          types = policy.chooseStorageTypes((short) lb.getLocations().length);
+        }
         final StorageTypeDiff diff = new StorageTypeDiff(types,
             lb.getStorageTypes());
         if (!diff.removeOverlap(true)) {
-          if (scheduleMoves4Block(diff, lb)) {
+          if (scheduleMoves4Block(diff, lb, ecPolicy)) {
             result.updateHasRemaining(diff.existing.size() > 1
                 && diff.expected.size() > 1);
             // One block scheduled successfully, set noBlockMoved to false
@@ -385,10 +403,13 @@
       }
     }
 
-    boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb) {
+    boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb,
+                                ErasureCodingPolicy ecPolicy) {
       final List<MLocation> locations = MLocation.toLocations(lb);
-      Collections.shuffle(locations);
-      final DBlock db = newDBlock(lb.getBlock().getLocalBlock(), locations);
+      if (!(lb instanceof LocatedStripedBlock)) {
+        Collections.shuffle(locations);
+      }
+      final DBlock db = newDBlock(lb, locations, ecPolicy);
 
       for (final StorageType t : diff.existing) {
         for (final MLocation ml : locations) {
@@ -781,4 +802,4 @@
       System.exit(-1);
     }
   }
-}
\ No newline at end of file
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
new file mode 100644
index 0000000..b77279b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * This manages erasure coding policies predefined and activated in the system.
+ * It loads customized policies and syncs with persisted ones in
+ * NameNode image.
+ *
+ * This class is instantiated by the FSNamesystem.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+public final class ErasureCodingPolicyManager {
+
+  /**
+   * TODO: HDFS-8095
+   */
+  private static final int DEFAULT_DATA_BLOCKS = 6;
+  private static final int DEFAULT_PARITY_BLOCKS = 3;
+  private static final int DEFAULT_CELLSIZE = 64 * 1024;
+  private static final String DEFAULT_CODEC_NAME = "rs";
+  private static final String DEFAULT_POLICY_NAME = "RS-6-3-64k";
+  private static final ECSchema SYS_DEFAULT_SCHEMA = new ECSchema(
+      DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS);
+  private static final ErasureCodingPolicy SYS_DEFAULT_POLICY =
+      new ErasureCodingPolicy(DEFAULT_POLICY_NAME, SYS_DEFAULT_SCHEMA,
+      DEFAULT_CELLSIZE);
+
+  //We may add more later.
+  private static ErasureCodingPolicy[] SYS_POLICY = new ErasureCodingPolicy[] {
+      SYS_DEFAULT_POLICY
+  };
+
+  /**
+   * All active policies maintained in NN memory for fast querying,
+   * identified and sorted by its name.
+   */
+  private final Map<String, ErasureCodingPolicy> activePolicies;
+
+  ErasureCodingPolicyManager() {
+
+    this.activePolicies = new TreeMap<>();
+    for (ErasureCodingPolicy policy : SYS_POLICY) {
+      activePolicies.put(policy.getName(), policy);
+    }
+
+    /**
+     * TODO: HDFS-7859 persist into NameNode
+     * load persistent policies from image and editlog, which is done only once
+     * during NameNode startup. This can be done here or in a separate method.
+     */
+  }
+
+  /**
+   * Get system defined policies.
+   * @return system policies
+   */
+  public static ErasureCodingPolicy[] getSystemPolices() {
+    return SYS_POLICY;
+  }
+
+  /**
+   * Get system-wide default policy, which can be used by default
+   * when no policy is specified for a path.
+   * @return ecPolicy
+   */
+  public static ErasureCodingPolicy getSystemDefaultPolicy() {
+    return SYS_DEFAULT_POLICY;
+  }
+
+  /**
+   * Get all policies that's available to use.
+   * @return all policies
+   */
+  public ErasureCodingPolicy[] getPolicies() {
+    ErasureCodingPolicy[] results = new ErasureCodingPolicy[activePolicies.size()];
+    return activePolicies.values().toArray(results);
+  }
+
+  /**
+   * Get the policy specified by the policy name.
+   */
+  public ErasureCodingPolicy getPolicy(String name) {
+    return activePolicies.get(name);
+  }
+
+  /**
+   * Clear and clean up
+   */
+  public void clear() {
+    activePolicies.clear();
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
index 53255e6..ae84f39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
@@ -106,6 +106,13 @@
                 + clientMachine);
       }
       final INodeFile file = INodeFile.valueOf(inode, path, true);
+
+      // not support appending file with striped blocks
+      if (file.isStriped()) {
+        throw new UnsupportedOperationException(
+            "Cannot append to files with striped block " + src);
+      }
+
       BlockManager blockManager = fsd.getBlockManager();
       final BlockStoragePolicy lpPolicy = blockManager
           .getStoragePolicy("LAZY_PERSIST");
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 df0bc20..46e172d 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
@@ -122,7 +122,7 @@
                                             " does not exist.");
       }
       boolean changed = unprotectedSetTimes(fsd, inode, mtime, atime, true,
-                                            iip.getLatestSnapshotId());
+          iip.getLatestSnapshotId());
       if (changed) {
         fsd.getEditLog().logTimes(src, mtime, atime);
       }
@@ -399,17 +399,18 @@
   static BlockInfo[] unprotectedSetReplication(
       FSDirectory fsd, String src, short replication)
       throws QuotaExceededException, UnresolvedLinkException,
-             SnapshotAccessControlException {
+      SnapshotAccessControlException, UnsupportedActionException {
     assert fsd.hasWriteLock();
 
     final BlockManager bm = fsd.getBlockManager();
     final INodesInPath iip = fsd.getINodesInPath4Write(src, true);
     final INode inode = iip.getLastINode();
-    if (inode == null || !inode.isFile()) {
+    if (inode == null || !inode.isFile() || inode.asFile().isStriped()) {
+      // TODO we do not support replication on stripe layout files yet
       return null;
     }
-    INodeFile file = inode.asFile();
 
+    INodeFile file = inode.asFile();
     // Make sure the directory has sufficient quotas
     short oldBR = file.getPreferredBlockReplication();
 
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 492994e..e8b2c54 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
@@ -144,6 +144,7 @@
         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() >
@@ -153,6 +154,11 @@
             + " 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);
     }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
new file mode 100644
index 0000000..d39da28
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -0,0 +1,277 @@
+/**
+ * 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 java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.XAttrHelper;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.WritableUtils;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_POLICY;
+
+/**
+ * Helper class to perform erasure coding related operations.
+ */
+final class FSDirErasureCodingOp {
+
+  /**
+   * Private constructor for preventing FSDirErasureCodingOp object
+   * creation. Static-only class.
+   */
+  private FSDirErasureCodingOp() {}
+
+  /**
+   * Set an erasure coding policy on the given path.
+   *
+   * @param fsn The namespace
+   * @param srcArg The path of the target directory.
+   * @param ecPolicy The erasure coding policy to set on the target directory.
+   * @param logRetryCache whether to record RPC ids in editlog for retry
+   *          cache rebuilding
+   * @return {@link HdfsFileStatus}
+   * @throws IOException
+   */
+  static HdfsFileStatus setErasureCodingPolicy(final FSNamesystem fsn,
+      final String srcArg, final ErasureCodingPolicy ecPolicy,
+      final boolean logRetryCache) throws IOException {
+    assert fsn.hasWriteLock();
+
+    String src = srcArg;
+    FSPermissionChecker pc = null;
+    byte[][] pathComponents = null;
+    pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    pc = fsn.getPermissionChecker();
+    FSDirectory fsd = fsn.getFSDirectory();
+    src = fsd.resolvePath(pc, src, pathComponents);
+    final INodesInPath iip;
+    List<XAttr> xAttrs;
+    fsd.writeLock();
+    try {
+      iip = fsd.getINodesInPath4Write(src, false);
+      xAttrs = createErasureCodingPolicyXAttr(fsn, iip, ecPolicy);
+    } finally {
+      fsd.writeUnlock();
+    }
+    fsn.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
+    return fsd.getAuditFileInfo(iip);
+  }
+
+  static List<XAttr> createErasureCodingPolicyXAttr(final FSNamesystem fsn,
+      final INodesInPath srcIIP, ErasureCodingPolicy ecPolicy) throws IOException {
+    FSDirectory fsd = fsn.getFSDirectory();
+    assert fsd.hasWriteLock();
+    Preconditions.checkNotNull(srcIIP, "INodes cannot be null");
+    String src = srcIIP.getPath();
+    if (srcIIP.getLastINode() != null &&
+        !srcIIP.getLastINode().isDirectory()) {
+      throw new IOException("Attempt to set an erasure coding policy " +
+          "for a file " + src);
+    }
+    if (getErasureCodingPolicyForPath(fsn, srcIIP) != null) {
+      throw new IOException("Directory " + src + " already has an " +
+          "erasure coding policy.");
+    }
+
+    // System default erasure coding policy will be used since no specified.
+    if (ecPolicy == null) {
+      ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    } else {
+      // If ecPolicy is specified check if it is one among active policies.
+      boolean validPolicy = false;
+      ErasureCodingPolicy[] activePolicies =
+          FSDirErasureCodingOp.getErasureCodingPolicies(fsd.getFSNamesystem());
+      for (ErasureCodingPolicy activePolicy : activePolicies) {
+        if (activePolicy.equals(ecPolicy)) {
+          validPolicy = true;
+          break;
+        }
+      }
+      if (!validPolicy) {
+        List<String> ecPolicyNames = new ArrayList<String>();
+        for (ErasureCodingPolicy activePolicy : activePolicies) {
+          ecPolicyNames.add(activePolicy.getName());
+        }
+        throw new HadoopIllegalArgumentException("Policy [ " +
+            ecPolicy.getName()+ " ] does not match any of the " +
+            "supported policies. Please select any one of " + ecPolicyNames);
+      }
+    }
+
+    final XAttr ecXAttr;
+    DataOutputStream dOut = null;
+    try {
+      ByteArrayOutputStream bOut = new ByteArrayOutputStream();
+      dOut = new DataOutputStream(bOut);
+      WritableUtils.writeString(dOut, ecPolicy.getName());
+      ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_POLICY,
+          bOut.toByteArray());
+    } finally {
+      IOUtils.closeStream(dOut);
+    }
+    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
+    xattrs.add(ecXAttr);
+    FSDirXAttrOp.unprotectedSetXAttrs(fsd, src, xattrs,
+        EnumSet.of(XAttrSetFlag.CREATE));
+    return xattrs;
+  }
+
+  /**
+   * Get the erasure coding policy information for specified path.
+   *
+   * @param fsn namespace
+   * @param src path
+   * @return {@link ErasureCodingPolicy}
+   * @throws IOException
+   */
+  static ErasureCodingPolicy getErasureCodingPolicy(final FSNamesystem fsn,
+      final String src) throws IOException {
+    assert fsn.hasReadLock();
+
+    final INodesInPath iip = getINodesInPath(fsn, src);
+    return getErasureCodingPolicyForPath(fsn, iip);
+  }
+
+  /**
+   * Check if the file or directory has an erasure coding policy.
+   *
+   * @param fsn namespace
+   * @param srcArg path
+   * @return Whether the file or directory has an erasure coding policy.
+   * @throws IOException
+   */
+  static boolean hasErasureCodingPolicy(final FSNamesystem fsn,
+      final String srcArg) throws IOException {
+    return hasErasureCodingPolicy(fsn, getINodesInPath(fsn, srcArg));
+  }
+
+  /**
+   * Check if the file or directory has an erasure coding policy.
+   *
+   * @param fsn namespace
+   * @param iip inodes in the path containing the file
+   * @return Whether the file or directory has an erasure coding policy.
+   * @throws IOException
+   */
+  static boolean hasErasureCodingPolicy(final FSNamesystem fsn,
+      final INodesInPath iip) throws IOException {
+    return getErasureCodingPolicy(fsn, iip) != null;
+  }
+
+  /**
+   * Get the erasure coding policy.
+   *
+   * @param fsn namespace
+   * @param iip inodes in the path containing the file
+   * @return {@link ErasureCodingPolicy}
+   * @throws IOException
+   */
+  static ErasureCodingPolicy getErasureCodingPolicy(final FSNamesystem fsn,
+      final INodesInPath iip) throws IOException {
+    assert fsn.hasReadLock();
+
+    return getErasureCodingPolicyForPath(fsn, iip);
+  }
+
+  /**
+   * Get available erasure coding polices.
+   *
+   * @param fsn namespace
+   * @return {@link ErasureCodingPolicy} array
+   */
+  static ErasureCodingPolicy[] getErasureCodingPolicies(final FSNamesystem fsn)
+      throws IOException {
+    assert fsn.hasReadLock();
+
+    return fsn.getErasureCodingPolicyManager().getPolicies();
+  }
+
+  private static INodesInPath getINodesInPath(final FSNamesystem fsn,
+      final String srcArg) throws IOException {
+    String src = srcArg;
+    final byte[][] pathComponents = FSDirectory
+        .getPathComponentsForReservedPath(src);
+    final FSDirectory fsd = fsn.getFSDirectory();
+    final FSPermissionChecker pc = fsn.getPermissionChecker();
+    src = fsd.resolvePath(pc, src, pathComponents);
+    INodesInPath iip = fsd.getINodesInPath(src, true);
+    if (fsn.isPermissionEnabled()) {
+      fsn.getFSDirectory().checkPathAccess(pc, iip, FsAction.READ);
+    }
+    return iip;
+  }
+
+  private static ErasureCodingPolicy getErasureCodingPolicyForPath(FSNamesystem fsn,
+      INodesInPath iip) throws IOException {
+    Preconditions.checkNotNull(iip, "INodes cannot be null");
+    FSDirectory fsd = fsn.getFSDirectory();
+    fsd.readLock();
+    try {
+      List<INode> inodes = iip.getReadOnlyINodes();
+      for (int i = inodes.size() - 1; i >= 0; i--) {
+        final INode inode = inodes.get(i);
+        if (inode == null) {
+          continue;
+        }
+        /**
+         * TODO: lookup {@link ErasureCodingPolicyManager#getSystemPolices()}
+         */
+        if (inode.isFile()) {
+          return inode.asFile().getErasureCodingPolicyID() == 0 ?
+              null : ErasureCodingPolicyManager.getSystemDefaultPolicy();
+        }
+        // We don't allow setting EC policies on paths with a symlink. Thus
+        // if a symlink is encountered, the dir shouldn't have EC policy.
+        // TODO: properly support symlinks
+        if (inode.isSymlink()) {
+          return null;
+        }
+        final XAttrFeature xaf = inode.getXAttrFeature();
+        if (xaf != null) {
+          XAttr xattr = xaf.getXAttr(XATTR_ERASURECODING_POLICY);
+          if (xattr != null) {
+            ByteArrayInputStream bIn = new ByteArrayInputStream(xattr.getValue());
+            DataInputStream dIn = new DataInputStream(bIn);
+            String ecPolicyName = WritableUtils.readString(dIn);
+            return fsd.getFSNamesystem().getErasureCodingPolicyManager().
+                getPolicy(ecPolicyName);
+          }
+        }
+      }
+    } finally {
+      fsd.readUnlock();
+    }
+    return null;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 98af592..67c6fc1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.commons.io.Charsets;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
@@ -29,6 +30,7 @@
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -179,11 +181,13 @@
 
       final FileEncryptionInfo feInfo = isReservedName ? null
           : FSDirEncryptionZoneOp.getFileEncryptionInfo(fsd, inode,
-              iip.getPathSnapshotId(), iip);
+          iip.getPathSnapshotId(), iip);
+      final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.
+          getErasureCodingPolicy(fsd.getFSNamesystem(), iip);
 
       final LocatedBlocks blocks = bm.createLocatedBlocks(
           inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
-          length, needBlockToken, iip.isSnapshot(), feInfo);
+          length, needBlockToken, iip.isSnapshot(), feInfo, ecPolicy);
 
       // Set caching information for the located blocks.
       for (LocatedBlock lb : blocks.getLocatedBlocks()) {
@@ -375,7 +379,7 @@
       if (fsd.getINode4DotSnapshot(srcs) != null) {
         return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
             HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
-            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
       }
       return null;
     }
@@ -443,6 +447,9 @@
     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
         .getFileEncryptionInfo(fsd, node, snapshot, iip);
 
+    final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
+        fsd.getFSNamesystem(), iip);
+
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
       size = fileNode.computeFileSize(snapshot);
@@ -474,7 +481,8 @@
         node.getId(),
         childrenNum,
         feInfo,
-        storagePolicy);
+        storagePolicy,
+        ecPolicy);
   }
 
   private static INodeAttributes getINodeAttributes(
@@ -497,6 +505,8 @@
     final boolean isEncrypted;
     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
         .getFileEncryptionInfo(fsd, node, snapshot, iip);
+    final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
+        fsd.getFSNamesystem(), iip);
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
       size = fileNode.computeFileSize(snapshot);
@@ -510,7 +520,7 @@
 
       loc = fsd.getBlockManager().createLocatedBlocks(
           fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
-          inSnapshot, feInfo);
+          inSnapshot, feInfo, ecPolicy);
       if (loc == null) {
         loc = new LocatedBlocks();
       }
@@ -531,7 +541,7 @@
           getPermissionForFileStatus(nodeAttrs, isEncrypted),
           nodeAttrs.getUserName(), nodeAttrs.getGroupName(),
           node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
-          node.getId(), loc, childrenNum, feInfo, storagePolicy);
+          node.getId(), loc, childrenNum, feInfo, storagePolicy, ecPolicy);
     // Set caching information for the located blocks.
     if (loc != null) {
       CacheManager cacheManager = fsd.getFSNamesystem().getCacheManager();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
index 6d37530..b46a195 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
@@ -84,6 +84,13 @@
         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
       }
       INodeFile file = INodeFile.valueOf(iip.getLastINode(), src);
+
+      // not support truncating file with striped blocks
+      if (file.isStriped()) {
+        throw new UnsupportedOperationException(
+            "Cannot truncate file with striped block " + src);
+      }
+
       final BlockStoragePolicy lpPolicy = fsd.getBlockManager()
           .getStoragePolicy("LAZY_PERSIST");
 
@@ -208,6 +215,7 @@
     assert fsn.hasWriteLock();
 
     INodeFile file = iip.getLastINode().asFile();
+    assert !file.isStriped();
     file.recordModification(iip.getLatestSnapshotId());
     file.toUnderConstruction(leaseHolder, clientMachine);
     assert file.isUnderConstruction() : "inode should be under construction.";
@@ -215,10 +223,11 @@
         file.getFileUnderConstructionFeature().getClientName(), file.getId());
     boolean shouldRecoverNow = (newBlock == null);
     BlockInfo oldBlock = file.getLastBlock();
+
     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(fsn, file, oldBlock);
     if (newBlock == null) {
-      newBlock = (shouldCopyOnTruncate) ? fsn.createNewBlock() : new Block(
-          oldBlock.getBlockId(), oldBlock.getNumBytes(),
+      newBlock = (shouldCopyOnTruncate) ? fsn.createNewBlock(false)
+          : new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
           fsn.nextGenerationStamp(fsn.getBlockIdManager().isLegacyBlock(
               oldBlock)));
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 575b1fd..4623264 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -35,6 +35,7 @@
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -44,6 +45,8 @@
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -88,7 +91,7 @@
 
     // update space consumed
     fsd.updateCount(iip, 0, -fileNode.getPreferredBlockSize(),
-                    fileNode.getPreferredBlockReplication(), true);
+        fileNode.getPreferredBlockReplication(), true);
     return true;
   }
 
@@ -131,6 +134,9 @@
     FSNamesystem fsn = fsd.getFSNamesystem();
     final INodeFile file = fsn.checkLease(src, holder, inode, fileId);
     Preconditions.checkState(file.isUnderConstruction());
+    if (file.isStriped()) {
+      return; // do not abandon block for striped file
+    }
 
     Block localBlock = ExtendedBlock.getLocalBlock(b);
     fsd.writeLock();
@@ -168,9 +174,10 @@
       String src, long fileId, String clientName,
       ExtendedBlock previous, LocatedBlock[] onRetryBlock) throws IOException {
     final long blockSize;
-    final int replication;
+    final short numTargets;
     final byte storagePolicyID;
     String clientMachine;
+    final boolean isStriped;
 
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsn.dir.resolvePath(pc, src, pathComponents);
@@ -196,18 +203,26 @@
     blockSize = pendingFile.getPreferredBlockSize();
     clientMachine = pendingFile.getFileUnderConstructionFeature()
         .getClientMachine();
-    replication = pendingFile.getFileReplication();
+    isStriped = pendingFile.isStriped();
+    ErasureCodingPolicy ecPolicy = null;
+    if (isStriped) {
+      ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(fsn, src);
+      numTargets = (short) (ecPolicy.getSchema().getNumDataUnits()
+          + ecPolicy.getSchema().getNumParityUnits());
+    } else {
+      numTargets = pendingFile.getFileReplication();
+    }
     storagePolicyID = pendingFile.getStoragePolicyID();
-    return new ValidateAddBlockResult(blockSize, replication, storagePolicyID,
-                                    clientMachine);
+    return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
+                                      clientMachine, isStriped);
   }
 
-  static LocatedBlock makeLocatedBlock(FSNamesystem fsn, Block blk,
+  static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
       DatanodeStorageInfo[] locs, long offset) throws IOException {
-    LocatedBlock lBlk = BlockManager.newLocatedBlock(fsn.getExtendedBlock(blk),
-                                                     locs, offset, false);
-    fsn.getFSDirectory().getBlockManager()
-        .setBlockToken(lBlk, BlockTokenIdentifier.AccessMode.WRITE);
+    LocatedBlock lBlk = BlockManager.newLocatedBlock(
+        fsn.getExtendedBlock(new Block(blk)), blk, locs, offset);
+    fsn.getBlockManager().setBlockToken(lBlk,
+        BlockTokenIdentifier.AccessMode.WRITE);
     return lBlk;
   }
 
@@ -237,8 +252,8 @@
       } else {
         // add new chosen targets to already allocated block and return
         BlockInfo lastBlockInFile = pendingFile.getLastBlock();
-        lastBlockInFile.getUnderConstructionFeature().
-            setExpectedLocations(lastBlockInFile, targets);
+        lastBlockInFile.getUnderConstructionFeature().setExpectedLocations(
+            lastBlockInFile, targets, pendingFile.isStriped());
         offset = pendingFile.computeFileSize();
         return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
       }
@@ -249,15 +264,17 @@
                                   ExtendedBlock.getLocalBlock(previous));
 
     // allocate new block, record block locations in INode.
-    Block newBlock = fsn.createNewBlock();
+    final boolean isStriped = pendingFile.isStriped();
+    // allocate new block, record block locations in INode.
+    Block newBlock = fsn.createNewBlock(isStriped);
     INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
-    saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets);
+    saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets, isStriped);
 
     persistNewBlock(fsn, src, pendingFile);
     offset = pendingFile.computeFileSize();
 
     // Return located block
-    return makeLocatedBlock(fsn, newBlock, targets, offset);
+    return makeLocatedBlock(fsn, fsn.getStoredBlock(newBlock), targets, offset);
   }
 
   static DatanodeStorageInfo[] chooseTargetForNewBlock(
@@ -278,9 +295,10 @@
         : Arrays.asList(favoredNodes);
 
     // choose targets for the new block to be allocated.
-    return bm.chooseTarget4NewBlock(src, r.replication, clientNode,
+    return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
                                     excludedNodesSet, r.blockSize,
-                                    favoredNodesList, r.storagePolicyID);
+                                    favoredNodesList, r.storagePolicyID,
+                                    r.isStriped);
   }
 
   /**
@@ -469,22 +487,22 @@
       long preferredBlockSize, boolean underConstruction, String clientName,
       String clientMachine, byte storagePolicyId) {
     final INodeFile newNode;
+    Preconditions.checkNotNull(existing);
     assert fsd.hasWriteLock();
-    if (underConstruction) {
-      newNode = newINodeFile(id, permissions, modificationTime,
-                                              modificationTime, replication,
-                                              preferredBlockSize,
-                                              storagePolicyId);
-      newNode.toUnderConstruction(clientName, clientMachine);
-    } else {
-      newNode = newINodeFile(id, permissions, modificationTime,
-                                              atime, replication,
-                                              preferredBlockSize,
-                                              storagePolicyId);
-    }
-
-    newNode.setLocalName(localName);
     try {
+      // check if the file has an EC policy
+      final boolean isStriped = FSDirErasureCodingOp.hasErasureCodingPolicy(
+          fsd.getFSNamesystem(), existing);
+      if (underConstruction) {
+        newNode = newINodeFile(id, permissions, modificationTime,
+            modificationTime, replication, preferredBlockSize, storagePolicyId,
+            isStriped);
+        newNode.toUnderConstruction(clientName, clientMachine);
+      } else {
+        newNode = newINodeFile(id, permissions, modificationTime, atime,
+            replication, preferredBlockSize, storagePolicyId, isStriped);
+      }
+      newNode.setLocalName(localName);
       INodesInPath iip = fsd.addINode(existing, newNode);
       if (iip != null) {
         if (aclEntries != null) {
@@ -508,23 +526,39 @@
   /**
    * Add a block to the file. Returns a reference to the added block.
    */
-  private static BlockInfo addBlock(
-      FSDirectory fsd, String path, INodesInPath inodesInPath, Block block,
-      DatanodeStorageInfo[] targets) throws IOException {
+  private static BlockInfo addBlock(FSDirectory fsd, String path,
+      INodesInPath inodesInPath, Block block, DatanodeStorageInfo[] targets,
+      boolean isStriped) throws IOException {
     fsd.writeLock();
     try {
       final INodeFile fileINode = inodesInPath.getLastINode().asFile();
       Preconditions.checkState(fileINode.isUnderConstruction());
 
-      // check quota limits and updated space consumed
-      fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
-          fileINode.getFileReplication(), true);
-
       // associate new last block for the file
-      BlockInfo blockInfo = new BlockInfoContiguous(block,
-          fileINode.getFileReplication());
-      blockInfo.convertToBlockUnderConstruction(
-          HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
+      final BlockInfo blockInfo;
+      if (isStriped) {
+        ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
+            fsd.getFSNamesystem(), inodesInPath);
+        short numDataUnits = (short) ecPolicy.getNumDataUnits();
+        short numParityUnits = (short) ecPolicy.getNumParityUnits();
+        short numLocations = (short) (numDataUnits + numParityUnits);
+
+        // check quota limits and updated space consumed
+        fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
+            numLocations, true);
+        blockInfo = new BlockInfoStriped(block, ecPolicy);
+        blockInfo.convertToBlockUnderConstruction(
+            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
+      } else {
+        // check quota limits and updated space consumed
+        fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
+            fileINode.getFileReplication(), true);
+
+        short numLocations = fileINode.getFileReplication();
+        blockInfo = new BlockInfoContiguous(block, numLocations);
+        blockInfo.convertToBlockUnderConstruction(
+            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
+      }
       fsd.getBlockManager().addBlockCollection(blockInfo, fileINode);
       fileINode.addBlock(blockInfo);
 
@@ -550,22 +584,24 @@
       String clientName, String clientMachine)
       throws IOException {
 
+    Preconditions.checkNotNull(existing);
     long modTime = now();
-    INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
-                                     modTime, modTime, replication, preferredBlockSize);
-    newNode.setLocalName(localName.getBytes(Charsets.UTF_8));
-    newNode.toUnderConstruction(clientName, clientMachine);
-
     INodesInPath newiip;
     fsd.writeLock();
     try {
+      final boolean isStriped = FSDirErasureCodingOp.hasErasureCodingPolicy(
+          fsd.getFSNamesystem(), existing);
+      INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
+          modTime, modTime, replication, preferredBlockSize, isStriped);
+      newNode.setLocalName(localName.getBytes(Charsets.UTF_8));
+      newNode.toUnderConstruction(clientName, clientMachine);
       newiip = fsd.addINode(existing, newNode);
     } finally {
       fsd.writeUnlock();
     }
     if (newiip == null) {
       NameNode.stateChangeLog.info("DIR* addFile: failed to add " +
-                                       existing.getPath() + "/" + localName);
+          existing.getPath() + "/" + localName);
       return null;
     }
 
@@ -578,7 +614,7 @@
   private static FileState analyzeFileState(
       FSNamesystem fsn, String src, long fileId, String clientName,
       ExtendedBlock previous, LocatedBlock[] onRetryBlock)
-          throws IOException  {
+      throws IOException {
     assert fsn.hasReadLock();
 
     checkBlock(fsn, previous);
@@ -687,14 +723,8 @@
     checkBlock(fsn, last);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsn.dir.resolvePath(pc, src, pathComponents);
-    boolean success = completeFileInternal(fsn, src, holder,
-                                           ExtendedBlock.getLocalBlock(last),
-                                           fileId);
-    if (success) {
-      NameNode.stateChangeLog.info("DIR* completeFile: " + srcArg
-                                       + " is closed by " + holder);
-    }
-    return success;
+    return completeFileInternal(fsn, src, holder,
+        ExtendedBlock.getLocalBlock(last), fileId);
   }
 
   private static boolean completeFileInternal(
@@ -759,16 +789,18 @@
 
   private static INodeFile newINodeFile(
       long id, PermissionStatus permissions, long mtime, long atime,
-      short replication, long preferredBlockSize, byte storagePolicyId) {
+      short replication, long preferredBlockSize, byte storagePolicyId,
+      boolean isStriped) {
     return new INodeFile(id, null, permissions, mtime, atime,
         BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
-        storagePolicyId);
+        storagePolicyId, isStriped);
   }
 
   private static INodeFile newINodeFile(long id, PermissionStatus permissions,
-      long mtime, long atime, short replication, long preferredBlockSize) {
+      long mtime, long atime, short replication, long preferredBlockSize,
+      boolean isStriped) {
     return newINodeFile(id, permissions, mtime, atime, replication,
-        preferredBlockSize, (byte)0);
+        preferredBlockSize, (byte)0, isStriped);
   }
 
   /**
@@ -796,13 +828,12 @@
    * @param targets target datanodes where replicas of the new block is placed
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
-  private static void saveAllocatedBlock(
-      FSNamesystem fsn, String src, INodesInPath inodesInPath, Block newBlock,
-      DatanodeStorageInfo[] targets)
-      throws IOException {
+  private static void saveAllocatedBlock(FSNamesystem fsn, String src,
+      INodesInPath inodesInPath, Block newBlock, DatanodeStorageInfo[] targets,
+      boolean isStriped) throws IOException {
     assert fsn.hasWriteLock();
-    BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock,
-                                     targets);
+    BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets,
+        isStriped);
     NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
   }
@@ -851,17 +882,19 @@
 
   static class ValidateAddBlockResult {
     final long blockSize;
-    final int replication;
+    final int numTargets;
     final byte storagePolicyID;
     final String clientMachine;
+    final boolean isStriped;
 
     ValidateAddBlockResult(
-        long blockSize, int replication, byte storagePolicyID,
-        String clientMachine) {
+        long blockSize, int numTargets, byte storagePolicyID,
+        String clientMachine, boolean isStriped) {
       this.blockSize = blockSize;
-      this.replication = replication;
+      this.numTargets = numTargets;
       this.storagePolicyID = storagePolicyID;
       this.clientMachine = clientMachine;
+      this.isStriped = isStriped;
     }
   }
 
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 866305f..5b144b8 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
@@ -688,7 +688,7 @@
     final INodeFile fileINode = iip.getLastINode().asFile();
     EnumCounters<StorageType> typeSpaceDeltas =
       getStorageTypeDeltas(fileINode.getStoragePolicyID(), ssDelta,
-          replication, replication);;
+          replication, replication);
     updateCount(iip, iip.length() - 1,
       new QuotaCounts.Builder().nameSpace(nsDelta).storageSpace(ssDelta * replication).
           typeSpaces(typeSpaceDeltas).build(),
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 c2cccb5..6819d8d 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
@@ -36,16 +36,20 @@
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -412,7 +416,9 @@
       // 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);
+      ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
+          fsDir.getFSNamesystem(), iip);
+      updateBlocks(fsDir, addCloseOp, iip, newFile, ecPolicy);
       break;
     }
     case OP_CLOSE: {
@@ -432,7 +438,9 @@
       // 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);
+      ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
+          fsDir.getFSNamesystem(), iip);
+      updateBlocks(fsDir, addCloseOp, iip, file, ecPolicy);
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -490,8 +498,10 @@
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
-      updateBlocks(fsDir, updateOp, iip, oldFile);
-      
+      ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
+          fsDir.getFSNamesystem(), iip);
+      updateBlocks(fsDir, updateOp, iip, oldFile, ecPolicy);
+
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
       }
@@ -504,9 +514,12 @@
         FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " new block id : " + addBlockOp.getLastBlock().getBlockId());
       }
-      INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
+      INodesInPath iip = fsDir.getINodesInPath(path, true);
+      INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // add the new block to the INodeFile
-      addNewBlock(addBlockOp, oldFile);
+      ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
+          fsDir.getFSNamesystem(), iip);
+      addNewBlock(addBlockOp, oldFile, ecPolicy);
       break;
     }
     case OP_SET_REPLICATION: {
@@ -786,8 +799,15 @@
     }
     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: {
@@ -940,14 +960,14 @@
   /**
    * Add a new block into the given INodeFile
    */
-  private void addNewBlock(AddBlockOp op, INodeFile file)
-      throws IOException {
+  private void addNewBlock(AddBlockOp op, INodeFile file,
+      ErasureCodingPolicy ecPolicy) 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
       BlockInfo oldLastBlock = oldBlocks[oldBlocks.length - 1];
       if (oldLastBlock.getBlockId() != pBlock.getBlockId()
@@ -967,12 +987,18 @@
       Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
     }
     // add the new block
-    BlockInfo newBI = new BlockInfoContiguous(newBlock,
-        file.getPreferredBlockReplication());
-    newBI.convertToBlockUnderConstruction(
-        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
-    fsNamesys.getBlockManager().addBlockCollection(newBI, file);
-    file.addBlock(newBI);
+    final BlockInfo newBlockInfo;
+    boolean isStriped = ecPolicy != null;
+    if (isStriped) {
+      newBlockInfo = new BlockInfoStriped(newBlock, ecPolicy);
+    } else {
+      newBlockInfo = new BlockInfoContiguous(newBlock,
+          file.getPreferredBlockReplication());
+    }
+    newBlockInfo.convertToBlockUnderConstruction(
+        BlockUCState.UNDER_CONSTRUCTION, null);
+    fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file);
+    file.addBlock(newBlockInfo);
     fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
   }
   
@@ -981,7 +1007,8 @@
    * @throws IOException
    */
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
-      INodesInPath iip, INodeFile file) throws IOException {
+      INodesInPath iip, INodeFile file, ErasureCodingPolicy ecPolicy)
+      throws IOException {
     // Update its block list
     BlockInfo[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
@@ -1040,27 +1067,37 @@
         throw new IOException("Trying to delete non-existant block " + oldBlock);
       }
     } else if (newBlocks.length > oldBlocks.length) {
+      final boolean isStriped = ecPolicy != null;
       // We're adding blocks
       for (int i = oldBlocks.length; i < newBlocks.length; i++) {
         Block newBlock = newBlocks[i];
-        BlockInfo 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 BlockInfoContiguous(newBlock,
-              file.getPreferredBlockReplication());
+          if (isStriped) {
+            newBI = new BlockInfoStriped(newBlock, ecPolicy);
+          } else {
+            newBI = new BlockInfoContiguous(newBlock,
+                file.getPreferredBlockReplication());
+          }
           newBI.convertToBlockUnderConstruction(
-              HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
+              BlockUCState.UNDER_CONSTRUCTION, null);
         } else {
           // OP_CLOSE should add finalized blocks. This code path
           // is only executed when loading edits written by prior
           // versions of Hadoop. Current versions always log
           // OP_ADD operations as each block is allocated.
-          newBI = new BlockInfoContiguous(newBlock,
-              file.getFileReplication());
+          if (isStriped) {
+            newBI = new BlockInfoStriped(newBlock,
+                ErasureCodingPolicyManager.getSystemDefaultPolicy());
+          } else {
+            newBI = new BlockInfoContiguous(newBlock,
+                file.getFileReplication());
+          }
         }
-        fsNamesys.getBlockManager().addBlockCollection(newBI, file);
+        fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);
         file.addBlock(newBI);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
       }
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 92f333a..0b1902f 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
@@ -48,6 +48,7 @@
 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;
@@ -358,7 +359,8 @@
 
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
-          namesystem.getBlockIdManager().setLastAllocatedBlockId(maxSequentialBlockId);
+          namesystem.getBlockIdManager().setLastAllocatedContiguousBlockId(
+              maxSequentialBlockId);
         } else {
 
           long startingGenStamp = namesystem.getBlockIdManager()
@@ -690,7 +692,7 @@
       if (blocks != null) {
         final BlockManager bm = namesystem.getBlockManager();
         for (int i = 0; i < blocks.length; i++) {
-          file.setBlock(i, bm.addBlockCollection(blocks[i], file));
+          file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file));
         } 
       }
     }
@@ -753,7 +755,7 @@
       // file
       
       // read blocks
-      BlockInfo[] blocks = new BlockInfo[numBlocks];
+      BlockInfo[] blocks = new BlockInfoContiguous[numBlocks];
       for (int j = 0; j < numBlocks; j++) {
         blocks[j] = new BlockInfoContiguous(replication);
         blocks[j].readFields(in);
@@ -790,14 +792,15 @@
         counter.increment();
       }
 
-      final INodeFile file = new INodeFile(inodeId, localName, permissions,
-          modificationTime, atime, blocks, replication, blockSize);
+      INodeFile 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();
@@ -896,7 +899,7 @@
       final long preferredBlockSize = in.readLong();
 
       return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
-          accessTime, replication, preferredBlockSize, (byte) 0, null);
+          accessTime, replication, preferredBlockSize, (byte) 0, null, false);
     }
 
     public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)
@@ -960,8 +963,8 @@
         if (oldnode.numBlocks() > 0) {
           BlockInfo ucBlock = cons.getLastBlock();
           // we do not replace the inode, just replace the last block of oldnode
-          BlockInfo info = namesystem.getBlockManager().addBlockCollection(
-              ucBlock, oldnode);
+          BlockInfo info = namesystem.getBlockManager()
+              .addBlockCollectionWithCheck(ucBlock, oldnode);
           oldnode.setBlock(oldnode.numBlocks() - 1, info);
         }
 
@@ -1140,7 +1143,7 @@
       + " option to automatically rename these paths during upgrade.";
 
   /**
-   * Same as {@link #renameReservedPathsOnUpgrade(String)}, but for a single
+   * Same as {@link #renameReservedPathsOnUpgrade}, but for a single
    * byte array path component.
    */
   private static byte[] renameReservedComponentOnUpgrade(byte[] component,
@@ -1160,7 +1163,7 @@
   }
 
   /**
-   * Same as {@link #renameReservedPathsOnUpgrade(String)}, but for a single
+   * Same as {@link #renameReservedPathsOnUpgrade}, but for a single
    * byte array path component.
    */
   private static byte[] renameReservedRootComponentOnUpgrade(byte[] component,
@@ -1268,7 +1271,7 @@
         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(context.getTxId());
         out.writeLong(sourceNamesystem.dir.getLastInodeId());
 
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 0ae739c..34b28e4 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
@@ -39,11 +39,13 @@
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 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.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
@@ -218,7 +220,7 @@
       final BlockInfo[] blocks = file.getBlocks();
       if (blocks != null) {
         for (int i = 0; i < blocks.length; i++) {
-          file.setBlock(i, bm.addBlockCollection(blocks[i], file));
+          file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file));
         }
       }
     }
@@ -331,27 +333,35 @@
       INodeSection.INodeFile f = n.getFile();
       List<BlockProto> bp = f.getBlocksList();
       short replication = (short) f.getReplication();
+      boolean isStriped = f.getIsStriped();
       LoaderContext state = parent.getLoaderContext();
+      ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 
       BlockInfo[] blocks = new BlockInfo[bp.size()];
-      for (int i = 0, e = bp.size(); i < e; ++i) {
-        blocks[i] =
-            new BlockInfoContiguous(PBHelperClient.convert(bp.get(i)), replication);
+      for (int i = 0; i < bp.size(); ++i) {
+        BlockProto b = bp.get(i);
+        if (isStriped) {
+          blocks[i] = new BlockInfoStriped(PBHelperClient.convert(b), ecPolicy);
+        } else {
+          blocks[i] = new BlockInfoContiguous(PBHelperClient.convert(b),
+              replication);
+        }
       }
+
       final PermissionStatus permissions = loadPermission(f.getPermission(),
           parent.getLoaderContext().getStringTable());
 
       final INodeFile file = new INodeFile(n.getId(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
           f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
-          (byte)f.getStoragePolicyID());
+          (byte)f.getStoragePolicyID(), isStriped);
 
       if (f.hasAcl()) {
         int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
             f.getAcl(), state.getStringTable()));
         file.addAclFeature(new AclFeature(entries));
       }
-      
+
       if (f.hasXAttrs()) {
         file.addXAttrFeature(new XAttrFeature(
             loadXAttrs(f.getXAttrs(), state.getStringTable())));
@@ -363,8 +373,18 @@
         file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         if (blocks.length > 0) {
           BlockInfo lastBlk = file.getLastBlock();
-          lastBlk.convertToBlockUnderConstruction(
+          // replace the last block of file
+          final BlockInfo ucBlk;
+          if (isStriped) {
+            BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
+            ucBlk = new BlockInfoStriped(striped, ecPolicy);
+          } else {
+            ucBlk = new BlockInfoContiguous(lastBlk,
+                replication);
+          }
+          ucBlk.convertToBlockUnderConstruction(
               HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
+          file.setBlock(file.numBlocks() - 1, ucBlk);
         }
       }
       return file;
@@ -478,7 +498,8 @@
           .setPermission(buildPermissionStatus(file, state.getStringMap()))
           .setPreferredBlockSize(file.getPreferredBlockSize())
           .setReplication(file.getFileReplication())
-          .setStoragePolicyID(file.getLocalStoragePolicyID());
+          .setStoragePolicyID(file.getLocalStoragePolicyID())
+          .setIsStriped(file.isStriped());
 
       AclFeature f = file.getAclFeature();
       if (f != null) {
@@ -632,8 +653,9 @@
     private void save(OutputStream out, INodeFile n) throws IOException {
       INodeSection.INodeFile.Builder b = buildINodeFile(n,
           parent.getSaverContext());
+      BlockInfo[] blocks = n.getBlocks();
 
-      if (n.getBlocks() != null) {
+      if (blocks != null) {
         for (Block block : n.getBlocks()) {
           b.addBlocks(PBHelperClient.convert(block));
         }
@@ -667,7 +689,7 @@
       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()));
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 7c8a857..ef0cc1d 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
@@ -297,7 +297,11 @@
       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()) {
@@ -549,7 +553,8 @@
           .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
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 96e4ecb..419a704 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
@@ -33,7 +33,6 @@
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
@@ -124,22 +123,25 @@
     short blockReplication = in.readShort();
     long modificationTime = in.readLong();
     long preferredBlockSize = in.readLong();
-  
+
     int numBlocks = in.readInt();
-    BlockInfo[] blocks = new BlockInfo[numBlocks];
+
+    final BlockInfoContiguous[] blocksContiguous =
+        new BlockInfoContiguous[numBlocks];
     Block blk = new Block();
     int i = 0;
-    for (; i < numBlocks-1; i++) {
+    for (; i < numBlocks - 1; i++) {
       blk.readFields(in);
-      blocks[i] = new BlockInfoContiguous(blk, blockReplication);
+      blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication);
     }
     // last block is UNDER_CONSTRUCTION
     if(numBlocks > 0) {
       blk.readFields(in);
-      blocks[i] = new BlockInfoContiguous(blk, blockReplication);
-      blocks[i].convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
-          null);
+      blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication);
+      blocksContiguous[i].convertToBlockUnderConstruction(
+          BlockUCState.UNDER_CONSTRUCTION, null);
     }
+
     PermissionStatus perm = PermissionStatus.read(in);
     String clientName = readString(in);
     String clientMachine = readString(in);
@@ -152,7 +154,7 @@
     // 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);
+        modificationTime, blocksContiguous, blockReplication, preferredBlockSize);
     file.toUnderConstruction(clientName, clientMachine);
     return file;
   }
@@ -305,7 +307,7 @@
     if (!isWithName) {
       Preconditions.checkState(ref instanceof INodeReference.DstReference);
       // dst snapshot id
-      out.writeInt(((INodeReference.DstReference) ref).getDstSnapshotId());
+      out.writeInt(ref.getDstSnapshotId());
     } else {
       out.writeInt(((INodeReference.WithName) ref).getLastSnapshotId());
     }
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 4dda27d..a94b61c 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
@@ -57,9 +57,9 @@
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
@@ -134,6 +134,7 @@
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.crypto.key.KeyProvider.Metadata;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
@@ -144,6 +145,7 @@
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
@@ -152,10 +154,8 @@
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.ServiceFailedException;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -163,6 +163,7 @@
 import org.apache.hadoop.hdfs.UnknownCryptoProtocolVersionException;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
@@ -171,15 +172,17 @@
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -194,6 +197,8 @@
 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.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -409,6 +414,7 @@
   private final BlockManager blockManager;
   private final SnapshotManager snapshotManager;
   private final CacheManager cacheManager;
+  private final ErasureCodingPolicyManager ecPolicyManager;
   private final DatanodeStatistics datanodeStatistics;
 
   private String nameserviceId;
@@ -582,6 +588,7 @@
     leaseManager.removeAllLeases();
     snapshotManager.clearSnapshottableDirs();
     cacheManager.clear();
+    ecPolicyManager.clear();
     setImageLoaded(false);
     blockManager.clear();
   }
@@ -819,6 +826,7 @@
       this.dir = new FSDirectory(this, conf);
       this.snapshotManager = new SnapshotManager(dir);
       this.cacheManager = new CacheManager(this, conf, blockManager);
+      this.ecPolicyManager = new ErasureCodingPolicyManager();
       this.safeMode = new SafeModeInfo(conf);
       this.topConf = new TopConf(conf);
       this.auditLoggers = initAuditLoggers(conf);
@@ -1740,8 +1748,14 @@
 
     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();
@@ -2076,7 +2090,19 @@
     if (!DFSUtil.isValidName(src)) {
       throw new InvalidPathException(src);
     }
-    blockManager.verifyReplication(src, replication, clientMachine);
+
+    checkOperation(OperationCategory.READ);
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      if (!FSDirErasureCodingOp.hasErasureCodingPolicy(this, src)) {
+        blockManager.verifyReplication(src, replication, clientMachine);
+      }
+    } finally {
+      readUnlock();
+    }
+    
+    checkOperation(OperationCategory.WRITE);
     if (blockSize < minBlockSize) {
       throw new IOException("Specified block size is less than configured" +
           " minimum value (" + DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY
@@ -2418,6 +2444,7 @@
     final long preferredblocksize;
     final byte storagePolicyID;
     final List<DatanodeStorageInfo> chosen;
+    final boolean isStriped;
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     FSPermissionChecker pc = getPermissionChecker();
@@ -2444,6 +2471,7 @@
       clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
       preferredblocksize = file.getPreferredBlockSize();
       storagePolicyID = file.getStoragePolicyID();
+      isStriped = file.isStriped();
 
       //find datanode storages
       final DatanodeManager dm = blockManager.getDatanodeManager();
@@ -2459,7 +2487,7 @@
     // choose new datanodes.
     final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
         src, numAdditionalNodes, clientnode, chosen, 
-        excludes, preferredblocksize, storagePolicyID);
+        excludes, preferredblocksize, storagePolicyID, isStriped);
     final LocatedBlock lb = BlockManager.newLocatedBlock(
         blk, targets, -1, false);
     blockManager.setBlockToken(lb, BlockTokenIdentifier.AccessMode.COPY);
@@ -2553,15 +2581,20 @@
       writeUnlock();
     }
     getEditLog().logSync();
+    if (success) {
+      NameNode.stateChangeLog.info("DIR* completeFile: " + src
+          + " is closed by " + holder);
+    }
     return success;
   }
 
   /**
    * 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;
@@ -2755,7 +2788,7 @@
       if (trackBlockCounts) {
         if (b.isComplete()) {
           numRemovedComplete++;
-          if (blockManager.checkMinReplication(b)) {
+          if (blockManager.hasMinStorage(b, b.numNodes())) {
             numRemovedSafe++;
           }
         }
@@ -2987,7 +3020,7 @@
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
         break;
-      assert blockManager.checkMinReplication(curBlock) :
+      assert blockManager.hasMinStorage(curBlock) :
               "A COMPLETE block is not minimally replicated in " + src;
     }
 
@@ -3022,8 +3055,8 @@
     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.hasMinStorage(penultimateBlock);
 
     switch(lastBlockState) {
     case COMPLETE:
@@ -3031,8 +3064,8 @@
       break;
     case COMMITTED:
       // Close file if committed blocks are minimally replicated
-      if(penultimateBlockMinReplication &&
-          blockManager.checkMinReplication(lastBlock)) {
+      if(penultimateBlockMinStorage &&
+          blockManager.hasMinStorage(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId());
         NameNode.stateChangeLog.warn("BLOCK*"
@@ -3067,8 +3100,8 @@
 
       // setup the last block locations from the blockManager if not known
       if (uc.getNumExpectedLocations() == 0) {
-        uc.setExpectedLocations(lastBlock,
-            blockManager.getStorages(lastBlock));
+        uc.setExpectedLocations(lastBlock, blockManager.getStorages(lastBlock),
+            lastBlock.isStriped());
       }
 
       if (uc.getNumExpectedLocations() == 0 && lastBlock.getNumBytes() == 0) {
@@ -3133,10 +3166,31 @@
     }
 
     // Adjust disk space consumption if required
-    final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();    
+    final long diff;
+    final short replicationFactor;
+    if (fileINode.isStriped()) {
+      final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp
+          .getErasureCodingPolicy(this, iip);
+      final short numDataUnits = (short) ecPolicy.getNumDataUnits();
+      final short numParityUnits = (short) ecPolicy.getNumParityUnits();
+
+      final long numBlocks = numDataUnits + numParityUnits;
+      final long fullBlockGroupSize =
+          fileINode.getPreferredBlockSize() * numBlocks;
+
+      final BlockInfoStriped striped = new BlockInfoStriped(commitBlock,
+          ecPolicy);
+      final long actualBlockGroupSize = striped.spaceConsumed();
+
+      diff = fullBlockGroupSize - actualBlockGroupSize;
+      replicationFactor = (short) 1;
+    } else {
+      diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();
+      replicationFactor = fileINode.getFileReplication();
+    }
     if (diff > 0) {
       try {
-        dir.updateSpaceConsumed(iip, 0, -diff, fileINode.getFileReplication());
+        dir.updateSpaceConsumed(iip, 0, -diff, replicationFactor);
       } catch (IOException e) {
         LOG.warn("Unexpected exception while updating disk space.", e);
       }
@@ -3286,7 +3340,7 @@
       }
 
       truncatedBlock = iFile.getLastBlock();
-      long recoveryId = truncatedBlock.getUnderConstructionFeature()
+      final long recoveryId = truncatedBlock.getUnderConstructionFeature()
           .getBlockRecoveryId();
       copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId();
       if(recoveryId != newgenerationstamp) {
@@ -3301,8 +3355,7 @@
         if (remove) {
           blockManager.removeBlock(storedBlock);
         }
-      }
-      else {
+      } else {
         // update last block
         if(!copyTruncate) {
           storedBlock.setGenerationStamp(newgenerationstamp);
@@ -3336,9 +3389,9 @@
                 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);
               }
             }
           }
@@ -3354,8 +3407,9 @@
         } else {
           iFile.convertLastBlockToUC(storedBlock, trimmedStorageInfos);
           if (closeFile) {
-            blockManager.markBlockReplicasAsCorrupt(storedBlock,
-                oldGenerationStamp, oldNumBytes, trimmedStorageInfos);
+            blockManager.markBlockReplicasAsCorrupt(oldBlock.getLocalBlock(),
+                storedBlock, oldGenerationStamp, oldNumBytes,
+                trimmedStorageInfos);
           }
         }
       }
@@ -3363,7 +3417,7 @@
       if (closeFile) {
         if(copyTruncate) {
           src = closeFileCommitBlocks(iFile, truncatedBlock);
-          if(!iFile.isBlockInLatestSnapshot(storedBlock)) {
+          if(!iFile.isBlockInLatestSnapshot((BlockInfoContiguous) storedBlock)) {
             blockManager.removeBlock(storedBlock);
           }
         } else {
@@ -4353,10 +4407,16 @@
     /**
      * 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).getRealDataBlockNum() : safeReplication;
+      if (storageNum == safe) {
         this.blockSafe++;
 
         // Report startup progress only if we haven't completed startup yet.
@@ -4631,12 +4691,12 @@
   }
 
   @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
@@ -5136,11 +5196,13 @@
 
   /**
    * 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.nextStripedBlockId() : blockIdManager.nextContiguousBlockId();
     getEditLog().logAllocateBlockId(blockId);
     // NB: callers sync the log
     return blockId;
@@ -5245,29 +5307,32 @@
    * Get a new generation stamp together with an access token for 
    * a block under construction
    * 
-   * This method is called for recovering a failed pipeline or setting up
-   * a pipeline to append to a block.
+   * This method is called for recovering a failed write or setting up
+   * a block for appended.
    * 
    * @param block a block
    * @param clientName the name of a client
    * @return a located block with a new generation stamp and an access token
    * @throws IOException if any error occurs
    */
-  LocatedBlock updateBlockForPipeline(ExtendedBlock block, 
+  LocatedBlock bumpBlockGenerationStamp(ExtendedBlock block,
       String clientName) throws IOException {
-    LocatedBlock locatedBlock;
+    final LocatedBlock locatedBlock;
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
 
       // check vadility of parameters
-      checkUCBlock(block, clientName);
+      final INodeFile file = checkUCBlock(block, clientName);
   
       // get a new generation stamp and an access token
       block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock())));
-      locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]);
-      blockManager.setBlockToken(locatedBlock, BlockTokenIdentifier.AccessMode.WRITE);
+
+      locatedBlock = BlockManager.newLocatedBlock(
+          block, file.getLastBlock(), null, -1);
+      blockManager.setBlockToken(locatedBlock,
+          BlockTokenIdentifier.AccessMode.WRITE);
     } finally {
       writeUnlock();
     }
@@ -5319,28 +5384,34 @@
     assert hasWriteLock();
     // check the vadility of the block and lease holder name
     final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
-    final BlockInfo blockinfo = pendingFile.getLastBlock();
-    assert !blockinfo.isComplete();
+    final BlockInfo lastBlock = pendingFile.getLastBlock();
+    assert !lastBlock.isComplete();
 
     // check new GS & length: this is not expected
-    if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() ||
-        newBlock.getNumBytes() < blockinfo.getNumBytes()) {
-      String msg = "Update " + oldBlock + " (len = " + 
-        blockinfo.getNumBytes() + ") to an older state: " + newBlock + 
-        " (len = " + newBlock.getNumBytes() +")";
+    if (newBlock.getGenerationStamp() <= lastBlock.getGenerationStamp()) {
+      final String msg = "Update " + oldBlock + " but the new block " + newBlock
+          + " does not have a larger generation stamp than the last block "
+          + lastBlock;
+      LOG.warn(msg);
+      throw new IOException(msg);
+    }
+    if (newBlock.getNumBytes() < lastBlock.getNumBytes()) {
+      final String msg = "Update " + oldBlock + " (size="
+          + oldBlock.getNumBytes() + ") to a smaller size block " + newBlock
+          + " (size=" + newBlock.getNumBytes() + ")";
       LOG.warn(msg);
       throw new IOException(msg);
     }
 
     // Update old block with the new generation stamp and new length
-    blockinfo.setNumBytes(newBlock.getNumBytes());
-    blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
+    lastBlock.setNumBytes(newBlock.getNumBytes());
+    lastBlock.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
 
     // find the DatanodeDescriptor objects
     final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
         .getDatanodeStorageInfos(newNodes, newStorageIDs);
-    blockinfo.getUnderConstructionFeature().setExpectedLocations(
-        blockinfo, storages);
+    lastBlock.getUnderConstructionFeature().setExpectedLocations(lastBlock,
+        storages, lastBlock.isStriped());
 
     String src = pendingFile.getFullPathName();
     FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, logRetryCache);
@@ -6110,17 +6181,24 @@
   public FSDirectory getFSDirectory() {
     return dir;
   }
+
   /** Set the FSDirectory. */
   @VisibleForTesting
   public void setFSDirectory(FSDirectory dir) {
     this.dir = dir;
   }
+
   /** @return the cache manager. */
   @Override
   public CacheManager getCacheManager() {
     return cacheManager;
   }
 
+  /** @return the ErasureCodingPolicyManager. */
+  public ErasureCodingPolicyManager getErasureCodingPolicyManager() {
+    return ecPolicyManager;
+  }
+
   @Override
   public HAContext getHAContext() {
     return haContext;
@@ -7024,6 +7102,68 @@
     }
   }
 
+  /**
+   * Set an erasure coding policy on the given path.
+   * @param srcArg  The path of the target directory.
+   * @param ecPolicy The erasure coding policy to set on the target directory.
+   * @throws AccessControlException  if the caller is not the superuser.
+   * @throws UnresolvedLinkException if the path can't be resolved.
+   * @throws SafeModeException       if the Namenode is in safe mode.
+   */
+  void setErasureCodingPolicy(final String srcArg, final ErasureCodingPolicy
+      ecPolicy, final boolean logRetryCache) throws IOException,
+      UnresolvedLinkException, SafeModeException, AccessControlException {
+    checkSuperuserPrivilege();
+    checkOperation(OperationCategory.WRITE);
+    HdfsFileStatus resultingStat = null;
+    boolean success = false;
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot set erasure coding policy on " + srcArg);
+      resultingStat = FSDirErasureCodingOp.setErasureCodingPolicy(this,
+          srcArg, ecPolicy, logRetryCache);
+      success = true;
+    } finally {
+      writeUnlock();
+      if (success) {
+        getEditLog().logSync();
+      }
+      logAuditEvent(success, "setErasureCodingPolicy", srcArg, null,
+          resultingStat);
+    }
+  }
+
+  /**
+   * Get the erasure coding policy information for specified path
+   */
+  ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws AccessControlException, UnresolvedLinkException, IOException {
+    checkOperation(OperationCategory.READ);
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      return getErasureCodingPolicyForPath(src);
+    } finally {
+      readUnlock();
+    }
+  }
+
+  /**
+   * Get available erasure coding polices
+   */
+  ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
+    checkOperation(OperationCategory.READ);
+    waitForLoadingFSImage();
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      return FSDirErasureCodingOp.getErasureCodingPolicies(this);
+    } finally {
+      readUnlock();
+    }
+  }
+
   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
                 boolean logRetryCache)
       throws IOException {
@@ -7205,7 +7345,6 @@
       logger.addAppender(asyncAppender);        
     }
   }
-
   /**
    * Return total number of Sync Operations on FSEditLog.
    */
@@ -7230,5 +7369,11 @@
       return "";
     }
   }
+
+  @Override
+  public ErasureCodingPolicy getErasureCodingPolicyForPath(String src)
+      throws IOException {
+    return FSDirErasureCodingOp.getErasureCodingPolicy(this, src);
+  }
 }
 
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 8565522..242c456 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,9 +34,11 @@
 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.ErasureCodingPolicy;
 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.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -79,12 +81,14 @@
 
   /** 
    * Bit format:
-   * [4-bit storagePolicyID][12-bit replication][48-bit preferredBlockSize]
+   * [4-bit storagePolicyID][1-bit isStriped]
+   * [11-bit replication][48-bit preferredBlockSize]
    */
   enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
-    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 1),
-    STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
+    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 0),
+    IS_STRIPED(REPLICATION.BITS, 1, 0),
+    STORAGE_POLICY_ID(IS_STRIPED.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
         0);
 
     private final LongBitFormat BITS;
@@ -105,14 +109,27 @@
       return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
     }
 
+    static boolean isStriped(long header) {
+      long isStriped = IS_STRIPED.BITS.retrieve(header);
+      Preconditions.checkState(isStriped == 0 || isStriped == 1);
+      return isStriped == 1;
+    }
+
     static long toLong(long preferredBlockSize, short replication,
-        byte storagePolicyID) {
+        boolean isStriped, byte storagePolicyID) {
       long h = 0;
       if (preferredBlockSize == 0) {
         preferredBlockSize = PREFERRED_BLOCK_SIZE.BITS.getMin();
       }
       h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h);
-      h = REPLICATION.BITS.combine(replication, h);
+      // Replication factor for striped files is zero
+      if (isStriped) {
+        h = REPLICATION.BITS.combine(0L, h);
+        h = IS_STRIPED.BITS.combine(1L, h);
+      } else {
+        h = REPLICATION.BITS.combine(replication, h);
+        h = IS_STRIPED.BITS.combine(0L, h);
+      }
       h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h);
       return h;
     }
@@ -127,15 +144,21 @@
             long atime, BlockInfo[] blklist, short replication,
             long preferredBlockSize) {
     this(id, name, permissions, mtime, atime, blklist, replication,
-         preferredBlockSize, (byte) 0);
+        preferredBlockSize, (byte) 0, false);
   }
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
       long atime, BlockInfo[] blklist, short replication,
-      long preferredBlockSize, byte storagePolicyID) {
+      long preferredBlockSize, byte storagePolicyID, boolean isStriped) {
     super(id, name, permissions, mtime, atime);
-    header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID);
-    this.blocks = blklist;
+    header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
+        storagePolicyID);
+    if (blklist != null && blklist.length > 0) {
+      for (BlockInfo b : blklist) {
+        Preconditions.checkArgument(b.isStriped() == isStriped);
+      }
+    }
+    setBlocks(blklist);
   }
   
   public INodeFile(INodeFile that) {
@@ -227,6 +250,7 @@
 
   @Override // BlockCollection
   public void setBlock(int index, BlockInfo blk) {
+    Preconditions.checkArgument(blk.isStriped() == this.isStriped());
     this.blocks[index] = blk;
   }
 
@@ -235,7 +259,6 @@
       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.");
     }
@@ -350,6 +373,7 @@
 
   /** The same as getFileReplication(null). */
   @Override // INodeFileAttributes
+  // TODO properly handle striped files
   public final short getFileReplication() {
     return getFileReplication(CURRENT_STATE_ID);
   }
@@ -364,7 +388,13 @@
       }
       max = maxInSnapshot > max ? maxInSnapshot : max;
     }
-    return max;
+    if(!isStriped()){
+      return max;
+    }
+    // TODO support more policies based on policyId
+    ErasureCodingPolicy ecPolicy =
+        ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits());
   }
 
   /** Set the replication factor of this file. */
@@ -412,6 +442,29 @@
     setStoragePolicyID(storagePolicyId);
   }
 
+
+  /**
+   * @return The ID of the erasure coding policy on the file. 0 represents no
+   *          EC policy (file is in contiguous format). 1 represents the system
+   *          default EC policy:
+   *          {@link ErasureCodingPolicyManager#SYS_DEFAULT_POLICY}.
+   * TODO: support more policies by reusing {@link HeaderFormat#REPLICATION}.
+   */
+  @VisibleForTesting
+  @Override
+  public byte getErasureCodingPolicyID() {
+    return isStriped() ? (byte)1 : (byte)0;
+  }
+
+  /**
+   * @return true if the file is in the striping layout.
+   */
+  @VisibleForTesting
+  @Override
+  public boolean isStriped() {
+    return HeaderFormat.isStriped(header);
+  }
+
   @Override // INodeFileAttributes
   public long getHeaderLong() {
     return header;
@@ -425,10 +478,12 @@
 
   /** @return blocks of the file corresponding to the snapshot. */
   public BlockInfo[] getBlocks(int snapshot) {
-    if(snapshot == CURRENT_STATE_ID || getDiffs() == null) {
+    if (snapshot == CURRENT_STATE_ID || getDiffs() == null) {
       return getBlocks();
     }
+    // find blocks stored in snapshot diffs (for truncate)
     FileDiff diff = getDiffs().getDiffById(snapshot);
+    // note that currently FileDiff can only store contiguous blocks
     BlockInfo[] snapshotBlocks = diff == null ? getBlocks() : diff.getBlocks();
     if (snapshotBlocks != null) {
       return snapshotBlocks;
@@ -446,6 +501,7 @@
     int size = this.blocks.length;
     int totalAddedBlocks = 0;
     for(INodeFile f : inodes) {
+      Preconditions.checkState(f.isStriped() == this.isStriped());
       totalAddedBlocks += f.blocks.length;
     }
     
@@ -473,6 +529,7 @@
    * add a block to the block list
    */
   void addBlock(BlockInfo newblock) {
+    Preconditions.checkArgument(newblock.isStriped() == this.isStriped());
     if (this.blocks == null) {
       this.setBlocks(new BlockInfo[]{newblock});
     } else {
@@ -586,6 +643,10 @@
 
     final long ssDeltaNoReplication;
     short replication;
+    if (isStriped()) {
+      return computeQuotaUsageWithStriped(bsp, counts);
+    }
+
     if (last < lastSnapshotId) {
       ssDeltaNoReplication = computeFileSize(true, false);
       replication = getFileReplication();
@@ -608,6 +669,18 @@
     return counts;
   }
 
+  /**
+   * Compute quota of striped file. Note that currently EC files do not support
+   * append/hflush/hsync, thus the file length recorded in snapshots should be
+   * the same with the current file length.
+   */
+  public final QuotaCounts computeQuotaUsageWithStriped(
+      BlockStoragePolicy bsp, QuotaCounts counts) {
+    counts.addNameSpace(1);
+    counts.add(storagespaceConsumed(bsp));
+    return counts;
+  }
+
   @Override
   public final ContentSummaryComputationContext computeContentSummary(
       int snapshotId, final ContentSummaryComputationContext summary) {
@@ -678,17 +751,21 @@
       return 0;
     }
     final int last = blocks.length - 1;
-    //check if the last block is under-construction
-    long size = blocks[last].getNumBytes();
-    if (!blocks[last].isComplete()) {
+    //check if the last block is BlockInfoUnderConstruction
+    BlockInfo lastBlk = blocks[last];
+    long size = lastBlk.getNumBytes();
+    if (!lastBlk.isComplete()) {
        if (!includesLastUcBlock) {
          size = 0;
        } else if (usePreferredBlockSize4LastUcBlock) {
-         size = getPreferredBlockSize();
+         size = isStriped()?
+             getPreferredBlockSize() *
+                 ((BlockInfoStriped)lastBlk).getDataBlockNum() :
+             getPreferredBlockSize();
        }
     }
     //sum other blocks
-    for(int i = 0; i < last; i++) {
+    for (int i = 0; i < last; i++) {
       size += blocks[i].getNumBytes();
     }
     return size;
@@ -700,6 +777,32 @@
    * Use preferred block size for the last block if it is under construction.
    */
   public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
+    if (isStriped()) {
+      return storagespaceConsumedStriped();
+    } else {
+      return storagespaceConsumedContiguous(bsp);
+    }
+  }
+
+  // TODO: support EC with heterogeneous storage
+  public final QuotaCounts storagespaceConsumedStriped() {
+    QuotaCounts counts = new QuotaCounts.Builder().build();
+    if (blocks == null || blocks.length == 0) {
+      return counts;
+    }
+
+    for (BlockInfo b : blocks) {
+      Preconditions.checkState(b.isStriped());
+      long blockSize = b.isComplete() ?
+          ((BlockInfoStriped)b).spaceConsumed() : getPreferredBlockSize() *
+          ((BlockInfoStriped)b).getTotalBlockNum();
+      counts.addStorageSpace(blockSize);
+    }
+    return  counts;
+  }
+
+  public final QuotaCounts storagespaceConsumedContiguous(
+      BlockStoragePolicy bsp) {
     QuotaCounts counts = new QuotaCounts.Builder().build();
     final Iterable<BlockInfo> blocks;
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
@@ -803,6 +906,7 @@
   /**
    * compute the quota usage change for a truncate op
    * @param newLength the length for truncation
+   * TODO: properly handle striped blocks (HDFS-7622)
    **/
   void computeQuotaDeltaForTruncate(
       long newLength, BlockStoragePolicy bsps,
@@ -866,8 +970,15 @@
     setBlocks(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(BlockInfo[] snapshotBlocks,
                                           BlocksMapUpdateInfo collectedBlocks) {
+    Preconditions.checkState(!isStriped());
     BlockInfo[] oldBlocks = getBlocks();
     if(snapshotBlocks == null || oldBlocks == null)
       return;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
index 204c8ac..8cd481a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
@@ -20,7 +20,7 @@
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile.HeaderFormat;
-import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
+
 /**
  * The attributes of a file.
  */
@@ -29,6 +29,12 @@
   /** @return the file replication. */
   public short getFileReplication();
 
+  /** @return whether the file is striped (instead of contiguous) */
+  public boolean isStriped();
+
+  /** @return whether the file is striped (instead of contiguous) */
+  public byte getErasureCodingPolicyID();
+
   /** @return preferred block size in bytes */
   public long getPreferredBlockSize();
 
@@ -47,10 +53,10 @@
     public SnapshotCopy(byte[] name, PermissionStatus permissions,
         AclFeature aclFeature, long modificationTime, long accessTime,
         short replication, long preferredBlockSize,
-        byte storagePolicyID, XAttrFeature xAttrsFeature) {
+        byte storagePolicyID, XAttrFeature xAttrsFeature, boolean isStriped) {
       super(name, permissions, aclFeature, modificationTime, accessTime, 
           xAttrsFeature);
-      header = HeaderFormat.toLong(preferredBlockSize, replication,
+      header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
           storagePolicyID);
     }
 
@@ -70,6 +76,16 @@
     }
 
     @Override
+    public boolean isStriped() {
+      return HeaderFormat.isStriped(header);
+    }
+
+    @Override
+    public byte getErasureCodingPolicyID() {
+      return isStriped() ? (byte)1 : (byte)0;
+    }
+
+    @Override
     public long getPreferredBlockSize() {
       return HeaderFormat.getPreferredBlockSize(header);
     }
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 908af45..cec9313 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
@@ -121,8 +121,8 @@
       for(BlockInfo b : blocks) {
         if(!b.isComplete())
           numUCBlocks++;
+        }
       }
-    }
     LOG.info("Number of blocks under construction: " + numUCBlocks);
     return numUCBlocks;
   }
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 3a5dc12..2943fc2 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
@@ -87,7 +87,8 @@
     BLOCK_STORAGE_POLICY(-60, -60, "Block Storage policy"),
     TRUNCATE(-61, -61, "Truncate"),
     APPEND_NEW_BLOCK(-62, -61, "Support appending to new block"),
-    QUOTA_BY_STORAGE_TYPE(-63, -61, "Support quota for specific storage types");
+    QUOTA_BY_STORAGE_TYPE(-63, -61, "Support quota for specific storage types"),
+    ERASURE_CODING(-64, -61, "Support erasure coding");
 
     private final FeatureInfo info;
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 551a0e2..328b3c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -84,6 +84,7 @@
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
@@ -798,7 +799,7 @@
   public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName)
       throws IOException {
     checkNNStartup();
-    return namesystem.updateBlockForPipeline(block, clientName);
+    return namesystem.bumpBlockGenerationStamp(block, clientName);
   }
 
 
@@ -1425,8 +1426,8 @@
 
   @Override // RefreshAuthorizationPolicyProtocol
   public void refreshUserToGroupsMappings() throws IOException {
-    LOG.info("Refreshing all user-to-groups mappings. Requested by user: " + 
-             getRemoteUser().getShortUserName());
+    LOG.info("Refreshing all user-to-groups mappings. Requested by user: " +
+        getRemoteUser().getShortUserName());
     Groups.getUserToGroupsMappingService().refresh();
   }
 
@@ -1559,7 +1560,7 @@
     }
     namesystem.checkOperation(OperationCategory.WRITE);
     CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache,
-      null);
+        null);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return (String) cacheEntry.getPayload();
     }
@@ -1851,6 +1852,23 @@
   }
 
   @Override // ClientProtocol
+  public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy)
+      throws IOException {
+    checkNNStartup();
+    final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return;
+    }
+    boolean success = false;
+    try {
+      namesystem.setErasureCodingPolicy(src, ecPolicy, cacheEntry != null);
+      success = true;
+    } finally {
+      RetryCache.setState(cacheEntry, success);
+    }
+  }
+
+  @Override // ClientProtocol
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
     checkNNStartup();
@@ -2045,4 +2063,16 @@
     namesystem.checkSuperuserPrivilege();
     nn.tracerConfigurationManager.removeSpanReceiver(id);
   }
+
+  @Override // ClientProtocol
+  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
+    checkNNStartup();
+    return namesystem.getErasureCodingPolicies();
+  }
+
+  @Override // ClientProtocol
+  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
+    checkNNStartup();
+    return namesystem.getErasureCodingPolicy(src);
+  }
 }
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 309012e..404a71e 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
@@ -66,8 +66,9 @@
 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.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -125,6 +126,9 @@
   private final int totalDatanodes;
   private final InetAddress remoteAddress;
 
+  private long totalDirs = 0L;
+  private long totalSymlinks = 0L;
+
   private String lostFound = null;
   private boolean lfInited = false;
   private boolean lfInitedOk = false;
@@ -174,7 +178,7 @@
   private final PrintWriter out;
   private List<String> snapshottableDirs = null;
 
-  private final BlockPlacementPolicy bpPolicy;
+  private final BlockPlacementPolicies bpPolicies;
   private StoragePolicySummary storageTypeSummary = null;
 
   /**
@@ -196,7 +200,7 @@
     this.out = out;
     this.totalDatanodes = totalDatanodes;
     this.remoteAddress = remoteAddress;
-    this.bpPolicy = BlockPlacementPolicy.getInstance(conf, null,
+    this.bpPolicies = new BlockPlacementPolicies(conf, null,
         networktopology,
         namenode.getNamesystem().getBlockManager().getDatanodeManager()
         .getHost2DatanodeMap());
@@ -261,7 +265,8 @@
       NumberReplicas numberReplicas= bm.countNodes(blockInfo);
       out.println("Block Id: " + blockId);
       out.println("Block belongs to: "+iNode.getFullPathName());
-      out.println("No. of Expected Replica: " + blockInfo.getReplication());
+      out.println("No. of Expected Replica: " +
+          bm.getExpectedReplicaNum(blockInfo));
       out.println("No. of live Replica: " + numberReplicas.liveReplicas());
       out.println("No. of excess Replica: " + numberReplicas.excessReplicas());
       out.println("No. of stale Replica: " +
@@ -362,13 +367,21 @@
               namenode.getNamesystem().getBlockManager().getStoragePolicies());
         }
 
-        Result res = new Result(conf);
+        Result replRes = new ReplicationResult(conf);
+        Result ecRes = new ErasureCodingResult(conf);
 
-        check(path, file, res);
+        check(path, file, replRes, ecRes);
 
-        out.println(res);
-        out.println(" Number of data-nodes:\t\t" + totalDatanodes);
+        out.print("\nStatus: ");
+        out.println(replRes.isHealthy() && ecRes.isHealthy() ? "HEALTHY" : "CORRUPT");
+        out.println(" Number of data-nodes:\t" + totalDatanodes);
         out.println(" Number of racks:\t\t" + networktopology.getNumOfRacks());
+        out.println(" Total dirs:\t\t\t" + totalDirs);
+        out.println(" Total symlinks:\t\t" + totalSymlinks);
+        out.println("\nReplicated Blocks:");
+        out.println(replRes);
+        out.println("\nErasure Coded Block Groups:");
+        out.println(ecRes);
 
         if (this.showStoragePolcies) {
           out.print(storageTypeSummary.toString());
@@ -388,7 +401,7 @@
         // of file system and return appropriate code. Changing the output
         // string might break testcases. Also note this must be the last line
         // of the report.
-        if (res.isHealthy()) {
+        if (replRes.isHealthy() && ecRes.isHealthy()) {
           out.print("\n\nThe filesystem under path '" + path + "' " + HEALTHY_STATUS);
         } else {
           out.print("\n\nThe filesystem under path '" + path + "' " + CORRUPT_STATUS);
@@ -431,42 +444,49 @@
   }
 
   @VisibleForTesting
-  void check(String parent, HdfsFileStatus file, Result res) throws IOException {
+  void check(String parent, HdfsFileStatus file, Result replRes, Result ecRes)
+      throws IOException {
     String path = file.getFullName(parent);
     if (file.isDir()) {
-      checkDir(path, res);
+      checkDir(path, replRes, ecRes);
       return;
     }
     if (file.isSymlink()) {
       if (showFiles) {
         out.println(path + " <symlink>");
       }
-      res.totalSymlinks++;
+      totalSymlinks++;
       return;
     }
     LocatedBlocks blocks = getBlockLocations(path, file);
     if (blocks == null) { // the file is deleted
       return;
     }
-    collectFileSummary(path, file, res, blocks);
-    collectBlocksSummary(parent, file, res, blocks);
+
+    final Result r = file.getErasureCodingPolicy() != null ? ecRes: replRes;
+    collectFileSummary(path, file, r, blocks);
+    if (showprogress && (replRes.totalFiles + ecRes.totalFiles) % 100 == 0) {
+      out.println();
+      out.flush();
+    }
+    collectBlocksSummary(parent, file, r, blocks);
   }
 
-  private void checkDir(String path, Result res) throws IOException {
+  private void checkDir(String path, Result replRes, Result ecRes) throws IOException {
     if (snapshottableDirs != null && snapshottableDirs.contains(path)) {
       String snapshotPath = (path.endsWith(Path.SEPARATOR) ? path : path
           + Path.SEPARATOR)
           + HdfsConstants.DOT_SNAPSHOT_DIR;
       HdfsFileStatus snapshotFileInfo = namenode.getRpcServer().getFileInfo(
           snapshotPath);
-      check(snapshotPath, snapshotFileInfo, res);
+      check(snapshotPath, snapshotFileInfo, replRes, ecRes);
     }
     byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;
     DirectoryListing thisListing;
     if (showFiles) {
       out.println(path + " <dir>");
     }
-    res.totalDirs++;
+    totalDirs++;
     do {
       assert lastReturnedName != null;
       thisListing = namenode.getRpcServer().getListing(
@@ -476,7 +496,7 @@
       }
       HdfsFileStatus[] files = thisListing.getPartialListing();
       for (int i = 0; i < files.length; i++) {
-        check(path, files[i], res);
+        check(path, files[i], replRes, ecRes);
       }
       lastReturnedName = thisListing.getLastName();
     } while (thisListing.hasMore());
@@ -524,10 +544,6 @@
     } else if (showprogress) {
       out.print('.');
     }
-    if ((showprogress) && res.totalFiles % 100 == 0) {
-      out.println();
-      out.flush();
-    }
   }
 
   private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res,
@@ -558,9 +574,10 @@
 
       final BlockInfo storedBlock = bm.getStoredBlock(
           block.getLocalBlock());
+      final int minReplication = bm.getMinStorageNum(storedBlock);
       // count decommissionedReplicas / decommissioningReplicas
       NumberReplicas numberReplicas = bm.countNodes(storedBlock);
-      int decommissionedReplicas = numberReplicas.decommissioned();;
+      int decommissionedReplicas = numberReplicas.decommissioned();
       int decommissioningReplicas = numberReplicas.decommissioning();
       res.decommissionedReplicas +=  decommissionedReplicas;
       res.decommissioningReplicas += decommissioningReplicas;
@@ -572,11 +589,18 @@
       res.totalReplicas += totalReplicasPerBlock;
 
       // count expected replicas
-      short targetFileReplication = file.getReplication();
+      short targetFileReplication;
+      if (file.getErasureCodingPolicy() != null) {
+        assert storedBlock instanceof BlockInfoStriped;
+        targetFileReplication = ((BlockInfoStriped) storedBlock)
+            .getRealTotalBlockNum();
+      } else {
+        targetFileReplication = file.getReplication();
+      }
       res.numExpectedReplicas += targetFileReplication;
 
       // count under min repl'd blocks
-      if(totalReplicasPerBlock < res.minReplication){
+      if(totalReplicasPerBlock < minReplication){
         res.numUnderMinReplicatedBlocks++;
       }
 
@@ -597,7 +621,7 @@
       }
 
       // count minimally replicated blocks
-      if (totalReplicasPerBlock >= res.minReplication)
+      if (totalReplicasPerBlock >= minReplication)
         res.numMinReplicatedBlocks++;
 
       // count missing replicas / under replicated blocks
@@ -617,7 +641,7 @@
       }
 
       // count mis replicated blocks
-      BlockPlacementStatus blockPlacementStatus = bpPolicy
+      BlockPlacementStatus blockPlacementStatus = bpPolicies.getPolicy(false)
           .verifyBlockPlacement(path, lBlk, targetFileReplication);
       if (!blockPlacementStatus.isPlacementPolicySatisfied()) {
         res.numMisReplicatedBlocks++;
@@ -652,9 +676,9 @@
         report.append(" Live_repl=" + liveReplicas);
         if (showLocations || showRacks || showReplicaDetails) {
           StringBuilder sb = new StringBuilder("[");
-          Iterable<DatanodeStorageInfo> storages = bm.getStorages(block.getLocalBlock());
-          for (Iterator<DatanodeStorageInfo> iterator = storages.iterator(); iterator.hasNext();) {
-            DatanodeStorageInfo storage = iterator.next();
+          DatanodeStorageInfo[] storages = bm.getStorages(storedBlock);
+          for (int i = 0; i < storages.length; i++) {
+            DatanodeStorageInfo storage = storages[i];
             DatanodeDescriptor dnDesc = storage.getDatanodeDescriptor();
             if (showRacks) {
               sb.append(NodeBase.getPath(dnDesc));
@@ -663,7 +687,7 @@
                   .getStorageType()));
             }
             if (showReplicaDetails) {
-              LightWeightLinkedSet<Block> blocksExcess =
+              LightWeightLinkedSet<BlockInfo> blocksExcess =
                   bm.excessReplicateMap.get(dnDesc.getDatanodeUuid());
               Collection<DatanodeDescriptor> corruptReplicas =
                   bm.getCorruptReplicas(block.getLocalBlock());
@@ -684,7 +708,7 @@
                 sb.append("LIVE)");
               }
             }
-            if (iterator.hasNext()) {
+            if (i < storages.length - 1) {
               sb.append(", ");
             }
           }
@@ -999,7 +1023,7 @@
     long missingReplicas = 0L;
     long decommissionedReplicas = 0L;
     long decommissioningReplicas = 0L;
-    long numUnderMinReplicatedBlocks=0L;
+    long numUnderMinReplicatedBlocks = 0L;
     long numOverReplicatedBlocks = 0L;
     long numUnderReplicatedBlocks = 0L;
     long numMisReplicatedBlocks = 0L;  // blocks that do not satisfy block placement policy
@@ -1009,22 +1033,10 @@
     long totalOpenFilesBlocks = 0L;
     long totalFiles = 0L;
     long totalOpenFiles = 0L;
-    long totalDirs = 0L;
-    long totalSymlinks = 0L;
     long totalSize = 0L;
     long totalOpenFilesSize = 0L;
     long totalReplicas = 0L;
 
-    final short replication;
-    final int minReplication;
-
-    Result(Configuration conf) {
-      this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
-                                            DFSConfigKeys.DFS_REPLICATION_DEFAULT);
-      this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
-                                            DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
-    }
-
     /**
      * DFS is considered healthy if there are no missing blocks.
      */
@@ -1050,19 +1062,29 @@
         return 0.0f;
       return (float) (totalReplicas) / (float) totalBlocks;
     }
+  }
+
+  @VisibleForTesting
+  static class ReplicationResult extends Result {
+    final short replication;
+    final short minReplication;
+
+    ReplicationResult(Configuration conf) {
+      this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
+                                            DFSConfigKeys.DFS_REPLICATION_DEFAULT);
+      this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
+                                            DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
+    }
 
     @Override
     public String toString() {
       StringBuilder res = new StringBuilder();
-      res.append("Status: ").append((isHealthy() ? "HEALTHY" : "CORRUPT"))
-          .append("\n Total size:\t").append(totalSize).append(" B");
+      res.append(" Total size:\t").append(totalSize).append(" B");
       if (totalOpenFilesSize != 0) {
         res.append(" (Total open files size: ").append(totalOpenFilesSize)
             .append(" B)");
       }
-      res.append("\n Total dirs:\t").append(totalDirs).append(
-          "\n Total files:\t").append(totalFiles);
-      res.append("\n Total symlinks:\t\t").append(totalSymlinks);
+      res.append("\n Total files:\t").append(totalFiles);
       if (totalOpenFiles != 0) {
         res.append(" (Files currently being written: ").append(totalOpenFiles)
             .append(")");
@@ -1152,4 +1174,110 @@
       return res.toString();
     }
   }
+
+  @VisibleForTesting
+  static class ErasureCodingResult extends Result {
+    final String defaultECPolicy;
+
+    ErasureCodingResult(Configuration conf) {
+      defaultECPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy()
+          .getName();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder res = new StringBuilder();
+      res.append(" Total size:\t").append(totalSize).append(" B");
+      if (totalOpenFilesSize != 0) {
+        res.append(" (Total open files size: ").append(totalOpenFilesSize)
+            .append(" B)");
+      }
+      res.append("\n Total files:\t").append(totalFiles);
+      if (totalOpenFiles != 0) {
+        res.append(" (Files currently being written: ").append(totalOpenFiles)
+            .append(")");
+      }
+      res.append("\n Total block groups (validated):\t").append(totalBlocks);
+      if (totalBlocks > 0) {
+        res.append(" (avg. block group size ").append((totalSize / totalBlocks))
+            .append(" B)");
+      }
+      if (totalOpenFilesBlocks != 0) {
+        res.append(" (Total open file block groups (not validated): ").append(
+            totalOpenFilesBlocks).append(")");
+      }
+      if (corruptFiles > 0 || numUnderMinReplicatedBlocks > 0) {
+        res.append("\n  ********************************");
+        if(numUnderMinReplicatedBlocks>0){
+          res.append("\n  UNRECOVERABLE BLOCK GROUPS:\t").append(numUnderMinReplicatedBlocks);
+          if(totalBlocks>0){
+            res.append(" (").append(
+                ((float) (numUnderMinReplicatedBlocks * 100) / (float) totalBlocks))
+                .append(" %)");
+          }
+        }
+        if(corruptFiles>0) {
+          res.append(
+              "\n  CORRUPT FILES:\t").append(corruptFiles);
+          if (missingSize > 0) {
+            res.append("\n  MISSING BLOCK GROUPS:\t").append(missingIds.size()).append(
+                "\n  MISSING SIZE:\t\t").append(missingSize).append(" B");
+          }
+          if (corruptBlocks > 0) {
+            res.append("\n  CORRUPT BLOCK GROUPS: \t").append(corruptBlocks).append(
+                "\n  CORRUPT SIZE:\t\t").append(corruptSize).append(" B");
+          }
+        }
+        res.append("\n  ********************************");
+      }
+      res.append("\n Minimally erasure-coded block groups:\t").append(
+          numMinReplicatedBlocks);
+      if (totalBlocks > 0) {
+        res.append(" (").append(
+            ((float) (numMinReplicatedBlocks * 100) / (float) totalBlocks))
+            .append(" %)");
+      }
+      res.append("\n Over-erasure-coded block groups:\t")
+          .append(numOverReplicatedBlocks);
+      if (totalBlocks > 0) {
+        res.append(" (").append(
+            ((float) (numOverReplicatedBlocks * 100) / (float) totalBlocks))
+            .append(" %)");
+      }
+      res.append("\n Under-erasure-coded block groups:\t").append(
+          numUnderReplicatedBlocks);
+      if (totalBlocks > 0) {
+        res.append(" (").append(
+            ((float) (numUnderReplicatedBlocks * 100) / (float) totalBlocks))
+            .append(" %)");
+      }
+      res.append("\n Unsatisfactory placement block groups:\t")
+          .append(numMisReplicatedBlocks);
+      if (totalBlocks > 0) {
+        res.append(" (").append(
+            ((float) (numMisReplicatedBlocks * 100) / (float) totalBlocks))
+            .append(" %)");
+      }
+      res.append("\n Default ecPolicy:\t\t").append(defaultECPolicy)
+          .append("\n Average block group size:\t").append(
+          getReplicationFactor()).append("\n Missing block groups:\t\t").append(
+          missingIds.size()).append("\n Corrupt block groups:\t\t").append(
+          corruptBlocks).append("\n Missing internal blocks:\t").append(
+          missingReplicas);
+      if (totalReplicas > 0) {
+        res.append(" (").append(
+            ((float) (missingReplicas * 100) / (float) numExpectedReplicas)).append(
+            " %)");
+      }
+      if (decommissionedReplicas > 0) {
+        res.append("\n Decommissioned internal blocks:\t").append(
+            decommissionedReplicas);
+      }
+      if (decommissioningReplicas > 0) {
+        res.append("\n Decommissioning internal blocks:\t").append(
+            decommissioningReplicas);
+      }
+      return res.toString();
+    }
+  }
 }
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 5bc4033..b1012c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -17,8 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.IOException;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
@@ -49,8 +52,19 @@
 
   void checkOperation(OperationCategory read) throws StandbyException;
 
+  /**
+   * Gets the erasure coding policy for the path
+   * @param src
+   *          - path
+   * @return {@link ErasureCodingPolicy}
+   * @throws IOException
+   */
+  ErasureCodingPolicy getErasureCodingPolicyForPath(String src)
+      throws IOException;
+
   boolean isInSnapshot(BlockInfo blockUC);
 
   CacheManager getCacheManager();
+
   HAContext getHAContext();
-}
\ No newline at end of file
+}
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 06a8219..98deed2 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
@@ -41,9 +41,10 @@
 
   /**
    * 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);
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 cf21411..ae23783 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
@@ -236,19 +236,22 @@
               .toByteArray(), permission, acl, fileInPb.getModificationTime(),
               fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
               fileInPb.getPreferredBlockSize(),
-              (byte)fileInPb.getStoragePolicyID(), xAttrs);
+              (byte)fileInPb.getStoragePolicyID(), xAttrs,
+              fileInPb.getIsStriped());
         }
 
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
             pbf.getFileSize());
         List<BlockProto> bpl = pbf.getBlocksList();
+        // in file diff there can only be contiguous blocks
         BlockInfo[] blocks = new BlockInfo[bpl.size()];
         for(int j = 0, e = bpl.size(); j < e; ++j) {
           Block blk = PBHelperClient.convert(bpl.get(j));
           BlockInfo storedBlock = bm.getStoredBlock(blk);
           if(storedBlock == null) {
-            storedBlock = bm.addBlockCollection(
-                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
+            storedBlock = (BlockInfoContiguous) fsn.getBlockManager()
+                .addBlockCollectionWithCheck(new BlockInfoContiguous(blk,
+                    copy.getFileReplication()), file);
           }
           blocks[j] = storedBlock;
         }
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 fc27ed8..9dcd4d8 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
@@ -22,6 +22,7 @@
 
 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.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -54,7 +55,9 @@
     final FileDiff diff =
         super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
     if (withBlocks) {  // Store blocks if this is the first update
-      diff.setBlocks(iNodeFile.getBlocks());
+      BlockInfo[] blks = iNodeFile.getBlocks();
+      assert blks != null;
+      diff.setBlocks(blks);
     }
   }
 
@@ -114,10 +117,10 @@
       earlierDiff.setBlocks(removedBlocks);
     }
     BlockInfo[] earlierBlocks =
-        (earlierDiff == null ? new BlockInfo[]{} : earlierDiff.getBlocks());
+        (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
     // Find later snapshot (or file itself) with blocks
     BlockInfo[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
-    laterBlocks = (laterBlocks==null) ? file.getBlocks() : laterBlocks;
+    laterBlocks = (laterBlocks == null) ? file.getBlocks() : laterBlocks;
     // Skip blocks, which belong to either the earlier or the later lists
     int i = 0;
     for(; i < removedBlocks.length; i++) {
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..55ae7b6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.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.protocol;
+
+import com.google.common.base.Joiner;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+
+import java.util.Arrays;
+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();
+  }
+
+  /** Block and targets pair */
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  public static class BlockECRecoveryInfo {
+    private final ExtendedBlock block;
+    private final DatanodeInfo[] sources;
+    private DatanodeInfo[] targets;
+    private String[] targetStorageIDs;
+    private StorageType[] targetStorageTypes;
+    private final short[] liveBlockIndices;
+    private final ErasureCodingPolicy ecPolicy;
+
+    public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources,
+        DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices,
+        ErasureCodingPolicy ecPolicy) {
+      this(block, sources, DatanodeStorageInfo
+          .toDatanodeInfos(targetDnStorageInfo), DatanodeStorageInfo
+          .toStorageIDs(targetDnStorageInfo), DatanodeStorageInfo
+          .toStorageTypes(targetDnStorageInfo), liveBlockIndices, ecPolicy);
+    }
+
+    public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources,
+        DatanodeInfo[] targets, String[] targetStorageIDs,
+        StorageType[] targetStorageTypes, short[] liveBlockIndices,
+        ErasureCodingPolicy ecPolicy) {
+      this.block = block;
+      this.sources = sources;
+      this.targets = targets;
+      this.targetStorageIDs = targetStorageIDs;
+      this.targetStorageTypes = targetStorageTypes;
+      this.liveBlockIndices = liveBlockIndices;
+      this.ecPolicy = ecPolicy;
+    }
+
+    public ExtendedBlock getExtendedBlock() {
+      return block;
+    }
+
+    public DatanodeInfo[] getSourceDnInfos() {
+      return sources;
+    }
+
+    public DatanodeInfo[] getTargetDnInfos() {
+      return targets;
+    }
+
+    public String[] getTargetStorageIDs() {
+      return targetStorageIDs;
+    }
+    
+    public StorageType[] getTargetStorageTypes() {
+      return targetStorageTypes;
+    }
+
+    public short[] getLiveBlockIndices() {
+      return liveBlockIndices;
+    }
+    
+    public ErasureCodingPolicy getErasureCodingPolicy() {
+      return ecPolicy;
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().append("BlockECRecoveryInfo(\n  ")
+          .append("Recovering ").append(block).append(" From: ")
+          .append(Arrays.asList(sources)).append(" To: [")
+          .append(Arrays.asList(targets)).append(")\n")
+          .append(" Block Indices: ").append(Arrays.asList(liveBlockIndices))
+          .toString();
+    }
+  }
+
+  public Collection<BlockECRecoveryInfo> getECTasks() {
+    return this.ecTasks;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java
index a985dbd..b6eddb6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.StorageType;
@@ -91,6 +92,35 @@
     }
   }
 
+  public static class StripedBlockWithLocations extends BlockWithLocations {
+    final byte[] indices;
+    final short dataBlockNum;
+    final int cellSize;
+
+    public StripedBlockWithLocations(BlockWithLocations blk, byte[] indices,
+         short dataBlockNum, int cellSize) {
+      super(blk.getBlock(), blk.getDatanodeUuids(), blk.getStorageIDs(),
+          blk.getStorageTypes());
+      Preconditions.checkArgument(
+          blk.getDatanodeUuids().length == indices.length);
+      this.indices = indices;
+      this.dataBlockNum = dataBlockNum;
+      this.cellSize = cellSize;
+    }
+
+    public byte[] getIndices() {
+      return indices;
+    }
+
+    public short getDataBlockNum() {
+      return dataBlockNum;
+    }
+
+    public int getCellSize() {
+      return cellSize;
+    }
+  }
+
   private final BlockWithLocations[] blocks;
 
   /** Constructor with one parameter */
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 dfe0813..add4e736 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 @@
   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_ERASURE_CODING_RECOVERY = 11; // erasure coding recovery command
 
   /** 
    * Register Datanode.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
new file mode 100644
index 0000000..4ed9d0a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.tools.erasurecode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.shell.CommandFactory;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * CLI for the erasure code encoding operations.
+ */
+public class ECCli extends FsShell {
+
+  private final static String usagePrefix =
+      "Usage: hdfs erasurecode [generic options]";
+
+  @Override
+  protected String getUsagePrefix() {
+    return usagePrefix;
+  }
+
+  @Override
+  protected void registerCommands(CommandFactory factory) {
+    factory.registerCommands(ECCommand.class);
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    int res = ToolRunner.run(conf, new ECCli(), args);
+    System.exit(res);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
new file mode 100644
index 0000000..978fe47
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java
@@ -0,0 +1,214 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.tools.erasurecode;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.Command;
+import org.apache.hadoop.fs.shell.CommandFactory;
+import org.apache.hadoop.fs.shell.PathData;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Erasure Coding CLI commands
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class ECCommand extends Command {
+
+  public static void registerCommands(CommandFactory factory) {
+    // Register all commands of Erasure CLI, with a '-' at the beginning in name
+    // of the command.
+    factory.addClass(SetECPolicyCommand.class, "-" + SetECPolicyCommand.NAME);
+    factory.addClass(GetECPolicyCommand.class, "-"
+        + GetECPolicyCommand.NAME);
+    factory.addClass(ListPolicies.class, "-" + ListPolicies.NAME);
+  }
+
+  @Override
+  public String getCommandName() {
+    return getName();
+  }
+
+  @Override
+  protected void run(Path path) throws IOException {
+    throw new RuntimeException("Not suppose to get here");
+  }
+
+  @Deprecated
+  @Override
+  public int runAll() {
+    return run(args);
+  }
+
+  @Override
+  protected void processPath(PathData item) throws IOException {
+    if (!(item.fs instanceof DistributedFileSystem)) {
+      throw new UnsupportedActionException(
+          "Erasure commands are only supported for the HDFS paths");
+    }
+  }
+
+  /**
+   * A command to set the erasure coding policy for a directory, with the name
+   * of the policy.
+   */
+  static class SetECPolicyCommand extends ECCommand {
+    public static final String NAME = "setPolicy";
+    public static final String USAGE = "[-p <policyName>] <path>";
+    public static final String DESCRIPTION = 
+        "Set a specified erasure coding policy to a directory\n"
+        + "Options :\n"
+        + "  -p <policyName> : erasure coding policy name to encode files. "
+        + "If not passed the default policy will be used\n"
+        + "  <path>  : Path to a directory. Under this directory "
+        + "files will be encoded using specified erasure coding policy";
+    private String ecPolicyName;
+    private ErasureCodingPolicy ecPolicy = null;
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      ecPolicyName = StringUtils.popOptionWithArgument("-p", args);
+      if (args.isEmpty()) {
+        throw new HadoopIllegalArgumentException("<path> is missing");
+      }
+      if (args.size() > 1) {
+        throw new HadoopIllegalArgumentException("Too many arguments");
+      }
+    }
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      super.processPath(item);
+      DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
+      try {
+        if (ecPolicyName != null) {
+          ErasureCodingPolicy[] ecPolicies = dfs.getClient().getErasureCodingPolicies();
+          for (ErasureCodingPolicy ecPolicy : ecPolicies) {
+            if (ecPolicyName.equals(ecPolicy.getName())) {
+              this.ecPolicy = ecPolicy;
+              break;
+            }
+          }
+          if (ecPolicy == null) {
+            StringBuilder sb = new StringBuilder();
+            sb.append("Policy '");
+            sb.append(ecPolicyName);
+            sb.append("' does not match any of the supported policies.");
+            sb.append(" Please select any one of ");
+            List<String> ecPolicyNames = new ArrayList<String>();
+            for (ErasureCodingPolicy ecPolicy : ecPolicies) {
+              ecPolicyNames.add(ecPolicy.getName());
+            }
+            sb.append(ecPolicyNames);
+            throw new HadoopIllegalArgumentException(sb.toString());
+          }
+        }
+        dfs.setErasureCodingPolicy(item.path, ecPolicy);
+        out.println("EC policy set successfully at " + item.path);
+      } catch (IOException e) {
+        throw new IOException("Unable to set EC policy for the path "
+            + item.path + ". " + e.getMessage());
+      }
+    }
+  }
+
+  /**
+   * Get the erasure coding policy of a file or directory
+   */
+  static class GetECPolicyCommand extends ECCommand {
+    public static final String NAME = "getPolicy";
+    public static final String USAGE = "<path>";
+    public static final String DESCRIPTION =
+        "Get erasure coding policy information about at specified path\n";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      if (args.isEmpty()) {
+        throw new HadoopIllegalArgumentException("<path> is missing");
+      }
+      if (args.size() > 1) {
+        throw new HadoopIllegalArgumentException("Too many arguments");
+      }
+    }
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      super.processPath(item);
+      DistributedFileSystem dfs = (DistributedFileSystem) item.fs;
+      try {
+        ErasureCodingPolicy ecPolicy = dfs.getErasureCodingPolicy(item.path);
+        if (ecPolicy != null) {
+          out.println(ecPolicy.toString());
+        } else {
+          out.println("Path " + item.path + " is not erasure coded.");
+        }
+      } catch (IOException e) {
+        throw new IOException("Unable to get EC policy for the path "
+            + item.path + ". " + e.getMessage());
+      }
+    }
+  }
+
+  /**
+   * List all supported erasure coding policies
+   */
+  static class ListPolicies extends ECCommand {
+    public static final String NAME = "listPolicies";
+    public static final String USAGE = "";
+    public static final String DESCRIPTION = 
+        "Get the list of erasure coding policies supported\n";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) throws IOException {
+      if (!args.isEmpty()) {
+        throw new HadoopIllegalArgumentException("Too many parameters");
+      }
+
+      FileSystem fs = FileSystem.get(getConf());
+      if (fs instanceof DistributedFileSystem == false) {
+        throw new UnsupportedActionException(
+            "Erasure commands are only supported for the HDFS");
+      }
+      DistributedFileSystem dfs = (DistributedFileSystem) fs;
+
+      ErasureCodingPolicy[] ecPolicies = dfs.getClient().getErasureCodingPolicies();
+      StringBuilder sb = new StringBuilder();
+      int i = 0;
+      while (i < ecPolicies.length) {
+        ErasureCodingPolicy ecPolicy = ecPolicies[i];
+        sb.append(ecPolicy.getName());
+        i++;
+        if (i < ecPolicies.length) {
+          sb.append(", ");
+        }
+      }
+      out.println(sb.toString());
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 727259f..6dd7b89 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -33,6 +33,7 @@
 
 import "HAServiceProtocol.proto";
 import "hdfs.proto";
+import "erasurecoding.proto";
 import "HdfsServer.proto";
 
 /**
@@ -59,6 +60,7 @@
     UnusedUpgradeCommand = 6;
     NullDatanodeCommand = 7;
     BlockIdCommand = 8;
+    BlockECRecoveryCommand = 9;
   }
 
   required Type cmdType = 1;    // Type of the command
@@ -72,6 +74,7 @@
   optional KeyUpdateCommandProto keyUpdateCmd = 6;
   optional RegisterCommandProto registerCmd = 7;
   optional BlockIdCommandProto blkIdCmd = 8;
+  optional BlockECRecoveryCommandProto blkECRecoveryCmd = 9;
 }
 
 /**
@@ -146,6 +149,13 @@
 }
 
 /**
+ * Block Erasure coding recovery command
+ */
+message BlockECRecoveryCommandProto {
+  repeated BlockECRecoveryInfoProto blockECRecoveryinfo = 1;
+}
+
+/**
  * registration - Information of the datanode registering with the namenode
  */
 message RegisterDatanodeRequestProto {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
index 3b60e51..66b2a33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
@@ -34,13 +34,6 @@
 import "hdfs.proto";
 
 /**
- * A list of storage IDs.
- */
-message StorageUuidsProto {
-  repeated string storageUuids = 1;
-}
-
-/**
  * Block access token information
  */
 message BlockKeyProto {
@@ -68,6 +61,10 @@
   repeated string datanodeUuids = 2; // Datanodes with replicas of the block
   repeated string storageUuids = 3;  // Storages with replicas of the block
   repeated StorageTypeProto storageTypes = 4;
+
+  optional bytes indices = 5;
+  optional uint32 dataBlockNum = 6;
+  optional uint32 cellSize = 7;
 }
 
 /**
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..43a1fc5 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 @@
   optional uint64 lastAllocatedBlockId = 5;
   optional uint64 transactionId = 6;
   optional uint64 rollingUpgradeStartTime = 7;
+  optional uint64 lastAllocatedStripedBlockId = 8;
 }
 
 /**
@@ -139,6 +140,7 @@
     optional AclFeatureProto acl = 8;
     optional XAttrFeatureProto xAttrs = 9;
     optional uint32 storagePolicyID = 10;
+    optional bool isStriped = 11;
   }
 
   message QuotaByStorageTypeEntryProto {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 77b556e..1475ecd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2413,11 +2413,11 @@
   </description>
 </property>
 
-  <property>
-    <name>dfs.datanode.block-pinning.enabled</name>
-    <value>false</value>
-    <description>Whether pin blocks on favored DataNode.</description>
-  </property>
+<property>
+  <name>dfs.datanode.block-pinning.enabled</name>
+  <value>false</value>
+  <description>Whether pin blocks on favored DataNode.</description>
+</property>
 
 <property>
   <name>dfs.client.block.write.locateFollowingBlock.initial.delay.ms</name>
@@ -2439,6 +2439,27 @@
 </property>
 
 <property>
+  <name>dfs.datanode.stripedread.timeout.millis</name>
+  <value>5000</value>
+  <description>Datanode striped read timeout in milliseconds.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.stripedread.threads</name>
+  <value>20</value>
+  <description>Number of threads used by the Datanode for background recovery work.
+  </description>
+</property>
+
+<property>
+  <name>dfs.datanode.stripedread.buffer.size</name>
+  <value>262144</value>
+  <description>Datanode striped read buffer size.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.quota.init-threads</name>
   <value>4</value>
   <description>
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 718463a..604fde4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -36,6 +36,7 @@
     * [crypto](#crypto)
     * [datanode](#datanode)
     * [dfsadmin](#dfsadmin)
+    * [erasurecode](#erasurecode)
     * [haadmin](#haadmin)
     * [journalnode](#journalnode)
     * [mover](#mover)
@@ -383,6 +384,25 @@
 
 Runs a HDFS dfsadmin client.
 
+### `erasurecode`
+
+Usage:
+
+       hdfs erasurecode [generic options]
+         [-setPolicy [-s <policyName>] <path>]
+         [-getPolicy <path>]
+         [-listPolicies]
+         [-usage [cmd ...]]
+         [-help [cmd ...]]
+
+| COMMAND\_OPTION | Description |
+|:---- |:---- |
+|-setPolicy| Set a specified ErasureCoding policy to a directory|
+|-getPolicy| Get ErasureCoding policy information about a specified path|
+|-listPolicies| Lists all supported ErasureCoding policies|
+
+Runs the ErasureCoding CLI. See [HDFS ErasureCoding](./HDFSErasureCoding.html#Administrative_commands) for more information on this command.
+
 ### `haadmin`
 
 Usage:
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
new file mode 100644
index 0000000..5b3aa34
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -0,0 +1,135 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+HDFS Erasure Coding
+===================
+
+* [HDFS Erasure Coding](#HDFS_Erasure_Coding)
+    * [Purpose](#Purpose)
+    * [Background](#Background)
+    * [Architecture](#Architecture)
+    * [Deployment](#Deployment)
+        * [Cluster and hardware configuration](#Cluster_and_hardware_configuration)
+        * [Configuration keys](#Configuration_keys)
+        * [Administrative commands](#Administrative_commands)
+
+Purpose
+-------
+  Replication is expensive -- the default 3x replication scheme in HDFS has 200% overhead in storage space and other resources (e.g., network bandwidth).
+  However, for warm and cold datasets with relatively low I/O activities, additional block replicas are rarely accessed during normal operations, but still consume the same amount of resources as the first replica.
+
+  Therefore, a natural improvement is to use Erasure Coding (EC) in place of replication, which provides the same level of fault-tolerance with much less storage space. In typical Erasure Coding (EC) setups, the storage overhead is no more than 50%.
+
+Background
+----------
+
+  In storage systems, the most notable usage of EC is Redundant Array of Inexpensive Disks (RAID). RAID implements EC through striping, which divides logically sequential data (such as a file) into smaller units (such as bit, byte, or block) and stores consecutive units on different disks. In the rest of this guide this unit of striping distribution is termed a striping cell (or cell). For each stripe of original data cells, a certain number of parity cells are calculated and stored -- the process of which is called encoding. The error on any striping cell can be recovered through decoding calculation based on surviving data and parity cells.
+
+  Integrating EC with HDFS can improve storage efficiency while still providing similar data durability as traditional replication-based HDFS deployments.
+  As an example, a 3x replicated file with 6 blocks will consume 6*3 = 18 blocks of disk space. But with EC (6 data, 3 parity) deployment, it will only consume 9 blocks of disk space.
+
+Architecture
+------------
+  In the context of EC, striping has several critical advantages. First, it enables online EC (writing data immediately in EC format), avoiding a conversion phase and immediately saving storage space. Online EC also enhances sequential I/O performance by leveraging multiple disk spindles in parallel; this is especially desirable in clusters with high end networking. Second, it naturally distributes a small file to multiple DataNodes and eliminates the need to bundle multiple files into a single coding group. This greatly simplifies file operations such as deletion, quota reporting, and migration between federated namespaces.
+
+  In typical HDFS clusters, small files can account for over 3/4 of total storage consumption. To better support small files, in this first phase of work HDFS supports EC with striping. In the future, HDFS will also support a contiguous EC layout. See the design doc and discussion on [HDFS-7285](https://issues.apache.org/jira/browse/HDFS-7285) for more information.
+
+ *  **NameNode Extensions** - Striped HDFS files are logically composed of block groups, each of which contains a certain number of internal blocks.
+    To reduce NameNode memory consumption from these additional blocks, a new hierarchical block naming protocol was introduced. The ID of a block group can be inferred from the ID of any of its internal blocks. This allows management at the level of the block group rather than the block.
+
+ *  **Client Extensions** - The client read and write paths were enhanced to work on multiple internal blocks in a block group in parallel.
+    On the output / write path, DFSStripedOutputStream manages a set of data streamers, one for each DataNode storing an internal block in the current block group. The streamers mostly
+    work asynchronously. A coordinator takes charge of operations on the entire block group, including ending the current block group, allocating a new block group, and so forth.
+    On the input / read path, DFSStripedInputStream translates a requested logical byte range of data as ranges into internal blocks stored on DataNodes. It then issues read requests in
+    parallel. Upon failures, it issues additional read requests for decoding.
+
+ *  **DataNode Extensions** - The DataNode runs an additional ErasureCodingWorker (ECWorker) task for background recovery of failed erasure coded blocks. Failed EC blocks are detected by the NameNode, which then chooses a DataNode to do the recovery work. The recovery task is passed as a heartbeat response. This process is similar to how replicated blocks are re-replicated on failure. Reconstruction performs three key tasks:
+
+      1. _Read the data from source nodes:_ Input data is read in parallel from source nodes using a dedicated thread pool.
+        Based on the EC policy, it schedules the read requests to all source targets and reads only the minimum number of input blocks for reconstruction.
+
+      1. _Decode the data and generate the output data:_ New data and parity blocks are decoded from the input data. All missing data and parity blocks are decoded together.
+
+      1. _Transfer the generated data blocks to target nodes:_ Once decoding is finished, the recovered blocks are transferred to target DataNodes.
+
+ *  **ErasureCoding policy**
+    To accommodate heterogeneous workloads, we allow files and directories in an HDFS cluster to have different replication and EC policies.
+    Information on how to encode/decode a file is encapsulated in an ErasureCodingPolicy class. Each policy is defined by the following 2 pieces of information:
+
+      1. _The ECSchema:_ This includes the numbers of data and parity blocks in an EC group (e.g., 6+3), as well as the codec algorithm (e.g., Reed-Solomon).
+
+      1. _The size of a striping cell._ This determines the granularity of striped reads and writes, including buffer sizes and encoding work.
+
+    Currently, HDFS supports the Reed-Solomon and XOR erasure coding algorithms. Additional algorithms are planned as future work.
+    The system default scheme is Reed-Solomon (6, 3) with a cell size of 64KB.
+
+
+Deployment
+----------
+
+### Cluster and hardware configuration
+
+  Erasure coding places additional demands on the cluster in terms of CPU and network.
+
+  Encoding and decoding work consumes additional CPU on both HDFS clients and DataNodes.
+
+  Erasure coded files are also spread across racks for rack fault-tolerance.
+  This means that when reading and writing striped files, most operations are off-rack.
+  Network bisection bandwidth is thus very important.
+
+  For rack fault-tolerance, it is also important to have at least as many racks as the configured EC stripe width.
+  For the default EC policy of RS (6,3), this means minimally 9 racks, and ideally 10 or 11 to handle planned and unplanned outages.
+  For clusters with fewer racks than the stripe width, HDFS cannot maintain rack fault-tolerance, but will still attempt
+  to spread a striped file across multiple nodes to preserve node-level fault-tolerance.
+
+### Configuration keys
+
+  The codec implementation for Reed-Solomon and XOR can be configured with the following client and DataNode configuration keys:
+  `io.erasurecode.codec.rs.rawcoder` and `io.erasurecode.codec.xor.rawcoder`.
+  The default implementations for both of these codecs are pure Java.
+
+  Erasure coding background recovery work on the DataNodes can also be tuned via the following configuration parameters:
+
+  1. `dfs.datanode.stripedread.timeout.millis` - Timeout for striped reads. Default value is 5000 ms.
+  1. `dfs.datanode.stripedread.threads` - Number of concurrent reader threads. Default value is 20 threads.
+  1. `dfs.datanode.stripedread.buffer.size` - Buffer size for reader service. Default value is 256KB.
+
+### Administrative commands
+
+  HDFS provides an `erasurecode` subcommand to perform administrative commands related to erasure coding.
+
+       hdfs erasurecode [generic options]
+         [-setPolicy [-p <policyName>] <path>]
+         [-getPolicy <path>]
+         [-listPolicies]
+         [-usage [cmd ...]]
+         [-help [cmd ...]]
+
+Below are the details about each command.
+
+ *  `[-setPolicy [-p <policyName>] <path>]`
+
+    Sets an ErasureCoding policy on a directory at the specified path.
+
+      `path`: An directory in HDFS. This is a mandatory parameter. Setting a policy only affects newly created files, and does not affect existing files.
+
+      `policyName`: The ErasureCoding policy to be used for files under this directory. This is an optional parameter, specified using ‘-s’ flag. If no policy is specified, the system default ErasureCodingPolicy will be used.
+
+ *  `[-getPolicy <path>]`
+
+     Get details of the ErasureCoding policy of a file or directory at the specified path.
+
+ *  `[-listPolicies]`
+
+     Lists all supported ErasureCoding policies. These names are suitable for use with the `setPolicy` command.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java
new file mode 100644
index 0000000..0499a2b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.cli;
+
+import org.apache.hadoop.cli.util.CLICommandErasureCodingCli;
+import org.apache.hadoop.cli.util.CLICommandTypes;
+import org.apache.hadoop.cli.util.CLITestCmd;
+import org.apache.hadoop.cli.util.CommandExecutor;
+import org.apache.hadoop.cli.util.ErasureCodingCliCmdExecutor;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.tools.erasurecode.ECCli;
+
+public class CLITestCmdErasureCoding extends CLITestCmd {
+  public CLITestCmdErasureCoding(String str, CLICommandTypes type) {
+    super(str, type);
+  }
+
+  @Override
+  public CommandExecutor getExecutor(String tag, Configuration conf) throws IllegalArgumentException {
+    if (getType() instanceof CLICommandErasureCodingCli)
+      return new ErasureCodingCliCmdExecutor(tag, new ECCli());
+    return super.getExecutor(tag, conf);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
new file mode 100644
index 0000000..29ec98e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
@@ -0,0 +1,115 @@
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.cli;
+
+import org.apache.hadoop.cli.util.CLICommand;
+import org.apache.hadoop.cli.util.CLICommandErasureCodingCli;
+import org.apache.hadoop.cli.util.CommandExecutor.Result;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.xml.sax.SAXException;
+
+public class TestErasureCodingCLI extends CLITestHelper {
+  private final int NUM_OF_DATANODES = 3;
+  private MiniDFSCluster dfsCluster = null;
+  private FileSystem fs = null;
+  private String namenode = null;
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+
+    dfsCluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(NUM_OF_DATANODES).build();
+    dfsCluster.waitClusterUp();
+    namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///");
+
+    username = System.getProperty("user.name");
+
+    fs = dfsCluster.getFileSystem();
+  }
+
+  @Override
+  protected String getTestFile() {
+    return "testErasureCodingConf.xml";
+  }
+
+  @After
+  @Override
+  public void tearDown() throws Exception {
+    if (fs != null) {
+      fs.close();
+    }
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+    }
+    Thread.sleep(2000);
+    super.tearDown();
+  }
+
+  @Override
+  protected String expandCommand(final String cmd) {
+    String expCmd = cmd;
+    expCmd = expCmd.replaceAll("NAMENODE", namenode);
+    expCmd = expCmd.replaceAll("#LF#", System.getProperty("line.separator"));
+    expCmd = super.expandCommand(expCmd);
+    return expCmd;
+  }
+
+  @Override
+  protected TestConfigFileParser getConfigParser() {
+    return new TestErasureCodingAdmin();
+  }
+
+  private class TestErasureCodingAdmin extends
+      CLITestHelper.TestConfigFileParser {
+    @Override
+    public void endElement(String uri, String localName, String qName)
+        throws SAXException {
+      if (qName.equals("ec-admin-command")) {
+        if (testCommands != null) {
+          testCommands.add(new CLITestCmdErasureCoding(charString,
+              new CLICommandErasureCodingCli()));
+        } else if (cleanupCommands != null) {
+          cleanupCommands.add(new CLITestCmdErasureCoding(charString,
+              new CLICommandErasureCodingCli()));
+        }
+      } else {
+        super.endElement(uri, localName, qName);
+      }
+    }
+  }
+
+  @Override
+  protected Result execute(CLICommand cmd) throws Exception {
+    return cmd.getExecutor(namenode, conf).executeCommand(cmd.getCmd());
+  }
+
+  @Test
+  @Override
+  public void testAll() {
+    super.testAll();
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java
new file mode 100644
index 0000000..aafcd9f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.cli.util;
+
+public class CLICommandErasureCodingCli implements CLICommandTypes {
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java
new file mode 100644
index 0000000..e993313
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.cli.util;
+
+import org.apache.hadoop.hdfs.tools.erasurecode.ECCli;
+import org.apache.hadoop.util.ToolRunner;
+
+public class ErasureCodingCliCmdExecutor extends CommandExecutor {
+  protected String namenode = null;
+  protected ECCli admin = null;
+
+  public ErasureCodingCliCmdExecutor(String namenode, ECCli admin) {
+    this.namenode = namenode;
+    this.admin = admin;
+  }
+
+  @Override
+  protected void execute(final String cmd) throws Exception {
+    String[] args = getCommandAsArgs(cmd, "NAMENODE", this.namenode);
+    ToolRunner.run(admin, args);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
index ba25d97..1ca1ca5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
@@ -165,20 +165,19 @@
    */
   public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToRead)
       throws IOException {
-    return getBlockReader(cluster, testBlock, offset, lenToRead);
+    return getBlockReader(cluster.getFileSystem(), testBlock, offset, lenToRead);
   }
 
   /**
    * Get a BlockReader for the given block.
    */
-  public static BlockReader getBlockReader(MiniDFSCluster cluster,
-      LocatedBlock testBlock, int offset, int lenToRead) throws IOException {
+  public static BlockReader getBlockReader(final DistributedFileSystem fs,
+      LocatedBlock testBlock, int offset, long lenToRead) throws IOException {
     InetSocketAddress targetAddr = null;
     ExtendedBlock block = testBlock.getBlock();
     DatanodeInfo[] nodes = testBlock.getLocations();
     targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr());
 
-    final DistributedFileSystem fs = cluster.getFileSystem();
     return new BlockReaderFactory(fs.getClient().getConf()).
       setInetSocketAddress(targetAddr).
       setBlock(block).
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 bce8b64..155abd0 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
@@ -66,6 +66,12 @@
 import java.util.UUID;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.UnhandledException;
@@ -106,6 +112,7 @@
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 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.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -125,14 +132,19 @@
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.hdfs.tools.JMXGet;
 import org.apache.hadoop.io.IOUtils;
@@ -154,12 +166,8 @@
 import org.mockito.internal.util.reflection.Whitebox;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Charsets;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
 
 /** Utilities for HDFS tests */
 public class DFSTestUtil {
@@ -809,15 +817,21 @@
     return os.toByteArray();
   }
 
-  /* Write the given string to the given file */
-  public static void writeFile(FileSystem fs, Path p, String s) 
+  /* Write the given bytes to the given file */
+  public static void writeFile(FileSystem fs, Path p, byte[] bytes)
       throws IOException {
     if (fs.exists(p)) {
       fs.delete(p, true);
     }
-    InputStream is = new ByteArrayInputStream(s.getBytes());
+    InputStream is = new ByteArrayInputStream(bytes);
     FSDataOutputStream os = fs.create(p);
-    IOUtils.copyBytes(is, os, s.length(), true);
+    IOUtils.copyBytes(is, os, bytes.length, true);
+  }
+
+  /* Write the given string to the given file */
+  public static void writeFile(FileSystem fs, Path p, String s)
+      throws IOException {
+    writeFile(fs, p, s.getBytes());
   }
 
   /* Append the given string to the given file */
@@ -1834,7 +1848,7 @@
     dn.setLastUpdate(Time.now() + offset);
     dn.setLastUpdateMonotonic(Time.monotonicNow() + offset);
   }
-
+  
   /**
    * This method takes a set of block locations and fills the provided buffer
    * with expected bytes based on simulated content from
@@ -1858,6 +1872,132 @@
     }
   }
 
+  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;
+  }
+
+  /**
+   * Creates the metadata of a file in striped layout. This method only
+   * manipulates the NameNode state without injecting data to DataNode.
+   * You should disable periodical heartbeat before use this.
+   *  @param file Path of the file to create
+   * @param dir Parent path of the file
+   * @param numBlocks Number of striped block groups to add to the file
+   * @param numStripesPerBlk Number of striped cells in each block
+   * @param toMkdir
+   */
+  public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir,
+      int numBlocks, int numStripesPerBlk, boolean toMkdir) throws Exception {
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    // If outer test already set EC policy, dir should be left as null
+    if (toMkdir) {
+      assert dir != null;
+      dfs.mkdirs(dir);
+      try {
+        dfs.getClient().setErasureCodingPolicy(dir.toString(), null);
+      } catch (IOException e) {
+        if (!e.getMessage().contains("non-empty directory")) {
+          throw e;
+        }
+      }
+    }
+
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1); // create an empty file
+
+      FSNamesystem ns = cluster.getNamesystem();
+      FSDirectory fsdir = ns.getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+
+      ExtendedBlock previous = null;
+      for (int i = 0; i < numBlocks; i++) {
+        Block newBlock = addStripedBlockToFile(cluster.getDataNodes(), dfs, ns,
+            file.toString(), fileNode, dfs.getClient().getClientName(),
+            previous, numStripesPerBlk);
+        previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
+      }
+
+      dfs.getClient().namenode.complete(file.toString(),
+          dfs.getClient().getClientName(), previous, fileNode.getId());
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+  }
+
+  /**
+   * Adds a striped block group to a file. This method only manipulates NameNode
+   * states of the file and the block without injecting data to DataNode.
+   * It does mimic block reports.
+   * You should disable periodical heartbeat before use this.
+   * @param dataNodes List DataNodes to host the striped block group
+   * @param previous Previous block in the file
+   * @param numStripes Number of stripes in each block group
+   * @return The added block group
+   */
+  public static Block addStripedBlockToFile(List<DataNode> dataNodes,
+      DistributedFileSystem fs, FSNamesystem ns, String file, INodeFile fileNode,
+      String clientName, ExtendedBlock previous, int numStripes)
+      throws Exception {
+    fs.getClient().namenode.addBlock(file, clientName, previous, null,
+        fileNode.getId(), null);
+
+    final BlockInfo lastBlock = fileNode.getLastBlock();
+    final int groupSize = fileNode.getPreferredBlockReplication();
+    assert dataNodes.size() >= groupSize;
+    // 1. RECEIVING_BLOCK IBR
+    for (int i = 0; i < groupSize; i++) {
+      DataNode dn = dataNodes.get(i);
+      final Block block = new Block(lastBlock.getBlockId() + i, 0,
+          lastBlock.getGenerationStamp());
+      DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+      StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+          .makeReportForReceivedBlock(block,
+              ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
+      for (StorageReceivedDeletedBlocks report : reports) {
+        ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
+      }
+    }
+
+    // 2. RECEIVED_BLOCK IBR
+    for (int i = 0; i < groupSize; i++) {
+      DataNode dn = dataNodes.get(i);
+      final Block block = new Block(lastBlock.getBlockId() + i,
+          numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp());
+      DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+      StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+          .makeReportForReceivedBlock(block,
+              ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+      for (StorageReceivedDeletedBlocks report : reports) {
+        ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
+      }
+    }
+
+    lastBlock.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS);
+    return lastBlock;
+  }
+
+  /**
+   * Because currently DFSStripedOutputStream does not support hflush/hsync,
+   * tests can use this method to flush all the buffered data to DataNodes.
+   */
+  public static ExtendedBlock flushInternal(DFSStripedOutputStream out)
+      throws IOException {
+    out.flushAllInternals();
+    return out.getBlock();
+  }
+
+  public static ExtendedBlock flushBuffer(DFSStripedOutputStream out)
+      throws IOException {
+    out.flush();
+    return out.getBlock();
+  }
+
   public static void waitForMetric(final JMXGet jmx, final String metricName, final int expectedValue)
       throws TimeoutException, InterruptedException {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 40414c8..71a4bd2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -121,6 +121,7 @@
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.ToolRunner;
 
 import com.google.common.base.Joiner;
@@ -2115,8 +2116,6 @@
     int node = -1;
     for (int i = 0; i < dataNodes.size(); i++) {
       DataNode dn = dataNodes.get(i).datanode;
-      LOG.info("DN name=" + dnName + " found DN=" + dn +
-          " with name=" + dn.getDisplayName());
       if (dnName.equals(dn.getDatanodeId().getXferAddr())) {
         node = i;
         break;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
new file mode 100644
index 0000000..12453fa
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
@@ -0,0 +1,491 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import com.google.common.base.Joiner;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.hdfs.web.ByteRangeInputStream;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+import org.junit.Assert;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.assertEquals;
+
+public class StripedFileTestUtil {
+  public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class);
+  /*
+   * These values correspond to the values used by the system default erasure
+   * coding policy.
+   */
+  public static final short NUM_DATA_BLOCKS = (short) 6;
+  public static final short NUM_PARITY_BLOCKS = (short) 3;
+  public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024;
+
+  static final int stripesPerBlock = 4;
+  static final int blockSize = BLOCK_STRIPED_CELL_SIZE * stripesPerBlock;
+  static final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS + 2;
+  static final int BLOCK_GROUP_SIZE = blockSize * NUM_DATA_BLOCKS;
+
+
+  static byte[] generateBytes(int cnt) {
+    byte[] bytes = new byte[cnt];
+    for (int i = 0; i < cnt; i++) {
+      bytes[i] = getByte(i);
+    }
+    return bytes;
+  }
+
+  static byte getByte(long pos) {
+    final int mod = 29;
+    return (byte) (pos % mod + 1);
+  }
+
+  static int readAll(FSDataInputStream in, byte[] buf) throws IOException {
+    int readLen = 0;
+    int ret;
+    while ((ret = in.read(buf, readLen, buf.length - readLen)) >= 0 &&
+        readLen <= buf.length) {
+      readLen += ret;
+    }
+    return readLen;
+  }
+
+  static void verifyLength(FileSystem fs, Path srcPath, int fileLength)
+      throws IOException {
+    FileStatus status = fs.getFileStatus(srcPath);
+    assertEquals("File length should be the same", fileLength, status.getLen());
+  }
+
+  static void verifyPread(FileSystem fs, Path srcPath,  int fileLength,
+      byte[] expected, byte[] buf) throws IOException {
+    try (FSDataInputStream in = fs.open(srcPath)) {
+      int[] startOffsets = {0, 1, BLOCK_STRIPED_CELL_SIZE - 102, BLOCK_STRIPED_CELL_SIZE, BLOCK_STRIPED_CELL_SIZE + 102,
+          BLOCK_STRIPED_CELL_SIZE * (NUM_DATA_BLOCKS - 1), BLOCK_STRIPED_CELL_SIZE * (NUM_DATA_BLOCKS - 1) + 102,
+          BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS, fileLength - 102, fileLength - 1};
+      for (int startOffset : startOffsets) {
+        startOffset = Math.max(0, Math.min(startOffset, fileLength - 1));
+        int remaining = fileLength - startOffset;
+        int offset = startOffset;
+        final byte[] result = new byte[remaining];
+        while (remaining > 0) {
+          int target = Math.min(remaining, buf.length);
+          in.readFully(offset, buf, 0, target);
+          System.arraycopy(buf, 0, result, offset - startOffset, target);
+          remaining -= target;
+          offset += target;
+        }
+        for (int i = 0; i < fileLength - startOffset; i++) {
+          assertEquals("Byte at " + (startOffset + i) + " is different, " + "the startOffset is " + startOffset, expected[startOffset + i], result[i]);
+        }
+      }
+    }
+  }
+
+  static void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength,
+      byte[] expected, byte[] buf) throws IOException {
+    try (FSDataInputStream in = fs.open(srcPath)) {
+      final byte[] result = new byte[fileLength];
+      int readLen = 0;
+      int ret;
+      while ((ret = in.read(buf, 0, buf.length)) >= 0) {
+        System.arraycopy(buf, 0, result, readLen, ret);
+        readLen += ret;
+      }
+      assertEquals("The length of file should be the same to write size", fileLength, readLen);
+      Assert.assertArrayEquals(expected, result);
+    }
+  }
+
+  static void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength,
+      byte[] expected, ByteBuffer buf) throws IOException {
+    try (FSDataInputStream in = fs.open(srcPath)) {
+      ByteBuffer result = ByteBuffer.allocate(fileLength);
+      int readLen = 0;
+      int ret;
+      while ((ret = in.read(buf)) >= 0) {
+        readLen += ret;
+        buf.flip();
+        result.put(buf);
+        buf.clear();
+      }
+      assertEquals("The length of file should be the same to write size", fileLength, readLen);
+      Assert.assertArrayEquals(expected, result.array());
+    }
+  }
+
+  static void verifySeek(FileSystem fs, Path srcPath, int fileLength)
+      throws IOException {
+    try (FSDataInputStream in = fs.open(srcPath)) {
+      // seek to 1/2 of content
+      int pos = fileLength / 2;
+      assertSeekAndRead(in, pos, fileLength);
+
+      // seek to 1/3 of content
+      pos = fileLength / 3;
+      assertSeekAndRead(in, pos, fileLength);
+
+      // seek to 0 pos
+      pos = 0;
+      assertSeekAndRead(in, pos, fileLength);
+
+      if (fileLength > BLOCK_STRIPED_CELL_SIZE) {
+        // seek to cellSize boundary
+        pos = BLOCK_STRIPED_CELL_SIZE - 1;
+        assertSeekAndRead(in, pos, fileLength);
+      }
+
+      if (fileLength > BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS) {
+        // seek to striped cell group boundary
+        pos = BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS - 1;
+        assertSeekAndRead(in, pos, fileLength);
+      }
+
+      if (fileLength > blockSize * NUM_DATA_BLOCKS) {
+        // seek to striped block group boundary
+        pos = blockSize * NUM_DATA_BLOCKS - 1;
+        assertSeekAndRead(in, pos, fileLength);
+      }
+
+      if (!(in.getWrappedStream() instanceof ByteRangeInputStream)) {
+        try {
+          in.seek(-1);
+          Assert.fail("Should be failed if seek to negative offset");
+        } catch (EOFException e) {
+          // expected
+        }
+
+        try {
+          in.seek(fileLength + 1);
+          Assert.fail("Should be failed if seek after EOF");
+        } catch (EOFException e) {
+          // expected
+        }
+      }
+    }
+  }
+
+  static void assertSeekAndRead(FSDataInputStream fsdis, int pos,
+      int writeBytes) throws IOException {
+    fsdis.seek(pos);
+    byte[] buf = new byte[writeBytes];
+    int readLen = StripedFileTestUtil.readAll(fsdis, buf);
+    assertEquals(readLen, writeBytes - pos);
+    for (int i = 0; i < readLen; i++) {
+      assertEquals("Byte at " + i + " should be the same", StripedFileTestUtil.getByte(pos + i), buf[i]);
+    }
+  }
+
+  static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out,
+      final int dnIndex, final AtomicInteger pos) {
+    final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex);
+    final DatanodeInfo datanode = getDatanodes(s);
+    assert datanode != null;
+    LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos);
+    cluster.stopDataNode(datanode.getXferAddr());
+  }
+
+  static DatanodeInfo getDatanodes(StripedDataStreamer streamer) {
+    for(;;) {
+      final DatanodeInfo[] datanodes = streamer.getNodes();
+      if (datanodes != null) {
+        assertEquals(1, datanodes.length);
+        Assert.assertNotNull(datanodes[0]);
+        return datanodes[0];
+      }
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException ignored) {
+        return null;
+      }
+    }
+  }
+
+  /**
+   * If the length of blockGroup is less than a full stripe, it returns the the
+   * number of actual data internal blocks. Otherwise returns NUM_DATA_BLOCKS.
+   */
+  public static short getRealDataBlockNum(int numBytes) {
+    return (short) Math.min(NUM_DATA_BLOCKS,
+        (numBytes - 1) / BLOCK_STRIPED_CELL_SIZE + 1);
+  }
+
+  public static short getRealTotalBlockNum(int numBytes) {
+    return (short) (getRealDataBlockNum(numBytes) + NUM_PARITY_BLOCKS);
+  }
+
+  /**
+   * Wait for all the internalBlocks of the blockGroups of the given file to be reported.
+   */
+  public static void waitBlockGroupsReported(DistributedFileSystem fs, String src)
+      throws IOException, InterruptedException, TimeoutException {
+    boolean success;
+    final int ATTEMPTS = 40;
+    int count = 0;
+
+    do {
+      success = true;
+      count++;
+      LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0);
+      for (LocatedBlock lb : lbs.getLocatedBlocks()) {
+        short expected = getRealTotalBlockNum((int) lb.getBlockSize());
+        int reported = lb.getLocations().length;
+        if (reported != expected){
+          success = false;
+          System.out.println("blockGroup " + lb.getBlock() + " of file " + src
+              + " has reported internalBlocks " + reported
+              + " (desired " + expected + "); locations "
+              + Joiner.on(' ').join(lb.getLocations()));
+          Thread.sleep(1000);
+          break;
+        }
+      }
+      if (success) {
+        System.out.println("All blockGroups of file " + src
+            + " verified to have all internalBlocks.");
+      }
+    } while (!success && count < ATTEMPTS);
+
+    if (count == ATTEMPTS) {
+      throw new TimeoutException("Timed out waiting for " + src +
+          " to have all the internalBlocks");
+    }
+  }
+
+  /**
+   * Generate n random and different numbers within
+   * specified non-negative integer range
+   * @param min minimum of the range
+   * @param max maximum of the range
+   * @param n number to be generated
+   */
+  public static int[] randomArray(int min, int max, int n){
+    if (n > (max - min + 1) || max < min || min < 0 || max < 0) {
+      return null;
+    }
+    int[] result = new int[n];
+    for (int i = 0; i < n; i++) {
+      result[i] = -1;
+    }
+
+    int count = 0;
+    while(count < n) {
+      int num = (int) (Math.random() * (max - min)) + min;
+      boolean flag = true;
+      for (int j = 0; j < n; j++) {
+        if(num == result[j]){
+          flag = false;
+          break;
+        }
+      }
+      if(flag){
+        result[count] = num;
+        count++;
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Verify that blocks in striped block group are on different nodes, and every
+   * internal blocks exists.
+   */
+  public static void verifyLocatedStripedBlocks(LocatedBlocks lbs, int groupSize) {
+    for (LocatedBlock lb : lbs.getLocatedBlocks()) {
+      assert lb instanceof LocatedStripedBlock;
+      HashSet<DatanodeInfo> locs = new HashSet<>();
+      Collections.addAll(locs, lb.getLocations());
+      assertEquals(groupSize, lb.getLocations().length);
+      assertEquals(groupSize, locs.size());
+
+      // verify that every internal blocks exists
+      int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices();
+      assertEquals(groupSize, blockIndices.length);
+      HashSet<Integer> found = new HashSet<>();
+      for (int index : blockIndices) {
+        assert index >=0;
+        found.add(index);
+      }
+      assertEquals(groupSize, found.size());
+    }
+  }
+
+  static void checkData(DistributedFileSystem dfs, Path srcPath, int length,
+      int[] killedDnIndex, long oldGS) throws IOException {
+
+    StripedFileTestUtil.verifyLength(dfs, srcPath, length);
+    Arrays.sort(killedDnIndex);
+    List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
+    LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(srcPath.toString(), 0L,
+        Long.MAX_VALUE);
+    int expectedNumGroup = 0;
+    if (length > 0) {
+      expectedNumGroup = (length - 1) / BLOCK_GROUP_SIZE + 1;
+    }
+    assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size());
+
+    for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
+      Assert.assertTrue(firstBlock instanceof LocatedStripedBlock);
+
+      final long gs = firstBlock.getBlock().getGenerationStamp();
+      final String s = "gs=" + gs + ", oldGS=" + oldGS;
+      LOG.info(s);
+      Assert.assertTrue(s, gs >= oldGS);
+
+      LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
+          (LocatedStripedBlock) firstBlock, BLOCK_STRIPED_CELL_SIZE,
+          NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+      blockGroupList.add(Arrays.asList(blocks));
+    }
+
+    // test each block group
+    for (int group = 0; group < blockGroupList.size(); group++) {
+      final boolean isLastGroup = group == blockGroupList.size() - 1;
+      final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE
+          : length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE;
+      final int numCellInGroup = (groupSize - 1)/BLOCK_STRIPED_CELL_SIZE + 1;
+      final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS;
+      final int lastCellSize = groupSize - (numCellInGroup - 1)*BLOCK_STRIPED_CELL_SIZE;
+
+      //get the data of this block
+      List<LocatedBlock> blockList = blockGroupList.get(group);
+      byte[][] dataBlockBytes = new byte[NUM_DATA_BLOCKS][];
+      byte[][] parityBlockBytes = new byte[NUM_PARITY_BLOCKS][];
+
+      // for each block, use BlockReader to read data
+      for (int i = 0; i < blockList.size(); i++) {
+        final int j = i >= NUM_DATA_BLOCKS? 0: i;
+        final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS
+            + (j <= lastCellIndex? 1: 0);
+        final int blockSize = numCellInBlock*BLOCK_STRIPED_CELL_SIZE
+            + (isLastGroup && j == lastCellIndex? lastCellSize - BLOCK_STRIPED_CELL_SIZE: 0);
+
+        final byte[] blockBytes = new byte[blockSize];
+        if (i < NUM_DATA_BLOCKS) {
+          dataBlockBytes[i] = blockBytes;
+        } else {
+          parityBlockBytes[i - NUM_DATA_BLOCKS] = blockBytes;
+        }
+
+        final LocatedBlock lb = blockList.get(i);
+        LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock
+            + ", blockSize=" + blockSize + ", lb=" + lb);
+        if (lb == null) {
+          continue;
+        }
+        final ExtendedBlock block = lb.getBlock();
+        assertEquals(blockSize, block.getNumBytes());
+
+        if (block.getNumBytes() == 0) {
+          continue;
+        }
+
+        if (Arrays.binarySearch(killedDnIndex, i) < 0) {
+          final BlockReader blockReader = BlockReaderTestUtil.getBlockReader(
+              dfs, lb, 0, block.getNumBytes());
+          blockReader.readAll(blockBytes, 0, (int) block.getNumBytes());
+          blockReader.close();
+        }
+      }
+
+      // check data
+      final int groupPosInFile = group*BLOCK_GROUP_SIZE;
+      for (int i = 0; i < dataBlockBytes.length; i++) {
+        boolean killed = false;
+        if (Arrays.binarySearch(killedDnIndex, i) >= 0){
+          killed = true;
+        }
+        final byte[] actual = dataBlockBytes[i];
+        for (int posInBlk = 0; posInBlk < actual.length; posInBlk++) {
+          final long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(
+              BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, posInBlk, i) + groupPosInFile;
+          Assert.assertTrue(posInFile < length);
+          final byte expected = getByte(posInFile);
+
+          if (killed) {
+            actual[posInBlk] = expected;
+          } else {
+            if(expected != actual[posInBlk]){
+              String s = "expected=" + expected + " but actual=" + actual[posInBlk]
+                  + ", posInFile=" + posInFile + ", posInBlk=" + posInBlk
+                  + ". group=" + group + ", i=" + i;
+              Assert.fail(s);
+            }
+          }
+        }
+      }
+
+      // check parity
+      verifyParityBlocks(dfs.getConf(), lbs.getLocatedBlocks().get(group)
+              .getBlockSize(),
+          BLOCK_STRIPED_CELL_SIZE, dataBlockBytes, parityBlockBytes, killedDnIndex);
+    }
+  }
+
+  static void verifyParityBlocks(Configuration conf, final long size, final int cellSize,
+      byte[][] dataBytes, byte[][] parityBytes, int[] killedDnIndex) {
+    Arrays.sort(killedDnIndex);
+    // verify the parity blocks
+    int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength(
+        size, cellSize, dataBytes.length, dataBytes.length);
+    final byte[][] expectedParityBytes = new byte[parityBytes.length][];
+    for (int i = 0; i < parityBytes.length; i++) {
+      expectedParityBytes[i] = new byte[parityBlkSize];
+    }
+    for (int i = 0; i < dataBytes.length; i++) {
+      if (dataBytes[i] == null) {
+        dataBytes[i] = new byte[dataBytes[0].length];
+      } else if (dataBytes[i].length < dataBytes[0].length) {
+        final byte[] tmp = dataBytes[i];
+        dataBytes[i] = new byte[dataBytes[0].length];
+        System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length);
+      }
+    }
+    final RawErasureEncoder encoder =
+        CodecUtil.createRSRawEncoder(conf, dataBytes.length, parityBytes.length);
+    encoder.encode(dataBytes, expectedParityBytes);
+    for (int i = 0; i < parityBytes.length; i++) {
+      if (Arrays.binarySearch(killedDnIndex, dataBytes.length + i) < 0){
+        Assert.assertArrayEquals("i=" + i + ", killedDnIndex=" + Arrays.toString(killedDnIndex),
+            expectedParityBytes[i], parityBytes[i]);
+      }
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
index a8ca9c7..a392c6c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java
@@ -250,8 +250,8 @@
           LocatedBlock lblock = locatedBlocks.get(0); // first block
           BlockReader blockReader = null;
           try {
-            blockReader = BlockReaderTestUtil.
-                getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
+            blockReader = BlockReaderTestUtil.getBlockReader(
+                cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN);
             Assert.fail("expected getBlockReader to fail the first time.");
           } catch (Throwable t) { 
             Assert.assertTrue("expected to see 'TCP reads were disabled " +
@@ -265,8 +265,8 @@
 
           // Second time should succeed.
           try {
-            blockReader = BlockReaderTestUtil.
-                getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
+            blockReader = BlockReaderTestUtil.getBlockReader(
+                cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN);
           } catch (Throwable t) { 
             LOG.error("error trying to retrieve a block reader " +
                 "the second time.", t);
@@ -474,8 +474,8 @@
           while (true) {
             BlockReader blockReader = null;
             try {
-              blockReader = BlockReaderTestUtil.
-                  getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
+              blockReader = BlockReaderTestUtil.getBlockReader(
+                  cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN);
               sem.release();
               try {
                 blockReader.readAll(buf, 0, TEST_FILE_LEN);
@@ -514,8 +514,8 @@
     // getting a ClosedChannelException.
     BlockReader blockReader = null;
     try {
-      blockReader = BlockReaderTestUtil.
-          getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
+      blockReader = BlockReaderTestUtil.getBlockReader(
+          cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN);
       blockReader.readFully(buf, 0, TEST_FILE_LEN);
     } finally {
       if (blockReader != null) blockReader.close();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index 6a224f9..e41c06a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -256,12 +256,12 @@
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null, (byte) 0)).when(mockNN).getFileInfo(anyString());
+                1010, 0, null, (byte) 0, null)).when(mockNN).getFileInfo(anyString());
     
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null, (byte) 0))
+                1010, 0, null, (byte) 0, null))
         .when(mockNN)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),
@@ -549,7 +549,7 @@
       badBlocks.add(badLocatedBlock);
       return new LocatedBlocks(goodBlockList.getFileLength(), false,
                                badBlocks, null, true,
-                               null);
+                               null, null);
     }
   }
   
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
new file mode 100644
index 0000000..549a744
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
@@ -0,0 +1,335 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.io.erasurecode.CodecUtil;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+public class TestDFSStripedInputStream {
+
+  public static final Log LOG = LogFactory.getLog(TestDFSStripedInputStream.class);
+
+  private MiniDFSCluster cluster;
+  private Configuration conf = new Configuration();
+  private DistributedFileSystem fs;
+  private final Path dirPath = new Path("/striped");
+  private Path filePath = new Path(dirPath, "file");
+  private final ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private final int NUM_STRIPE_PER_BLOCK = 2;
+  private final int INTERNAL_BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE;
+  private final int BLOCK_GROUP_SIZE =  DATA_BLK_NUM * INTERNAL_BLOCK_SIZE;
+
+  @Before
+  public void setup() throws IOException {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    SimulatedFSDataset.setFactory(conf);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
+        DATA_BLK_NUM + PARITY_BLK_NUM).build();
+    cluster.waitActive();
+    for (DataNode dn : cluster.getDataNodes()) {
+      DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
+    }
+    fs = cluster.getFileSystem();
+    fs.mkdirs(dirPath);
+    fs.getClient().setErasureCodingPolicy(dirPath.toString(), null);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test {@link DFSStripedInputStream#getBlockAt(long)}
+   */
+  @Test
+  public void testRefreshBlock() throws Exception {
+    final int numBlocks = 4;
+    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
+    final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
+        filePath.toString(), false, ecPolicy, null);
+
+    List<LocatedBlock> lbList = lbs.getLocatedBlocks();
+    for (LocatedBlock aLbList : lbList) {
+      LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList;
+      LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb,
+          CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM);
+      for (int j = 0; j < DATA_BLK_NUM; j++) {
+        LocatedBlock refreshed = in.refreshLocatedBlock(blks[j]);
+        assertEquals(blks[j].getBlock(), refreshed.getBlock());
+        assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset());
+        assertArrayEquals(blks[j].getLocations(), refreshed.getLocations());
+      }
+    }
+  }
+
+  @Test
+  public void testPread() throws Exception {
+    final int numBlocks = 2;
+    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
+    int fileLen = BLOCK_GROUP_SIZE * numBlocks;
+
+    byte[] expected = new byte[fileLen];
+    assertEquals(numBlocks, lbs.getLocatedBlocks().size());
+    for (int bgIdx = 0; bgIdx < numBlocks; bgIdx++) {
+      LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(bgIdx));
+      for (int i = 0; i < DATA_BLK_NUM; i++) {
+        Block blk = new Block(bg.getBlock().getBlockId() + i,
+            NUM_STRIPE_PER_BLOCK * CELLSIZE,
+            bg.getBlock().getGenerationStamp());
+        blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
+        cluster.injectBlocks(i, Arrays.asList(blk),
+            bg.getBlock().getBlockPoolId());
+      }
+
+      /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
+      for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) {
+        for (int j = 0; j < DATA_BLK_NUM; j++) {
+          for (int k = 0; k < CELLSIZE; k++) {
+            int posInBlk = i * CELLSIZE + k;
+            int posInFile = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k;
+            expected[bgIdx*BLOCK_GROUP_SIZE + posInFile] =
+                SimulatedFSDataset.simulatedByte(
+                    new Block(bg.getBlock().getBlockId() + j), posInBlk);
+          }
+        }
+      }
+    }
+    DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
+        filePath.toString(), false, ecPolicy, null);
+
+    int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102,
+        CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102,
+        BLOCK_GROUP_SIZE - 102, BLOCK_GROUP_SIZE, BLOCK_GROUP_SIZE + 102,
+        fileLen - 1};
+    for (int startOffset : startOffsets) {
+      startOffset = Math.max(0, Math.min(startOffset, fileLen - 1));
+      int remaining = fileLen - startOffset;
+      byte[] buf = new byte[fileLen];
+      int ret = in.read(startOffset, buf, 0, fileLen);
+      assertEquals(remaining, ret);
+      for (int i = 0; i < remaining; i++) {
+        Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " +
+                "same",
+            expected[startOffset + i], buf[i]);
+      }
+    }
+    in.close();
+  }
+
+  @Test
+  public void testPreadWithDNFailure() throws Exception {
+    final int numBlocks = 4;
+    final int failedDNIdx = 2;
+    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCK_GROUP_SIZE);
+
+    assert lbs.get(0) instanceof LocatedStripedBlock;
+    LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0));
+    for (int i = 0; i < DATA_BLK_NUM + PARITY_BLK_NUM; i++) {
+      Block blk = new Block(bg.getBlock().getBlockId() + i,
+          NUM_STRIPE_PER_BLOCK * CELLSIZE,
+          bg.getBlock().getGenerationStamp());
+      blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
+      cluster.injectBlocks(i, Arrays.asList(blk),
+          bg.getBlock().getBlockPoolId());
+    }
+    DFSStripedInputStream in =
+        new DFSStripedInputStream(fs.getClient(), filePath.toString(), false,
+            ErasureCodingPolicyManager.getSystemDefaultPolicy(), null);
+    int readSize = BLOCK_GROUP_SIZE;
+    byte[] readBuffer = new byte[readSize];
+    byte[] expected = new byte[readSize];
+    cluster.stopDataNode(failedDNIdx);
+    /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
+    for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) {
+      for (int j = 0; j < DATA_BLK_NUM; j++) {
+        for (int k = 0; k < CELLSIZE; k++) {
+          int posInBlk = i * CELLSIZE + k;
+          int posInFile = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k;
+          expected[posInFile] = SimulatedFSDataset.simulatedByte(
+              new Block(bg.getBlock().getBlockId() + j), posInBlk);
+        }
+      }
+    }
+
+    // Update the expected content for decoded data
+    for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) {
+      byte[][] decodeInputs = new byte[DATA_BLK_NUM + PARITY_BLK_NUM][CELLSIZE];
+      int[] missingBlkIdx = new int[]{failedDNIdx + PARITY_BLK_NUM, 1, 2};
+      byte[][] decodeOutputs = new byte[PARITY_BLK_NUM][CELLSIZE];
+      for (int j = 0; j < DATA_BLK_NUM; j++) {
+        int posInBuf = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE;
+        if (j != failedDNIdx) {
+          System.arraycopy(expected, posInBuf, decodeInputs[j + PARITY_BLK_NUM],
+              0, CELLSIZE);
+        }
+      }
+      for (int k = 0; k < CELLSIZE; k++) {
+        int posInBlk = i * CELLSIZE + k;
+        decodeInputs[0][k] = SimulatedFSDataset.simulatedByte(
+            new Block(bg.getBlock().getBlockId() + DATA_BLK_NUM), posInBlk);
+      }
+      for (int m : missingBlkIdx) {
+        decodeInputs[m] = null;
+      }
+      RawErasureDecoder rawDecoder = CodecUtil.createRSRawDecoder(conf,
+          DATA_BLK_NUM, PARITY_BLK_NUM);
+      rawDecoder.decode(decodeInputs, missingBlkIdx, decodeOutputs);
+      int posInBuf = i * CELLSIZE * DATA_BLK_NUM + failedDNIdx * CELLSIZE;
+      System.arraycopy(decodeOutputs[0], 0, expected, posInBuf, CELLSIZE);
+    }
+    int delta = 10;
+    int done = 0;
+    // read a small delta, shouldn't trigger decode
+    // |cell_0 |
+    // |10     |
+    done += in.read(0, readBuffer, 0, delta);
+    assertEquals(delta, done);
+    // both head and trail cells are partial
+    // |c_0      |c_1    |c_2 |c_3 |c_4      |c_5         |
+    // |256K - 10|missing|256K|256K|256K - 10|not in range|
+    done += in.read(delta, readBuffer, delta,
+        CELLSIZE * (DATA_BLK_NUM - 1) - 2 * delta);
+    assertEquals(CELLSIZE * (DATA_BLK_NUM - 1) - delta, done);
+    // read the rest
+    done += in.read(done, readBuffer, done, readSize - done);
+    assertEquals(readSize, done);
+    assertArrayEquals(expected, readBuffer);
+  }
+
+  @Test
+  public void testStatefulRead() throws Exception {
+    testStatefulRead(false, false);
+    testStatefulRead(true, false);
+    testStatefulRead(true, true);
+  }
+
+  private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket)
+      throws Exception {
+    final int numBlocks = 2;
+    final int fileSize = numBlocks * BLOCK_GROUP_SIZE;
+    if (cellMisalignPacket) {
+      conf.setInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT + 1);
+      tearDown();
+      setup();
+    }
+    DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(filePath.toString(), 0, fileSize);
+
+    assert lbs.getLocatedBlocks().size() == numBlocks;
+    for (LocatedBlock lb : lbs.getLocatedBlocks()) {
+      assert lb instanceof LocatedStripedBlock;
+      LocatedStripedBlock bg = (LocatedStripedBlock)(lb);
+      for (int i = 0; i < DATA_BLK_NUM; i++) {
+        Block blk = new Block(bg.getBlock().getBlockId() + i,
+            NUM_STRIPE_PER_BLOCK * CELLSIZE,
+            bg.getBlock().getGenerationStamp());
+        blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
+        cluster.injectBlocks(i, Arrays.asList(blk),
+            bg.getBlock().getBlockPoolId());
+      }
+    }
+
+    DFSStripedInputStream in =
+        new DFSStripedInputStream(fs.getClient(), filePath.toString(),
+            false, ecPolicy, null);
+
+    byte[] expected = new byte[fileSize];
+
+    for (LocatedBlock bg : lbs.getLocatedBlocks()) {
+      /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
+      for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) {
+        for (int j = 0; j < DATA_BLK_NUM; j++) {
+          for (int k = 0; k < CELLSIZE; k++) {
+            int posInBlk = i * CELLSIZE + k;
+            int posInFile = (int) bg.getStartOffset() +
+                i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k;
+            expected[posInFile] = SimulatedFSDataset.simulatedByte(
+                new Block(bg.getBlock().getBlockId() + j), posInBlk);
+          }
+        }
+      }
+    }
+
+    if (useByteBuffer) {
+      ByteBuffer readBuffer = ByteBuffer.allocate(fileSize);
+      int done = 0;
+      while (done < fileSize) {
+        int ret = in.read(readBuffer);
+        assertTrue(ret > 0);
+        done += ret;
+      }
+      assertArrayEquals(expected, readBuffer.array());
+    } else {
+      byte[] readBuffer = new byte[fileSize];
+      int done = 0;
+      while (done < fileSize) {
+        int ret = in.read(readBuffer, done, fileSize - done);
+        assertTrue(ret > 0);
+        done += ret;
+      }
+      assertArrayEquals(expected, readBuffer);
+    }
+    fs.delete(filePath, true);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
new file mode 100644
index 0000000..d78e88b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+import 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.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestDFSStripedOutputStream {
+  public static final Log LOG = LogFactory.getLog(
+      TestDFSStripedOutputStream.class);
+
+  static {
+    GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
+  }
+
+  private int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private Configuration conf;
+  private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private final int stripesPerBlock = 4;
+  private final int blockSize = cellSize * stripesPerBlock;
+
+  @Before
+  public void setup() throws IOException {
+    int numDNs = dataBlocks + parityBlocks + 2;
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
+    fs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testFileEmpty() throws Exception {
+    testOneFile("/EmptyFile", 0);
+  }
+
+  @Test
+  public void testFileSmallerThanOneCell1() throws Exception {
+    testOneFile("/SmallerThanOneCell", 1);
+  }
+
+  @Test
+  public void testFileSmallerThanOneCell2() throws Exception {
+    testOneFile("/SmallerThanOneCell", cellSize - 1);
+  }
+
+  @Test
+  public void testFileEqualsWithOneCell() throws Exception {
+    testOneFile("/EqualsWithOneCell", cellSize);
+  }
+
+  @Test
+  public void testFileSmallerThanOneStripe1() throws Exception {
+    testOneFile("/SmallerThanOneStripe", cellSize * dataBlocks - 1);
+  }
+
+  @Test
+  public void testFileSmallerThanOneStripe2() throws Exception {
+    testOneFile("/SmallerThanOneStripe", cellSize + 123);
+  }
+
+  @Test
+  public void testFileEqualsWithOneStripe() throws Exception {
+    testOneFile("/EqualsWithOneStripe", cellSize * dataBlocks);
+  }
+
+  @Test
+  public void testFileMoreThanOneStripe1() throws Exception {
+    testOneFile("/MoreThanOneStripe1", cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void testFileMoreThanOneStripe2() throws Exception {
+    testOneFile("/MoreThanOneStripe2", cellSize * dataBlocks
+            + cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void testFileLessThanFullBlockGroup() throws Exception {
+    testOneFile("/LessThanFullBlockGroup",
+        cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize);
+  }
+
+  @Test
+  public void testFileFullBlockGroup() throws Exception {
+    testOneFile("/FullBlockGroup", blockSize * dataBlocks);
+  }
+
+  @Test
+  public void testFileMoreThanABlockGroup1() throws Exception {
+    testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void testFileMoreThanABlockGroup2() throws Exception {
+    testOneFile("/MoreThanABlockGroup2",
+        blockSize * dataBlocks + cellSize+ 123);
+  }
+
+
+  @Test
+  public void testFileMoreThanABlockGroup3() throws Exception {
+    testOneFile("/MoreThanABlockGroup3",
+        blockSize * dataBlocks * 3 + cellSize * dataBlocks
+        + cellSize + 123);
+  }
+
+  private void testOneFile(String src, int writeBytes) throws Exception {
+    src += "_" + writeBytes;
+    Path testPath = new Path(src);
+
+    byte[] bytes = StripedFileTestUtil.generateBytes(writeBytes);
+    DFSTestUtil.writeFile(fs, testPath, new String(bytes));
+    StripedFileTestUtil.waitBlockGroupsReported(fs, src);
+
+    StripedFileTestUtil.checkData(fs, testPath, writeBytes,
+        new int[]{}, 0);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
new file mode 100644
index 0000000..f6c2566
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -0,0 +1,473 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
+import org.apache.hadoop.hdfs.security.token.block.SecurityTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.base.Preconditions;
+
+
+public class TestDFSStripedOutputStreamWithFailure {
+  public static final Log LOG = LogFactory.getLog(
+      TestDFSStripedOutputStreamWithFailure.class);
+  static {
+    GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
+  }
+
+  private static final int NUM_DATA_BLOCKS = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private static final int NUM_PARITY_BLOCKS = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private static final int CELL_SIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private static final int STRIPES_PER_BLOCK = 4;
+  private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK;
+  private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS;
+
+  private static final int FLUSH_POS
+      = 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
+
+  static {
+    System.out.println("NUM_DATA_BLOCKS  = " + NUM_DATA_BLOCKS);
+    System.out.println("NUM_PARITY_BLOCKS= " + NUM_PARITY_BLOCKS);
+    System.out.println("CELL_SIZE        = " + CELL_SIZE
+        + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(CELL_SIZE, "B", 2) + ")");
+    System.out.println("BLOCK_SIZE       = " + BLOCK_SIZE
+        + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(BLOCK_SIZE, "B", 2) + ")");
+    System.out.println("BLOCK_GROUP_SIZE = " + BLOCK_GROUP_SIZE
+        + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(BLOCK_GROUP_SIZE, "B", 2) + ")");
+  }
+
+  static List<Integer> newLengths() {
+    final List<Integer> lengths = new ArrayList<>();
+    lengths.add(FLUSH_POS + 2);
+    for(int b = 0; b <= 2; b++) {
+      for(int c = 0; c < STRIPES_PER_BLOCK*NUM_DATA_BLOCKS; c++) {
+        for(int delta = -1; delta <= 1; delta++) {
+          final int length = b*BLOCK_GROUP_SIZE + c*CELL_SIZE + delta;
+          System.out.println(lengths.size() + ": length=" + length
+              + ", (b, c, d) = (" + b + ", " + c + ", " + delta + ")");
+          lengths.add(length);
+        }
+      }
+    }
+    return lengths;
+  }
+
+  private static final int[][] dnIndexSuite = {
+      {0, 1},
+      {0, 5},
+      {0, 6},
+      {0, 8},
+      {1, 5},
+      {1, 6},
+      {6, 8},
+      {0, 1, 2},
+      {3, 4, 5},
+      {0, 1, 6},
+      {0, 5, 6},
+      {0, 5, 8},
+      {0, 6, 7},
+      {5, 6, 7},
+      {6, 7, 8},
+  };
+
+  private int[] getKillPositions(int fileLen, int num) {
+    int[] positions = new int[num];
+    for (int i = 0; i < num; i++) {
+      positions[i] = fileLen * (i + 1) / (num + 1);
+    }
+    return positions;
+  }
+
+  private static final List<Integer> LENGTHS = newLengths();
+
+  static int getLength(int i) {
+    return LENGTHS.get(i);
+  }
+
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem dfs;
+  private final Path dir = new Path("/"
+      + TestDFSStripedOutputStreamWithFailure.class.getSimpleName());
+
+  private void setup(Configuration conf) throws IOException {
+    final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.waitActive();
+    dfs = cluster.getFileSystem();
+    dfs.mkdirs(dir);
+    dfs.setErasureCodingPolicy(dir, null);
+  }
+
+  private void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private HdfsConfiguration newHdfsConfiguration() {
+    final HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    return conf;
+  }
+
+  @Test(timeout=240000)
+  public void testDatanodeFailure56() throws Exception {
+    runTest(getLength(56));
+  }
+
+  @Test(timeout=240000)
+  public void testMultipleDatanodeFailure56() throws Exception {
+    runTestWithMultipleFailure(getLength(56));
+  }
+
+  @Test(timeout=240000)
+  public void testBlockTokenExpired() throws Exception {
+    final int length = NUM_DATA_BLOCKS * (BLOCK_SIZE - CELL_SIZE);
+    final HdfsConfiguration conf = newHdfsConfiguration();
+
+    conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
+    conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
+    // Set short retry timeouts so this test runs faster
+    conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
+    for (int dn = 0; dn < 9; dn += 2) {
+      try {
+        setup(conf);
+        runTest(length, new int[]{length/2}, new int[]{dn}, true);
+      } catch (Exception e) {
+        LOG.error("failed, dn=" + dn + ", length=" + length);
+        throw e;
+      } finally {
+        tearDown();
+      }
+    }
+  }
+
+  @Test(timeout = 90000)
+  public void testAddBlockWhenNoSufficientDataBlockNumOfNodes()
+      throws IOException {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    try {
+      setup(conf);
+      ArrayList<DataNode> dataNodes = cluster.getDataNodes();
+      // shutdown few datanodes to avoid getting sufficient data blocks number
+      // of datanodes
+      int killDns = dataNodes.size() / 2;
+      int numDatanodes = dataNodes.size() - killDns;
+      for (int i = 0; i < killDns; i++) {
+        cluster.stopDataNode(i);
+      }
+      cluster.restartNameNodes();
+      cluster.triggerHeartbeats();
+      DatanodeInfo[] info = dfs.getClient().datanodeReport(
+          DatanodeReportType.LIVE);
+      assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
+      final Path dirFile = new Path(dir, "ecfile");
+      FSDataOutputStream out = null;
+      try {
+        out = dfs.create(dirFile, true);
+        out.write("something".getBytes());
+        out.flush();
+        out.close();
+        Assert.fail("Failed to validate available dns against blkGroupSize");
+      } catch (IOException ioe) {
+        // expected
+        GenericTestUtils.assertExceptionContains("Failed to get 6 nodes from" +
+            " namenode: blockGroupSize= 9, blocks.length= 5", ioe);
+      }
+    } finally {
+      tearDown();
+    }
+  }
+
+  @Test(timeout = 90000)
+  public void testAddBlockWhenNoSufficientParityNumOfNodes() throws IOException {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    try {
+      setup(conf);
+      ArrayList<DataNode> dataNodes = cluster.getDataNodes();
+      // shutdown few data nodes to avoid writing parity blocks
+      int killDns = (NUM_PARITY_BLOCKS - 1);
+      int numDatanodes = dataNodes.size() - killDns;
+      for (int i = 0; i < killDns; i++) {
+        cluster.stopDataNode(i);
+      }
+      cluster.restartNameNodes();
+      cluster.triggerHeartbeats();
+      DatanodeInfo[] info = dfs.getClient().datanodeReport(
+          DatanodeReportType.LIVE);
+      assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
+      Path srcPath = new Path(dir, "testAddBlockWhenNoSufficientParityNodes");
+      int fileLength = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE - 1000;
+      final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
+      DFSTestUtil.writeFile(dfs, srcPath, new String(expected));
+      LOG.info("writing finished. Seek and read the file to verify.");
+      StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength);
+    } finally {
+      tearDown();
+    }
+  }
+
+  void runTest(final int length) {
+    final HdfsConfiguration conf = newHdfsConfiguration();
+    for (int dn = 0; dn < 9; dn++) {
+      try {
+        setup(conf);
+        runTest(length, new int[]{length/2}, new int[]{dn}, false);
+      } catch (Throwable e) {
+        final String err = "failed, dn=" + dn + ", length=" + length
+            + StringUtils.stringifyException(e);
+        LOG.error(err);
+        Assert.fail(err);
+      } finally {
+        tearDown();
+      }
+    }
+  }
+
+  void runTestWithMultipleFailure(final int length) throws Exception {
+    final HdfsConfiguration conf = newHdfsConfiguration();
+    for(int i=0;i<dnIndexSuite.length;i++){
+      int[] dnIndex = dnIndexSuite[i];
+      int[] killPos = getKillPositions(length, dnIndex.length);
+      try {
+        setup(conf);
+        runTest(length, killPos, dnIndex, false);
+      } catch (Throwable e) {
+        final String err = "failed, killPos=" + Arrays.toString(killPos)
+            + ", dnIndex=" + Arrays.toString(dnIndex) + ", length=" + length;
+        LOG.error(err);
+        throw e;
+      } finally {
+        tearDown();
+      }
+    }
+  }
+
+  /**
+   * runTest implementation
+   * @param length file length
+   * @param killPos killing positions in ascending order
+   * @param dnIndex DN index to kill when meets killing positions
+   * @param tokenExpire wait token to expire when kill a DN
+   * @throws Exception
+   */
+  private void runTest(final int length, final int[] killPos,
+      final int[] dnIndex, final boolean tokenExpire) throws Exception {
+    if (killPos[0] <= FLUSH_POS) {
+      LOG.warn("killPos=" + Arrays.toString(killPos) + " <= FLUSH_POS=" + FLUSH_POS
+          + ", length=" + length + ", dnIndex=" + Arrays.toString(dnIndex));
+      return; //skip test
+    }
+    Preconditions.checkArgument(length > killPos[0], "length=%s <= killPos=%s",
+        length, killPos);
+    Preconditions.checkArgument(killPos.length == dnIndex.length);
+
+    final Path p = new Path(dir, "dn" + Arrays.toString(dnIndex)
+        + "len" + length + "kill" +  Arrays.toString(killPos));
+    final String fullPath = p.toString();
+    LOG.info("fullPath=" + fullPath);
+
+    if (tokenExpire) {
+      final NameNode nn = cluster.getNameNode();
+      final BlockManager bm = nn.getNamesystem().getBlockManager();
+      final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
+
+      // set a short token lifetime (1 second)
+      SecurityTestUtil.setBlockTokenLifetime(sm, 1000L);
+    }
+
+    final AtomicInteger pos = new AtomicInteger();
+    final FSDataOutputStream out = dfs.create(p);
+    final DFSStripedOutputStream stripedOut
+        = (DFSStripedOutputStream)out.getWrappedStream();
+
+    long firstGS = -1;  // first GS of this block group which never proceeds blockRecovery
+    long oldGS = -1; // the old GS before bumping
+    int numKilled=0;
+    for(; pos.get() < length; ) {
+      final int i = pos.getAndIncrement();
+      if (numKilled < killPos.length &&  i == killPos[numKilled]) {
+        assertTrue(firstGS != -1);
+        final long gs = getGenerationStamp(stripedOut);
+        if (numKilled == 0) {
+          assertEquals(firstGS, gs);
+        } else {
+          //TODO: implement hflush/hsync and verify gs strict greater than oldGS
+          assertTrue(gs >= oldGS);
+        }
+        oldGS = gs;
+
+        if (tokenExpire) {
+          DFSTestUtil.flushInternal(stripedOut);
+          waitTokenExpires(out);
+        }
+
+        killDatanode(cluster, stripedOut, dnIndex[numKilled], pos);
+        numKilled++;
+      }
+
+      write(out, i);
+
+      if (i % BLOCK_GROUP_SIZE == FLUSH_POS) {
+        firstGS = getGenerationStamp(stripedOut);
+        oldGS = firstGS;
+      }
+    }
+    out.close();
+    assertEquals(dnIndex.length, numKilled);
+
+    short expectedReported = StripedFileTestUtil.getRealTotalBlockNum(length);
+    for(int idx :dnIndex) {
+      if (length > idx * CELL_SIZE || idx >= NUM_DATA_BLOCKS) {
+        expectedReported--;
+      }
+    }
+    DFSTestUtil.waitReplication(dfs, p, expectedReported);
+
+    cluster.triggerBlockReports();
+    StripedFileTestUtil.checkData(dfs, p, length, dnIndex, oldGS);
+  }
+
+  static void write(FSDataOutputStream out, int i) throws IOException {
+    try {
+      out.write(StripedFileTestUtil.getByte(i));
+    } catch(IOException ioe) {
+      throw new IOException("Failed at i=" + i, ioe);
+    }
+  }
+
+  static long getGenerationStamp(DFSStripedOutputStream out)
+      throws IOException {
+    DFSTestUtil.flushBuffer(out);
+    final long gs = DFSTestUtil.flushInternal(out).getGenerationStamp();
+    LOG.info("getGenerationStamp returns " + gs);
+    return gs;
+  }
+
+  static DatanodeInfo getDatanodes(StripedDataStreamer streamer) {
+    for(;;) {
+      DatanodeInfo[] datanodes = streamer.getNodes();
+      if (datanodes == null) {
+        // try peeking following block.
+        final LocatedBlock lb = streamer.peekFollowingBlock();
+        if (lb != null) {
+          datanodes = lb.getLocations();
+        }
+      }
+
+      if (datanodes != null) {
+        Assert.assertEquals(1, datanodes.length);
+        Assert.assertNotNull(datanodes[0]);
+        return datanodes[0];
+      }
+
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException ie) {
+        Assert.fail(StringUtils.stringifyException(ie));
+        return null;
+      }
+    }
+  }
+
+  static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out,
+      final int dnIndex, final AtomicInteger pos) {
+    final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex);
+    final DatanodeInfo datanode = getDatanodes(s);
+    LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos);
+    cluster.stopDataNode(datanode.getXferAddr());
+  }
+
+
+  private void waitTokenExpires(FSDataOutputStream out) throws IOException {
+    Token<BlockTokenIdentifier> token = DFSTestUtil.getBlockToken(out);
+    while (!SecurityTestUtil.isBlockTokenExpired(token)) {
+      try {
+        Thread.sleep(10);
+      } catch (InterruptedException ignored) {
+      }
+    }
+  }
+
+  public static abstract class TestBase {
+    static final long TIMEOUT = 240000;
+
+    int getBase() {
+      final String name = getClass().getSimpleName();
+      int i = name.length() - 1;
+      for(; i >= 0 && Character.isDigit(name.charAt(i)); i--);
+      return Integer.parseInt(name.substring(i + 1));
+    }
+
+    private final TestDFSStripedOutputStreamWithFailure test
+        = new TestDFSStripedOutputStreamWithFailure();
+    private void run(int offset) {
+      final int i = offset + getBase();
+      final int length = getLength(i);
+      System.out.println("Run test " + i + ", length=" + length);
+      test.runTest(length);
+    }
+
+    @Test(timeout=TIMEOUT) public void test0() {run(0);}
+    @Test(timeout=TIMEOUT) public void test1() {run(1);}
+    @Test(timeout=TIMEOUT) public void test2() {run(2);}
+    @Test(timeout=TIMEOUT) public void test3() {run(3);}
+    @Test(timeout=TIMEOUT) public void test4() {run(4);}
+    @Test(timeout=TIMEOUT) public void test5() {run(5);}
+    @Test(timeout=TIMEOUT) public void test6() {run(6);}
+    @Test(timeout=TIMEOUT) public void test7() {run(7);}
+    @Test(timeout=TIMEOUT) public void test8() {run(8);}
+    @Test(timeout=TIMEOUT) public void test9() {run(9);}
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java
new file mode 100644
index 0000000..b4fb1b8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure000.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.TestDFSStripedOutputStreamWithFailure.TestBase;
+
+public class TestDFSStripedOutputStreamWithFailure000 extends TestBase {}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java
new file mode 100644
index 0000000..8489c3d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure010.java
@@ -0,0 +1,22 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.TestDFSStripedOutputStreamWithFailure.TestBase;
+
+public class TestDFSStripedOutputStreamWithFailure010 extends TestBase {}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index f22deaf..19b8c69 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -110,7 +110,7 @@
     l2.setCorrupt(true);
 
     List<LocatedBlock> ls = Arrays.asList(l1, l2);
-    LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null);
+    LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null, null);
 
     BlockLocation[] bs = DFSUtilClient.locatedBlocks2Locations(lbs);
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
index 9cdb763..989e9fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java
@@ -81,7 +81,7 @@
   public void testDataDirectories() throws IOException {
     File dataDir = new File(BASE_DIR, "data").getCanonicalFile();
     Configuration conf = cluster.getConfiguration(0);
-    // 1. Test unsupported schema. Only "file:" is supported.
+    // 1. Test unsupported ecPolicy. Only "file:" is supported.
     String dnDir = makeURI("shv", null, fileAsURI(dataDir).getPath());
     conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dnDir);
     DataNode dn = null;
@@ -97,7 +97,7 @@
     }
     assertNull("Data-node startup should have failed.", dn);
 
-    // 2. Test "file:" schema and no schema (path-only). Both should work.
+    // 2. Test "file:" ecPolicy and no ecPolicy (path-only). Both should work.
     String dnDir1 = fileAsURI(dataDir).toString() + "1";
     String dnDir2 = makeURI("file", "localhost",
                     fileAsURI(dataDir).getPath() + "2");
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 c1fdd25..3ae9e25 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
@@ -674,8 +674,7 @@
       GenericTestUtils.waitFor(new Supplier<Boolean>() {
         @Override
         public Boolean get() {
-          BlockInfo 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++) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index 567a70a..64daeb3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -743,7 +743,7 @@
             version, new byte[suite.getAlgorithmBlockSize()],
             new byte[suite.getAlgorithmBlockSize()],
             "fakeKey", "fakeVersion"),
-            (byte) 0))
+            (byte) 0, null))
         .when(mcp)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
new file mode 100644
index 0000000..0ababed
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -0,0 +1,249 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.junit.Assert.*;
+
+public class TestErasureCodingPolicies {
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private static final int BLOCK_SIZE = 1024;
+  private FSNamesystem namesystem;
+
+  @Before
+  public void setupCluster() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    cluster = new MiniDFSCluster.Builder(conf).
+        numDataNodes(1).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    namesystem = cluster.getNamesystem();
+  }
+
+  @After
+  public void shutdownCluster() throws IOException {
+    cluster.shutdown();
+  }
+
+  @Test
+  public void testBasicSetECPolicy()
+      throws IOException, InterruptedException {
+    final Path testDir = new Path("/ec");
+    fs.mkdir(testDir, FsPermission.getDirDefault());
+
+    /* Normal creation of an erasure coding directory */
+    fs.getClient().setErasureCodingPolicy(testDir.toString(), null);
+
+    /* Verify files under the directory are striped */
+    final Path ECFilePath = new Path(testDir, "foo");
+    fs.create(ECFilePath);
+    INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString());
+    assertTrue(inode.asFile().isStriped());
+
+    /**
+     * Verify that setting EC policy on non-empty directory only affects
+     * newly created files under the directory.
+     */
+    final Path notEmpty = new Path("/nonEmpty");
+    fs.mkdir(notEmpty, FsPermission.getDirDefault());
+    final Path oldFile = new Path(notEmpty, "old");
+    fs.create(oldFile);
+    fs.getClient().setErasureCodingPolicy(notEmpty.toString(), null);
+    final Path newFile = new Path(notEmpty, "new");
+    fs.create(newFile);
+    INode oldInode = namesystem.getFSDirectory().getINode(oldFile.toString());
+    assertFalse(oldInode.asFile().isStriped());
+    INode newInode = namesystem.getFSDirectory().getINode(newFile.toString());
+    assertTrue(newInode.asFile().isStriped());
+
+    /* Verify that nested EC policies not supported */
+    final Path dir1 = new Path("/dir1");
+    final Path dir2 = new Path(dir1, "dir2");
+    fs.mkdir(dir1, FsPermission.getDirDefault());
+    fs.getClient().setErasureCodingPolicy(dir1.toString(), null);
+    fs.mkdir(dir2, FsPermission.getDirDefault());
+    try {
+      fs.getClient().setErasureCodingPolicy(dir2.toString(), null);
+      fail("Nested erasure coding policies");
+    } catch (IOException e) {
+      assertExceptionContains("already has an erasure coding policy", e);
+    }
+
+    /* Verify that EC policy cannot be set on a file */
+    final Path fPath = new Path("/file");
+    fs.create(fPath);
+    try {
+      fs.getClient().setErasureCodingPolicy(fPath.toString(), null);
+      fail("Erasure coding policy on file");
+    } catch (IOException e) {
+      assertExceptionContains("erasure coding policy for a file", e);
+    }
+  }
+
+  @Test
+  public void testMoveValidity() throws IOException, InterruptedException {
+    final Path srcECDir = new Path("/srcEC");
+    final Path dstECDir = new Path("/dstEC");
+    fs.mkdir(srcECDir, FsPermission.getDirDefault());
+    fs.mkdir(dstECDir, FsPermission.getDirDefault());
+    fs.getClient().setErasureCodingPolicy(srcECDir.toString(), null);
+    fs.getClient().setErasureCodingPolicy(dstECDir.toString(), null);
+    final Path srcFile = new Path(srcECDir, "foo");
+    fs.create(srcFile);
+
+    // Test move dir
+    // Move EC dir under non-EC dir
+    final Path newDir = new Path("/srcEC_new");
+    fs.rename(srcECDir, newDir);
+    fs.rename(newDir, srcECDir); // move back
+
+    // Move EC dir under another EC dir
+    fs.rename(srcECDir, dstECDir);
+    fs.rename(new Path("/dstEC/srcEC"), srcECDir); // move back
+
+    // Test move file
+    /* Verify that a file can be moved between 2 EC dirs */
+    fs.rename(srcFile, dstECDir);
+    fs.rename(new Path(dstECDir, "foo"), srcECDir); // move back
+
+    /* Verify that a file can be moved from a non-EC dir to an EC dir */
+    final Path nonECDir = new Path("/nonEC");
+    fs.mkdir(nonECDir, FsPermission.getDirDefault());
+    fs.rename(srcFile, nonECDir);
+
+    /* Verify that a file can be moved from an EC dir to a non-EC dir */
+    final Path nonECFile = new Path(nonECDir, "nonECFile");
+    fs.create(nonECFile);
+    fs.rename(nonECFile, dstECDir);
+  }
+
+  @Test
+  public void testReplication() throws IOException {
+    final Path testDir = new Path("/ec");
+    fs.mkdir(testDir, FsPermission.getDirDefault());
+    fs.setErasureCodingPolicy(testDir, null);
+    final Path fooFile = new Path(testDir, "foo");
+    // create ec file with replication=0
+    fs.create(fooFile, FsPermission.getFileDefault(), true,
+        conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        (short)0, fs.getDefaultBlockSize(fooFile), null);
+    // set replication should be a no-op
+    fs.setReplication(fooFile, (short) 3);
+  }
+
+  @Test
+  public void testGetErasureCodingPolicyWithSystemDefaultECPolicy() throws Exception {
+    String src = "/ec";
+    final Path ecDir = new Path(src);
+    fs.mkdir(ecDir, FsPermission.getDirDefault());
+    // dir EC policy should be null
+    assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy());
+    // dir EC policy after setting
+    fs.getClient().setErasureCodingPolicy(src, null); //Default one will be used.
+    ErasureCodingPolicy sysDefaultECPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    verifyErasureCodingInfo(src, sysDefaultECPolicy);
+    fs.create(new Path(ecDir, "child1")).close();
+    // verify for the files in ec dir
+    verifyErasureCodingInfo(src + "/child1", sysDefaultECPolicy);
+  }
+
+  @Test
+  public void testGetErasureCodingPolicy() throws Exception {
+    ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager.getSystemPolices();
+    assertTrue("System ecPolicies should be of only 1 for now",
+        sysECPolicies.length == 1);
+
+    ErasureCodingPolicy usingECPolicy = sysECPolicies[0];
+    String src = "/ec2";
+    final Path ecDir = new Path(src);
+    fs.mkdir(ecDir, FsPermission.getDirDefault());
+    // dir ECInfo before being set
+    assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy());
+    // dir ECInfo after set
+    fs.getClient().setErasureCodingPolicy(src, usingECPolicy);
+    verifyErasureCodingInfo(src, usingECPolicy);
+    fs.create(new Path(ecDir, "child1")).close();
+    // verify for the files in ec dir
+    verifyErasureCodingInfo(src + "/child1", usingECPolicy);
+  }
+
+  private void verifyErasureCodingInfo(
+      String src, ErasureCodingPolicy usingECPolicy) throws IOException {
+    HdfsFileStatus hdfsFileStatus = fs.getClient().getFileInfo(src);
+    ErasureCodingPolicy ecPolicy = hdfsFileStatus.getErasureCodingPolicy();
+    assertNotNull(ecPolicy);
+    assertEquals("Actually used ecPolicy should be equal with target ecPolicy",
+        usingECPolicy, ecPolicy);
+  }
+
+  @Test
+  public void testCreationErasureCodingZoneWithInvalidPolicy()
+      throws IOException {
+    ECSchema rsSchema = new ECSchema("rs", 4, 2);
+    String policyName = "RS-4-2-128k";
+    int cellSize = 128 * 1024;
+    ErasureCodingPolicy ecPolicy=
+        new ErasureCodingPolicy(policyName,rsSchema,cellSize);
+    String src = "/ecZone4-2";
+    final Path ecDir = new Path(src);
+    try {
+      fs.mkdir(ecDir, FsPermission.getDirDefault());
+      fs.getClient().setErasureCodingPolicy(src, ecPolicy);
+      fail("HadoopIllegalArgumentException should be thrown for"
+          + "setting an invalid erasure coding policy");
+    } catch (Exception e) {
+      assertExceptionContains("Policy [ RS-4-2-128k ] does not match " +
+          "any of the supported policies",e);
+    }
+  }
+
+  @Test
+  public void testGetAllErasureCodingPolicies() throws Exception {
+    ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager
+        .getSystemPolices();
+    assertTrue("System ecPolicies should be of only 1 for now",
+        sysECPolicies.length == 1);
+
+    Collection<ErasureCodingPolicy> allECPolicies = fs
+        .getAllErasureCodingPolicies();
+    assertTrue("All ecPolicies should be of only 1 for now",
+        allECPolicies.size() == 1);
+    assertEquals("Erasure coding policy mismatches",
+        sysECPolicies[0], allECPolicies.iterator().next());
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java
new file mode 100644
index 0000000..64063d2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java
@@ -0,0 +1,65 @@
+package org.apache.hadoop.hdfs;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFileStatusWithECPolicy {
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private DFSClient client;
+
+  @Before
+  public void before() throws IOException {
+    cluster =
+        new MiniDFSCluster.Builder(new Configuration()).numDataNodes(1).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    client = fs.getClient();
+  }
+
+  @After
+  public void after() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testFileStatusWithECPolicy() throws Exception {
+    // test directory doesn't have an EC policy
+    final Path dir = new Path("/foo");
+    assertTrue(fs.mkdir(dir, FsPermission.getDirDefault()));
+    assertNull(client.getFileInfo(dir.toString()).getErasureCodingPolicy());
+    // test file doesn't have an EC policy
+    final Path file = new Path(dir, "foo");
+    fs.create(file).close();
+    assertNull(client.getFileInfo(file.toString()).getErasureCodingPolicy());
+    fs.delete(file, true);
+
+    final ErasureCodingPolicy ecPolicy1 = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    // set EC policy on dir
+    fs.setErasureCodingPolicy(dir, ecPolicy1);
+    final ErasureCodingPolicy ecPolicy2 = client.getFileInfo(dir.toUri().getPath()).getErasureCodingPolicy();
+    assertNotNull(ecPolicy2);
+    assertTrue(ecPolicy1.equals(ecPolicy2));
+
+    // test file doesn't have an EC policy
+    fs.create(file).close();
+    final ErasureCodingPolicy ecPolicy3 =
+        fs.getClient().getFileInfo(file.toUri().getPath()).getErasureCodingPolicy();
+    assertNotNull(ecPolicy3);
+    assertTrue(ecPolicy1.equals(ecPolicy3));
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
index bacdc994..20596c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
@@ -354,12 +354,12 @@
     Mockito.doReturn(
         new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
             (short) 777), "owner", "group", new byte[0], new byte[0],
-            1010, 0, null, (byte) 0)).when(mcp).getFileInfo(anyString());
+            1010, 0, null, (byte) 0, null)).when(mcp).getFileInfo(anyString());
     Mockito
         .doReturn(
             new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null, (byte) 0))
+                1010, 0, null, (byte) 0, null))
         .when(mcp)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
new file mode 100644
index 0000000..9cf970b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
@@ -0,0 +1,362 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.Block;
+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.protocol.LocatedStripedBlock;
+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.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.NameNodeAdapter;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
+
+public class TestReadStripedFileWithDecoding {
+  static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class);
+
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private final int smallFileLength = blockSize * dataBlocks - 123;
+  private final int largeFileLength = blockSize * dataBlocks + 123;
+  private final int[] fileLengths = {smallFileLength, largeFileLength};
+  private final int[] dnFailureNums = {1, 2, 3};
+
+  @Before
+  public void setup() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
+        .numDataNodes(numDNs).build();
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
+    fs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Shutdown tolerable number of Datanode before reading.
+   * Verify the decoding works correctly.
+   */
+  @Test(timeout=300000)
+  public void testReadWithDNFailure() throws Exception {
+    for (int fileLength : fileLengths) {
+      for (int dnFailureNum : dnFailureNums) {
+        try {
+          // setup a new cluster with no dead datanode
+          setup();
+          testReadWithDNFailure(fileLength, dnFailureNum);
+        } catch (IOException ioe) {
+          String fileType = fileLength < (blockSize * dataBlocks) ?
+              "smallFile" : "largeFile";
+          LOG.error("Failed to read file with DN failure:"
+              + " fileType = "+ fileType
+              + ", dnFailureNum = " + dnFailureNum);
+        } finally {
+          // tear down the cluster
+          tearDown();
+        }
+      }
+    }
+  }
+
+  /**
+   * Corrupt tolerable number of block before reading.
+   * Verify the decoding works correctly.
+   */
+  @Test(timeout=300000)
+  public void testReadCorruptedData() throws IOException {
+    for (int fileLength : fileLengths) {
+      for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) {
+        for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) {
+          String src = "/corrupted_" + dataDelNum + "_" + parityDelNum;
+          testReadWithBlockCorrupted(src, fileLength,
+              dataDelNum, parityDelNum, false);
+        }
+      }
+    }
+  }
+
+  /**
+   * Delete tolerable number of block before reading.
+   * Verify the decoding works correctly.
+   */
+  @Test(timeout=300000)
+  public void testReadCorruptedDataByDeleting() throws IOException {
+    for (int fileLength : fileLengths) {
+      for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) {
+        for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) {
+          String src = "/deleted_" + dataDelNum + "_" + parityDelNum;
+          testReadWithBlockCorrupted(src, fileLength,
+              dataDelNum, parityDelNum, true);
+        }
+      }
+    }
+  }
+
+  private int findFirstDataNode(Path file, long length) throws IOException {
+    BlockLocation[] locs = fs.getFileBlockLocations(file, 0, length);
+    String name = (locs[0].getNames())[0];
+    int dnIndex = 0;
+    for (DataNode dn : cluster.getDataNodes()) {
+      int port = dn.getXferPort();
+      if (name.contains(Integer.toString(port))) {
+        return dnIndex;
+      }
+      dnIndex++;
+    }
+    return -1;
+  }
+
+  private void verifyRead(Path testPath, int length, byte[] expected)
+      throws IOException {
+    byte[] buffer = new byte[length + 100];
+    StripedFileTestUtil.verifyLength(fs, testPath, length);
+    StripedFileTestUtil.verifyPread(fs, testPath, length, expected, buffer);
+    StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected, buffer);
+    StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected,
+        ByteBuffer.allocate(length + 100));
+    StripedFileTestUtil.verifySeek(fs, testPath, length);
+  }
+
+  private void testReadWithDNFailure(int fileLength, int dnFailureNum)
+      throws Exception {
+    String fileType = fileLength < (blockSize * dataBlocks) ?
+        "smallFile" : "largeFile";
+    String src = "/dnFailure_" + dnFailureNum + "_" + fileType;
+    LOG.info("testReadWithDNFailure: file = " + src
+        + ", fileSize = " + fileLength
+        + ", dnFailureNum = " + dnFailureNum);
+
+    Path testPath = new Path(src);
+    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
+    DFSTestUtil.writeFile(fs, testPath, bytes);
+    StripedFileTestUtil.waitBlockGroupsReported(fs, src);
+
+    // shut down the DN that holds an internal data block
+    BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5,
+        cellSize);
+    for (int failedDnIdx = 0; failedDnIdx < dnFailureNum; failedDnIdx++) {
+      String name = (locs[0].getNames())[failedDnIdx];
+      for (DataNode dn : cluster.getDataNodes()) {
+        int port = dn.getXferPort();
+        if (name.contains(Integer.toString(port))) {
+          dn.shutdown();
+        }
+      }
+    }
+
+    // check file length, pread, stateful read and seek
+    verifyRead(testPath, fileLength, bytes);
+  }
+
+  /**
+   * After reading a corrupted block, make sure the client can correctly report
+   * the corruption to the NameNode.
+   */
+  @Test
+  public void testReportBadBlock() throws IOException {
+    // create file
+    final Path file = new Path("/corrupted");
+    final int length = 10; // length of "corruption"
+    final byte[] bytes = StripedFileTestUtil.generateBytes(length);
+    DFSTestUtil.writeFile(fs, file, bytes);
+
+    // corrupt the first data block
+    int dnIndex = findFirstDataNode(file, cellSize * dataBlocks);
+    Assert.assertNotEquals(-1, dnIndex);
+    LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient()
+        .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0);
+    final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb,
+        cellSize, dataBlocks, parityBlocks);
+    // find the first block file
+    File storageDir = cluster.getInstanceStorageDir(dnIndex, 0);
+    File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock());
+    Assert.assertTrue("Block file does not exist", blkFile.exists());
+    // corrupt the block file
+    LOG.info("Deliberately corrupting file " + blkFile.getName());
+    try (FileOutputStream out = new FileOutputStream(blkFile)) {
+      out.write("corruption".getBytes());
+    }
+
+    // disable the heartbeat from DN so that the corrupted block record is kept
+    // in NameNode
+    for (DataNode dn : cluster.getDataNodes()) {
+      DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
+    }
+
+    try {
+      // do stateful read
+      StripedFileTestUtil.verifyStatefulRead(fs, file, length, bytes,
+          ByteBuffer.allocate(1024));
+
+      // check whether the corruption has been reported to the NameNode
+      final FSNamesystem ns = cluster.getNamesystem();
+      final BlockManager bm = ns.getBlockManager();
+      BlockInfo blockInfo = (ns.getFSDirectory().getINode4Write(file.toString())
+          .asFile().getBlocks())[0];
+      Assert.assertEquals(1, bm.getCorruptReplicas(blockInfo).size());
+    } finally {
+      for (DataNode dn : cluster.getDataNodes()) {
+        DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false);
+      }
+    }
+  }
+
+  @Test
+  public void testInvalidateBlock() throws IOException {
+    final Path file = new Path("/invalidate");
+    final int length = 10;
+    final byte[] bytes = StripedFileTestUtil.generateBytes(length);
+    DFSTestUtil.writeFile(fs, file, bytes);
+
+    int dnIndex = findFirstDataNode(file, cellSize * dataBlocks);
+    Assert.assertNotEquals(-1, dnIndex);
+    LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient()
+        .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0);
+    final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb,
+        cellSize, dataBlocks, parityBlocks);
+    final Block b = blks[0].getBlock().getLocalBlock();
+
+    DataNode dn = cluster.getDataNodes().get(dnIndex);
+    // disable the heartbeat from DN so that the invalidated block record is kept
+    // in NameNode until heartbeat expires and NN mark the dn as dead
+    DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
+
+    try {
+      // delete the file
+      fs.delete(file, true);
+      // check the block is added to invalidateBlocks
+      final FSNamesystem fsn = cluster.getNamesystem();
+      final BlockManager bm = fsn.getBlockManager();
+      DatanodeDescriptor dnd = NameNodeAdapter.getDatanode(fsn, dn.getDatanodeId());
+      Assert.assertTrue(bm.containsInvalidateBlock(
+          blks[0].getLocations()[0], b) || dnd.containsInvalidateBlock(b));
+    } finally {
+      DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false);
+    }
+  }
+
+  /**
+   * Test reading a file with some blocks(data blocks or parity blocks or both)
+   * deleted or corrupted.
+   * @param src file path
+   * @param fileLength file length
+   * @param dataBlkDelNum the deleted or corrupted number of data blocks.
+   * @param parityBlkDelNum the deleted or corrupted number of parity blocks.
+   * @param deleteBlockFile whether block file is deleted or corrupted.
+   *                        true is to delete the block file.
+   *                        false is to corrupt the content of the block file.
+   * @throws IOException
+   */
+  private void testReadWithBlockCorrupted(String src, int fileLength,
+      int dataBlkDelNum, int parityBlkDelNum, boolean deleteBlockFile)
+      throws IOException {
+    LOG.info("testReadWithBlockCorrupted: file = " + src
+        + ", dataBlkDelNum = " + dataBlkDelNum
+        + ", parityBlkDelNum = " + parityBlkDelNum
+        + ", deleteBlockFile? " + deleteBlockFile);
+    int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
+    Assert.assertTrue("dataBlkDelNum and parityBlkDelNum should be positive",
+        dataBlkDelNum >= 0 && parityBlkDelNum >= 0);
+    Assert.assertTrue("The sum of dataBlkDelNum and parityBlkDelNum " +
+        "should be between 1 ~ " + parityBlocks, recoverBlkNum <= parityBlocks);
+
+    // write a file with the length of writeLen
+    Path srcPath = new Path(src);
+    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
+    DFSTestUtil.writeFile(fs, srcPath, bytes);
+
+    // delete or corrupt some blocks
+    corruptBlocks(srcPath, dataBlkDelNum, parityBlkDelNum, deleteBlockFile);
+
+    // check the file can be read after some blocks were deleted
+    verifyRead(srcPath, fileLength, bytes);
+  }
+
+  private void corruptBlocks(Path srcPath, int dataBlkDelNum,
+      int parityBlkDelNum, boolean deleteBlockFile) throws IOException {
+    int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
+
+    LocatedBlocks locatedBlocks = getLocatedBlocks(srcPath);
+    LocatedStripedBlock lastBlock =
+        (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
+
+    int[] delDataBlkIndices = StripedFileTestUtil.randomArray(0, dataBlocks,
+        dataBlkDelNum);
+    Assert.assertNotNull(delDataBlkIndices);
+    int[] delParityBlkIndices = StripedFileTestUtil.randomArray(dataBlocks,
+        dataBlocks + parityBlocks, parityBlkDelNum);
+    Assert.assertNotNull(delParityBlkIndices);
+
+    int[] delBlkIndices = new int[recoverBlkNum];
+    System.arraycopy(delDataBlkIndices, 0,
+        delBlkIndices, 0, delDataBlkIndices.length);
+    System.arraycopy(delParityBlkIndices, 0,
+        delBlkIndices, delDataBlkIndices.length, delParityBlkIndices.length);
+
+    ExtendedBlock[] delBlocks = new ExtendedBlock[recoverBlkNum];
+    for (int i = 0; i < recoverBlkNum; i++) {
+      delBlocks[i] = StripedBlockUtil
+          .constructInternalBlock(lastBlock.getBlock(),
+              cellSize, dataBlocks, delBlkIndices[i]);
+      if (deleteBlockFile) {
+        // delete the block file
+        cluster.corruptBlockOnDataNodesByDeletingBlockFile(delBlocks[i]);
+      } else {
+        // corrupt the block file
+        cluster.corruptBlockOnDataNodes(delBlocks[i]);
+      }
+    }
+  }
+
+  private LocatedBlocks getLocatedBlocks(Path filePath) throws IOException {
+    return fs.getClient().getLocatedBlocks(filePath.toString(),
+        0, Long.MAX_VALUE);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
new file mode 100644
index 0000000..f521d8e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
+
+/**
+ * Test reading a striped file when some of its blocks are missing (not included
+ * in the block locations returned by the NameNode).
+ */
+public class TestReadStripedFileWithMissingBlocks {
+  public static final Log LOG = LogFactory
+      .getLog(TestReadStripedFileWithMissingBlocks.class);
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem fs;
+  private static Configuration conf = new HdfsConfiguration();
+  private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private final int fileLength = blockSize * dataBlocks + 123;
+
+  @Before
+  public void setup() throws IOException {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
+    fs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testReadFileWithMissingBlocks1() throws Exception {
+    readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 0);
+  }
+
+  @Test
+  public void testReadFileWithMissingBlocks2() throws Exception {
+    readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 1);
+  }
+
+  @Test
+  public void testReadFileWithMissingBlocks3() throws Exception {
+    readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 2);
+  }
+
+  @Test
+  public void testReadFileWithMissingBlocks4() throws Exception {
+    readFileWithMissingBlocks(new Path("/foo"), fileLength, 2, 0);
+  }
+
+  @Test
+  public void testReadFileWithMissingBlocks5() throws Exception {
+    readFileWithMissingBlocks(new Path("/foo"), fileLength, 2, 1);
+  }
+
+  @Test
+  public void testReadFileWithMissingBlocks6() throws Exception {
+    readFileWithMissingBlocks(new Path("/foo"), fileLength, 3, 0);
+  }
+
+  private void readFileWithMissingBlocks(Path srcPath, int fileLength,
+      int missingDataNum, int missingParityNum)
+      throws Exception {
+    LOG.info("readFileWithMissingBlocks: (" + missingDataNum + ","
+        + missingParityNum + ")");
+    final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
+    DFSTestUtil.writeFile(fs, srcPath, new String(expected));
+    StripedFileTestUtil.waitBlockGroupsReported(fs, srcPath.toUri().getPath());
+    StripedFileTestUtil.verifyLength(fs, srcPath, fileLength);
+    int dataBlocks = (fileLength - 1) / cellSize + 1;
+    BlockLocation[] locs = fs.getFileBlockLocations(srcPath, 0, cellSize);
+
+    int[] missingDataNodes = new int[missingDataNum + missingParityNum];
+    for (int i = 0; i < missingDataNum; i++) {
+      missingDataNodes[i] = i;
+    }
+    for (int i = 0; i < missingParityNum; i++) {
+      missingDataNodes[i + missingDataNum] = i +
+          Math.min(StripedFileTestUtil.NUM_DATA_BLOCKS, dataBlocks);
+    }
+    stopDataNodes(locs, missingDataNodes);
+
+    // make sure there are missing block locations
+    BlockLocation[] newLocs = fs.getFileBlockLocations(srcPath, 0, cellSize);
+    Assert.assertTrue(newLocs[0].getNames().length < locs[0].getNames().length);
+
+    byte[] smallBuf = new byte[1024];
+    byte[] largeBuf = new byte[fileLength + 100];
+    StripedFileTestUtil.verifySeek(fs, srcPath, fileLength);
+    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
+        smallBuf);
+    StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf);
+
+    // delete the file
+    fs.delete(srcPath, true);
+  }
+
+  private void stopDataNodes(BlockLocation[] locs, int[] datanodes)
+      throws IOException {
+    if (locs != null && locs.length > 0) {
+      for (int failedDNIdx : datanodes) {
+        String name = (locs[0].getNames())[failedDNIdx];
+        for (DataNode dn : cluster.getDataNodes()) {
+          int port = dn.getXferPort();
+          if (name.contains(Integer.toString(port))) {
+            dn.shutdown();
+            cluster.setDataNodeDead(dn.getDatanodeId());
+            LOG.info("stop datanode " + failedDNIdx);
+            break;
+          }
+        }
+      }
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
new file mode 100644
index 0000000..75dc6a02
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java
@@ -0,0 +1,418 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+
+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.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestRecoverStripedFile {
+  public static final Log LOG = LogFactory.getLog(TestRecoverStripedFile.class);
+  
+  private static final int dataBlkNum = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private static final int parityBlkNum = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private static final int blockSize = cellSize * 3;
+  private static final int groupSize = dataBlkNum + parityBlkNum;
+  private static final int dnNum = groupSize + parityBlkNum;
+  
+  private MiniDFSCluster cluster;
+  private Configuration conf;
+  private DistributedFileSystem fs;
+  // Map: DatanodeID -> datanode index in cluster
+  private Map<DatanodeID, Integer> dnMap = new HashMap<DatanodeID, Integer>();
+
+  @Before
+  public void setup() throws IOException {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY, cellSize - 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dnNum).build();;
+    cluster.waitActive();
+    
+    fs = cluster.getFileSystem();
+    fs.getClient().setErasureCodingPolicy("/", null);
+
+    List<DataNode> datanodes = cluster.getDataNodes();
+    for (int i = 0; i < dnNum; i++) {
+      dnMap.put(datanodes.get(i).getDatanodeId(), i);
+    }
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverOneParityBlock() throws Exception {
+    int fileLen = 10 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverOneParityBlock", fileLen, 0, 1);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverOneParityBlock1() throws Exception {
+    int fileLen = cellSize + cellSize/10;
+    assertFileBlocksRecovery("/testRecoverOneParityBlock1", fileLen, 0, 1);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverOneParityBlock2() throws Exception {
+    int fileLen = 1;
+    assertFileBlocksRecovery("/testRecoverOneParityBlock2", fileLen, 0, 1);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverOneParityBlock3() throws Exception {
+    int fileLen = 3 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverOneParityBlock3", fileLen, 0, 1);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverThreeParityBlocks() throws Exception {
+    int fileLen = 10 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverThreeParityBlocks", fileLen, 0, 3);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverThreeDataBlocks() throws Exception {
+    int fileLen = 10 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverThreeDataBlocks", fileLen, 1, 3);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverThreeDataBlocks1() throws Exception {
+    int fileLen = 3 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverThreeDataBlocks1", fileLen, 1, 3);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverOneDataBlock() throws Exception {
+    int fileLen = 10 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverOneDataBlock", fileLen, 1, 1);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverOneDataBlock1() throws Exception {
+    int fileLen = cellSize + cellSize/10;
+    assertFileBlocksRecovery("/testRecoverOneDataBlock1", fileLen, 1, 1);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverOneDataBlock2() throws Exception {
+    int fileLen = 1;
+    assertFileBlocksRecovery("/testRecoverOneDataBlock2", fileLen, 1, 1);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverAnyBlocks() throws Exception {
+    int fileLen = 3 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverAnyBlocks", fileLen, 2, 2);
+  }
+  
+  @Test(timeout = 120000)
+  public void testRecoverAnyBlocks1() throws Exception {
+    int fileLen = 10 * blockSize + blockSize/10;
+    assertFileBlocksRecovery("/testRecoverAnyBlocks1", fileLen, 2, 3);
+  }
+  
+  /**
+   * Test the file blocks recovery.
+   * 1. Check the replica is recovered in the target datanode, 
+   *    and verify the block replica length, generationStamp and content.
+   * 2. Read the file and verify content. 
+   */
+  private void assertFileBlocksRecovery(String fileName, int fileLen,
+      int recovery, int toRecoverBlockNum) throws Exception {
+    if (recovery != 0 && recovery != 1 && recovery != 2) {
+      Assert.fail("Invalid recovery: 0 is to recovery parity blocks,"
+          + "1 is to recovery data blocks, 2 is any.");
+    }
+    if (toRecoverBlockNum < 1 || toRecoverBlockNum > parityBlkNum) {
+      Assert.fail("toRecoverBlockNum should be between 1 ~ " + parityBlkNum);
+    }
+    
+    Path file = new Path(fileName);
+
+    final byte[] data = new byte[fileLen];
+    ThreadLocalRandom.current().nextBytes(data);
+    DFSTestUtil.writeFile(fs, file, data);
+    StripedFileTestUtil.waitBlockGroupsReported(fs, fileName);
+
+    LocatedBlocks locatedBlocks = getLocatedBlocks(file);
+    assertEquals(locatedBlocks.getFileLength(), fileLen);
+    
+    LocatedStripedBlock lastBlock = 
+        (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
+    
+    DatanodeInfo[] storageInfos = lastBlock.getLocations();
+    int[] indices = lastBlock.getBlockIndices();
+    
+    BitSet bitset = new BitSet(dnNum);
+    for (DatanodeInfo storageInfo : storageInfos) {
+      bitset.set(dnMap.get(storageInfo));
+    }
+    
+    int[] toDead = new int[toRecoverBlockNum];
+    int n = 0;
+    for (int i = 0; i < indices.length; i++) {
+      if (n < toRecoverBlockNum) {
+        if (recovery == 0) {
+          if (indices[i] >= dataBlkNum) {
+            toDead[n++] = i;
+          }
+        } else if (recovery == 1) {
+          if (indices[i] < dataBlkNum) {
+            toDead[n++] = i;
+          }
+        } else {
+          toDead[n++] = i;
+        }
+      } else {
+        break;
+      }
+    }
+    
+    DatanodeInfo[] dataDNs = new DatanodeInfo[toRecoverBlockNum];
+    int[] deadDnIndices = new int[toRecoverBlockNum];
+    ExtendedBlock[] blocks = new ExtendedBlock[toRecoverBlockNum];
+    File[] replicas = new File[toRecoverBlockNum];
+    File[] metadatas = new File[toRecoverBlockNum];
+    byte[][] replicaContents = new byte[toRecoverBlockNum][];
+    for (int i = 0; i < toRecoverBlockNum; i++) {
+      dataDNs[i] = storageInfos[toDead[i]];
+      deadDnIndices[i] = dnMap.get(dataDNs[i]);
+      
+      // Check the block replica file on deadDn before it dead.
+      blocks[i] = StripedBlockUtil.constructInternalBlock(
+          lastBlock.getBlock(), cellSize, dataBlkNum, indices[toDead[i]]);
+      replicas[i] = cluster.getBlockFile(deadDnIndices[i], blocks[i]);
+      metadatas[i] = cluster.getBlockMetadataFile(deadDnIndices[i], blocks[i]);
+      // the block replica on the datanode should be the same as expected
+      assertEquals(replicas[i].length(), 
+          StripedBlockUtil.getInternalBlockLength(
+          lastBlock.getBlockSize(), cellSize, dataBlkNum, indices[toDead[i]]));
+      assertTrue(metadatas[i].getName().
+          endsWith(blocks[i].getGenerationStamp() + ".meta"));
+      replicaContents[i] = readReplica(replicas[i]);
+    }
+    
+    int cellsNum = (fileLen - 1) / cellSize + 1;
+    int groupSize = Math.min(cellsNum, dataBlkNum) + parityBlkNum;
+
+    try {
+      DatanodeID[] dnIDs = new DatanodeID[toRecoverBlockNum];
+      for (int i = 0; i < toRecoverBlockNum; i++) {
+        /*
+         * Kill the datanode which contains one replica
+         * We need to make sure it dead in namenode: clear its update time and 
+         * trigger NN to check heartbeat.
+         */
+        DataNode dn = cluster.getDataNodes().get(deadDnIndices[i]);
+        dn.shutdown();
+        dnIDs[i] = dn.getDatanodeId();
+      }
+      setDataNodesDead(dnIDs);
+      
+      // Check the locatedBlocks of the file again
+      locatedBlocks = getLocatedBlocks(file);
+      lastBlock = (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
+      storageInfos = lastBlock.getLocations();
+      assertEquals(storageInfos.length, groupSize - toRecoverBlockNum);
+      
+      int[] targetDNs = new int[dnNum - groupSize];
+      n = 0;
+      for (int i = 0; i < dnNum; i++) {
+        if (!bitset.get(i)) { // not contain replica of the block.
+          targetDNs[n++] = i;
+        }
+      }
+      
+      waitForRecoveryFinished(file, groupSize);
+      
+      targetDNs = sortTargetsByReplicas(blocks, targetDNs);
+      
+      // Check the replica on the new target node.
+      for (int i = 0; i < toRecoverBlockNum; i++) {
+        File replicaAfterRecovery = cluster.getBlockFile(targetDNs[i], blocks[i]);
+        File metadataAfterRecovery = 
+            cluster.getBlockMetadataFile(targetDNs[i], blocks[i]);
+        assertEquals(replicaAfterRecovery.length(), replicas[i].length());
+        assertTrue(metadataAfterRecovery.getName().
+            endsWith(blocks[i].getGenerationStamp() + ".meta"));
+        byte[] replicaContentAfterRecovery = readReplica(replicaAfterRecovery);
+        
+        Assert.assertArrayEquals(replicaContents[i], replicaContentAfterRecovery);
+      }
+    } finally {
+      for (int i = 0; i < toRecoverBlockNum; i++) {
+        restartDataNode(toDead[i]);
+      }
+      cluster.waitActive();
+    }
+    fs.delete(file, true);
+  }
+  
+  private void setDataNodesDead(DatanodeID[] dnIDs) throws IOException {
+    for (DatanodeID dn : dnIDs) {
+      DatanodeDescriptor dnd =
+          NameNodeAdapter.getDatanode(cluster.getNamesystem(), dn);
+      DFSTestUtil.setDatanodeDead(dnd);
+    }
+    
+    BlockManagerTestUtil.checkHeartbeat(cluster.getNamesystem().getBlockManager());
+  }
+  
+  private void restartDataNode(int dn) {
+    try {
+      cluster.restartDataNode(dn, true, true);
+    } catch (IOException e) {
+    }
+  }
+  
+  private int[] sortTargetsByReplicas(ExtendedBlock[] blocks, int[] targetDNs) {
+    int[] result = new int[blocks.length];
+    for (int i = 0; i < blocks.length; i++) {
+      result[i] = -1;
+      for (int j = 0; j < targetDNs.length; j++) {
+        if (targetDNs[j] != -1) {
+          File replica = cluster.getBlockFile(targetDNs[j], blocks[i]);
+          if (replica != null) {
+            result[i] = targetDNs[j];
+            targetDNs[j] = -1;
+            break;
+          }
+        }
+      }
+      if (result[i] == -1) {
+        Assert.fail("Failed to recover striped block: " + blocks[i].getBlockId());
+      }
+    }
+    return result;
+  }
+  
+  private byte[] readReplica(File replica) throws IOException {
+    int length = (int)replica.length();
+    ByteArrayOutputStream content = new ByteArrayOutputStream(length);
+    FileInputStream in = new FileInputStream(replica);
+    try {
+      byte[] buffer = new byte[1024];
+      int total = 0;
+      while (total < length) {
+        int n = in.read(buffer);
+        if (n <= 0) {
+          break;
+        }
+        content.write(buffer, 0, n);
+        total += n;
+      }
+      if (total < length) {
+        Assert.fail("Failed to read all content of replica");
+      }
+      return content.toByteArray();
+    } finally {
+      in.close();
+    }
+  }
+  
+  private LocatedBlocks waitForRecoveryFinished(Path file, int groupSize) 
+      throws Exception {
+    final int ATTEMPTS = 60;
+    for (int i = 0; i < ATTEMPTS; i++) {
+      LocatedBlocks locatedBlocks = getLocatedBlocks(file);
+      LocatedStripedBlock lastBlock = 
+          (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
+      DatanodeInfo[] storageInfos = lastBlock.getLocations();
+      if (storageInfos.length >= groupSize) {
+        return locatedBlocks;
+      }
+      Thread.sleep(1000);
+    }
+    throw new IOException ("Time out waiting for EC block recovery.");
+  }
+  
+  private LocatedBlocks getLocatedBlocks(Path file) throws IOException {
+    return fs.getClient().getLocatedBlocks(file.toString(), 0, Long.MAX_VALUE);
+  }
+
+  /*
+   * Tests that processErasureCodingTasks should not throw exceptions out due to
+   * invalid ECTask submission.
+   */
+  @Test
+  public void testProcessErasureCodingTasksSubmitionShouldSucceed()
+      throws Exception {
+    DataNode dataNode = cluster.dataNodes.get(0).datanode;
+
+    // Pack invalid(dummy) parameters in ecTasks. Irrespective of parameters, each task
+    // thread pool submission should succeed, so that it will not prevent
+    // processing other tasks in the list if any exceptions.
+    int size = cluster.dataNodes.size();
+    short[] liveIndices = new short[size];
+    DatanodeInfo[] dataDNs = new DatanodeInfo[size + 1];
+    DatanodeStorageInfo targetDnInfos_1 = BlockManagerTestUtil
+        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
+            new DatanodeStorage("s01"));
+    DatanodeStorageInfo[] dnStorageInfo = new DatanodeStorageInfo[] {
+        targetDnInfos_1 };
+
+    BlockECRecoveryInfo invalidECInfo = new BlockECRecoveryInfo(
+        new ExtendedBlock("bp-id", 123456), dataDNs, dnStorageInfo, liveIndices,
+        ErasureCodingPolicyManager.getSystemDefaultPolicy());
+    List<BlockECRecoveryInfo> ecTasks = new ArrayList<BlockECRecoveryInfo>();
+    ecTasks.add(invalidECInfo);
+    dataNode.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
+  }
+}
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 6cea7e8..6b4e46a 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,7 @@
       if(cluster!= null) cluster.shutdown();
     }
   }
-  
+
   void checkGetBlockLocationsWorks(FileSystem fs, Path fileName) throws IOException {
     FileStatus stat = fs.getFileStatus(fileName);
     try {  
@@ -560,7 +560,7 @@
     } 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);
     }    
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
new file mode 100644
index 0000000..9853b8a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestSafeModeWithStripedFile {
+
+  static final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  static final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  static final int numDNs = DATA_BLK_NUM + PARITY_BLK_NUM;
+  static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  static final int blockSize = cellSize * 2;
+
+  static MiniDFSCluster cluster;
+  static Configuration conf;
+
+  @Before
+  public void setup() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 100);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
+    cluster.waitActive();
+
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testStripedFile0() throws IOException {
+    doTest(cellSize, 1);
+  }
+
+  @Test
+  public void testStripedFile1() throws IOException {
+    doTest(cellSize * 5, 5);
+  }
+
+  /**
+   * This util writes a small block group whose size is given by caller.
+   * Then write another 2 full stripe blocks.
+   * Then shutdown all DNs and start again one by one. and verify the safemode
+   * status accordingly.
+   *
+   * @param smallSize file size of the small block group
+   * @param minStorages minimum replicas needed by the block so it can be safe
+   */
+  private void doTest(int smallSize, int minStorages) throws IOException {
+    FileSystem fs = cluster.getFileSystem();
+    // add 1 block
+    byte[] data = StripedFileTestUtil.generateBytes(smallSize);
+    Path smallFilePath = new Path("/testStripedFile_" + smallSize);
+    DFSTestUtil.writeFile(fs, smallFilePath, data);
+
+    // If we only have 1 block, NN won't enter safemode in the first place
+    // because the threshold is 0 blocks.
+    // So we need to add another 2 blocks.
+    int bigSize = blockSize * DATA_BLK_NUM * 2;
+    Path bigFilePath = new Path("/testStripedFile_" + bigSize);
+    data = StripedFileTestUtil.generateBytes(bigSize);
+    DFSTestUtil.writeFile(fs, bigFilePath, data);
+    // now we have 3 blocks. NN needs 2 blocks to reach the threshold 0.9 of
+    // total blocks 3.
+
+    // stopping all DNs
+    List<MiniDFSCluster.DataNodeProperties> dnprops = Lists.newArrayList();
+    LocatedBlocks lbs = cluster.getNameNodeRpc()
+        .getBlockLocations(smallFilePath.toString(), 0, smallSize);
+    DatanodeInfo[] locations = lbs.get(0).getLocations();
+    for (DatanodeInfo loc : locations) {
+      // keep the DNs that have smallFile in the head of dnprops
+      dnprops.add(cluster.stopDataNode(loc.getName()));
+    }
+    for (int i = 0; i < numDNs - locations.length; i++) {
+      dnprops.add(cluster.stopDataNode(0));
+    }
+
+    cluster.restartNameNode(0);
+    NameNode nn = cluster.getNameNode();
+    assertTrue(cluster.getNameNode().isInSafeMode());
+    assertEquals(0, NameNodeAdapter.getSafeModeSafeBlocks(nn));
+
+    // the block of smallFile doesn't reach minStorages,
+    // so the safe blocks count doesn't increment.
+    for (int i = 0; i < minStorages - 1; i++) {
+      cluster.restartDataNode(dnprops.remove(0));
+      cluster.waitActive();
+      cluster.triggerBlockReports();
+      assertEquals(0, NameNodeAdapter.getSafeModeSafeBlocks(nn));
+    }
+
+    // the block of smallFile reaches minStorages,
+    // so the safe blocks count increment.
+    cluster.restartDataNode(dnprops.remove(0));
+    cluster.waitActive();
+    cluster.triggerBlockReports();
+    assertEquals(1, NameNodeAdapter.getSafeModeSafeBlocks(nn));
+
+    // the 2 blocks of bigFile need DATA_BLK_NUM storages to be safe
+    for (int i = minStorages; i < DATA_BLK_NUM - 1; i++) {
+      cluster.restartDataNode(dnprops.remove(0));
+      cluster.waitActive();
+      cluster.triggerBlockReports();
+      assertTrue(nn.isInSafeMode());
+    }
+
+    cluster.restartDataNode(dnprops.remove(0));
+    cluster.waitActive();
+    cluster.triggerBlockReports();
+    assertFalse(nn.isInSafeMode());
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
new file mode 100644
index 0000000..3d51f0c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
@@ -0,0 +1,251 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.stripesPerBlock;
+
+public class TestWriteReadStripedFile {
+  public static final Log LOG = LogFactory.getLog(TestWriteReadStripedFile.class);
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem fs;
+  private static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private static short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private static Configuration conf = new HdfsConfiguration();
+
+  static {
+    ((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class))
+        .getLogger().setLevel(Level.ALL);
+  }
+
+  @Before
+  public void setup() throws IOException {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
+    fs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testFileEmpty() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/EmptyFile", 0);
+    testOneFileUsingDFSStripedInputStream("/EmptyFile2", 0, true);
+  }
+
+  @Test
+  public void testFileSmallerThanOneCell1() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1);
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell2", 1, true);
+  }
+
+  @Test
+  public void testFileSmallerThanOneCell2() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1);
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell2", cellSize - 1,
+        true);
+  }
+
+  @Test
+  public void testFileEqualsWithOneCell() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize);
+    testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell2", cellSize, true);
+  }
+
+  @Test
+  public void testFileSmallerThanOneStripe1() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe",
+        cellSize * dataBlocks - 1);
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe2",
+        cellSize * dataBlocks - 1, true);
+  }
+
+  @Test
+  public void testFileSmallerThanOneStripe2() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe",
+        cellSize + 123);
+    testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe2",
+        cellSize + 123, true);
+  }
+
+  @Test
+  public void testFileEqualsWithOneStripe() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe",
+        cellSize * dataBlocks);
+    testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe2",
+        cellSize * dataBlocks, true);
+  }
+
+  @Test
+  public void testFileMoreThanOneStripe1() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1",
+        cellSize * dataBlocks + 123);
+    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe12",
+        cellSize * dataBlocks + 123, true);
+  }
+
+  @Test
+  public void testFileMoreThanOneStripe2() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2",
+        cellSize * dataBlocks + cellSize * dataBlocks + 123);
+    testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe22",
+        cellSize * dataBlocks + cellSize * dataBlocks + 123, true);
+  }
+
+  @Test
+  public void testLessThanFullBlockGroup() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup",
+        cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize);
+    testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup2",
+        cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize, true);
+  }
+
+  @Test
+  public void testFileFullBlockGroup() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/FullBlockGroup",
+        blockSize * dataBlocks);
+    testOneFileUsingDFSStripedInputStream("/FullBlockGroup2",
+        blockSize * dataBlocks, true);
+  }
+
+  @Test
+  public void testFileMoreThanABlockGroup1() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1",
+        blockSize * dataBlocks + 123);
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup12",
+        blockSize * dataBlocks + 123, true);
+  }
+
+  @Test
+  public void testFileMoreThanABlockGroup2() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2",
+        blockSize * dataBlocks + cellSize + 123);
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup22",
+        blockSize * dataBlocks + cellSize + 123, true);
+  }
+
+
+  @Test
+  public void testFileMoreThanABlockGroup3() throws Exception {
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3",
+        blockSize * dataBlocks * 3 + cellSize * dataBlocks
+            + cellSize + 123);
+    testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup32",
+        blockSize * dataBlocks * 3 + cellSize * dataBlocks
+            + cellSize + 123, true);
+  }
+
+  private void testOneFileUsingDFSStripedInputStream(String src, int fileLength)
+      throws Exception {
+    testOneFileUsingDFSStripedInputStream(src, fileLength, false);
+  }
+
+  private void testOneFileUsingDFSStripedInputStream(String src, int fileLength,
+      boolean withDataNodeFailure) throws Exception {
+    final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
+    Path srcPath = new Path(src);
+    DFSTestUtil.writeFile(fs, srcPath, new String(expected));
+    StripedFileTestUtil.waitBlockGroupsReported(fs, src);
+
+    StripedFileTestUtil.verifyLength(fs, srcPath, fileLength);
+
+    if (withDataNodeFailure) {
+      int dnIndex = 1; // TODO: StripedFileTestUtil.random.nextInt(dataBlocks);
+      LOG.info("stop DataNode " + dnIndex);
+      stopDataNode(srcPath, dnIndex);
+    }
+
+    byte[] smallBuf = new byte[1024];
+    byte[] largeBuf = new byte[fileLength + 100];
+    StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf);
+
+    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
+        largeBuf);
+    StripedFileTestUtil.verifySeek(fs, srcPath, fileLength);
+    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
+        ByteBuffer.allocate(fileLength + 100));
+    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
+        smallBuf);
+    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
+        ByteBuffer.allocate(1024));
+  }
+
+  private void stopDataNode(Path path, int failedDNIdx)
+      throws IOException {
+    BlockLocation[] locs = fs.getFileBlockLocations(path, 0, cellSize);
+    if (locs != null && locs.length > 0) {
+      String name = (locs[0].getNames())[failedDNIdx];
+      for (DataNode dn : cluster.getDataNodes()) {
+        int port = dn.getXferPort();
+        if (name.contains(Integer.toString(port))) {
+          dn.shutdown();
+          break;
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testWriteReadUsingWebHdfs() throws Exception {
+    int fileLength = blockSize * dataBlocks + cellSize + 123;
+
+    final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
+    FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+    Path srcPath = new Path("/testWriteReadUsingWebHdfs");
+    DFSTestUtil.writeFile(fs, srcPath, new String(expected));
+
+    StripedFileTestUtil.verifyLength(fs, srcPath, fileLength);
+
+    byte[] smallBuf = new byte[1024];
+    byte[] largeBuf = new byte[fileLength + 100];
+    // TODO: HDFS-8797
+    //StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf);
+
+    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf);
+    StripedFileTestUtil.verifySeek(fs, srcPath, fileLength);
+    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf);
+    // webhdfs doesn't support bytebuffer read
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
new file mode 100644
index 0000000..764527d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
@@ -0,0 +1,171 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
+
+public class TestWriteStripedFileWithFailure {
+  public static final Log LOG = LogFactory
+      .getLog(TestWriteStripedFileWithFailure.class);
+  private static MiniDFSCluster cluster;
+  private static FileSystem fs;
+  private static Configuration conf = new HdfsConfiguration();
+
+  static {
+    GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
+  }
+
+  private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final int smallFileLength = blockSize * dataBlocks - 123;
+  private final int largeFileLength = blockSize * dataBlocks + 123;
+  private final int[] fileLengths = {smallFileLength, largeFileLength};
+
+  public void setup() throws IOException {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
+    fs = cluster.getFileSystem();
+  }
+
+  public void tearDown() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  // Test writing file with some Datanodes failure
+  // TODO: enable this test after HDFS-8704 and HDFS-9040
+  @Ignore
+  @Test(timeout = 300000)
+  public void testWriteStripedFileWithDNFailure() throws IOException {
+    for (int fileLength : fileLengths) {
+      for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) {
+        for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) {
+          try {
+            // setup a new cluster with no dead datanode
+            setup();
+            writeFileWithDNFailure(fileLength, dataDelNum, parityDelNum);
+          } catch (IOException ioe) {
+            String fileType = fileLength < (blockSize * dataBlocks) ?
+                "smallFile" : "largeFile";
+            LOG.error("Failed to write file with DN failure:"
+                + " fileType = "+ fileType
+                + ", dataDelNum = " + dataDelNum
+                + ", parityDelNum = " + parityDelNum);
+            throw ioe;
+          } finally {
+            // tear down the cluster
+            tearDown();
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Test writing a file with shutting down some DNs(data DNs or parity DNs or both).
+   * @param fileLength file length
+   * @param dataDNFailureNum the shutdown number of data DNs
+   * @param parityDNFailureNum the shutdown number of parity DNs
+   * @throws IOException
+   */
+  private void writeFileWithDNFailure(int fileLength,
+      int dataDNFailureNum, int parityDNFailureNum) throws IOException {
+    String fileType = fileLength < (blockSize * dataBlocks) ?
+        "smallFile" : "largeFile";
+    String src = "/dnFailure_" + dataDNFailureNum + "_" + parityDNFailureNum
+        + "_" + fileType;
+    LOG.info("writeFileWithDNFailure: file = " + src
+        + ", fileType = " + fileType
+        + ", dataDNFailureNum = " + dataDNFailureNum
+        + ", parityDNFailureNum = " + parityDNFailureNum);
+
+    Path srcPath = new Path(src);
+    final AtomicInteger pos = new AtomicInteger();
+    final FSDataOutputStream out = fs.create(srcPath);
+    final DFSStripedOutputStream stripedOut
+        = (DFSStripedOutputStream)out.getWrappedStream();
+
+    int[] dataDNFailureIndices = StripedFileTestUtil.randomArray(0, dataBlocks,
+        dataDNFailureNum);
+    Assert.assertNotNull(dataDNFailureIndices);
+    int[] parityDNFailureIndices = StripedFileTestUtil.randomArray(dataBlocks,
+        dataBlocks + parityBlocks, parityDNFailureNum);
+    Assert.assertNotNull(parityDNFailureIndices);
+
+    int[] failedDataNodes = new int[dataDNFailureNum + parityDNFailureNum];
+    System.arraycopy(dataDNFailureIndices, 0, failedDataNodes,
+        0, dataDNFailureIndices.length);
+    System.arraycopy(parityDNFailureIndices, 0, failedDataNodes,
+        dataDNFailureIndices.length, parityDNFailureIndices.length);
+
+    final int killPos = fileLength/2;
+    for (; pos.get() < fileLength; ) {
+      final int i = pos.getAndIncrement();
+      if (i == killPos) {
+        for(int failedDn : failedDataNodes) {
+          StripedFileTestUtil.killDatanode(cluster, stripedOut, failedDn, pos);
+        }
+      }
+      write(out, i);
+    }
+    out.close();
+
+    // make sure the expected number of Datanode have been killed
+    int dnFailureNum = dataDNFailureNum + parityDNFailureNum;
+    Assert.assertEquals(cluster.getDataNodes().size(), numDNs - dnFailureNum);
+
+    byte[] smallBuf = new byte[1024];
+    byte[] largeBuf = new byte[fileLength + 100];
+    final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
+    StripedFileTestUtil.verifyLength(fs, srcPath, fileLength);
+    StripedFileTestUtil.verifySeek(fs, srcPath, fileLength);
+    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
+        smallBuf);
+    StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf);
+
+    // delete the file
+    fs.delete(srcPath, true);
+  }
+
+  void write(FSDataOutputStream out, int i) throws IOException {
+    try {
+      out.write(StripedFileTestUtil.getByte(i));
+    } catch (IOException e) {
+      throw new IOException("Failed at i=" + i, e);
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
index 9f8aef5..e944b81 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
@@ -126,7 +126,8 @@
     EnumSet<NameNodeLayoutVersion.Feature> compatibleFeatures = EnumSet.of(
         NameNodeLayoutVersion.Feature.TRUNCATE,
         NameNodeLayoutVersion.Feature.APPEND_NEW_BLOCK,
-        NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE);
+        NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE,
+        NameNodeLayoutVersion.Feature.ERASURE_CODING);
     for (LayoutFeature f : compatibleFeatures) {
       assertEquals(String.format("Expected minimum compatible layout version " +
           "%d for feature %s.", baseLV, f), baseLV,
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 2bfba98..4171d5c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@ -24,6 +24,7 @@
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -33,6 +34,7 @@
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -40,6 +42,7 @@
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
@@ -63,15 +66,21 @@
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -80,6 +89,7 @@
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
@@ -181,40 +191,59 @@
     assertEquals(b, b2);
   }
 
-  private static BlockWithLocations getBlockWithLocations(int bid) {
+  private static BlockWithLocations getBlockWithLocations(
+      int bid, boolean isStriped) {
     final String[] datanodeUuids = {"dn1", "dn2", "dn3"};
     final String[] storageIDs = {"s1", "s2", "s3"};
     final StorageType[] storageTypes = {
         StorageType.DISK, StorageType.DISK, StorageType.DISK};
-    return new BlockWithLocations(new Block(bid, 0, 1),
+    final byte[] indices = {0, 1, 2};
+    final short dataBlkNum = 6;
+    BlockWithLocations blkLocs = new BlockWithLocations(new Block(bid, 0, 1),
         datanodeUuids, storageIDs, storageTypes);
+    if (isStriped) {
+      blkLocs = new StripedBlockWithLocations(blkLocs, indices, dataBlkNum,
+          StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE);
+    }
+    return blkLocs;
   }
 
   private void compare(BlockWithLocations locs1, BlockWithLocations locs2) {
     assertEquals(locs1.getBlock(), locs2.getBlock());
     assertTrue(Arrays.equals(locs1.getStorageIDs(), locs2.getStorageIDs()));
+    if (locs1 instanceof StripedBlockWithLocations) {
+      assertTrue(Arrays.equals(((StripedBlockWithLocations) locs1).getIndices(),
+          ((StripedBlockWithLocations) locs2).getIndices()));
+    }
   }
 
   @Test
   public void testConvertBlockWithLocations() {
-    BlockWithLocations locs = getBlockWithLocations(1);
-    BlockWithLocationsProto locsProto = PBHelper.convert(locs);
-    BlockWithLocations locs2 = PBHelper.convert(locsProto);
-    compare(locs, locs2);
+    boolean[] testSuite = new boolean[]{false, true};
+    for (int i = 0; i < testSuite.length; i++) {
+      BlockWithLocations locs = getBlockWithLocations(1, testSuite[i]);
+      BlockWithLocationsProto locsProto = PBHelper.convert(locs);
+      BlockWithLocations locs2 = PBHelper.convert(locsProto);
+      compare(locs, locs2);
+    }
   }
 
   @Test
   public void testConvertBlocksWithLocations() {
-    BlockWithLocations[] list = new BlockWithLocations[] {
-        getBlockWithLocations(1), getBlockWithLocations(2) };
-    BlocksWithLocations locs = new BlocksWithLocations(list);
-    BlocksWithLocationsProto locsProto = PBHelper.convert(locs);
-    BlocksWithLocations locs2 = PBHelper.convert(locsProto);
-    BlockWithLocations[] blocks = locs.getBlocks();
-    BlockWithLocations[] blocks2 = locs2.getBlocks();
-    assertEquals(blocks.length, blocks2.length);
-    for (int i = 0; i < blocks.length; i++) {
-      compare(blocks[i], blocks2[i]);
+    boolean[] testSuite = new boolean[]{false, true};
+    for (int i = 0; i < testSuite.length; i++) {
+      BlockWithLocations[] list = new BlockWithLocations[]{
+          getBlockWithLocations(1, testSuite[i]),
+          getBlockWithLocations(2, testSuite[i])};
+      BlocksWithLocations locs = new BlocksWithLocations(list);
+      BlocksWithLocationsProto locsProto = PBHelper.convert(locs);
+      BlocksWithLocations locs2 = PBHelper.convert(locsProto);
+      BlockWithLocations[] blocks = locs.getBlocks();
+      BlockWithLocations[] blocks2 = locs2.getBlocks();
+      assertEquals(blocks.length, blocks2.length);
+      for (int j = 0; j < blocks.length; j++) {
+        compare(blocks[j], blocks2[j]);
+      }
     }
   }
 
@@ -489,16 +518,16 @@
   @Test
   public void testConvertLocatedBlock() {
     LocatedBlock lb = createLocatedBlock();
-    LocatedBlockProto lbProto = PBHelperClient.convert(lb);
-    LocatedBlock lb2 = PBHelperClient.convert(lbProto);
+    LocatedBlockProto lbProto = PBHelperClient.convertLocatedBlock(lb);
+    LocatedBlock lb2 = PBHelperClient.convertLocatedBlockProto(lbProto);
     compare(lb,lb2);
   }
 
   @Test
   public void testConvertLocatedBlockNoStorageMedia() {
     LocatedBlock lb = createLocatedBlockNoStorageMedia();
-    LocatedBlockProto lbProto = PBHelperClient.convert(lb);
-    LocatedBlock lb2 = PBHelperClient.convert(lbProto);
+    LocatedBlockProto lbProto = PBHelperClient.convertLocatedBlock(lb);
+    LocatedBlock lb2 = PBHelperClient.convertLocatedBlockProto(lbProto);
     compare(lb,lb2);
   }
 
@@ -508,8 +537,8 @@
     for (int i=0;i<3;i++) {
       lbl.add(createLocatedBlock());
     }
-    List<LocatedBlockProto> lbpl = PBHelperClient.convertLocatedBlock2(lbl);
-    List<LocatedBlock> lbl2 = PBHelperClient.convertLocatedBlock(lbpl);
+    List<LocatedBlockProto> lbpl = PBHelperClient.convertLocatedBlocks2(lbl);
+    List<LocatedBlock> lbl2 = PBHelperClient.convertLocatedBlocks(lbpl);
     assertEquals(lbl.size(), lbl2.size());
     for (int i=0;i<lbl.size();i++) {
       compare(lbl.get(i), lbl2.get(2));
@@ -522,8 +551,8 @@
     for (int i=0;i<3;i++) {
       lbl[i] = createLocatedBlock();
     }
-    LocatedBlockProto [] lbpl = PBHelperClient.convertLocatedBlock(lbl);
-    LocatedBlock [] lbl2 = PBHelperClient.convertLocatedBlock(lbpl);
+    LocatedBlockProto [] lbpl = PBHelperClient.convertLocatedBlocks(lbl);
+    LocatedBlock [] lbl2 = PBHelperClient.convertLocatedBlocks(lbpl);
     assertEquals(lbl.length, lbl2.length);
     for (int i=0;i<lbl.length;i++) {
       compare(lbl[i], lbl2[i]);
@@ -639,4 +668,97 @@
         .build();
     Assert.assertEquals(s, PBHelperClient.convert(PBHelperClient.convert(s)));
   }
+  
+  @Test
+  public void testBlockECRecoveryCommand() {
+    DatanodeInfo[] dnInfos0 = new DatanodeInfo[] {
+        DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
+    DatanodeStorageInfo targetDnInfos_0 = BlockManagerTestUtil
+        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
+            new DatanodeStorage("s00"));
+    DatanodeStorageInfo targetDnInfos_1 = BlockManagerTestUtil
+        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
+            new DatanodeStorage("s01"));
+    DatanodeStorageInfo[] targetDnInfos0 = new DatanodeStorageInfo[] {
+        targetDnInfos_0, targetDnInfos_1 };
+    short[] liveBlkIndices0 = new short[2];
+    BlockECRecoveryInfo blkECRecoveryInfo0 = new BlockECRecoveryInfo(
+        new ExtendedBlock("bp1", 1234), dnInfos0, targetDnInfos0,
+        liveBlkIndices0, ErasureCodingPolicyManager.getSystemDefaultPolicy());
+    DatanodeInfo[] dnInfos1 = new DatanodeInfo[] {
+        DFSTestUtil.getLocalDatanodeInfo(), DFSTestUtil.getLocalDatanodeInfo() };
+    DatanodeStorageInfo targetDnInfos_2 = BlockManagerTestUtil
+        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
+            new DatanodeStorage("s02"));
+    DatanodeStorageInfo targetDnInfos_3 = BlockManagerTestUtil
+        .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(),
+            new DatanodeStorage("s03"));
+    DatanodeStorageInfo[] targetDnInfos1 = new DatanodeStorageInfo[] {
+        targetDnInfos_2, targetDnInfos_3 };
+    short[] liveBlkIndices1 = new short[2];
+    BlockECRecoveryInfo blkECRecoveryInfo1 = new BlockECRecoveryInfo(
+        new ExtendedBlock("bp2", 3256), dnInfos1, targetDnInfos1,
+        liveBlkIndices1, ErasureCodingPolicyManager.getSystemDefaultPolicy());
+    List<BlockECRecoveryInfo> blkRecoveryInfosList = new ArrayList<BlockECRecoveryInfo>();
+    blkRecoveryInfosList.add(blkECRecoveryInfo0);
+    blkRecoveryInfosList.add(blkECRecoveryInfo1);
+    BlockECRecoveryCommand blkECRecoveryCmd = new BlockECRecoveryCommand(
+        DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY, blkRecoveryInfosList);
+    BlockECRecoveryCommandProto blkECRecoveryCmdProto = PBHelper
+        .convert(blkECRecoveryCmd);
+    blkECRecoveryCmd = PBHelper.convert(blkECRecoveryCmdProto);
+    Iterator<BlockECRecoveryInfo> iterator = blkECRecoveryCmd.getECTasks()
+        .iterator();
+    assertBlockECRecoveryInfoEquals(blkECRecoveryInfo0, iterator.next());
+    assertBlockECRecoveryInfoEquals(blkECRecoveryInfo1, iterator.next());
+  }
+
+  private void assertBlockECRecoveryInfoEquals(
+      BlockECRecoveryInfo blkECRecoveryInfo1,
+      BlockECRecoveryInfo blkECRecoveryInfo2) {
+    assertEquals(blkECRecoveryInfo1.getExtendedBlock(),
+        blkECRecoveryInfo2.getExtendedBlock());
+
+    DatanodeInfo[] sourceDnInfos1 = blkECRecoveryInfo1.getSourceDnInfos();
+    DatanodeInfo[] sourceDnInfos2 = blkECRecoveryInfo2.getSourceDnInfos();
+    assertDnInfosEqual(sourceDnInfos1, sourceDnInfos2);
+
+    DatanodeInfo[] targetDnInfos1 = blkECRecoveryInfo1.getTargetDnInfos();
+    DatanodeInfo[] targetDnInfos2 = blkECRecoveryInfo2.getTargetDnInfos();
+    assertDnInfosEqual(targetDnInfos1, targetDnInfos2);
+
+    String[] targetStorageIDs1 = blkECRecoveryInfo1.getTargetStorageIDs();
+    String[] targetStorageIDs2 = blkECRecoveryInfo2.getTargetStorageIDs();
+    assertEquals(targetStorageIDs1.length, targetStorageIDs2.length);
+    for (int i = 0; i < targetStorageIDs1.length; i++) {
+      assertEquals(targetStorageIDs1[i], targetStorageIDs2[i]);
+    }
+
+    short[] liveBlockIndices1 = blkECRecoveryInfo1.getLiveBlockIndices();
+    short[] liveBlockIndices2 = blkECRecoveryInfo2.getLiveBlockIndices();
+    for (int i = 0; i < liveBlockIndices1.length; i++) {
+      assertEquals(liveBlockIndices1[i], liveBlockIndices2[i]);
+    }
+    
+    ErasureCodingPolicy ecPolicy1 = blkECRecoveryInfo1.getErasureCodingPolicy();
+    ErasureCodingPolicy ecPolicy2 = blkECRecoveryInfo2.getErasureCodingPolicy();
+    // Compare ECPolicies same as default ECPolicy as we used system default
+    // ECPolicy used in this test
+    compareECPolicies(ErasureCodingPolicyManager.getSystemDefaultPolicy(), ecPolicy1);
+    compareECPolicies(ErasureCodingPolicyManager.getSystemDefaultPolicy(), ecPolicy2);
+  }
+
+  private void compareECPolicies(ErasureCodingPolicy ecPolicy1, ErasureCodingPolicy ecPolicy2) {
+    assertEquals(ecPolicy1.getName(), ecPolicy2.getName());
+    assertEquals(ecPolicy1.getNumDataUnits(), ecPolicy2.getNumDataUnits());
+    assertEquals(ecPolicy1.getNumParityUnits(), ecPolicy2.getNumParityUnits());
+  }
+
+  private void assertDnInfosEqual(DatanodeInfo[] dnInfos1,
+      DatanodeInfo[] dnInfos2) {
+    assertEquals(dnInfos1.length, dnInfos2.length);
+    for (int i = 0; i < dnInfos1.length; i++) {
+      compare(dnInfos1[i], dnInfos2[i]);
+    }
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index fad9f7b..2d7caba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -66,6 +66,7 @@
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -73,6 +74,7 @@
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.balancer.Balancer.Cli;
 import org.apache.hadoop.hdfs.server.balancer.Balancer.Result;
 import org.apache.hadoop.hdfs.server.balancer.BalancerParameters;
@@ -147,6 +149,23 @@
     conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
   }
 
+  int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  int groupSize = dataBlocks + parityBlocks;
+  private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private final static int stripesPerBlock = 4;
+  static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock;
+
+  static void initConfWithStripe(Configuration conf) {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    SimulatedFSDataset.setFactory(conf);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
+    conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
+    conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
+  }
+
   /* create a file with a length of <code>fileLen</code> */
   static void createFile(MiniDFSCluster cluster, Path filePath, long fileLen,
       short replicationFactor, int nnIndex)
@@ -935,9 +954,9 @@
   void testBalancer1Internal(Configuration conf) throws Exception {
     initConf(conf);
     testUnevenDistribution(conf,
-        new long[] {50*CAPACITY/100, 10*CAPACITY/100},
+        new long[]{50 * CAPACITY / 100, 10 * CAPACITY / 100},
         new long[]{CAPACITY, CAPACITY},
-        new String[] {RACK0, RACK1});
+        new String[]{RACK0, RACK1});
   }
   
   @Test(expected=HadoopIllegalArgumentException.class)
@@ -951,7 +970,7 @@
   public void testBalancerWithNonZeroThreadsForMove() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, 8);
-    testBalancer1Internal (conf);
+    testBalancer1Internal(conf);
   }
   
   @Test(timeout=100000)
@@ -961,8 +980,8 @@
   
   void testBalancer2Internal(Configuration conf) throws Exception {
     initConf(conf);
-    testBalancerDefaultConstructor(conf, new long[] { CAPACITY, CAPACITY },
-        new String[] { RACK0, RACK1 }, CAPACITY, RACK2);
+    testBalancerDefaultConstructor(conf, new long[]{CAPACITY, CAPACITY},
+        new String[]{RACK0, RACK1}, CAPACITY, RACK2);
   }
 
   private void testBalancerDefaultConstructor(Configuration conf,
@@ -1679,7 +1698,74 @@
       cluster.shutdown();
     }
   }
-  
+  public void integrationTestWithStripedFile(Configuration conf) throws Exception {
+    initConfWithStripe(conf);
+    doTestBalancerWithStripedFile(conf);
+  }
+
+  @Test(timeout = 100000)
+  public void testBalancerWithStripedFile() throws Exception {
+    Configuration conf = new Configuration();
+    initConfWithStripe(conf);
+    doTestBalancerWithStripedFile(conf);
+  }
+
+  private void doTestBalancerWithStripedFile(Configuration conf) throws Exception {
+    int numOfDatanodes = dataBlocks + parityBlocks + 2;
+    int numOfRacks = dataBlocks;
+    long capacity = 20 * DEFAULT_STRIPE_BLOCK_SIZE;
+    long[] capacities = new long[numOfDatanodes];
+    for (int i = 0; i < capacities.length; i++) {
+      capacities[i] = capacity;
+    }
+    String[] racks = new String[numOfDatanodes];
+    for (int i = 0; i < numOfDatanodes; i++) {
+      racks[i] = "/rack" + (i % numOfRacks);
+    }
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numOfDatanodes)
+        .racks(racks)
+        .simulatedCapacities(capacities)
+        .build();
+
+    try {
+      cluster.waitActive();
+      client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(),
+          ClientProtocol.class).getProxy();
+      client.setErasureCodingPolicy("/", null);
+
+      long totalCapacity = sum(capacities);
+
+      // fill up the cluster with 30% data. It'll be 45% full plus parity.
+      long fileLen = totalCapacity * 3 / 10;
+      long totalUsedSpace = fileLen * (dataBlocks + parityBlocks) / dataBlocks;
+      FileSystem fs = cluster.getFileSystem(0);
+      DFSTestUtil.createFile(fs, filePath, fileLen, (short) 3, r.nextLong());
+
+      // verify locations of striped blocks
+      LocatedBlocks locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
+
+      // add one datanode
+      String newRack = "/rack" + (++numOfRacks);
+      cluster.startDataNodes(conf, 1, true, null,
+          new String[]{newRack}, null, new long[]{capacity});
+      totalCapacity += capacity;
+      cluster.triggerHeartbeats();
+
+      // run balancer and validate results
+      BalancerParameters p = BalancerParameters.DEFAULT;
+      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      runBalancer(conf, totalUsedSpace, totalCapacity, p, 0);
+
+      // verify locations of striped blocks
+      locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   /**
    * @param args
    */
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..64d80bd 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 static int computeAllPendingWork(BlockManager bm) {
     int work = computeInvalidationWork(bm);
-    work += bm.computeReplicationWork(Integer.MAX_VALUE);
+    work += bm.computeBlockRecoveryWork(Integer.MAX_VALUE);
     return work;
   }
 
@@ -306,4 +306,12 @@
       throws ExecutionException, InterruptedException {
     dm.getDecomManager().runMonitor();
   }
+
+  /**
+   * add block to the replicateBlocks queue of the Datanode
+   */
+  public static void addBlockToBeReplicated(DatanodeDescriptor node,
+      Block block, DatanodeStorageInfo[] targets) {
+    node.addBlockToBeReplicated(block, targets);
+  }
 }
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 ceef9f2..d6213ff 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
@@ -64,25 +64,13 @@
 
     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));
   }
 
   @Test
-  public void testCopyConstructor() {
-    BlockInfoContiguous old = new BlockInfoContiguous((short) 3);
-    try {
-      BlockInfoContiguous copy = new BlockInfoContiguous(old);
-      assertEquals(old.getBlockCollectionId(), copy.getBlockCollectionId());
-      assertEquals(old.getCapacity(), copy.getCapacity());
-    } catch (Exception e) {
-      Assert.fail("Copy constructor throws exception: " + e);
-    }
-  }
-
-  @Test
   public void testReplaceStorage() throws Exception {
 
     // Create two dummy storages.
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..4650dce
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
@@ -0,0 +1,220 @@
+/**
+ * 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.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.junit.Assert;
+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.StripedFileTestUtil.NUM_DATA_BLOCKS;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_PARITY_BLOCKS;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * 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 static final ErasureCodingPolicy testECPolicy
+      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  private final BlockInfoStriped info = new BlockInfoStriped(baseBlock,
+      testECPolicy);
+
+  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 testWrite() {
+    long blkID = 1;
+    long numBytes = 1;
+    long generationStamp = 1;
+    ByteBuffer byteBuffer = ByteBuffer.allocate(Long.SIZE / Byte.SIZE * 3);
+    byteBuffer.putLong(blkID).putLong(numBytes).putLong(generationStamp);
+
+    ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+    DataOutput out = new DataOutputStream(byteStream);
+    BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes,
+        generationStamp), testECPolicy);
+
+    try {
+      blk.write(out);
+    } catch(Exception ex) {
+      fail("testWrite error:" + ex.getMessage());
+    }
+    assertEquals(byteBuffer.array().length, byteStream.toByteArray().length);
+    assertArrayEquals(byteBuffer.array(), byteStream.toByteArray());
+  }
+}
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 d40b1c7..094794b 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
@@ -388,7 +388,7 @@
     for (int i = 1; i < pipeline.length; i++) {
       DatanodeStorageInfo storage = pipeline[i];
       bm.addBlock(storage, blockInfo, null);
-      blockInfo.addStorage(storage);
+      blockInfo.addStorage(storage, blockInfo);
     }
   }
 
@@ -398,7 +398,7 @@
 
     for (DatanodeDescriptor dn : nodes) {
       for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
-        blockInfo.addStorage(storage);
+        blockInfo.addStorage(storage, blockInfo);
       }
     }
     return blockInfo;
@@ -462,8 +462,8 @@
     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);
 
@@ -517,35 +517,38 @@
     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 ArrayList<Short>(),
+            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,
+    assertEquals("Does not choose a source node for a less-than-highest-priority"
+            + " replication since all available source nodes have reached"
+            + " their replication limits.", 0,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED));
+            new ArrayList<Short>(),
+            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED).length);
 
     // 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 node for a highest-priority"
-        + " replication when all available nodes exceed the hard limit.",
-        bm.chooseSourceDatanode(
-            aBlock,
+    assertEquals("Does not choose a source node for a highest-priority"
+            + " replication when all available nodes exceed the hard limit.", 0,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY));
+            new ArrayList<Short>(),
+            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY).length);
   }
 
   @Test
@@ -566,30 +569,28 @@
     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(),
-            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
+            new NumberReplicas(), new LinkedList<Short>(),
+            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)[0]);
 
 
     // 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,
+    assertEquals("Does not choose a source decommissioning node for a normal"
+        + " replication when all available nodes exceed the hard limit.", 0,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
-            new NumberReplicas(),
-            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
+            new NumberReplicas(), new LinkedList<Short>(),
+            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED).length);
   }
 
-
-
   @Test
   public void testSafeModeIBR() throws Exception {
     DatanodeDescriptor node = spy(nodes.get(0));
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
index e39ab3e..8e65ff6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
@@ -24,7 +24,6 @@
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
-import java.util.EnumSet;
 import java.util.List;
 import java.util.Random;
 
@@ -70,28 +69,32 @@
 
 public class TestBlockTokenWithDFS {
 
-  private static final int BLOCK_SIZE = 1024;
-  private static final int FILE_SIZE = 2 * BLOCK_SIZE;
+  protected static int BLOCK_SIZE = 1024;
+  protected static int FILE_SIZE = 2 * BLOCK_SIZE;
   private static final String FILE_TO_READ = "/fileToRead.dat";
   private static final String FILE_TO_WRITE = "/fileToWrite.dat";
   private static final String FILE_TO_APPEND = "/fileToAppend.dat";
-  private final byte[] rawData = new byte[FILE_SIZE];
 
   {
     GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
-    Random r = new Random();
-    r.nextBytes(rawData);
   }
 
-  private void createFile(FileSystem fs, Path filename) throws IOException {
+  public static byte[] generateBytes(int fileSize){
+    Random r = new Random();
+    byte[] rawData = new byte[fileSize];
+    r.nextBytes(rawData);
+    return rawData;
+  }
+
+  private void createFile(FileSystem fs, Path filename, byte[] expected) throws IOException {
     FSDataOutputStream out = fs.create(filename);
-    out.write(rawData);
+    out.write(expected);
     out.close();
   }
 
   // read a file using blockSeekTo()
-  private boolean checkFile1(FSDataInputStream in) {
-    byte[] toRead = new byte[FILE_SIZE];
+  private boolean checkFile1(FSDataInputStream in, byte[] expected) {
+    byte[] toRead = new byte[expected.length];
     int totalRead = 0;
     int nRead = 0;
     try {
@@ -102,27 +105,27 @@
       return false;
     }
     assertEquals("Cannot read file.", toRead.length, totalRead);
-    return checkFile(toRead);
+    return checkFile(toRead, expected);
   }
 
   // read a file using fetchBlockByteRange()
-  private boolean checkFile2(FSDataInputStream in) {
-    byte[] toRead = new byte[FILE_SIZE];
+  private boolean checkFile2(FSDataInputStream in, byte[] expected) {
+    byte[] toRead = new byte[expected.length];
     try {
       assertEquals("Cannot read file", toRead.length, in.read(0, toRead, 0,
           toRead.length));
     } catch (IOException e) {
       return false;
     }
-    return checkFile(toRead);
+    return checkFile(toRead, expected);
   }
 
-  private boolean checkFile(byte[] fileToCheck) {
-    if (fileToCheck.length != rawData.length) {
+  private boolean checkFile(byte[] fileToCheck, byte[] expected) {
+    if (fileToCheck.length != expected.length) {
       return false;
     }
     for (int i = 0; i < fileToCheck.length; i++) {
-      if (fileToCheck[i] != rawData[i]) {
+      if (fileToCheck[i] != expected[i]) {
         return false;
       }
     }
@@ -138,7 +141,7 @@
   }
 
   // try reading a block using a BlockReader directly
-  private static void tryRead(final Configuration conf, LocatedBlock lblock,
+  protected void tryRead(final Configuration conf, LocatedBlock lblock,
       boolean shouldSucceed) {
     InetSocketAddress targetAddr = null;
     IOException ioe = null;
@@ -149,7 +152,7 @@
       targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr());
 
       blockReader = new BlockReaderFactory(new DfsClientConf(conf)).
-          setFileName(BlockReaderFactory.getFileName(targetAddr, 
+          setFileName(BlockReaderFactory.getFileName(targetAddr,
                         "test-blockpoolid", block.getBlockId())).
           setBlock(block).
           setBlockToken(lblock.getBlockToken()).
@@ -207,7 +210,7 @@
   }
 
   // get a conf for testing
-  private static Configuration getConf(int numDataNodes) {
+  protected Configuration getConf(int numDataNodes) {
     Configuration conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -243,16 +246,16 @@
       SecurityTestUtil.setBlockTokenLifetime(sm, 1000L);
       Path fileToAppend = new Path(FILE_TO_APPEND);
       FileSystem fs = cluster.getFileSystem();
-
+      byte[] expected = generateBytes(FILE_SIZE);
       // write a one-byte file
       FSDataOutputStream stm = writeFile(fs, fileToAppend,
           (short) numDataNodes, BLOCK_SIZE);
-      stm.write(rawData, 0, 1);
+      stm.write(expected, 0, 1);
       stm.close();
       // open the file again for append
       stm = fs.append(fileToAppend);
-      int mid = rawData.length - 1;
-      stm.write(rawData, 1, mid - 1);
+      int mid = expected.length - 1;
+      stm.write(expected, 1, mid - 1);
       stm.hflush();
 
       /*
@@ -269,11 +272,11 @@
       // remove a datanode to force re-establishing pipeline
       cluster.stopDataNode(0);
       // append the rest of the file
-      stm.write(rawData, mid, rawData.length - mid);
+      stm.write(expected, mid, expected.length - mid);
       stm.close();
       // check if append is successful
       FSDataInputStream in5 = fs.open(fileToAppend);
-      assertTrue(checkFile1(in5));
+      assertTrue(checkFile1(in5, expected));
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -305,11 +308,12 @@
       Path fileToWrite = new Path(FILE_TO_WRITE);
       FileSystem fs = cluster.getFileSystem();
 
+      byte[] expected = generateBytes(FILE_SIZE);
       FSDataOutputStream stm = writeFile(fs, fileToWrite, (short) numDataNodes,
           BLOCK_SIZE);
       // write a partial block
-      int mid = rawData.length - 1;
-      stm.write(rawData, 0, mid);
+      int mid = expected.length - 1;
+      stm.write(expected, 0, mid);
       stm.hflush();
 
       /*
@@ -326,11 +330,11 @@
       // remove a datanode to force re-establishing pipeline
       cluster.stopDataNode(0);
       // write the rest of the file
-      stm.write(rawData, mid, rawData.length - mid);
+      stm.write(expected, mid, expected.length - mid);
       stm.close();
       // check if write is successful
       FSDataInputStream in4 = fs.open(fileToWrite);
-      assertTrue(checkFile1(in4));
+      assertTrue(checkFile1(in4, expected));
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -348,125 +352,137 @@
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
       cluster.waitActive();
       assertEquals(numDataNodes, cluster.getDataNodes().size());
+      doTestRead(conf, cluster, false);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 
-      final NameNode nn = cluster.getNameNode();
-      final NamenodeProtocols nnProto = nn.getRpcServer();
-      final BlockManager bm = nn.getNamesystem().getBlockManager();
-      final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
+  protected void doTestRead(Configuration conf, MiniDFSCluster cluster,
+      boolean isStriped) throws Exception {
+    final int numDataNodes = cluster.getDataNodes().size();
+    final NameNode nn = cluster.getNameNode();
+    final NamenodeProtocols nnProto = nn.getRpcServer();
+    final BlockManager bm = nn.getNamesystem().getBlockManager();
+    final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
 
-      // set a short token lifetime (1 second) initially
-      SecurityTestUtil.setBlockTokenLifetime(sm, 1000L);
+    // set a short token lifetime (1 second) initially
+    SecurityTestUtil.setBlockTokenLifetime(sm, 1000L);
 
-      Path fileToRead = new Path(FILE_TO_READ);
-      FileSystem fs = cluster.getFileSystem();
-      createFile(fs, fileToRead);
+    Path fileToRead = new Path(FILE_TO_READ);
+    FileSystem fs = cluster.getFileSystem();
+    byte[] expected = generateBytes(FILE_SIZE);
+    createFile(fs, fileToRead, expected);
 
       /*
        * setup for testing expiration handling of cached tokens
        */
 
-      // read using blockSeekTo(). Acquired tokens are cached in in1
-      FSDataInputStream in1 = fs.open(fileToRead);
-      assertTrue(checkFile1(in1));
-      // read using blockSeekTo(). Acquired tokens are cached in in2
-      FSDataInputStream in2 = fs.open(fileToRead);
-      assertTrue(checkFile1(in2));
-      // read using fetchBlockByteRange(). Acquired tokens are cached in in3
-      FSDataInputStream in3 = fs.open(fileToRead);
-      assertTrue(checkFile2(in3));
+    // read using blockSeekTo(). Acquired tokens are cached in in1
+    FSDataInputStream in1 = fs.open(fileToRead);
+    assertTrue(checkFile1(in1,expected));
+    // read using blockSeekTo(). Acquired tokens are cached in in2
+    FSDataInputStream in2 = fs.open(fileToRead);
+    assertTrue(checkFile1(in2,expected));
+    // read using fetchBlockByteRange(). Acquired tokens are cached in in3
+    FSDataInputStream in3 = fs.open(fileToRead);
+    assertTrue(checkFile2(in3,expected));
 
       /*
        * testing READ interface on DN using a BlockReader
        */
-      DFSClient client = null;
-      try {
-        client = new DFSClient(new InetSocketAddress("localhost",
+    DFSClient client = null;
+    try {
+      client = new DFSClient(new InetSocketAddress("localhost",
           cluster.getNameNodePort()), conf);
-      } finally {
-        if (client != null) client.close();
-      }
-      List<LocatedBlock> locatedBlocks = nnProto.getBlockLocations(
-          FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks();
-      LocatedBlock lblock = locatedBlocks.get(0); // first block
-      Token<BlockTokenIdentifier> myToken = lblock.getBlockToken();
-      // verify token is not expired
-      assertFalse(SecurityTestUtil.isBlockTokenExpired(myToken));
-      // read with valid token, should succeed
-      tryRead(conf, lblock, true);
+    } finally {
+      if (client != null) client.close();
+    }
+    List<LocatedBlock> locatedBlocks = nnProto.getBlockLocations(
+        FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks();
+    LocatedBlock lblock = locatedBlocks.get(0); // first block
+    // verify token is not expired
+    assertFalse(isBlockTokenExpired(lblock));
+    // read with valid token, should succeed
+    tryRead(conf, lblock, true);
 
       /*
        * wait till myToken and all cached tokens in in1, in2 and in3 expire
        */
 
-      while (!SecurityTestUtil.isBlockTokenExpired(myToken)) {
-        try {
-          Thread.sleep(10);
-        } catch (InterruptedException ignored) {
-        }
+    while (!isBlockTokenExpired(lblock)) {
+      try {
+        Thread.sleep(10);
+      } catch (InterruptedException ignored) {
       }
+    }
 
       /*
        * continue testing READ interface on DN using a BlockReader
        */
 
-      // verify token is expired
-      assertTrue(SecurityTestUtil.isBlockTokenExpired(myToken));
-      // read should fail
-      tryRead(conf, lblock, false);
-      // use a valid new token
-      lblock.setBlockToken(sm.generateToken(lblock.getBlock(),
-              EnumSet.of(BlockTokenIdentifier.AccessMode.READ)));
-      // read should succeed
-      tryRead(conf, lblock, true);
-      // use a token with wrong blockID
-      ExtendedBlock wrongBlock = new ExtendedBlock(lblock.getBlock()
-          .getBlockPoolId(), lblock.getBlock().getBlockId() + 1);
-      lblock.setBlockToken(sm.generateToken(wrongBlock,
-          EnumSet.of(BlockTokenIdentifier.AccessMode.READ)));
-      // read should fail
-      tryRead(conf, lblock, false);
-      // use a token with wrong access modes
-      lblock.setBlockToken(sm.generateToken(lblock.getBlock(),
-          EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE,
-                     BlockTokenIdentifier.AccessMode.COPY,
-                     BlockTokenIdentifier.AccessMode.REPLACE)));
-      // read should fail
-      tryRead(conf, lblock, false);
+    // verify token is expired
+    assertTrue(isBlockTokenExpired(lblock));
+    // read should fail
+    tryRead(conf, lblock, false);
+    // use a valid new token
+    bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.READ);
+    // read should succeed
+    tryRead(conf, lblock, true);
+    // use a token with wrong blockID
+    long rightId = lblock.getBlock().getBlockId();
+    long wrongId = rightId + 1;
+    lblock.getBlock().setBlockId(wrongId);
+    bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.READ);
+    lblock.getBlock().setBlockId(rightId);
+    // read should fail
+    tryRead(conf, lblock, false);
+    // use a token with wrong access modes
+    bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.WRITE);
+    // read should fail
+    tryRead(conf, lblock, false);
 
-      // set a long token lifetime for future tokens
-      SecurityTestUtil.setBlockTokenLifetime(sm, 600 * 1000L);
+    // set a long token lifetime for future tokens
+    SecurityTestUtil.setBlockTokenLifetime(sm, 600 * 1000L);
 
       /*
        * testing that when cached tokens are expired, DFSClient will re-fetch
        * tokens transparently for READ.
        */
 
-      // confirm all tokens cached in in1 are expired by now
-      List<LocatedBlock> lblocks = DFSTestUtil.getAllBlocks(in1);
-      for (LocatedBlock blk : lblocks) {
-        assertTrue(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken()));
-      }
-      // verify blockSeekTo() is able to re-fetch token transparently
-      in1.seek(0);
-      assertTrue(checkFile1(in1));
+    // confirm all tokens cached in in1 are expired by now
+    List<LocatedBlock> lblocks = DFSTestUtil.getAllBlocks(in1);
+    for (LocatedBlock blk : lblocks) {
+      assertTrue(isBlockTokenExpired(blk));
+    }
+    // verify blockSeekTo() is able to re-fetch token transparently
+    in1.seek(0);
+    assertTrue(checkFile1(in1, expected));
 
-      // confirm all tokens cached in in2 are expired by now
-      List<LocatedBlock> lblocks2 = DFSTestUtil.getAllBlocks(in2);
-      for (LocatedBlock blk : lblocks2) {
-        assertTrue(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken()));
-      }
-      // verify blockSeekTo() is able to re-fetch token transparently (testing
-      // via another interface method)
+    // confirm all tokens cached in in2 are expired by now
+    List<LocatedBlock> lblocks2 = DFSTestUtil.getAllBlocks(in2);
+    for (LocatedBlock blk : lblocks2) {
+      assertTrue(isBlockTokenExpired(blk));
+    }
+    // verify blockSeekTo() is able to re-fetch token transparently (testing
+    // via another interface method)
+    if (isStriped) {
+      // striped block doesn't support seekToNewSource
+      in2.seek(0);
+    } else {
       assertTrue(in2.seekToNewSource(0));
-      assertTrue(checkFile1(in2));
+    }
+    assertTrue(checkFile1(in2,expected));
 
-      // confirm all tokens cached in in3 are expired by now
-      List<LocatedBlock> lblocks3 = DFSTestUtil.getAllBlocks(in3);
-      for (LocatedBlock blk : lblocks3) {
-        assertTrue(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken()));
-      }
-      // verify fetchBlockByteRange() is able to re-fetch token transparently
-      assertTrue(checkFile2(in3));
+    // confirm all tokens cached in in3 are expired by now
+    List<LocatedBlock> lblocks3 = DFSTestUtil.getAllBlocks(in3);
+    for (LocatedBlock blk : lblocks3) {
+      assertTrue(isBlockTokenExpired(blk));
+    }
+    // verify fetchBlockByteRange() is able to re-fetch token transparently
+    assertTrue(checkFile2(in3,expected));
 
       /*
        * testing that after datanodes are restarted on the same ports, cached
@@ -475,37 +491,42 @@
        * new tokens can be fetched from namenode).
        */
 
-      // restart datanodes on the same ports that they currently use
-      assertTrue(cluster.restartDataNodes(true));
-      cluster.waitActive();
-      assertEquals(numDataNodes, cluster.getDataNodes().size());
-      cluster.shutdownNameNode(0);
+    // restart datanodes on the same ports that they currently use
+    assertTrue(cluster.restartDataNodes(true));
+    cluster.waitActive();
+    assertEquals(numDataNodes, cluster.getDataNodes().size());
+    cluster.shutdownNameNode(0);
 
-      // confirm tokens cached in in1 are still valid
-      lblocks = DFSTestUtil.getAllBlocks(in1);
-      for (LocatedBlock blk : lblocks) {
-        assertFalse(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken()));
-      }
-      // verify blockSeekTo() still works (forced to use cached tokens)
-      in1.seek(0);
-      assertTrue(checkFile1(in1));
+    // confirm tokens cached in in1 are still valid
+    lblocks = DFSTestUtil.getAllBlocks(in1);
+    for (LocatedBlock blk : lblocks) {
+      assertFalse(isBlockTokenExpired(blk));
+    }
+    // verify blockSeekTo() still works (forced to use cached tokens)
+    in1.seek(0);
+    assertTrue(checkFile1(in1,expected));
 
-      // confirm tokens cached in in2 are still valid
-      lblocks2 = DFSTestUtil.getAllBlocks(in2);
-      for (LocatedBlock blk : lblocks2) {
-        assertFalse(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken()));
-      }
-      // verify blockSeekTo() still works (forced to use cached tokens)
+    // confirm tokens cached in in2 are still valid
+    lblocks2 = DFSTestUtil.getAllBlocks(in2);
+    for (LocatedBlock blk : lblocks2) {
+      assertFalse(isBlockTokenExpired(blk));
+    }
+
+    // verify blockSeekTo() still works (forced to use cached tokens)
+    if (isStriped) {
+      in2.seek(0);
+    } else {
       in2.seekToNewSource(0);
-      assertTrue(checkFile1(in2));
+    }
+    assertTrue(checkFile1(in2,expected));
 
-      // confirm tokens cached in in3 are still valid
-      lblocks3 = DFSTestUtil.getAllBlocks(in3);
-      for (LocatedBlock blk : lblocks3) {
-        assertFalse(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken()));
-      }
-      // verify fetchBlockByteRange() still works (forced to use cached tokens)
-      assertTrue(checkFile2(in3));
+    // confirm tokens cached in in3 are still valid
+    lblocks3 = DFSTestUtil.getAllBlocks(in3);
+    for (LocatedBlock blk : lblocks3) {
+      assertFalse(isBlockTokenExpired(blk));
+    }
+    // verify fetchBlockByteRange() still works (forced to use cached tokens)
+    assertTrue(checkFile2(in3,expected));
 
       /*
        * testing that when namenode is restarted, cached tokens should still
@@ -514,18 +535,23 @@
        * setup for this test depends on the previous test.
        */
 
-      // restart the namenode and then shut it down for test
-      cluster.restartNameNode(0);
-      cluster.shutdownNameNode(0);
+    // restart the namenode and then shut it down for test
+    cluster.restartNameNode(0);
+    cluster.shutdownNameNode(0);
 
-      // verify blockSeekTo() still works (forced to use cached tokens)
-      in1.seek(0);
-      assertTrue(checkFile1(in1));
-      // verify again blockSeekTo() still works (forced to use cached tokens)
+    // verify blockSeekTo() still works (forced to use cached tokens)
+    in1.seek(0);
+    assertTrue(checkFile1(in1,expected));
+    // verify again blockSeekTo() still works (forced to use cached tokens)
+    if (isStriped) {
+      in2.seek(0);
+    } else {
       in2.seekToNewSource(0);
-      assertTrue(checkFile1(in2));
-      // verify fetchBlockByteRange() still works (forced to use cached tokens)
-      assertTrue(checkFile2(in3));
+    }
+    assertTrue(checkFile1(in2,expected));
+
+    // verify fetchBlockByteRange() still works (forced to use cached tokens)
+    assertTrue(checkFile2(in3,expected));
 
       /*
        * testing that after both namenode and datanodes got restarted (namenode
@@ -534,58 +560,60 @@
        * setup of this test depends on the previous test.
        */
 
-      // restore the cluster and restart the datanodes for test
-      cluster.restartNameNode(0);
-      assertTrue(cluster.restartDataNodes(true));
-      cluster.waitActive();
-      assertEquals(numDataNodes, cluster.getDataNodes().size());
+    // restore the cluster and restart the datanodes for test
+    cluster.restartNameNode(0);
+    assertTrue(cluster.restartDataNodes(true));
+    cluster.waitActive();
+    assertEquals(numDataNodes, cluster.getDataNodes().size());
 
-      // shutdown namenode so that DFSClient can't get new tokens from namenode
-      cluster.shutdownNameNode(0);
+    // shutdown namenode so that DFSClient can't get new tokens from namenode
+    cluster.shutdownNameNode(0);
 
-      // verify blockSeekTo() fails (cached tokens become invalid)
-      in1.seek(0);
-      assertFalse(checkFile1(in1));
-      // verify fetchBlockByteRange() fails (cached tokens become invalid)
-      assertFalse(checkFile2(in3));
+    // verify blockSeekTo() fails (cached tokens become invalid)
+    in1.seek(0);
+    assertFalse(checkFile1(in1,expected));
+    // verify fetchBlockByteRange() fails (cached tokens become invalid)
+    assertFalse(checkFile2(in3,expected));
 
-      // restart the namenode to allow DFSClient to re-fetch tokens
-      cluster.restartNameNode(0);
-      // verify blockSeekTo() works again (by transparently re-fetching
-      // tokens from namenode)
-      in1.seek(0);
-      assertTrue(checkFile1(in1));
+    // restart the namenode to allow DFSClient to re-fetch tokens
+    cluster.restartNameNode(0);
+    // verify blockSeekTo() works again (by transparently re-fetching
+    // tokens from namenode)
+    in1.seek(0);
+    assertTrue(checkFile1(in1,expected));
+    if (isStriped) {
+      in2.seek(0);
+    } else {
       in2.seekToNewSource(0);
-      assertTrue(checkFile1(in2));
-      // verify fetchBlockByteRange() works again (by transparently
-      // re-fetching tokens from namenode)
-      assertTrue(checkFile2(in3));
+    }
+    assertTrue(checkFile1(in2,expected));
+    // verify fetchBlockByteRange() works again (by transparently
+    // re-fetching tokens from namenode)
+    assertTrue(checkFile2(in3,expected));
 
       /*
        * testing that when datanodes are restarted on different ports, DFSClient
        * is able to re-fetch tokens transparently to connect to them
        */
 
-      // restart datanodes on newly assigned ports
-      assertTrue(cluster.restartDataNodes(false));
-      cluster.waitActive();
-      assertEquals(numDataNodes, cluster.getDataNodes().size());
-      // verify blockSeekTo() is able to re-fetch token transparently
-      in1.seek(0);
-      assertTrue(checkFile1(in1));
-      // verify blockSeekTo() is able to re-fetch token transparently
+    // restart datanodes on newly assigned ports
+    assertTrue(cluster.restartDataNodes(false));
+    cluster.waitActive();
+    assertEquals(numDataNodes, cluster.getDataNodes().size());
+    // verify blockSeekTo() is able to re-fetch token transparently
+    in1.seek(0);
+    assertTrue(checkFile1(in1,expected));
+    // verify blockSeekTo() is able to re-fetch token transparently
+    if (isStriped) {
+      in2.seek(0);
+    } else {
       in2.seekToNewSource(0);
-      assertTrue(checkFile1(in2));
-      // verify fetchBlockByteRange() is able to re-fetch token transparently
-      assertTrue(checkFile2(in3));
-
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
     }
-  }
+    assertTrue(checkFile1(in2,expected));
+    // verify fetchBlockByteRange() is able to re-fetch token transparently
+    assertTrue(checkFile2(in3,expected));
 
+  }
   /**
    * Integration testing of access token, involving NN, DN, and Balancer
    */
@@ -595,4 +623,8 @@
     conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     new TestBalancer().integrationTest(conf);
   }
+
+  protected boolean isBlockTokenExpired(LocatedBlock lb) throws IOException {
+    return SecurityTestUtil.isBlockTokenExpired(lb.getBlockToken());
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
new file mode 100644
index 0000000..ae33ffe
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestBlockTokenWithDFSStriped extends TestBlockTokenWithDFS {
+
+  private final static int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final static int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private final static int stripesPerBlock = 4;
+  private final static int numDNs = dataBlocks + parityBlocks + 2;
+  private static MiniDFSCluster cluster;
+  private static Configuration conf;
+
+  {
+    BLOCK_SIZE = cellSize * stripesPerBlock;
+    FILE_SIZE =  BLOCK_SIZE * dataBlocks * 3;
+  }
+
+  private Configuration getConf() {
+    Configuration conf = super.getConf(numDNs);
+    conf.setInt("io.bytes.per.checksum", cellSize);
+    return conf;
+  }
+
+  @Test
+  @Override
+  public void testRead() throws Exception {
+    conf = getConf();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.getFileSystem().getClient()
+        .setErasureCodingPolicy("/", null);
+    try {
+      cluster.waitActive();
+      doTestRead(conf, cluster, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * tested at {@link org.apache.hadoop.hdfs.TestDFSStripedOutputStreamWithFailure#testBlockTokenExpired()}
+   */
+  @Test
+  @Override
+  public void testWrite(){
+  }
+
+  @Test
+  @Override
+  public void testAppend() throws Exception {
+    //TODO: support Append for striped file
+  }
+
+  @Test
+  @Override
+  public void testEnd2End() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
+    new TestBalancer().integrationTestWithStripedFile(conf);
+  }
+
+  @Override
+  protected void tryRead(final Configuration conf, LocatedBlock lblock,
+                         boolean shouldSucceed) {
+    LocatedStripedBlock lsb = (LocatedStripedBlock) lblock;
+    LocatedBlock[] internalBlocks = StripedBlockUtil.parseStripedBlockGroup
+        (lsb, cellSize, dataBlocks, parityBlocks);
+    for (LocatedBlock internalBlock : internalBlocks) {
+      super.tryRead(conf, internalBlock, shouldSucceed);
+    }
+  }
+
+  @Override
+  protected boolean isBlockTokenExpired(LocatedBlock lb) throws IOException {
+    LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
+    LocatedBlock[] internalBlocks = StripedBlockUtil.parseStripedBlockGroup
+        (lsb, cellSize, dataBlocks, parityBlocks);
+    for (LocatedBlock internalBlock : internalBlocks) {
+      if(super.isBlockTokenExpired(internalBlock)){
+        return true;
+      }
+    }
+    return false;
+  }
+}
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 cea6865..b11b48ae 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
@@ -185,9 +185,12 @@
       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();
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 1c3f075..c33667d 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
@@ -100,7 +100,7 @@
       DatanodeDescriptor nonExcessDN = null;
       for(DatanodeStorageInfo storage : bm.blocksMap.getStorages(block.getLocalBlock())) {
         final DatanodeDescriptor dn = storage.getDatanodeDescriptor();
-        Collection<Block> blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid());
+        Collection<BlockInfo> blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid());
         if (blocks == null || !blocks.contains(block.getLocalBlock()) ) {
           nonExcessDN = dn;
           break;
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 2d7bb44..83b3aa0 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
@@ -34,7 +34,6 @@
 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.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -42,7 +41,6 @@
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 public class TestOverReplicatedBlocks {
@@ -185,7 +183,7 @@
       // All replicas for deletion should be scheduled on lastDN.
       // And should not actually be deleted, because lastDN does not heartbeat.
       namesystem.readLock();
-      Collection<Block> dnBlocks = 
+      Collection<BlockInfo> dnBlocks =
         namesystem.getBlockManager().excessReplicateMap.get(lastDNid);
       assertEquals("Replicas on node " + lastDNid + " should have been deleted",
           SMALL_FILE_LENGTH / SMALL_BLOCK_SIZE, dnBlocks.size());
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 90bc1b0..a653d45 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
@@ -1275,8 +1275,17 @@
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
-    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    long blkID1 = ThreadLocalRandom.current().nextLong();
+    if (blkID1 < 0) {
+      blkID1 *= -1;
+    }
+    long blkID2 = ThreadLocalRandom.current().nextLong();
+    if (blkID2 < 0) {
+      blkID2 *= -1;
+    }
+
+    BlockInfo block1 = genBlockInfo(blkID1);
+    BlockInfo block2 = genBlockInfo(blkID2);
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1291,7 +1300,7 @@
     chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
-    final BlockInfo info = new BlockInfoContiguous(block1, (short) 1);
+    final BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
     final BlockCollection mbc = mock(BlockCollection.class);
     when(mbc.getId()).thenReturn(1000L);
     when(mbc.getLastBlock()).thenReturn(info);
@@ -1315,9 +1324,10 @@
     when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true);
     when(storage.addBlock(any(BlockInfo.class))).thenReturn
         (DatanodeStorageInfo.AddBlockResult.ADDED);
-    info.addStorage(storage);
+    info.addStorage(storage, info);
 
-    when(mbc.getLastBlock()).thenReturn(info);
+    BlockInfo lastBlk = mbc.getLastBlock();
+    when(mbc.getLastBlock()).thenReturn(lastBlk, info);
 
     bm.convertLastBlockToUnderConstruction(mbc, 0L);
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java
new file mode 100644
index 0000000..00b4e3b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java
@@ -0,0 +1,222 @@
+/**
+ * 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 static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BLOCK_GROUP_INDEX_MASK;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_BLOCKS_IN_GROUP;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+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.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Tests the sequential blockGroup ID generation mechanism and blockGroup ID
+ * collision handling.
+ */
+public class TestSequentialBlockGroupId {
+  private static final Log LOG = LogFactory
+      .getLog("TestSequentialBlockGroupId");
+
+  private final short REPLICATION = 1;
+  private final long SEED = 0;
+  private final int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+
+  private final int stripesPerBlock = 2;
+  private final int blockSize = cellSize * stripesPerBlock;
+  private final int numDNs = dataBlocks + parityBlocks + 2;
+  private final int blockGrpCount = 4;
+  private final int fileLen = blockSize * dataBlocks * blockGrpCount;
+
+  private MiniDFSCluster cluster;
+  private FileSystem fs;
+  private SequentialBlockGroupIdGenerator blockGrpIdGenerator;
+  private Path ecDir = new Path("/ecDir");
+
+  @Before
+  public void setup() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.waitActive();
+
+    fs = cluster.getFileSystem();
+    blockGrpIdGenerator = cluster.getNamesystem().getBlockIdManager()
+        .getBlockGroupIdGenerator();
+    fs.mkdirs(ecDir);
+    cluster.getFileSystem().getClient()
+        .setErasureCodingPolicy("/ecDir", null);
+  }
+
+  @After
+  public void teardown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test that blockGroup IDs are generating unique value.
+   */
+  @Test(timeout = 60000)
+  public void testBlockGroupIdGeneration() throws IOException {
+    long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue();
+
+    // Create a file that is 4 blocks long.
+    Path path = new Path(ecDir, "testBlockGrpIdGeneration.dat");
+    DFSTestUtil.createFile(fs, path, cellSize, fileLen, blockSize, REPLICATION,
+        SEED);
+    List<LocatedBlock> blocks = DFSTestUtil.getAllBlocks(fs, path);
+    assertThat("Wrong BlockGrps", blocks.size(), is(blockGrpCount));
+
+    // initialising the block group generator for verifying the block id
+    blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue);
+    // Ensure that the block IDs are generating unique value.
+    for (int i = 0; i < blocks.size(); ++i) {
+      blockGrpIdGenerator
+          .skipTo((blockGrpIdGenerator.getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK)
+              + MAX_BLOCKS_IN_GROUP);
+      long nextBlockExpectedId = blockGrpIdGenerator.getCurrentValue();
+      long nextBlockGrpId = blocks.get(i).getBlock().getBlockId();
+      LOG.info("BlockGrp" + i + " id is " + nextBlockGrpId);
+      assertThat("BlockGrpId mismatches!", nextBlockGrpId,
+          is(nextBlockExpectedId));
+    }
+  }
+
+  /**
+   * Test that collisions in the blockGroup ID space are handled gracefully.
+   */
+  @Test(timeout = 60000)
+  public void testTriggerBlockGroupIdCollision() throws IOException {
+    long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue();
+
+    // Create a file with a few blocks to rev up the global block ID
+    // counter.
+    Path path1 = new Path(ecDir, "testBlockGrpIdCollisionDetection_file1.dat");
+    DFSTestUtil.createFile(fs, path1, cellSize, fileLen, blockSize,
+        REPLICATION, SEED);
+    List<LocatedBlock> blocks1 = DFSTestUtil.getAllBlocks(fs, path1);
+    assertThat("Wrong BlockGrps", blocks1.size(), is(blockGrpCount));
+
+    // Rewind the block ID counter in the name system object. This will result
+    // in block ID collisions when we try to allocate new blocks.
+    blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue);
+
+    // Trigger collisions by creating a new file.
+    Path path2 = new Path(ecDir, "testBlockGrpIdCollisionDetection_file2.dat");
+    DFSTestUtil.createFile(fs, path2, cellSize, fileLen, blockSize,
+        REPLICATION, SEED);
+    List<LocatedBlock> blocks2 = DFSTestUtil.getAllBlocks(fs, path2);
+    assertThat("Wrong BlockGrps", blocks2.size(), is(blockGrpCount));
+
+    // Make sure that file1 and file2 block IDs are different
+    for (LocatedBlock locBlock1 : blocks1) {
+      long blockId1 = locBlock1.getBlock().getBlockId();
+      for (LocatedBlock locBlock2 : blocks2) {
+        long blockId2 = locBlock2.getBlock().getBlockId();
+        assertThat("BlockGrpId mismatches!", blockId1, is(not(blockId2)));
+      }
+    }
+  }
+
+  /**
+   * Test that collisions in the blockGroup ID when the id is occupied by legacy
+   * block.
+   */
+  @Test(timeout = 60000)
+  public void testTriggerBlockGroupIdCollisionWithLegacyBlockId()
+      throws Exception {
+    long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue();
+    blockGrpIdGenerator
+        .skipTo((blockGrpIdGenerator.getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK)
+            + MAX_BLOCKS_IN_GROUP);
+    final long curBlockGroupIdValue = blockGrpIdGenerator.getCurrentValue();
+
+    // Creates contiguous block with negative blockId so that it would trigger
+    // collision during blockGroup Id generation
+    FSNamesystem fsn = cluster.getNamesystem();
+    // Replace SequentialBlockIdGenerator with a spy
+    SequentialBlockIdGenerator blockIdGenerator = spy(fsn.getBlockIdManager()
+        .getBlockIdGenerator());
+    Whitebox.setInternalState(fsn.getBlockIdManager(), "blockIdGenerator",
+        blockIdGenerator);
+    SequentialBlockIdGenerator spySequentialBlockIdGenerator = new SequentialBlockIdGenerator(
+        null) {
+      @Override
+      public long nextValue() {
+        return curBlockGroupIdValue;
+      }
+    };
+    final Answer<Object> delegator = new GenericTestUtils.DelegateAnswer(
+        spySequentialBlockIdGenerator);
+    doAnswer(delegator).when(blockIdGenerator).nextValue();
+
+    Path path1 = new Path("/testCollisionWithLegacyBlock_file1.dat");
+    DFSTestUtil.createFile(fs, path1, 1024, REPLICATION, SEED);
+
+    List<LocatedBlock> contiguousBlocks = DFSTestUtil.getAllBlocks(fs, path1);
+    assertThat(contiguousBlocks.size(), is(1));
+    Assert.assertEquals("Unexpected BlockId!", curBlockGroupIdValue,
+        contiguousBlocks.get(0).getBlock().getBlockId());
+
+    // Reset back to the initial value to trigger collision
+    blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue);
+    // Trigger collisions by creating a new file.
+    Path path2 = new Path(ecDir, "testCollisionWithLegacyBlock_file2.dat");
+    DFSTestUtil.createFile(fs, path2, cellSize, fileLen, blockSize,
+        REPLICATION, SEED);
+    List<LocatedBlock> blocks2 = DFSTestUtil.getAllBlocks(fs, path2);
+    assertThat("Wrong BlockGrps", blocks2.size(), is(blockGrpCount));
+
+    // Make sure that file1 and file2 block IDs are different
+    for (LocatedBlock locBlock1 : contiguousBlocks) {
+      long blockId1 = locBlock1.getBlock().getBlockId();
+      for (LocatedBlock locBlock2 : blocks2) {
+        long blockId2 = locBlock2.getBlock().getBlockId();
+        assertThat("BlockGrpId mismatches!", blockId1, is(not(blockId2)));
+      }
+    }
+  }
+}
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 de36e07..7cd2e19 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
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -28,10 +30,19 @@
 
 public class TestUnderReplicatedBlockQueues {
 
+  private final ErasureCodingPolicy ecPolicy =
+      ErasureCodingPolicyManager.getSystemDefaultPolicy();
+
   private BlockInfo genBlockInfo(long id) {
     return new BlockInfoContiguous(new Block(id), (short) 3);
   }
 
+  private BlockInfo genStripedBlockInfo(long id, long numBytes) {
+    BlockInfoStriped sblk =  new BlockInfoStriped(new Block(id), ecPolicy);
+    sblk.setNumBytes(numBytes);
+    return sblk;
+  }
+
   /**
    * Test that adding blocks with different replication counts puts them
    * into different queues
@@ -85,6 +96,54 @@
     assertEquals(2, queues.getCorruptReplOneBlockSize());
   }
 
+  @Test
+  public void testStripedBlockPriorities() throws Throwable {
+    int dataBlkNum = ecPolicy.getNumDataUnits();
+    int parityBlkNUm = ecPolicy.getNumParityUnits();
+    doTestStripedBlockPriorities(1, parityBlkNUm);
+    doTestStripedBlockPriorities(dataBlkNum, parityBlkNUm);
+  }
+
+  private void doTestStripedBlockPriorities(int dataBlkNum, int parityBlkNum)
+      throws Throwable {
+    int groupSize = dataBlkNum + parityBlkNum;
+    long numBytes = ecPolicy.getCellSize() * dataBlkNum;
+    UnderReplicatedBlocks queues = new UnderReplicatedBlocks();
+
+    // add a striped block which been left NUM_DATA_BLOCKS internal blocks
+    BlockInfo block1 = genStripedBlockInfo(-100, numBytes);
+    assertAdded(queues, block1, dataBlkNum, 0, groupSize);
+    assertEquals(1, queues.getUnderReplicatedBlockCount());
+    assertEquals(1, queues.size());
+    assertInLevel(queues, block1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY);
+
+    // add a striped block which been left NUM_DATA_BLOCKS+1 internal blocks
+    BlockInfo block2 = genStripedBlockInfo(-200, numBytes);
+    assertAdded(queues, block2, dataBlkNum + 1, 0, groupSize);
+    assertEquals(2, queues.getUnderReplicatedBlockCount());
+    assertEquals(2, queues.size());
+    assertInLevel(queues, block2,
+        UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED);
+
+    // add a striped block which been left NUM_DATA_BLOCKS+2 internal blocks
+    BlockInfo block3 = genStripedBlockInfo(-300, numBytes);
+    assertAdded(queues, block3, dataBlkNum + 2, 0, groupSize);
+    assertEquals(3, queues.getUnderReplicatedBlockCount());
+    assertEquals(3, queues.size());
+    assertInLevel(queues, block3,
+        UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED);
+
+    // add a corrupted block
+    BlockInfo block_corrupt = genStripedBlockInfo(-400, numBytes);
+    assertEquals(0, queues.getCorruptBlockSize());
+    assertAdded(queues, block_corrupt, dataBlkNum - 1, 0, groupSize);
+    assertEquals(4, queues.size());
+    assertEquals(3, queues.getUnderReplicatedBlockCount());
+    assertEquals(1, queues.getCorruptBlockSize());
+    assertInLevel(queues, block_corrupt,
+        UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
+  }
+
   private void assertAdded(UnderReplicatedBlocks queues,
                            BlockInfo block,
                            int curReplicas,
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 acbd8a8..4316854 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
@@ -99,7 +99,7 @@
 
   public static byte simulatedByte(Block b, long offsetInBlk) {
     byte firstByte = (byte) (b.getBlockId() & BYTE_MASK);
-    return (byte) ((firstByte + offsetInBlk) & BYTE_MASK);
+    return (byte) ((firstByte + offsetInBlk % 29) & BYTE_MASK);
   }
   
   public static final String CONFIG_PROPERTY_CAPACITY =
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 989e216..d8c651f 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
@@ -227,15 +227,6 @@
     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.
@@ -251,8 +242,9 @@
     // 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);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index d3d814c..7cf5656 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@ -27,6 +27,7 @@
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -34,7 +35,12 @@
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+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.server.balancer.Dispatcher.DBlock;
 import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
@@ -99,7 +105,7 @@
       final LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
       final List<MLocation> locations = MLocation.toLocations(lb);
       final MLocation ml = locations.get(0);
-      final DBlock db = mover.newDBlock(lb.getBlock().getLocalBlock(), locations);
+      final DBlock db = mover.newDBlock(lb, locations, null);
 
       final List<StorageType> storageTypes = new ArrayList<StorageType>(
           Arrays.asList(StorageType.DEFAULT, StorageType.DEFAULT));
@@ -409,4 +415,119 @@
       cluster.shutdown();
     }
   }
+
+  int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private final static int stripesPerBlock = 4;
+  static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock;
+
+  static void initConfWithStripe(Configuration conf) {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
+  }
+
+  @Test(timeout = 300000)
+  public void testMoverWithStripedFile() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    initConfWithStripe(conf);
+
+    // start 10 datanodes
+    int numOfDatanodes =10;
+    int storagesPerDatanode=2;
+    long capacity = 10 * DEFAULT_STRIPE_BLOCK_SIZE;
+    long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
+    for (int i = 0; i < numOfDatanodes; i++) {
+      for(int j=0;j<storagesPerDatanode;j++){
+        capacities[i][j]=capacity;
+      }
+    }
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numOfDatanodes)
+        .storagesPerDatanode(storagesPerDatanode)
+        .storageTypes(new StorageType[][]{
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.DISK},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE},
+            {StorageType.DISK, StorageType.ARCHIVE}})
+        .storageCapacities(capacities)
+        .build();
+
+    try {
+      cluster.waitActive();
+
+      // set "/bar" directory with HOT storage policy.
+      ClientProtocol client = NameNodeProxies.createProxy(conf,
+          cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy();
+      String barDir = "/bar";
+      client.mkdirs(barDir, new FsPermission((short) 777), true);
+      client.setStoragePolicy(barDir,
+          HdfsConstants.HOT_STORAGE_POLICY_NAME);
+      // set an EC policy on "/bar" directory
+      client.setErasureCodingPolicy(barDir, null);
+
+      // write file to barDir
+      final String fooFile = "/bar/foo";
+      long fileLen = 20 * DEFAULT_STRIPE_BLOCK_SIZE ;
+      DFSTestUtil.createFile(cluster.getFileSystem(), new Path(fooFile),
+          fileLen,(short) 3, 0);
+
+      // verify storage types and locations
+      LocatedBlocks locatedBlocks =
+          client.getBlockLocations(fooFile, 0, fileLen);
+      for(LocatedBlock lb : locatedBlocks.getLocatedBlocks()){
+        for( StorageType type : lb.getStorageTypes()){
+          Assert.assertEquals(StorageType.DISK, type);
+        }
+      }
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
+          dataBlocks + parityBlocks);
+
+      // start 5 more datanodes
+      numOfDatanodes +=5;
+      capacities = new long[5][storagesPerDatanode];
+      for (int i = 0; i < 5; i++) {
+        for(int j=0;j<storagesPerDatanode;j++){
+          capacities[i][j]=capacity;
+        }
+      }
+      cluster.startDataNodes(conf, 5,
+          new StorageType[][]{
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE},
+              {StorageType.ARCHIVE, StorageType.ARCHIVE}},
+          true, null, null, null,capacities, null, false, false, false, null);
+      cluster.triggerHeartbeats();
+
+      // move file to ARCHIVE
+      client.setStoragePolicy(barDir, "COLD");
+      // run Mover
+      int rc = ToolRunner.run(conf, new Mover.Cli(),
+          new String[] { "-p", barDir });
+      Assert.assertEquals("Movement to ARCHIVE should be successfull", 0, rc);
+
+      // verify storage types and locations
+      locatedBlocks = client.getBlockLocations(fooFile, 0, fileLen);
+      for(LocatedBlock lb : locatedBlocks.getLocatedBlocks()){
+        for( StorageType type : lb.getStorageTypes()){
+          Assert.assertEquals(StorageType.ARCHIVE, type);
+        }
+      }
+      StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
+          dataBlocks + parityBlocks);
+
+    }finally{
+      cluster.shutdown();
+    }
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index 64f614d..fd11cc0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -71,8 +71,13 @@
   public static HdfsFileStatus getFileInfo(NameNode namenode, String src,
       boolean resolveLink) throws AccessControlException, UnresolvedLinkException,
         StandbyException, IOException {
-    return FSDirStatAndListingOp.getFileInfo(namenode.getNamesystem()
-            .getFSDirectory(), src, resolveLink);
+    namenode.getNamesystem().readLock();
+    try {
+      return FSDirStatAndListingOp.getFileInfo(namenode.getNamesystem()
+          .getFSDirectory(), src, resolveLink);
+    } finally {
+      namenode.getNamesystem().readUnlock();
+    }
   }
   
   public static boolean mkdirs(NameNode namenode, String src,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
new file mode 100644
index 0000000..abcdbc1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
@@ -0,0 +1,265 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.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.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestAddOverReplicatedStripedBlocks {
+
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private final Path dirPath = new Path("/striped");
+  private Path filePath = new Path(dirPath, "file");
+  private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final short GROUP_SIZE = (short) (DATA_BLK_NUM + PARITY_BLK_NUM);
+  private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private final int NUM_STRIPE_PER_BLOCK = 4;
+  private final int BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE;
+  private final int numDNs = GROUP_SIZE + 3;
+
+  @Before
+  public void setup() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    // disable block recovery
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
+    SimulatedFSDataset.setFactory(conf);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    fs.mkdirs(dirPath);
+    fs.getClient().setErasureCodingPolicy(dirPath.toString(), null);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testProcessOverReplicatedStripedBlock() throws Exception {
+    // create a file which has exact one block group to the first GROUP_SIZE DNs
+    long fileLen = DATA_BLK_NUM * BLOCK_SIZE;
+    DFSTestUtil.createStripedFile(cluster, filePath, null, 1,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations(
+        filePath.toString(), 0, fileLen);
+    LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0));
+    long gs = bg.getBlock().getGenerationStamp();
+    String bpid = bg.getBlock().getBlockPoolId();
+    long groupId = bg.getBlock().getBlockId();
+    Block blk = new Block(groupId, BLOCK_SIZE, gs);
+    for (int i = 0; i < GROUP_SIZE; i++) {
+      blk.setBlockId(groupId + i);
+      cluster.injectBlocks(i, Arrays.asList(blk), bpid);
+    }
+    cluster.triggerBlockReports();
+
+    // let a internal block be over replicated with 2 redundant blocks.
+    blk.setBlockId(groupId + 2);
+    cluster.injectBlocks(numDNs - 3, Arrays.asList(blk), bpid);
+    cluster.injectBlocks(numDNs - 2, Arrays.asList(blk), bpid);
+    // let a internal block be over replicated with 1 redundant block.
+    blk.setBlockId(groupId + 6);
+    cluster.injectBlocks(numDNs - 1, Arrays.asList(blk), bpid);
+
+    // update blocksMap
+    cluster.triggerBlockReports();
+    // add to invalidates
+    cluster.triggerHeartbeats();
+    // datanode delete block
+    cluster.triggerHeartbeats();
+    // update blocksMap
+    cluster.triggerBlockReports();
+
+    // verify that all internal blocks exists
+    lbs = cluster.getNameNodeRpc().getBlockLocations(
+        filePath.toString(), 0, fileLen);
+    StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE);
+  }
+
+  @Test
+  public void testProcessOverReplicatedSBSmallerThanFullBlocks()
+      throws Exception {
+    // Create a EC file which doesn't fill full internal blocks.
+    int fileLen = CELLSIZE * (DATA_BLK_NUM - 1);
+    byte[] content = new byte[fileLen];
+    DFSTestUtil.writeFile(fs, filePath, new String(content));
+    LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations(
+        filePath.toString(), 0, fileLen);
+    LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0));
+    long gs = bg.getBlock().getGenerationStamp();
+    String bpid = bg.getBlock().getBlockPoolId();
+    long groupId = bg.getBlock().getBlockId();
+    Block blk = new Block(groupId, BLOCK_SIZE, gs);
+    cluster.triggerBlockReports();
+    List<DatanodeInfo> infos = Arrays.asList(bg.getLocations());
+
+    // let a internal block be over replicated with 2 redundant blocks.
+    // Therefor number of internal blocks is over GROUP_SIZE. (5 data blocks +
+    // 3 parity blocks  + 2 redundant blocks > GROUP_SIZE)
+    blk.setBlockId(groupId + 2);
+    List<DataNode> dataNodeList = cluster.getDataNodes();
+    for (int i = 0; i < numDNs; i++) {
+      if (!infos.contains(dataNodeList.get(i).getDatanodeId())) {
+        cluster.injectBlocks(i, Arrays.asList(blk), bpid);
+        System.out.println("XXX: inject block into datanode " + i);
+      }
+    }
+
+    // update blocksMap
+    cluster.triggerBlockReports();
+    // add to invalidates
+    cluster.triggerHeartbeats();
+    // datanode delete block
+    cluster.triggerHeartbeats();
+    // update blocksMap
+    cluster.triggerBlockReports();
+
+    // verify that all internal blocks exists
+    lbs = cluster.getNameNodeRpc().getBlockLocations(
+        filePath.toString(), 0, fileLen);
+    StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1);
+  }
+
+  @Test
+  public void testProcessOverReplicatedAndCorruptStripedBlock()
+      throws Exception {
+    long fileLen = DATA_BLK_NUM * BLOCK_SIZE;
+    DFSTestUtil.createStripedFile(cluster, filePath, null, 1,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations(
+        filePath.toString(), 0, fileLen);
+    LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0));
+    long gs = bg.getBlock().getGenerationStamp();
+    String bpid = bg.getBlock().getBlockPoolId();
+    long groupId = bg.getBlock().getBlockId();
+    Block blk = new Block(groupId, BLOCK_SIZE, gs);
+    BlockInfoStriped blockInfo = new BlockInfoStriped(blk,
+        ErasureCodingPolicyManager.getSystemDefaultPolicy());
+    for (int i = 0; i < GROUP_SIZE; i++) {
+      blk.setBlockId(groupId + i);
+      cluster.injectBlocks(i, Arrays.asList(blk), bpid);
+    }
+    cluster.triggerBlockReports();
+
+    // let a internal block be corrupt
+    BlockManager bm = cluster.getNamesystem().getBlockManager();
+    List<DatanodeInfo> infos = Arrays.asList(bg.getLocations());
+    List<String> storages = Arrays.asList(bg.getStorageIDs());
+    cluster.getNamesystem().writeLock();
+    try {
+      bm.findAndMarkBlockAsCorrupt(lbs.getLastLocatedBlock().getBlock(),
+          infos.get(0), storages.get(0), "TEST");
+    } finally {
+      cluster.getNamesystem().writeUnlock();
+    }
+    assertEquals(1, bm.countNodes(blockInfo).corruptReplicas());
+
+    // let a internal block be over replicated with 2 redundant block.
+    blk.setBlockId(groupId + 2);
+    cluster.injectBlocks(numDNs - 3, Arrays.asList(blk), bpid);
+    cluster.injectBlocks(numDNs - 2, Arrays.asList(blk), bpid);
+
+    // update blocksMap
+    cluster.triggerBlockReports();
+    // add to invalidates
+    cluster.triggerHeartbeats();
+    // datanode delete block
+    cluster.triggerHeartbeats();
+    // update blocksMap
+    cluster.triggerBlockReports();
+
+    // verify that all internal blocks exists
+    lbs = cluster.getNameNodeRpc().getBlockLocations(
+        filePath.toString(), 0, fileLen);
+    StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE);
+  }
+
+  @Test
+  public void testProcessOverReplicatedAndMissingStripedBlock()
+      throws Exception {
+    long fileLen = CELLSIZE * DATA_BLK_NUM;
+    DFSTestUtil.createStripedFile(cluster, filePath, null, 1,
+        NUM_STRIPE_PER_BLOCK, false);
+    LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations(
+        filePath.toString(), 0, fileLen);
+    LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0));
+    long gs = bg.getBlock().getGenerationStamp();
+    String bpid = bg.getBlock().getBlockPoolId();
+    long groupId = bg.getBlock().getBlockId();
+    Block blk = new Block(groupId, BLOCK_SIZE, gs);
+    // only inject GROUP_SIZE - 1 blocks, so there is one block missing
+    for (int i = 0; i < GROUP_SIZE - 1; i++) {
+      blk.setBlockId(groupId + i);
+      cluster.injectBlocks(i, Arrays.asList(blk), bpid);
+    }
+    cluster.triggerBlockReports();
+
+    // let a internal block be over replicated with 2 redundant blocks.
+    // Therefor number of internal blocks is over GROUP_SIZE. (5 data blocks +
+    // 3 parity blocks  + 2 redundant blocks > GROUP_SIZE)
+    blk.setBlockId(groupId + 2);
+    cluster.injectBlocks(numDNs - 3, Arrays.asList(blk), bpid);
+    cluster.injectBlocks(numDNs - 2, Arrays.asList(blk), bpid);
+
+    // update blocksMap
+    cluster.triggerBlockReports();
+    Thread.sleep(2000);
+    // add to invalidates
+    cluster.triggerHeartbeats();
+    // datanode delete block
+    cluster.triggerHeartbeats();
+    // update blocksMap
+    cluster.triggerBlockReports();
+
+    // Since one block is missing, when over-replicated blocks got deleted,
+    // we are left GROUP_SIZE - 1 blocks.
+    lbs = cluster.getNameNodeRpc().getBlockLocations(
+        filePath.toString(), 0, fileLen);
+    StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1);
+  }
+
+}
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..836bff1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -0,0 +1,430 @@
+/**
+ * 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.DFSStripedOutputStream;
+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.StripedFileTestUtil;
+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.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
+import static org.junit.Assert.assertEquals;
+
+public class TestAddStripedBlocks {
+  private final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS +
+      StripedFileTestUtil.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.getClient().setErasureCodingPolicy("/", null);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Make sure the IDs of striped blocks do not conflict
+   */
+  @Test
+  public void testAllocateBlockId() throws Exception {
+    Path testPath = new Path("/testfile");
+    // create a file while allocates a new block
+    DFSTestUtil.writeFile(dfs, testPath, "hello, world!");
+    LocatedBlocks lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0);
+    final long firstId = lb.get(0).getBlock().getBlockId();
+    // delete the file
+    dfs.delete(testPath, true);
+
+    // allocate a new block, and make sure the new block's id does not conflict
+    // with the previous one
+    DFSTestUtil.writeFile(dfs, testPath, "hello again");
+    lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0);
+    final long secondId = lb.get(0).getBlock().getBlockId();
+    Assert.assertEquals(firstId + HdfsServerConstants.MAX_BLOCKS_IN_GROUP, secondId);
+  }
+
+  private static void writeAndFlushStripedOutputStream(
+      DFSStripedOutputStream out, int chunkSize) throws IOException {
+    // FSOutputSummer.BUFFER_NUM_CHUNKS == 9
+    byte[] toWrite = new byte[chunkSize * 9 + 1];
+    out.write(toWrite);
+    DFSTestUtil.flushInternal(out);
+  }
+
+  @Test (timeout=60000)
+  public void testAddStripedBlock() throws Exception {
+    final Path file = new Path("/file1");
+    // create an empty file
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1);
+      writeAndFlushStripedOutputStream(
+          (DFSStripedOutputStream) out.getWrappedStream(),
+          DFS_BYTES_PER_CHECKSUM_DEFAULT);
+
+      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(), true);
+
+      // restart NameNode to check editlog
+      cluster.restartNameNode(true);
+      fsdir = cluster.getNamesystem().getFSDirectory();
+      fileNode = fsdir.getINode4Write(file.toString()).asFile();
+      blocks = fileNode.getBlocks();
+      assertEquals(1, blocks.length);
+      Assert.assertTrue(blocks[0].isStriped());
+      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
+
+      // save namespace, restart namenode, and check
+      dfs = cluster.getFileSystem();
+      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+      dfs.saveNamespace();
+      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+      cluster.restartNameNode(true);
+      fsdir = cluster.getNamesystem().getFSDirectory();
+      fileNode = fsdir.getINode4Write(file.toString()).asFile();
+      blocks = fileNode.getBlocks();
+      assertEquals(1, blocks.length);
+      Assert.assertTrue(blocks[0].isStriped());
+      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+  }
+
+  private void checkStripedBlockUC(BlockInfoStriped block,
+      boolean checkReplica) {
+    assertEquals(0, block.numNodes());
+    Assert.assertFalse(block.isComplete());
+    Assert.assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, block.getDataBlockNum());
+    Assert.assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS,
+        block.getParityBlockNum());
+    Assert.assertEquals(0,
+        block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
+
+    Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+        block.getBlockUCState());
+    if (checkReplica) {
+      Assert.assertEquals(GROUP_SIZE,
+          block.getUnderConstructionFeature().getNumExpectedLocations());
+      DatanodeStorageInfo[] storages = block.getUnderConstructionFeature()
+          .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;
+  }
+
+  @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);
+      writeAndFlushStripedOutputStream(
+          (DFSStripedOutputStream) out.getWrappedStream(),
+          DFS_BYTES_PER_CHECKSUM_DEFAULT);
+
+      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+      BlockInfoStriped lastBlk = (BlockInfoStriped) fileNode.getLastBlock();
+      DatanodeInfo[] expectedDNs = DatanodeStorageInfo.toDatanodeInfos(
+          lastBlk.getUnderConstructionFeature().getExpectedStorageLocations());
+      int[] indices = lastBlk.getUnderConstructionFeature().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();
+
+      DatanodeStorageInfo[] locs = lastBlock.getUnderConstructionFeature()
+          .getExpectedStorageLocations();
+      int[] indices = lastBlock.getUnderConstructionFeature().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 BlockInfoUCStriped is correct
+      int i = 0;
+      for (DataNode dn : cluster.getDataNodes()) {
+        final Block block = new Block(lastBlock.getBlockId() + i++,
+            0, lastBlock.getGenerationStamp());
+        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 = lastBlock.getUnderConstructionFeature().getExpectedStorageLocations();
+      indices = lastBlock.getUnderConstructionFeature().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 = BlockListAsLongs.encode(blocks);
+      StorageBlockReport[] reports = {new StorageBlockReport(storage,
+          bll)};
+      cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId),
+          bpId, reports, null);
+    }
+
+    DatanodeStorageInfo[] locs = lastBlock.getUnderConstructionFeature()
+        .getExpectedStorageLocations();
+    int[] indices = lastBlock.getUnderConstructionFeature().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]);
+    }
+  }
+
+  @Test
+  public void testCheckStripedReplicaCorrupt() throws Exception {
+    final int numBlocks = 4;
+    final int numStripes = 4;
+    final Path filePath = new Path("/corrupt");
+    final FSNamesystem ns = cluster.getNameNode().getNamesystem();
+    final BlockManager bm = ns.getBlockManager();
+    DFSTestUtil.createStripedFile(cluster, filePath, null,
+        numBlocks, numStripes, false);
+
+    INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()).
+        asFile();
+    Assert.assertTrue(fileNode.isStriped());
+    BlockInfo stored = fileNode.getBlocks()[0];
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(0, ns.getCorruptReplicaBlocks());
+
+    // Now send a block report with correct size
+    DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+    final Block reported = new Block(stored);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
+    StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+        .makeReportForReceivedBlock(reported,
+            ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(0).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(0, ns.getCorruptReplicaBlocks());
+
+    // Now send a block report with wrong size
+    reported.setBlockId(stored.getBlockId() + 1);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE - 1);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+            ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(1).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
+
+    // Now send a parity block report with correct size
+    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(2).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
+
+    // Now send a parity block report with wrong size
+    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 1);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(3).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    // the total number of corrupted block info is still 1
+    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
+    // 2 internal blocks corrupted
+    Assert.assertEquals(2, bm.getCorruptReplicas(stored).size());
+
+    // Now change the size of stored block, and test verifying the last
+    // block size
+    stored.setNumBytes(stored.getNumBytes() + 10);
+    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS + 2);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(4).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
+    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
+
+    // Now send a parity block report with correct size based on adjusted
+    // size of stored block
+    /** Now stored block has {@link numStripes} full stripes + a cell + 10 */
+    stored.setNumBytes(stored.getNumBytes() + BLOCK_STRIPED_CELL_SIZE);
+    reported.setBlockId(stored.getBlockId());
+    reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(0).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
+    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
+
+    reported.setBlockId(stored.getBlockId() + 1);
+    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 10);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(5).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
+    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
+
+    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
+    reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE);
+    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+    ns.processIncrementalBlockReport(
+        cluster.getDataNodes().get(2).getDatanodeId(), reports[0]);
+    BlockManagerTestUtil.updateState(ns.getBlockManager());
+    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
+    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index c5262d4..ec3d924 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -162,7 +162,8 @@
     // choose the targets, but local node should not get selected as this is not
     // part of the cluster anymore
     DatanodeStorageInfo[] results = bm.chooseTarget4NewBlock("/hello", 3,
-        clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7);
+        clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7,
+        false);
     for (DatanodeStorageInfo datanodeStorageInfo : results) {
       assertFalse("Dead node should not be choosen", datanodeStorageInfo
           .getDatanodeDescriptor().equals(clientNode));
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index df07a62..2bb3d5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -21,6 +21,7 @@
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.junit.Assert.assertFalse;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.spy;
 
@@ -38,14 +39,23 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 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.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
@@ -65,6 +75,9 @@
   private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class);
 
   private static final int NUM_DATA_NODES = 0;
+
+  private static final ErasureCodingPolicy testECPolicy
+      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
   
   @Test
   public void testDisplayRecentEditLogOpCodes() throws IOException {
@@ -417,4 +430,260 @@
           fromByte(code), FSEditLogOpCodes.fromByte(code));
     }
   }
+
+  @Test
+  public void testAddNewStripedBlock() throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/ec";
+      String testFile = "testfile_001";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser1";
+      String clientMachine = "testMachine1";
+      long blkId = 1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+      short blockNum = StripedFileTestUtil.NUM_DATA_BLOCKS;
+      short parityNum = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+
+      //set the storage policy of the directory
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      fs.getClient().getNamenode().setErasureCodingPolicy(testDir, null);
+
+      // Create a file with striped block
+      Path p = new Path(testFilePath);
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      fns.leaveSafeMode();
+
+      // Add a striped block to the file
+      BlockInfoStriped stripedBlk = new BlockInfoStriped(
+          new Block(blkId, blkNumBytes, timestamp), testECPolicy);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(stripedBlk);
+      fns.getEditLog().logAddBlock(testFilePath, file);
+      file.toCompleteFile(System.currentTimeMillis());
+
+      //If the block by loaded is the same as above it means that
+      //we have successfully applied the edit log to the fsimage.
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+
+      INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory()
+          .getINode(testFilePath);
+
+      assertTrue(inodeLoaded.isStriped());
+
+      BlockInfo[] blks = inodeLoaded.getBlocks();
+      assertEquals(1, blks.length);
+      assertEquals(blkId, blks[0].getBlockId());
+      assertEquals(blkNumBytes, blks[0].getNumBytes());
+      assertEquals(timestamp, blks[0].getGenerationStamp());
+      assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum());
+      assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testUpdateStripedBlocks() throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/ec";
+      String testFile = "testfile_002";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser2";
+      String clientMachine = "testMachine2";
+      long blkId = 1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+      short blockNum = StripedFileTestUtil.NUM_DATA_BLOCKS;
+      short parityNum = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+
+      //set the storage policy of the directory
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      fs.getClient().getNamenode().setErasureCodingPolicy(testDir, null);
+
+      //create a file with striped blocks
+      Path p = new Path(testFilePath);
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoStriped stripedBlk = new BlockInfoStriped(
+          new Block(blkId, blkNumBytes, timestamp), testECPolicy);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(stripedBlk);
+      fns.getEditLog().logAddBlock(testFilePath, file);
+      file.toCompleteFile(System.currentTimeMillis());
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      fns.leaveSafeMode();
+
+      //update the last block
+      long newBlkNumBytes = 1024*8;
+      long newTimestamp = 1426222918+3600;
+      file.toUnderConstruction(clientName, clientMachine);
+      file.getLastBlock().setNumBytes(newBlkNumBytes);
+      file.getLastBlock().setGenerationStamp(newTimestamp);
+      fns.getEditLog().logUpdateBlocks(testFilePath, file, true);
+      file.toCompleteFile(System.currentTimeMillis());
+
+      //After the namenode restarts if the block by loaded is the same as above
+      //(new block size and timestamp) it means that we have successfully
+      //applied the edit log to the fsimage.
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+
+      INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory()
+          .getINode(testFilePath);
+
+      assertTrue(inodeLoaded.isStriped());
+
+      BlockInfo[] blks = inodeLoaded.getBlocks();
+      assertEquals(1, blks.length);
+      assertTrue(blks[0].isStriped());
+      assertEquals(blkId, blks[0].getBlockId());
+      assertEquals(newBlkNumBytes, blks[0].getNumBytes());
+      assertEquals(newTimestamp, blks[0].getGenerationStamp());
+      assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum());
+      assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForAddBlock() throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_addblock";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_addblock";
+      String clientMachine = "testMachine_addblock";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      //check whether the hasNonEcBlockUsingStripedID is set
+      //after loading a addblock-editlog
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      fns.getEditLog().logAddBlock(testFilePath, file);
+      file.toCompleteFile(System.currentTimeMillis());
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForUpdateBlocks()
+      throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_002";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser2";
+      String clientMachine = "testMachine1";
+      long blkId = 100;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      file.toCompleteFile(System.currentTimeMillis());
+
+      long newBlkNumBytes = 1024*8;
+      long newTimestamp = 1426222918+3600;
+      file.toUnderConstruction(clientName, clientMachine);
+      file.getLastBlock().setBlockId(-100);
+      file.getLastBlock().setNumBytes(newBlkNumBytes);
+      file.getLastBlock().setGenerationStamp(newTimestamp);
+      fns.getEditLog().logUpdateBlocks(testFilePath, file, true);
+      file.toCompleteFile(System.currentTimeMillis());
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
 }
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 df20fd6..e45d08d 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
@@ -22,11 +22,25 @@
 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.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
 
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -34,15 +48,15 @@
 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;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 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.BlockInfo;
 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.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
@@ -50,10 +64,15 @@
 import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
+import static org.junit.Assert.assertArrayEquals;
+
 public class TestFSImage {
 
   private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ =
       "image-with-zero-block-size.tar.gz";
+  private static final ErasureCodingPolicy testECPolicy
+      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+
   @Test
   public void testPersist() throws IOException {
     Configuration conf = new Configuration();
@@ -120,6 +139,123 @@
     }
   }
 
+  private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf,
+                                               boolean isUC) throws IOException{
+    // contruct a INode with StripedBlock for saving and loading
+    fsn.setErasureCodingPolicy("/", null, false);
+    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;
+    INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
+        blks, replication, preferredBlockSize, (byte) 0, true);
+    ByteArrayOutputStream bs = new ByteArrayOutputStream();
+
+    //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),
+              testECPolicy);
+      file.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());
+    // TODO for striped blocks, we currently save and load them as contiguous
+    // blocks to/from legacy fsimage
+    assertEquals(3, fileByLoaded.getBlocks().length);
+    assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
+
+    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 testSaveAndLoadStripedINodeFile() throws IOException{
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      testSaveAndLoadStripedINodeFile(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 testSaveAndLoadStripedINodeFileUC() throws IOException {
+    // construct a INode with StripedBlock for saving and loading
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
    /**
    * On checkpointing , stale fsimage checkpoint file should be deleted.
    */
@@ -260,8 +396,7 @@
         .format(false)
         .manageDataDfsDirs(false)
         .manageNameDfsDirs(false)
-        .waitSafeMode(false)
-        .startupOption(StartupOption.UPGRADE)
+        .waitSafeMode(false).startupOption(StartupOption.UPGRADE)
         .build();
     try {
       FileSystem fs = cluster.getFileSystem();
@@ -274,4 +409,207 @@
       FileUtil.fullyDelete(dfsDir);
     }
   }
+
+  /**
+   * Ensure that FSImage supports BlockGroup.
+   */
+  @Test
+  public void testSupportBlockGroup() throws IOException {
+    final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS
+        + StripedFileTestUtil.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.getClient().getNamenode().setErasureCodingPolicy("/", null);
+      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();
+      assertTrue(inode.isStriped());
+      BlockInfo[] blks = inode.getBlocks();
+      assertEquals(1, blks.length);
+      assertTrue(blks[0].isStriped());
+      assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum());
+      assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForLoadFile() throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_loadfile";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_loadfile";
+      String clientMachine = "testMachine_loadfile";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      file.toCompleteFile(System.currentTimeMillis());
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      //after nonEcBlockUsingStripedID is deleted
+      //the hasNonEcBlockUsingStripedID is set to false
+      fs = cluster.getFileSystem();
+      fs.delete(p,false);
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertFalse(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForLoadUCFile()
+      throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_loaducfile";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_loaducfile";
+      String clientMachine = "testMachine_loaducfile";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForLoadSnapshot()
+      throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_loadSnapshot";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_loadSnapshot";
+      String clientMachine = "testMachine_loadSnapshot";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      Path d = new Path(testDir);
+      fs.mkdir(d, new FsPermission("755"));
+      fs.allowSnapshot(d);
+
+      Path p = new Path(testFilePath);
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      file.toCompleteFile(System.currentTimeMillis());
+
+      fs.createSnapshot(d,"testHasNonEcBlockUsingStripeID");
+      fs.truncate(p,0);
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }
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 34677ef..0423d41 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
@@ -56,6 +56,7 @@
 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.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -1033,7 +1034,8 @@
     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();
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 a84ddd0..6df88fd 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
@@ -86,6 +86,8 @@
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result;
+import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ReplicationResult;
+import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ErasureCodingResult;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSck;
 import org.apache.hadoop.io.IOUtils;
@@ -1071,13 +1073,14 @@
       final HdfsFileStatus file = 
           namenode.getRpcServer().getFileInfo(pathString);
       assertNotNull(file);
-      Result res = new Result(conf);
-      fsck.check(pathString, file, res);
+      Result replRes = new ReplicationResult(conf);
+      Result ecRes = new ErasureCodingResult(conf);
+      fsck.check(pathString, file, replRes, ecRes);
       // Also print the output from the fsck, for ex post facto sanity checks
       System.out.println(result.toString());
-      assertEquals(res.missingReplicas, 
+      assertEquals(replRes.missingReplicas,
           (NUM_BLOCKS*REPL_FACTOR) - (NUM_BLOCKS*NUM_REPLICAS));
-      assertEquals(res.numExpectedReplicas, NUM_BLOCKS*REPL_FACTOR);
+      assertEquals(replRes.numExpectedReplicas, NUM_BLOCKS*REPL_FACTOR);
     } finally {
       if(dfs != null) {
         dfs.close();
@@ -1148,10 +1151,11 @@
       final HdfsFileStatus file = 
           namenode.getRpcServer().getFileInfo(pathString);
       assertNotNull(file);
-      Result res = new Result(conf);
-      fsck.check(pathString, file, res);
+      Result replRes = new ReplicationResult(conf);
+      Result ecRes = new ErasureCodingResult(conf);
+      fsck.check(pathString, file, replRes, ecRes);
       // check misReplicatedBlock number.
-      assertEquals(res.numMisReplicatedBlocks, NUM_BLOCKS);
+      assertEquals(replRes.numMisReplicatedBlocks, NUM_BLOCKS);
     } finally {
       if(dfs != null) {
         dfs.close();
@@ -1211,15 +1215,16 @@
 
     HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication,
         blockSize, modTime, accessTime, perms, owner, group, symlink,
-        path, fileId, numChildren, null, storagePolicy);
-    Result res = new Result(conf);
+        path, fileId, numChildren, null, storagePolicy, null);
+    Result replRes = new ReplicationResult(conf);
+    Result ecRes = new ErasureCodingResult(conf);
 
     try {
-      fsck.check(pathString, file, res);
+      fsck.check(pathString, file, replRes, ecRes);
     } catch (Exception e) {
       fail("Unexpected exception " + e.getMessage());
     }
-    assertTrue(res.toString().contains("HEALTHY"));
+    assertTrue(replRes.isHealthy());
   }
 
   /** Test fsck with symlinks in the filesystem */
@@ -1657,4 +1662,60 @@
       }
     }
   }
+
+  @Test
+  public void testECFsck() throws Exception {
+    MiniDFSCluster cluster = null;
+    FileSystem fs = null;
+    try {
+      Configuration conf = new HdfsConfiguration();
+      final long precision = 1L;
+      conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
+      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
+      int totalSize = ErasureCodingPolicyManager.getSystemDefaultPolicy().getNumDataUnits()
+                      + ErasureCodingPolicyManager.getSystemDefaultPolicy().getNumParityUnits();
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(totalSize).build();
+      fs = cluster.getFileSystem();
+
+      // create a contiguous file
+      Path replDirPath = new Path("/replicated");
+      Path replFilePath = new Path(replDirPath, "replfile");
+      final short factor = 3;
+      DFSTestUtil.createFile(fs, replFilePath, 1024, factor, 0);
+      DFSTestUtil.waitReplication(fs, replFilePath, factor);
+
+      // create a large striped file
+      Path ecDirPath = new Path("/striped");
+      Path largeFilePath = new Path(ecDirPath, "largeFile");
+      DFSTestUtil.createStripedFile(cluster, largeFilePath, ecDirPath, 1, 2, true);
+
+      // create a small striped file
+      Path smallFilePath = new Path(ecDirPath, "smallFile");
+      DFSTestUtil.writeFile(fs, smallFilePath, "hello world!");
+
+      long replTime = fs.getFileStatus(replFilePath).getAccessTime();
+      long ecTime = fs.getFileStatus(largeFilePath).getAccessTime();
+      Thread.sleep(precision);
+      setupAuditLogs();
+      String outStr = runFsck(conf, 0, true, "/");
+      verifyAuditLogs();
+      assertEquals(replTime, fs.getFileStatus(replFilePath).getAccessTime());
+      assertEquals(ecTime, fs.getFileStatus(largeFilePath).getAccessTime());
+      System.out.println(outStr);
+      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
+      if (fs != null) {try{fs.close();} catch(Exception e){}}
+      cluster.shutdown();
+
+      // restart the cluster; bring up namenode but not the data nodes
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0).format(false).build();
+      outStr = runFsck(conf, 1, true, "/");
+      // expect the result is corrupt
+      assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
+      System.out.println(outStr);
+    } finally {
+      if (fs != null) {try{fs.close();} catch(Exception e){}}
+      if (cluster != null) { cluster.shutdown(); }
+    }
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
index 9f5850a..c33e668 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
@@ -96,7 +96,7 @@
 
   private static INodeFile createINodeFile(byte storagePolicyID) {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-        null, (short)3, 1024L, storagePolicyID);
+        null, (short)3, 1024L, storagePolicyID, false);
   }
 
   @Test
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
new file mode 100644
index 0000000..26f9b8e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+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;
+
+/**
+ * Make sure we correctly update the quota usage with the striped blocks.
+ */
+public class TestQuotaWithStripedBlocks {
+  private static final int BLOCK_SIZE = 1024 * 1024;
+  private static final long DISK_QUOTA = BLOCK_SIZE * 10;
+  private static final ErasureCodingPolicy ecPolicy =
+      ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  private static final int NUM_DATA_BLOCKS = ecPolicy.getNumDataUnits();
+  private static final int NUM_PARITY_BLOCKS = ecPolicy.getNumParityUnits();
+  private static final int GROUP_SIZE = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
+  private static final Path ecDir = new Path("/ec");
+
+  private MiniDFSCluster cluster;
+  private FSDirectory dir;
+  private DistributedFileSystem dfs;
+
+  @Before
+  public void setUp() throws IOException {
+    final Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE).build();
+    cluster.waitActive();
+
+    dir = cluster.getNamesystem().getFSDirectory();
+    dfs = cluster.getFileSystem();
+
+    dfs.mkdirs(ecDir);
+    dfs.getClient().setErasureCodingPolicy(ecDir.toString(), ecPolicy);
+    dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA);
+    dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA);
+    dfs.setStoragePolicy(ecDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testUpdatingQuotaCount() throws Exception {
+    final Path file = new Path(ecDir, "file");
+    FSDataOutputStream out = null;
+
+    try {
+      out = dfs.create(file, (short) 1);
+
+      INodeFile fileNode = dir.getINode4Write(file.toString()).asFile();
+      ExtendedBlock previous = null;
+      // Create striped blocks which have a cell in each block.
+      Block newBlock = DFSTestUtil.addStripedBlockToFile(cluster.getDataNodes(),
+          dfs, cluster.getNamesystem(), file.toString(), fileNode,
+          dfs.getClient().getClientName(), previous, 1);
+      previous = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(),
+          newBlock);
+
+      final INodeDirectory dirNode = dir.getINode4Write(ecDir.toString())
+          .asDirectory();
+      final long spaceUsed = dirNode.getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().getStorageSpace();
+      final long diskUsed = dirNode.getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
+      // When we add a new block we update the quota using the full block size.
+      Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, spaceUsed);
+      Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, diskUsed);
+
+      dfs.getClient().getNamenode().complete(file.toString(),
+          dfs.getClient().getClientName(), previous, fileNode.getId());
+
+      final long actualSpaceUsed = dirNode.getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().getStorageSpace();
+      final long actualDiskUsed = dirNode.getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
+      // In this case the file's real size is cell size * block group size.
+      Assert.assertEquals(StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
+          actualSpaceUsed);
+      Assert.assertEquals(StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
+          actualDiskUsed);
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
new file mode 100644
index 0000000..6774aed
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -0,0 +1,169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
+import org.junit.Test;
+import java.util.List;
+
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS;
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_PARITY_BLOCKS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestRecoverStripedBlocks {
+  private final short GROUP_SIZE =
+      (short) (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS);
+  private MiniDFSCluster cluster;
+  private final Path dirPath = new Path("/dir");
+  private Path filePath = new Path(dirPath, "file");
+  private int maxReplicationStreams =
+      DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT;
+
+  private void initConf(Configuration conf) {
+    // 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);
+  }
+
+  @Test
+  public void testMissingStripedBlock() throws Exception {
+    doTestMissingStripedBlock(1, 0);
+  }
+
+  @Test
+  public void testMissingStripedBlockWithBusyNode1() throws Exception {
+    doTestMissingStripedBlock(2, 1);
+  }
+
+  @Test
+  public void testMissingStripedBlockWithBusyNode2() throws Exception {
+    doTestMissingStripedBlock(3, 1);
+  }
+
+  /**
+   * Start GROUP_SIZE + 1 datanodes.
+   * Inject striped blocks to first GROUP_SIZE datanodes.
+   * Then make numOfBusy datanodes busy, make numOfMissed datanodes missed.
+   * Then trigger BlockManager to compute recovery works. (so all recovery work
+   * will be scheduled to the last datanode)
+   * Finally, verify the recovery work of the last datanode.
+   */
+  private void doTestMissingStripedBlock(int numOfMissed, int numOfBusy)
+      throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    initConf(conf);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE + 1)
+        .build();
+
+    try {
+      cluster.waitActive();
+      final int numBlocks = 4;
+      DFSTestUtil.createStripedFile(cluster, filePath,
+          dirPath, numBlocks, 1, true);
+      // all blocks will be located at first GROUP_SIZE DNs, the last DN is
+      // empty because of the util function createStripedFile
+
+      // make sure the file is complete in NN
+      final INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
+          .getINode4Write(filePath.toString()).asFile();
+      assertFalse(fileNode.isUnderConstruction());
+      assertTrue(fileNode.isStriped());
+      BlockInfo[] blocks = fileNode.getBlocks();
+      assertEquals(numBlocks, blocks.length);
+      for (BlockInfo blk : blocks) {
+        assertTrue(blk.isStriped());
+        assertTrue(blk.isComplete());
+        assertEquals(BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS,
+            blk.getNumBytes());
+        final BlockInfoStriped sb = (BlockInfoStriped) blk;
+        assertEquals(GROUP_SIZE, sb.numNodes());
+      }
+
+      final BlockManager bm = cluster.getNamesystem().getBlockManager();
+      BlockInfo firstBlock = fileNode.getBlocks()[0];
+      DatanodeStorageInfo[] storageInfos = bm.getStorages(firstBlock);
+
+      // make numOfBusy nodes busy
+      int i = 0;
+      for (; i < numOfBusy; i++) {
+        DatanodeDescriptor busyNode = storageInfos[i].getDatanodeDescriptor();
+        for (int j = 0; j < maxReplicationStreams + 1; j++) {
+          BlockManagerTestUtil.addBlockToBeReplicated(busyNode, new Block(j),
+              new DatanodeStorageInfo[]{storageInfos[0]});
+        }
+      }
+
+      // make numOfMissed internal blocks missed
+      for (; i < numOfBusy + numOfMissed; i++) {
+        DatanodeDescriptor missedNode = storageInfos[i].getDatanodeDescriptor();
+        assertEquals(numBlocks, missedNode.numBlocks());
+        bm.getDatanodeManager().removeDatanode(missedNode);
+      }
+
+      BlockManagerTestUtil.getComputedDatanodeWork(bm);
+
+      // all the recovery work will be scheduled on the last DN
+      DataNode lastDn = cluster.getDataNodes().get(GROUP_SIZE);
+      DatanodeDescriptor last =
+          bm.getDatanodeManager().getDatanode(lastDn.getDatanodeId());
+      assertEquals("Counting the number of outstanding EC tasks", numBlocks,
+          last.getNumberOfBlocksToBeErasureCoded());
+      List<BlockECRecoveryInfo> recovery =
+          last.getErasureCodeCommand(numBlocks);
+      for (BlockECRecoveryInfo info : recovery) {
+        assertEquals(1, info.getTargetDnInfos().length);
+        assertEquals(last, info.getTargetDnInfos()[0]);
+        assertEquals(info.getSourceDnInfos().length,
+            info.getLiveBlockIndices().length);
+        if (GROUP_SIZE - numOfMissed == NUM_DATA_BLOCKS) {
+          // It's a QUEUE_HIGHEST_PRIORITY block, so the busy DNs will be chosen
+          // to make sure we have NUM_DATA_BLOCKS DNs to do recovery work.
+          assertEquals(NUM_DATA_BLOCKS, info.getSourceDnInfos().length);
+        } else {
+          // The block has no highest priority, so we don't use the busy DNs as
+          // sources
+          assertEquals(GROUP_SIZE - numOfMissed - numOfBusy,
+              info.getSourceDnInfos().length);
+        }
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
new file mode 100644
index 0000000..9f7ea3e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@ -0,0 +1,285 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+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.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+
+import org.junit.Test;
+
+/**
+ * This class tests INodeFile with striped feature.
+ */
+public class TestStripedINodeFile {
+  public static final Log LOG = LogFactory.getLog(TestINodeFile.class);
+
+  private static final PermissionStatus perm = new PermissionStatus(
+      "userName", null, FsPermission.getDefault());
+
+  private final BlockStoragePolicySuite defaultSuite =
+      BlockStoragePolicySuite.createDefaultSuite();
+  private final BlockStoragePolicy defaultPolicy =
+      defaultSuite.getDefaultPolicy();
+
+  private static final ErasureCodingPolicy testECPolicy
+      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+
+  private static INodeFile createStripedINodeFile() {
+    return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+        null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID, true);
+  }
+
+  @Test
+  public void testBlockStripedFeature()
+      throws IOException, InterruptedException{
+    INodeFile inf = createStripedINodeFile();
+    assertTrue(inf.isStriped());
+  }
+
+  @Test
+  public void testBlockStripedTotalBlockCount() {
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk, testECPolicy);
+    assertEquals(9, blockInfoStriped.getTotalBlockNum());
+  }
+
+  @Test
+  public void testBlockStripedLength()
+      throws IOException, InterruptedException {
+    INodeFile inf = createStripedINodeFile();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk, testECPolicy);
+    inf.addBlock(blockInfoStriped);
+    assertEquals(1, inf.getBlocks().length);
+  }
+
+  @Test
+  public void testBlockStripedConsumedSpace()
+      throws IOException, InterruptedException {
+    INodeFile inf = createStripedINodeFile();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk, testECPolicy);
+    blockInfoStriped.setNumBytes(1);
+    inf.addBlock(blockInfoStriped);
+    //   0. Calculate the total bytes per stripes <Num Bytes per Stripes>
+    //   1. Calculate the number of stripes in this block group. <Num Stripes>
+    //   2. Calculate the last remaining length which does not make a stripe. <Last Stripe Length>
+    //   3. Total consumed space is the total of
+    //     a. The total of the full cells of data blocks and parity blocks.
+    //     b. The remaining of data block which does not make a stripe.
+    //     c. The last parity block cells. These size should be same
+    //        to the first cell in this stripe.
+    // So the total consumed space is the sum of
+    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
+    //  b. <Num Bytes> % <Num Bytes per Stripes> = 1
+    //  c. <Last Stripe Length> * <Parity Block Num> = 1 * 3
+    assertEquals(4, inf.storagespaceConsumedStriped().getStorageSpace());
+    assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
+  }
+
+  @Test
+  public void testMultipleBlockStripedConsumedSpace()
+      throws IOException, InterruptedException {
+    INodeFile inf = createStripedINodeFile();
+    Block blk1 = new Block(1);
+    BlockInfoStriped blockInfoStriped1
+        = new BlockInfoStriped(blk1, testECPolicy);
+    blockInfoStriped1.setNumBytes(1);
+    Block blk2 = new Block(2);
+    BlockInfoStriped blockInfoStriped2
+        = new BlockInfoStriped(blk2, testECPolicy);
+    blockInfoStriped2.setNumBytes(1);
+    inf.addBlock(blockInfoStriped1);
+    inf.addBlock(blockInfoStriped2);
+    // This is the double size of one block in above case.
+    assertEquals(4 * 2, inf.storagespaceConsumedStriped().getStorageSpace());
+    assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
+  }
+
+  @Test
+  public void testBlockStripedFileSize()
+      throws IOException, InterruptedException {
+    INodeFile inf = createStripedINodeFile();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk, testECPolicy);
+    blockInfoStriped.setNumBytes(100);
+    inf.addBlock(blockInfoStriped);
+    // Compute file size should return actual data
+    // size which is retained by this file.
+    assertEquals(100, inf.computeFileSize());
+    assertEquals(100, inf.computeFileSize(false, false));
+  }
+
+  @Test
+  public void testBlockStripedUCFileSize()
+      throws IOException, InterruptedException {
+    INodeFile inf = createStripedINodeFile();
+    Block blk = new Block(1);
+    BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy);
+    bInfoUCStriped.convertToBlockUnderConstruction(
+        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
+    bInfoUCStriped.setNumBytes(100);
+    inf.addBlock(bInfoUCStriped);
+    assertEquals(100, inf.computeFileSize());
+    assertEquals(0, inf.computeFileSize(false, false));
+  }
+
+  @Test
+  public void testBlockStripedComputeQuotaUsage()
+      throws IOException, InterruptedException {
+    INodeFile inf = createStripedINodeFile();
+    Block blk = new Block(1);
+    BlockInfoStriped blockInfoStriped
+        = new BlockInfoStriped(blk, testECPolicy);
+    blockInfoStriped.setNumBytes(100);
+    inf.addBlock(blockInfoStriped);
+
+    QuotaCounts counts =
+        inf.computeQuotaUsageWithStriped(defaultPolicy,
+            new QuotaCounts.Builder().build());
+    assertEquals(1, counts.getNameSpace());
+    // The total consumed space is the sum of
+    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
+    //  b. <Num Bytes> % <Num Bytes per Stripes> = 100
+    //  c. <Last Stripe Length> * <Parity Block Num> = 100 * 3
+    assertEquals(400, counts.getStorageSpace());
+  }
+
+  @Test
+  public void testBlockStripedUCComputeQuotaUsage()
+      throws IOException, InterruptedException {
+    INodeFile inf = createStripedINodeFile();
+    Block blk = new Block(1);
+    BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy);
+    bInfoUCStriped.convertToBlockUnderConstruction(
+        HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null);
+    bInfoUCStriped.setNumBytes(100);
+    inf.addBlock(bInfoUCStriped);
+
+    QuotaCounts counts
+        = inf.computeQuotaUsageWithStriped(defaultPolicy,
+              new QuotaCounts.Builder().build());
+    assertEquals(1024, inf.getPreferredBlockSize());
+    assertEquals(1, counts.getNameSpace());
+    // Consumed space in the case of BlockInfoStripedUC can be calculated
+    // by using preferred block size. This is 1024 and total block num
+    // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216.
+    assertEquals(9216, counts.getStorageSpace());
+  }
+
+  /**
+   * Test the behavior of striped and contiguous block deletions.
+   */
+  @Test(timeout = 60000)
+  public void testDeleteOp() throws Exception {
+    MiniDFSCluster cluster = null;
+    try {
+      final int len = 1024;
+      final Path parentDir = new Path("/parentDir");
+      final Path ecDir = new Path(parentDir, "ecDir");
+      final Path ecFile = new Path(ecDir, "ecFile");
+      final Path contiguousFile = new Path(parentDir, "someFile");
+      final DistributedFileSystem dfs;
+      final Configuration conf = new Configuration();
+      final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS
+          + StripedFileTestUtil.NUM_PARITY_BLOCKS);
+      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 2);
+
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE)
+          .build();
+      cluster.waitActive();
+
+      FSNamesystem fsn = cluster.getNamesystem();
+      dfs = cluster.getFileSystem();
+      dfs.mkdirs(ecDir);
+
+      // set erasure coding policy
+      dfs.setErasureCodingPolicy(ecDir, null);
+      DFSTestUtil.createFile(dfs, ecFile, len, (short) 1, 0xFEED);
+      DFSTestUtil.createFile(dfs, contiguousFile, len, (short) 1, 0xFEED);
+      final FSDirectory fsd = fsn.getFSDirectory();
+
+      // Case-1: Verify the behavior of striped blocks
+      // Get blocks of striped file
+      INode inodeStriped = fsd.getINode("/parentDir/ecDir/ecFile");
+      assertTrue("Failed to get INodeFile for /parentDir/ecDir/ecFile",
+          inodeStriped instanceof INodeFile);
+      INodeFile inodeStripedFile = (INodeFile) inodeStriped;
+      BlockInfo[] stripedBlks = inodeStripedFile.getBlocks();
+      for (BlockInfo blockInfo : stripedBlks) {
+        assertFalse("Mistakenly marked the block as deleted!",
+            blockInfo.isDeleted());
+      }
+
+      // delete directory with erasure coding policy
+      dfs.delete(ecDir, true);
+      for (BlockInfo blockInfo : stripedBlks) {
+        assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
+      }
+
+      // Case-2: Verify the behavior of contiguous blocks
+      // Get blocks of contiguous file
+      INode inode = fsd.getINode("/parentDir/someFile");
+      assertTrue("Failed to get INodeFile for /parentDir/someFile",
+          inode instanceof INodeFile);
+      INodeFile inodeFile = (INodeFile) inode;
+      BlockInfo[] contiguousBlks = inodeFile.getBlocks();
+      for (BlockInfo blockInfo : contiguousBlks) {
+        assertFalse("Mistakenly marked the block as deleted!",
+            blockInfo.isDeleted());
+      }
+
+      // delete parent directory
+      dfs.delete(parentDir, true);
+      for (BlockInfo blockInfo : contiguousBlks) {
+        assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index 4af9c75..f3221b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -73,6 +73,7 @@
 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.DatanodeManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -736,7 +737,13 @@
       DatanodeInfo[] newNodes = new DatanodeInfo[2];
       newNodes[0] = nodes[0];
       newNodes[1] = nodes[1];
-      String[] storageIDs = {"s0", "s1"};
+      final DatanodeManager dm = cluster.getNamesystem(0).getBlockManager()
+          .getDatanodeManager();
+      final String storageID1 = dm.getDatanode(newNodes[0]).getStorageInfos()[0]
+          .getStorageID();
+      final String storageID2 = dm.getDatanode(newNodes[1]).getStorageInfos()[0]
+          .getStorageID();
+      String[] storageIDs = {storageID1, storageID2};
       
       client.getNamenode().updatePipeline(client.getClientName(), oldBlock,
           newBlock, newNodes, storageIDs);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
index 29d22729..8b19b6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
@@ -175,7 +175,7 @@
    * localName (className@hashCode) parent permission group user
    * 
    * Specific information for different types of INode: 
-   * {@link INodeDirectory}:childrenSize 
+   * {@link INodeDirectory}:childrenSize
    * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()}
    * and {@link BlockUnderConstructionFeature#toString()} for detailed information.
    * </pre>
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
new file mode 100644
index 0000000..6d4d797
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
@@ -0,0 +1,162 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.tools.offlineImageViewer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestOfflineImageViewerWithStripedBlocks {
+  private static int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private static int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem fs;
+  private static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private static final int stripesPerBlock = 3;
+  private static final int blockSize = cellSize * stripesPerBlock;
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    int numDNs = dataBlocks + parityBlocks + 2;
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.waitActive();
+    cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null);
+    fs = cluster.getFileSystem();
+    Path eczone = new Path("/eczone");
+    fs.mkdirs(eczone);
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testFileEqualToOneStripe() throws Exception {
+    int numBytes = cellSize;
+    testFileSize(numBytes);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileLessThanOneStripe() throws Exception {
+    int numBytes = cellSize - 100;
+    testFileSize(numBytes);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileHavingMultipleBlocks() throws Exception {
+    int numBytes = blockSize * 3;
+    testFileSize(numBytes);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileLargerThanABlockGroup1() throws IOException {
+    testFileSize(blockSize * dataBlocks + cellSize + 123);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileLargerThanABlockGroup2() throws IOException {
+    testFileSize(blockSize * dataBlocks * 3 + cellSize * dataBlocks + cellSize
+        + 123);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileFullBlockGroup() throws IOException {
+    testFileSize(blockSize * dataBlocks);
+  }
+
+  @Test(timeout = 60000)
+  public void testFileMoreThanOneStripe() throws Exception {
+    int numBytes = blockSize + blockSize / 2;
+    testFileSize(numBytes);
+  }
+
+  private void testFileSize(int numBytes) throws IOException,
+      UnresolvedLinkException, SnapshotAccessControlException {
+    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    File orgFsimage = null;
+    Path file = new Path("/eczone/striped");
+    FSDataOutputStream out = fs.create(file, true);
+    byte[] bytes = DFSTestUtil.generateSequentialBytes(0, numBytes);
+    out.write(bytes);
+    out.close();
+
+    // Write results to the fsimage file
+    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
+    fs.saveNamespace();
+
+    // Determine location of fsimage file
+    orgFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil
+        .getFSImage(cluster.getNameNode()).getStorage().getStorageDir(0));
+    if (orgFsimage == null) {
+      throw new RuntimeException("Didn't generate or can't find fsimage");
+    }
+    FSImageLoader loader = FSImageLoader.load(orgFsimage.getAbsolutePath());
+    String fileStatus = loader.getFileStatus("/eczone/striped");
+    long expectedFileSize = bytes.length;
+
+    // Verify space consumed present in BlockInfoStriped
+    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+    INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+    assertTrue("Invalid block size", fileNode.getBlocks().length > 0);
+    long actualFileSize = 0;
+    for (BlockInfo blockInfo : fileNode.getBlocks()) {
+      assertTrue("Didn't find block striped information",
+          blockInfo instanceof BlockInfoStriped);
+      actualFileSize += blockInfo.getNumBytes();
+    }
+
+    assertEquals("Wrongly computed file size contains striped blocks",
+        expectedFileSize, actualFileSize);
+
+    // Verify space consumed present in filestatus
+    String EXPECTED_FILE_SIZE = "\"length\":"
+        + String.valueOf(expectedFileSize);
+    assertTrue(
+        "Wrongly computed file size contains striped blocks, file status:"
+            + fileStatus + ". Expected file size is : " + EXPECTED_FILE_SIZE,
+        fileStatus.contains(EXPECTED_FILE_SIZE));
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
new file mode 100644
index 0000000..95b0135
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
@@ -0,0 +1,279 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.util;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.*;
+
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Need to cover the following combinations:
+ * 1. Block group size:
+ *  1.1 One byte
+ *  1.2 Smaller than cell
+ *  1.3 One full cell
+ *  1.4 x full cells, where x is smaller than number of data blocks
+ *  1.5 x full cells plus a partial cell
+ *  1.6 One full stripe
+ *  1.7 One full stripe plus a partial cell
+ *  1.8 One full stripe plus x full cells
+ *  1.9 One full stripe plus x full cells plus a partial cell
+ *  1.10 y full stripes, but smaller than full block group size
+ *  1.11 Full block group size
+ *
+ * 2. Byte range start
+ *  2.1 Zero
+ *  2.2 Within first cell
+ *  2.3 End of first cell
+ *  2.4 Start of a middle* cell in the first stripe (* neither first or last)
+ *  2.5 End of middle cell in the first stripe
+ *  2.6 Within a middle cell in the first stripe
+ *  2.7 Start of the last cell in the first stripe
+ *  2.8 Within the last cell in the first stripe
+ *  2.9 End of the last cell in the first stripe
+ *  2.10 Start of a middle stripe
+ *  2.11 Within a middle stripe
+ *  2.12 End of a middle stripe
+ *  2.13 Start of the last stripe
+ *  2.14 Within the last stripe
+ *  2.15 End of the last stripe (last byte)
+ *
+ * 3. Byte range length: same settings as block group size
+ *
+ * We should test in total 11 x 15 x 11 = 1815 combinations
+ * TODO: test parity block logic
+ */
+public class TestStripedBlockUtil {
+  private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
+  private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final short BLK_GROUP_WIDTH = (short) (DATA_BLK_NUM + PARITY_BLK_NUM);
+  private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private final int FULL_STRIPE_SIZE = DATA_BLK_NUM * CELLSIZE;
+  /** number of full stripes in a full block group */
+  private final int BLK_GROUP_STRIPE_NUM = 16;
+  private final ErasureCodingPolicy ECPOLICY = ErasureCodingPolicyManager.
+      getSystemDefaultPolicy();
+  private final Random random = new Random();
+
+  private int[] blockGroupSizes;
+  private int[] byteRangeStartOffsets;
+  private int[] byteRangeSizes;
+
+  @Before
+  public void setup(){
+    blockGroupSizes = new int[]{1, getDelta(CELLSIZE), CELLSIZE,
+        getDelta(DATA_BLK_NUM) * CELLSIZE,
+        getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE),
+        FULL_STRIPE_SIZE, FULL_STRIPE_SIZE + getDelta(CELLSIZE),
+        FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE,
+        FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE),
+        getDelta(BLK_GROUP_STRIPE_NUM) * FULL_STRIPE_SIZE,
+        BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE};
+    byteRangeStartOffsets = new int[] {0, getDelta(CELLSIZE), CELLSIZE - 1};
+    byteRangeSizes = new int[]{1, getDelta(CELLSIZE), CELLSIZE,
+        getDelta(DATA_BLK_NUM) * CELLSIZE,
+        getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE),
+        FULL_STRIPE_SIZE, FULL_STRIPE_SIZE + getDelta(CELLSIZE),
+        FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE,
+        FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE),
+        getDelta(BLK_GROUP_STRIPE_NUM) * FULL_STRIPE_SIZE,
+        BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE};
+  }
+
+  private int getDelta(int size) {
+    return 1 + random.nextInt(size - 2);
+  }
+  private byte hashIntToByte(int i) {
+    int BYTE_MASK = 0xff;
+    return (byte) (((i + 13) * 29) & BYTE_MASK);
+  }
+
+  private LocatedStripedBlock createDummyLocatedBlock(int bgSize) {
+    final long blockGroupID = -1048576;
+    DatanodeInfo[] locs = new DatanodeInfo[BLK_GROUP_WIDTH];
+    String[] storageIDs = new String[BLK_GROUP_WIDTH];
+    StorageType[] storageTypes = new StorageType[BLK_GROUP_WIDTH];
+    int[] indices = new int[BLK_GROUP_WIDTH];
+    for (int i = 0; i < BLK_GROUP_WIDTH; i++) {
+      indices[i] = (i + 2) % DATA_BLK_NUM;
+      // Location port always equal to logical index of a block,
+      // for easier verification
+      locs[i] = DFSTestUtil.getLocalDatanodeInfo(indices[i]);
+      storageIDs[i] = locs[i].getDatanodeUuid();
+      storageTypes[i] = StorageType.DISK;
+    }
+    return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID,
+        bgSize, 1001), locs, storageIDs, storageTypes, indices, 0, false,
+        null);
+  }
+
+  private byte[][] createInternalBlkBuffers(int bgSize) {
+    byte[][] bufs = new byte[DATA_BLK_NUM + PARITY_BLK_NUM][];
+    int[] pos = new int[DATA_BLK_NUM + PARITY_BLK_NUM];
+    for (int i = 0; i < DATA_BLK_NUM + PARITY_BLK_NUM; i++) {
+      int bufSize = (int) getInternalBlockLength(
+          bgSize, CELLSIZE, DATA_BLK_NUM, i);
+      bufs[i] = new byte[bufSize];
+      pos[i] = 0;
+    }
+    int done = 0;
+    while (done < bgSize) {
+      Preconditions.checkState(done % CELLSIZE == 0);
+      StripingCell cell = new StripingCell(ECPOLICY, CELLSIZE, done / CELLSIZE, 0);
+      int idxInStripe = cell.idxInStripe;
+      int size = Math.min(CELLSIZE, bgSize - done);
+      for (int i = 0; i < size; i++) {
+        bufs[idxInStripe][pos[idxInStripe] + i] = hashIntToByte(done + i);
+      }
+      done += size;
+      pos[idxInStripe] += size;
+    }
+
+    return bufs;
+  }
+
+  @Test
+  public void testParseDummyStripedBlock() {
+    LocatedStripedBlock lsb = createDummyLocatedBlock(
+        BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE);
+    LocatedBlock[] blocks = parseStripedBlockGroup(
+        lsb, CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM);
+    assertEquals(DATA_BLK_NUM + PARITY_BLK_NUM, blocks.length);
+    for (int i = 0; i < DATA_BLK_NUM; i++) {
+      assertFalse(blocks[i].isStriped());
+      assertEquals(i,
+          BlockIdManager.getBlockIndex(blocks[i].getBlock().getLocalBlock()));
+      assertEquals(0, blocks[i].getStartOffset());
+      assertEquals(1, blocks[i].getLocations().length);
+      assertEquals(i, blocks[i].getLocations()[0].getIpcPort());
+      assertEquals(i, blocks[i].getLocations()[0].getXferPort());
+    }
+  }
+
+  private void verifyInternalBlocks (int numBytesInGroup, int[] expected) {
+    for (int i = 1; i < BLK_GROUP_WIDTH; i++) {
+      assertEquals(expected[i],
+          getInternalBlockLength(numBytesInGroup, CELLSIZE, DATA_BLK_NUM, i));
+    }
+  }
+
+  @Test
+  public void testGetInternalBlockLength () {
+    // A small delta that is smaller than a cell
+    final int delta = 10;
+
+    // Block group is smaller than a cell
+    verifyInternalBlocks(CELLSIZE - delta,
+        new int[] {CELLSIZE - delta, 0, 0, 0, 0, 0,
+            CELLSIZE - delta, CELLSIZE - delta, CELLSIZE - delta});
+
+    // Block group is exactly as large as a cell
+    verifyInternalBlocks(CELLSIZE,
+        new int[] {CELLSIZE, 0, 0, 0, 0, 0,
+            CELLSIZE, CELLSIZE, CELLSIZE});
+
+    // Block group is a little larger than a cell
+    verifyInternalBlocks(CELLSIZE + delta,
+        new int[] {CELLSIZE, delta, 0, 0, 0, 0,
+            CELLSIZE, CELLSIZE, CELLSIZE});
+
+    // Block group contains multiple stripes and ends at stripe boundary
+    verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE,
+        new int[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE});
+
+    // Block group contains multiple stripes and ends at cell boundary
+    // (not ending at stripe boundary)
+    verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE + CELLSIZE,
+        new int[] {3 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            3 * CELLSIZE, 3 * CELLSIZE, 3 * CELLSIZE});
+
+    // Block group contains multiple stripes and doesn't end at cell boundary
+    verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE - delta,
+        new int[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE - delta,
+            2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE});
+  }
+
+  /**
+   * Test dividing a byte range into aligned stripes and verify the aligned
+   * ranges can be translated back to the byte range.
+   */
+  @Test
+  public void testDivideByteRangeIntoStripes() {
+    byte[] assembled = new byte[BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE];
+    for (int bgSize : blockGroupSizes) {
+      LocatedStripedBlock blockGroup = createDummyLocatedBlock(bgSize);
+      byte[][] internalBlkBufs = createInternalBlkBuffers(bgSize);
+      for (int brStart : byteRangeStartOffsets) {
+        for (int brSize : byteRangeSizes) {
+          if (brStart + brSize > bgSize) {
+            continue;
+          }
+          AlignedStripe[] stripes = divideByteRangeIntoStripes(ECPOLICY,
+              CELLSIZE, blockGroup, brStart, brStart + brSize - 1, assembled, 0);
+
+          for (AlignedStripe stripe : stripes) {
+            for (int i = 0; i < DATA_BLK_NUM; i++) {
+              StripingChunk chunk = stripe.chunks[i];
+              if (chunk == null || chunk.state != StripingChunk.REQUESTED) {
+                continue;
+              }
+              int done = 0;
+              for (int j = 0; j < chunk.byteArray.getLengths().length; j++) {
+                System.arraycopy(internalBlkBufs[i],
+                    (int) stripe.getOffsetInBlock() + done, assembled,
+                    chunk.byteArray.getOffsets()[j],
+                    chunk.byteArray.getLengths()[j]);
+                done += chunk.byteArray.getLengths()[j];
+              }
+            }
+          }
+          for (int i = 0; i < brSize; i++) {
+            if (hashIntToByte(brStart + i) != assembled[i]) {
+              System.out.println("Oops");
+            }
+            assertEquals("Byte at " + (brStart + i) + " should be the same",
+                hashIntToByte(brStart + i), assembled[i]);
+          }
+        }
+      }
+    }
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
index 391f190..8947c5b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
@@ -65,7 +65,7 @@
     final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26,
         now, now + 10, new FsPermission((short) 0644), "user", "group",
         DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"),
-        HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0);
+        HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null);
     final FileStatus fstatus = toFileStatus(status, parent);
     System.out.println("status  = " + status);
     System.out.println("fstatus = " + fstatus);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
new file mode 100644
index 0000000..e72d862
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -0,0 +1,377 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="testConf.xsl"?>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<configuration>
+  <!-- Normal mode is test. To run just the commands and dump the output
+       to the log, set it to nocompare -->
+  <mode>test</mode>
+
+  <!--  Comparator types:
+           ExactComparator
+           SubstringComparator
+           RegexpComparator
+           TokenComparator
+           -->
+  <tests>
+
+  <!-- Test help options -->
+    <test>
+      <description>help: help for erasure coding command</description>
+      <test-commands>
+        <ec-admin-command>-help</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Usage: hdfs erasurecode [generic options]</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>help: setPolicy command</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -help setPolicy</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^[ \t]*Set a specified erasure coding policy to a directory( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-setPolicy \[-p &lt;policyName&gt;\] &lt;path&gt;(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>help: getPolicy command</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -help getPolicy</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Get erasure coding policy information about at specified path</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-getPolicy &lt;path&gt;(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>help: listPolicies command</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -help listPolicies</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Get the list of erasure coding policies supported</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-listPolicies (.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+  <!-- Test erasure code commands -->
+    <test>
+      <description>setPolicy : set erasure coding policy on a directory to encode files</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>EC policy set successfully at NAMENODE/ecdir</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>setPolicy : set a policy twice</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Directory /ecdir already has an erasure coding policy</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>setPolicy : default policy</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>ErasureCodingPolicy=[Name=RS-6-3-64k</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /noec</command>
+        <ec-admin-command>-fs NAMENODE -getPolicy /noec</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /noec</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Path NAMENODE/noec is not erasure coded</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir</ec-admin-command>
+        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>ErasureCodingPolicy=[Name=RS-6-3-64k</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getPolicy : get EC policy information at specified path, which doesn't have an EC policy</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir</ec-admin-command>
+        <command>-fs NAMENODE -touchz /ecdir/ecfile</command>
+        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir/ecfile</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /ecdir/ecfile</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>ErasureCodingPolicy=[Name=RS-6-3-64k</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>listPolicies : get the list of ECPolicies supported</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -listPolicies</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>RS-6-3</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+<!-- Test illegal parameters -->
+    <test>
+      <description>setPolicy : illegal parameters - path is missing</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-setPolicy: &lt;path&gt; is missing(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>setPolicy : illegal parameters - policy name is missing</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -p</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-setPolicy: option -p requires 1 argument(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>setPolicy : illegal parameters - too many arguments</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir1 /ecdir2</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-setPolicy: Too many arguments</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>setPolicy : illegal parameters - invalidpolicy</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -p invalidpolicy /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Policy 'invalidpolicy' does not match any of the supported policies. Please select any one of [RS-6-3-64k]</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>setPolicy : illegal parameters - no such file</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -setPolicy /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^setPolicy: `/ecdir': No such file or directory(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getPolicy : illegal parameters - path is missing</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -getPolicy </ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-getPolicy: &lt;path&gt; is missing(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getPolicy : illegal parameters - too many arguments</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-getPolicy: Too many arguments</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>getPolicy : illegal parameters - no such file</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -getPolicy /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^getPolicy: `/ecdir': No such file or directory(.)*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>listPolicies : illegal parameters - too many parameters</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -listPolicies /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-listPolicies: Too many parameters</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+  </tests>
+</configuration>
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 272ea7b..5c92d73 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -98,6 +98,7 @@
       <item name="Storage Policies" href="hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html"/>
       <item name="Memory Storage Support" href="hadoop-project-dist/hadoop-hdfs/MemoryStorage.html"/>
       <item name="Synthetic Load Generator" href="hadoop-project-dist/hadoop-hdfs/SLGUserGuide.html"/>
+      <item name="Erasure Coding" href="hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html"/>
     </menu>
 
     <menu name="MapReduce" inherit="top">