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 dr...@apache.org on 2015/04/07 11:34:33 UTC

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

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 a84274f90 -> 1ceda4394


HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng


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

Branch: refs/heads/HDFS-7285
Commit: a948cb76e4c9c9dabcf268040ba191b1b0be1fd7
Parents: a84274f
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Apr 8 01:26:40 2015 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Apr 8 01:26:40 2015 +0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a948cb76/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
index ba32f04..e2c5051 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -4,9 +4,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
-import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 
 /**
  * Reed-Solomon erasure decoder that decodes a block group.
@@ -56,7 +56,7 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
       rsRawDecoder = createRawDecoder(
           CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
       if (rsRawDecoder == null) {
-        rsRawDecoder = new JRSRawDecoder();
+        rsRawDecoder = new RSRawDecoder();
       }
       rsRawDecoder.initialize(getNumDataUnits(),
           getNumParityUnits(), getChunkSize());
@@ -66,7 +66,7 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
 
   private RawErasureDecoder checkCreateXorRawDecoder() {
     if (xorRawDecoder == null) {
-      xorRawDecoder = new XorRawDecoder();
+      xorRawDecoder = new XORRawDecoder();
       xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize());
     }
     return xorRawDecoder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a948cb76/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
index 430749d..a7d02b5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
@@ -3,7 +3,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 
 /**
@@ -30,7 +30,7 @@ public class RSErasureEncoder extends AbstractErasureEncoder {
       rawEncoder = createRawEncoder(
           CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
       if (rawEncoder == null) {
-        rawEncoder = new JRSRawEncoder();
+        rawEncoder = new RSRawEncoder();
       }
       rawEncoder.initialize(getNumDataUnits(),
           getNumParityUnits(), getChunkSize());

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a948cb76/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
deleted file mode 100644
index 39e5deb..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.io.erasurecode.rawcoder;
-
-import org.apache.hadoop.io.erasurecode.ECChunk;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-
-/**
- * Test raw Reed-solomon encoding and decoding.
- */
-public class TestJRSRawCoder extends TestRawCoderBase {
-
-  private static int symbolSize = 0;
-  private static int symbolMax = 0;
-
-  static {
-    symbolSize = (int) Math.round(Math.log(
-        RSUtil.GF.getFieldSize()) / Math.log(2));
-    symbolMax = (int) Math.pow(2, symbolSize);
-  }
-
-  @Before
-  public void setup() {
-    this.encoderClass = JRSRawEncoder.class;
-    this.decoderClass = JRSRawDecoder.class;
-  }
-
-  @Test
-  public void testCodingNoDirectBuffer_10x4() {
-    prepare(null, 10, 4, null);
-    testCoding(false);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_10x4() {
-    prepare(null, 10, 4, null);
-    testCoding(true);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
-    prepare(null, 10, 4, new int[] {2, 4});
-    testCoding(true);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_10x4_erasing_all() {
-    prepare(null, 10, 4, new int[] {0, 1, 2, 3});
-    testCoding(true);
-  }
-
-  @Test
-  public void testCodingNoDirectBuffer_3x3() {
-    prepare(null, 3, 3, null);
-    testCoding(false);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_3x3() {
-    prepare(null, 3, 3, null);
-    testCoding(true);
-  }
-
-  @Override
-  protected ECChunk generateDataChunk() {
-    ByteBuffer buffer = allocateOutputBuffer();
-    for (int i = 0; i < chunkSize; i++) {
-      buffer.put((byte) RAND.nextInt(symbolMax));
-    }
-    buffer.flip();
-
-    return new ECChunk(buffer);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a948cb76/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
new file mode 100644
index 0000000..8bb5d0f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Test raw Reed-solomon encoding and decoding.
+ */
+public class TestRSRawCoder extends TestRawCoderBase {
+
+  private static int symbolSize = 0;
+  private static int symbolMax = 0;
+
+  static {
+    symbolSize = (int) Math.round(Math.log(
+        RSUtil.GF.getFieldSize()) / Math.log(2));
+    symbolMax = (int) Math.pow(2, symbolSize);
+  }
+
+  @Before
+  public void setup() {
+    this.encoderClass = RSRawEncoder.class;
+    this.decoderClass = RSRawDecoder.class;
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_10x4() {
+    prepare(null, 10, 4, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4() {
+    prepare(null, 10, 4, null);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
+    prepare(null, 10, 4, new int[] {2, 4});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_all() {
+    prepare(null, 10, 4, new int[] {0, 1, 2, 3});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_3x3() {
+    prepare(null, 3, 3, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_3x3() {
+    prepare(null, 3, 3, null);
+    testCoding(true);
+  }
+
+  @Override
+  protected ECChunk generateDataChunk() {
+    ByteBuffer buffer = allocateOutputBuffer();
+    for (int i = 0; i < chunkSize; i++) {
+      buffer.put((byte) RAND.nextInt(symbolMax));
+    }
+    buffer.flip();
+
+    return new ECChunk(buffer);
+  }
+}

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

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


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

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


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

Branch: refs/heads/HDFS-7285
Commit: 1ceda439494e62865f2e87271cfcc1d0052e3240
Parents: a948cb7
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Apr 8 01:31:46 2015 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Apr 8 01:31:46 2015 +0800

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


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