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

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

HADOOP-11646. Erasure Coder API for encoding and decoding of block group ( Contributed by Kai Zheng )


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

Branch: refs/heads/HDFS-7285
Commit: a5804bf3fb6684a940339ef11f1a705fa4ae87d0
Parents: 803d4da
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Mar 9 12:32:26 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 23 11:12:35 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5804bf3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java
new file mode 100644
index 0000000..956954a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode;
+
+/**
+ * A wrapper of block level data source/output that {@link ECChunk}s can be
+ * extracted from. For HDFS, it can be an HDFS block (250MB). Note it only cares
+ * about erasure coding specific logic thus avoids coupling with any HDFS block
+ * details. We can have something like HdfsBlock extend it.
+ */
+public class ECBlock {
+
+  private boolean isParity;
+  private boolean isErased;
+
+  /**
+   * A default constructor. isParity and isErased are false by default.
+   */
+  public ECBlock() {
+    this(false, false);
+  }
+
+  /**
+   * A constructor specifying isParity and isErased.
+   * @param isParity
+   * @param isErased
+   */
+  public ECBlock(boolean isParity, boolean isErased) {
+    this.isParity = isParity;
+    this.isErased = isErased;
+  }
+
+  /**
+   * Set true if it's for a parity block.
+   * @param isParity
+   */
+  public void setParity(boolean isParity) {
+    this.isParity = isParity;
+  }
+
+  /**
+   * Set true if the block is missing.
+   * @param isMissing
+   */
+  public void setErased(boolean isMissing) {
+    this.isErased = isMissing;
+  }
+
+  /**
+   *
+   * @return true if it's parity block, otherwise false
+   */
+  public boolean isParity() {
+    return isParity;
+  }
+
+  /**
+   *
+   * @return true if it's missing or corrupt due to erasure, otherwise false
+   */
+  public boolean isErased() {
+    return isErased;
+  }
+
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5804bf3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
new file mode 100644
index 0000000..f2cc041
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+/**
+ * A common class of basic facilities to be shared by encoder and decoder
+ *
+ * It implements the {@link ErasureCoder} interface.
+ */
+public abstract class AbstractErasureCoder implements ErasureCoder {
+
+  private int numDataUnits;
+  private int numParityUnits;
+  private int chunkSize;
+
+  @Override
+  public void initialize(int numDataUnits, int numParityUnits,
+                         int chunkSize) {
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+    this.chunkSize = chunkSize;
+  }
+
+  @Override
+  public int getNumDataUnits() {
+    return numDataUnits;
+  }
+
+  @Override
+  public int getNumParityUnits() {
+    return numParityUnits;
+  }
+
+  @Override
+  public int getChunkSize() {
+    return chunkSize;
+  }
+
+  @Override
+  public boolean preferNativeBuffer() {
+    return false;
+  }
+
+  @Override
+  public void release() {
+    // Nothing to do by default
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5804bf3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
new file mode 100644
index 0000000..c429d49
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+
+/**
+ * Abstract class for common facilities shared by {@link ErasureEncodingStep}
+ * and {@link ErasureDecodingStep}.
+ *
+ * It implements {@link ErasureEncodingStep}.
+ */
+public abstract class AbstractErasureCodingStep implements ErasureCodingStep {
+
+  private ECBlock[] inputBlocks;
+  private ECBlock[] outputBlocks;
+
+  /**
+   * Constructor given input blocks and output blocks.
+   * @param inputBlocks
+   * @param outputBlocks
+   */
+  public AbstractErasureCodingStep(ECBlock[] inputBlocks,
+                                   ECBlock[] outputBlocks) {
+    this.inputBlocks = inputBlocks;
+    this.outputBlocks = outputBlocks;
+  }
+
+  @Override
+  public ECBlock[] getInputBlocks() {
+    return inputBlocks;
+  }
+
+  @Override
+  public ECBlock[] getOutputBlocks() {
+    return outputBlocks;
+  }
+
+  @Override
+  public void finish() {
+    // NOOP by default
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5804bf3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
new file mode 100644
index 0000000..df71528
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+
+/**
+ * An abstract erasure decoder that's to be inherited by new decoders.
+ *
+ * It implements the {@link ErasureDecoder} interface.
+ */
+public abstract class AbstractErasureDecoder extends AbstractErasureCoder
+    implements ErasureDecoder {
+
+  @Override
+  public ErasureCodingStep decode(ECBlockGroup blockGroup) {
+    return performDecoding(blockGroup);
+  }
+
+  /**
+   * Perform decoding against a block blockGroup.
+   * @param blockGroup
+   * @return decoding step for caller to do the real work
+   */
+  protected abstract ErasureCodingStep performDecoding(ECBlockGroup blockGroup);
+
+  /**
+   * We have all the data blocks and parity blocks as input blocks for
+   * recovering by default. It's codec specific
+   * @param blockGroup
+   * @return
+   */
+  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
+    ECBlock[] inputBlocks = new ECBlock[getNumParityUnits()
+        + getNumDataUnits()];
+
+    System.arraycopy(blockGroup.getParityBlocks(), 0, inputBlocks, 0,
+        getNumParityUnits());
+    System.arraycopy(blockGroup.getDataBlocks(), 0, inputBlocks,
+        getNumParityUnits(), getNumDataUnits());
+
+    return inputBlocks;
+  }
+
+  /**
+   * Which blocks were erased ? We only care data blocks here. Sub-classes can
+   * override this behavior.
+   * @param blockGroup
+   * @return output blocks to recover
+   */
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    ECBlock[] outputBlocks = new ECBlock[
+        getNumErasedBlocks(blockGroup.getDataBlocks())];
+
+    int idx = 0;
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      if (blockGroup.getDataBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
+      }
+    }
+
+    return outputBlocks;
+  }
+
+  /**
+   * Get the number of erased blocks in the block group.
+   * @param blockGroup
+   * @return number of erased blocks
+   */
+  protected int getNumErasedBlocks(ECBlockGroup blockGroup) {
+    int num = getNumErasedBlocks(blockGroup.getParityBlocks());
+    num += getNumErasedBlocks(blockGroup.getDataBlocks());
+    return num;
+  }
+
+  /**
+   * Find out how many blocks are erased.
+   * @param inputBlocks all the input blocks
+   * @return number of erased blocks
+   */
+  protected static int getNumErasedBlocks(ECBlock[] inputBlocks) {
+    int numErased = 0;
+    for (int i = 0; i < inputBlocks.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        numErased ++;
+      }
+    }
+
+    return numErased;
+  }
+
+  /**
+   * Get indexes of erased blocks from inputBlocks
+   * @param inputBlocks
+   * @return indexes of erased blocks from inputBlocks
+   */
+  protected int[] getErasedIndexes(ECBlock[] inputBlocks) {
+    int numErased = getNumErasedBlocks(inputBlocks);
+    if (numErased == 0) {
+      return new int[0];
+    }
+
+    int[] erasedIndexes = new int[numErased];
+    int i = 0, j = 0;
+    for (; i < inputBlocks.length && j < erasedIndexes.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        erasedIndexes[j++] = i;
+      }
+    }
+
+    return erasedIndexes;
+  }
+
+  /**
+   * Get erased input blocks from inputBlocks
+   * @param inputBlocks
+   * @return an array of erased blocks from inputBlocks
+   */
+  protected ECBlock[] getErasedBlocks(ECBlock[] inputBlocks) {
+    int numErased = getNumErasedBlocks(inputBlocks);
+    if (numErased == 0) {
+      return new ECBlock[0];
+    }
+
+    ECBlock[] erasedBlocks = new ECBlock[numErased];
+    int i = 0, j = 0;
+    for (; i < inputBlocks.length && j < erasedBlocks.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        erasedBlocks[j++] = inputBlocks[i];
+      }
+    }
+
+    return erasedBlocks;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5804bf3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
new file mode 100644
index 0000000..80b9875
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+
+/**
+ * An abstract erasure encoder that's to be inherited by new encoders.
+ *
+ * It implements the {@link ErasureEncoder} interface.
+ */
+public abstract class AbstractErasureEncoder extends AbstractErasureCoder
+    implements ErasureEncoder {
+
+  @Override
+  public ErasureCodingStep encode(ECBlockGroup blockGroup) {
+    return performEncoding(blockGroup);
+  }
+
+  /**
+   * Perform encoding against a block group.
+   * @param blockGroup
+   * @return encoding step for caller to do the real work
+   */
+  protected abstract ErasureCodingStep performEncoding(ECBlockGroup blockGroup);
+
+  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
+    return blockGroup.getDataBlocks();
+  }
+
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    return blockGroup.getParityBlocks();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5804bf3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
new file mode 100644
index 0000000..68875c0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+/**
+ * An erasure coder to perform encoding or decoding given a group. Generally it
+ * involves calculating necessary internal steps according to codec logic. For
+ * each step,it calculates necessary input blocks to read chunks from and output
+ * parity blocks to write parity chunks into from the group. It also takes care
+ * of appropriate raw coder to use for the step. And encapsulates all the
+ * necessary info (input blocks, output blocks and raw coder) into a step
+ * represented by {@link ErasureCodingStep}. ErasureCoder callers can use the
+ * step to do the real work with retrieved input and output chunks.
+ *
+ * Note, currently only one coding step is supported. Will support complex cases
+ * of multiple coding steps.
+ *
+ */
+public interface ErasureCoder {
+
+  /**
+   * Initialize with the important parameters for the code.
+   * @param numDataUnits how many data inputs for the coding
+   * @param numParityUnits how many parity outputs the coding generates
+   * @param chunkSize the size of the input/output buffer
+   */
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize);
+
+  /**
+   * The number of data input units for the coding. A unit can be a byte,
+   * chunk or buffer or even a block.
+   * @return count of data input units
+   */
+  public int getNumDataUnits();
+
+  /**
+   * The number of parity output units for the coding. A unit can be a byte,
+   * chunk, buffer or even a block.
+   * @return count of parity output units
+   */
+  public int getNumParityUnits();
+
+  /**
+   * Chunk buffer size for the input/output
+   * @return chunk buffer size
+   */
+  public int getChunkSize();
+
+  /**
+   * Tell if native or off-heap buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, either on heap or off heap.
+   * It will return false by default.
+   * @return true if native buffer is preferred for performance consideration,
+   * otherwise false.
+   */
+  public boolean preferNativeBuffer();
+
+  /**
+   * Release the resources if any. Good chance to invoke RawErasureCoder#release.
+   */
+  public void release();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5804bf3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
new file mode 100644
index 0000000..a3b177f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECChunk;
+
+/**
+ * Erasure coding step that's involved in encoding/decoding of a block group.
+ */
+public interface ErasureCodingStep {
+
+  /**
+   * Input blocks of readable data involved in this step, may be data blocks
+   * or parity blocks.
+   * @return input blocks
+   */
+  public ECBlock[] getInputBlocks();
+
+  /**
+   * Output blocks of writable buffers involved in this step, may be data
+   * blocks or parity blocks.
+   * @return output blocks
+   */
+  public ECBlock[] getOutputBlocks();
+
+  /**
+   * Perform encoding or decoding given the input chunks, and generated results
+   * will be written to the output chunks.
+   * @param inputChunks
+   * @param outputChunks
+   */
+  public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks);
+
+  /**
+   * Notify erasure coder that all the chunks of input blocks are processed so
+   * the coder can be able to update internal states, considering next step.
+   */
+  public void finish();
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5804bf3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
new file mode 100644
index 0000000..980c580
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+
+/**
+ * Erasure decoding step, a wrapper of all the necessary information to perform
+ * a decoding step involved in the whole process of decoding a block group.
+ */
+public class ErasureDecodingStep extends AbstractErasureCodingStep {
+  private int[] erasedIndexes;
+  private RawErasureDecoder rawDecoder;
+
+  /**
+   * The constructor with all the necessary info.
+   * @param inputBlocks
+   * @param erasedIndexes the indexes of erased blocks in inputBlocks array
+   * @param outputBlocks
+   * @param rawDecoder
+   */
+  public ErasureDecodingStep(ECBlock[] inputBlocks, int[] erasedIndexes,
+                             ECBlock[] outputBlocks,
+                             RawErasureDecoder rawDecoder) {
+    super(inputBlocks, outputBlocks);
+    this.erasedIndexes = erasedIndexes;
+    this.rawDecoder = rawDecoder;
+  }
+
+  @Override
+  public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) {
+    rawDecoder.decode(inputChunks, erasedIndexes, outputChunks);
+  }
+
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5804bf3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
new file mode 100644
index 0000000..bd7587f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+
+/**
+ * Erasure encoding step, a wrapper of all the necessary information to perform
+ * an encoding step involved in the whole process of encoding a block group.
+ */
+public class ErasureEncodingStep extends AbstractErasureCodingStep {
+
+  private RawErasureEncoder rawEncoder;
+
+  /**
+   * The constructor with all the necessary info.
+   * @param inputBlocks
+   * @param outputBlocks
+   * @param rawEncoder
+   */
+  public ErasureEncodingStep(ECBlock[] inputBlocks, ECBlock[] outputBlocks,
+                             RawErasureEncoder rawEncoder) {
+    super(inputBlocks, outputBlocks);
+    this.rawEncoder = rawEncoder;
+  }
+
+  @Override
+  public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) {
+    rawEncoder.encode(inputChunks, outputChunks);
+  }
+
+}

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5804bf3/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
new file mode 100644
index 0000000..ca5c1c9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
@@ -0,0 +1,266 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.TestCoderBase;
+
+/**
+ * Erasure coder test base with utilities.
+ */
+public abstract class TestErasureCoderBase extends TestCoderBase {
+  protected Class<? extends ErasureEncoder> encoderClass;
+  protected Class<? extends ErasureDecoder> decoderClass;
+
+  protected int numChunksInBlock = 16;
+
+  /**
+   * It's just a block for this test purpose. We don't use HDFS block here
+   * at all for simple.
+   */
+  protected static class TestBlock extends ECBlock {
+    private ECChunk[] chunks;
+
+    // For simple, just assume the block have the chunks already ready.
+    // In practice we need to read/write chunks from/to the block via file IO.
+    public TestBlock(ECChunk[] chunks) {
+      this.chunks = chunks;
+    }
+  }
+
+  /**
+   * Generating source data, encoding, recovering and then verifying.
+   * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
+   * it supports two kinds of ByteBuffers, one is array backed, the other is
+   * direct ByteBuffer. Have usingDirectBuffer to indicate which case to test.
+   * @param usingDirectBuffer
+   */
+  protected void testCoding(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+
+    ErasureEncoder encoder = createEncoder();
+    // Generate data and encode
+    ECBlockGroup blockGroup = prepareBlockGroupForEncoding();
+    // Backup all the source chunks for later recovering because some coders
+    // may affect the source data.
+    TestBlock[] clonedDataBlocks = cloneBlocksWithData((TestBlock[])
+        blockGroup.getDataBlocks());
+    // Make a copy of a strip for later comparing
+    TestBlock[] toEraseBlocks = copyDataBlocksToErase(clonedDataBlocks);
+
+    ErasureCodingStep codingStep = encoder.encode(blockGroup);
+    performCodingStep(codingStep);
+    // Erase the copied sources
+    eraseSomeDataBlocks(clonedDataBlocks);
+
+    //Decode
+    blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
+    ErasureDecoder decoder = createDecoder();
+    codingStep = decoder.decode(blockGroup);
+    performCodingStep(codingStep);
+
+    //Compare
+    compareAndVerify(toEraseBlocks, codingStep.getOutputBlocks());
+  }
+
+  /**
+   * This is typically how a coding step should be performed.
+   * @param codingStep
+   */
+  private void performCodingStep(ErasureCodingStep codingStep) {
+    // Pretend that we're opening these input blocks and output blocks.
+    ECBlock[] inputBlocks = codingStep.getInputBlocks();
+    ECBlock[] outputBlocks = codingStep.getOutputBlocks();
+    // We allocate input and output chunks accordingly.
+    ECChunk[] inputChunks = new ECChunk[inputBlocks.length];
+    ECChunk[] outputChunks = new ECChunk[outputBlocks.length];
+
+    for (int i = 0; i < numChunksInBlock; ++i) {
+      // Pretend that we're reading input chunks from input blocks.
+      for (int j = 0; j < inputBlocks.length; ++j) {
+        inputChunks[j] = ((TestBlock) inputBlocks[j]).chunks[i];
+      }
+
+      // Pretend that we allocate and will write output results to the blocks.
+      for (int j = 0; j < outputBlocks.length; ++j) {
+        outputChunks[j] = allocateOutputChunk();
+        ((TestBlock) outputBlocks[j]).chunks[i] = outputChunks[j];
+      }
+
+      // Given the input chunks and output chunk buffers, just call it !
+      codingStep.performCoding(inputChunks, outputChunks);
+    }
+
+    codingStep.finish();
+  }
+
+  /**
+   * Compare and verify if recovered blocks data are the same with the erased
+   * blocks data.
+   * @param erasedBlocks
+   * @param recoveredBlocks
+   */
+  protected void compareAndVerify(ECBlock[] erasedBlocks,
+                                  ECBlock[] recoveredBlocks) {
+    for (int i = 0; i < erasedBlocks.length; ++i) {
+      compareAndVerify(((TestBlock) erasedBlocks[i]).chunks,
+          ((TestBlock) recoveredBlocks[i]).chunks);
+    }
+  }
+
+  /**
+   * Create erasure encoder for test.
+   * @return
+   */
+  private ErasureEncoder createEncoder() {
+    ErasureEncoder encoder;
+    try {
+      encoder = encoderClass.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create encoder", e);
+    }
+
+    encoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    return encoder;
+  }
+
+  /**
+   * Create the erasure decoder for the test.
+   * @return
+   */
+  private ErasureDecoder createDecoder() {
+    ErasureDecoder decoder;
+    try {
+      decoder = decoderClass.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create decoder", e);
+    }
+
+    decoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    return decoder;
+  }
+
+  /**
+   * Prepare a block group for encoding.
+   * @return
+   */
+  protected ECBlockGroup prepareBlockGroupForEncoding() {
+    ECBlock[] dataBlocks = new TestBlock[numDataUnits];
+    ECBlock[] parityBlocks = new TestBlock[numParityUnits];
+
+    for (int i = 0; i < numDataUnits; i++) {
+      dataBlocks[i] = generateDataBlock();
+    }
+
+    for (int i = 0; i < numParityUnits; i++) {
+      parityBlocks[i] = allocateOutputBlock();
+    }
+
+    return new ECBlockGroup(dataBlocks, parityBlocks);
+  }
+
+  /**
+   * Generate random data and return a data block.
+   * @return
+   */
+  protected ECBlock generateDataBlock() {
+    ECChunk[] chunks = new ECChunk[numChunksInBlock];
+
+    for (int i = 0; i < numChunksInBlock; ++i) {
+      chunks[i] = generateDataChunk();
+    }
+
+    return new TestBlock(chunks);
+  }
+
+  /**
+   * Copy those data blocks that's to be erased for later comparing and
+   * verifying.
+   * @param dataBlocks
+   * @return
+   */
+  protected TestBlock[] copyDataBlocksToErase(TestBlock[] dataBlocks) {
+    TestBlock[] copiedBlocks = new TestBlock[erasedDataIndexes.length];
+
+    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+      copiedBlocks[i] = cloneBlockWithData(dataBlocks[erasedDataIndexes[i]]);
+    }
+
+    return copiedBlocks;
+  }
+
+  /**
+   * Allocate an output block. Note the chunk buffer will be allocated by the
+   * up caller when performing the coding step.
+   * @return
+   */
+  protected TestBlock allocateOutputBlock() {
+    ECChunk[] chunks = new ECChunk[numChunksInBlock];
+
+    return new TestBlock(chunks);
+  }
+
+  /**
+   * Clone blocks with data copied along with, avoiding affecting the original
+   * blocks.
+   * @param blocks
+   * @return
+   */
+  protected static TestBlock[] cloneBlocksWithData(TestBlock[] blocks) {
+    TestBlock[] results = new TestBlock[blocks.length];
+    for (int i = 0; i < blocks.length; ++i) {
+      results[i] = cloneBlockWithData(blocks[i]);
+    }
+
+    return results;
+  }
+
+  /**
+   * Clone exactly a block, avoiding affecting the original block.
+   * @param block
+   * @return a new block
+   */
+  protected static TestBlock cloneBlockWithData(TestBlock block) {
+    ECChunk[] newChunks = cloneChunksWithData(block.chunks);
+
+    return new TestBlock(newChunks);
+  }
+
+  /**
+   * Erase some data blocks specified by the indexes from the data blocks.
+   * @param dataBlocks
+   */
+  protected void eraseSomeDataBlocks(TestBlock[] dataBlocks) {
+    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+      eraseDataFromBlock(dataBlocks, erasedDataIndexes[i]);
+    }
+  }
+
+  /**
+   * Erase data from a block specified by erased index.
+   * @param blocks
+   * @param erasedIndex
+   */
+  protected void eraseDataFromBlock(TestBlock[] blocks, int erasedIndex) {
+    TestBlock theBlock = blocks[erasedIndex];
+    eraseDataFromChunks(theBlock.chunks);
+    theBlock.setErased(true);
+  }
+}

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