Selaa lähdekoodia

HADOOP-12826. Rename the new Java coder and make it default. Contributed by Rui Li.

Zhe Zhang 9 vuotta sitten
vanhempi
commit
19e8f07691

+ 10 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java

@@ -20,7 +20,14 @@ package org.apache.hadoop.io.erasurecode;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.erasurecode.rawcoder.*;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawEncoder;
 
 /**
  * A codec & coder utility to help create raw coders conveniently.
@@ -43,7 +50,7 @@ public final class CodecUtil {
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
         true, numDataUnits, numParityUnits);
     if (rawCoder == null) {
-      rawCoder = new RSRawEncoderLegacy(numDataUnits, numParityUnits);
+      rawCoder = new RSRawEncoder(numDataUnits, numParityUnits);
     }
 
     return (RawErasureEncoder) rawCoder;
@@ -62,7 +69,7 @@ public final class CodecUtil {
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
         false, numDataUnits, numParityUnits);
     if (rawCoder == null) {
-      rawCoder = new RSRawDecoderLegacy(numDataUnits, numParityUnits);
+      rawCoder = new RSRawDecoder(numDataUnits, numParityUnits);
     }
 
     return (RawErasureDecoder) rawCoder;

+ 6 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder2.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java

@@ -23,7 +23,6 @@ import org.apache.hadoop.io.erasurecode.rawcoder.util.CoderUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.GF256;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil2;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
@@ -35,7 +34,7 @@ import java.util.Arrays;
  * from HDFS-RAID, and also compatible with the native/ISA-L coder.
  */
 @InterfaceAudience.Private
-public class RSRawDecoder2 extends AbstractRawErasureDecoder {
+public class RSRawDecoder extends AbstractRawErasureDecoder {
   //relevant to schema and won't change during decode calls
   private byte[] encodeMatrix;
 
@@ -55,7 +54,7 @@ public class RSRawDecoder2 extends AbstractRawErasureDecoder {
   private int numErasedDataUnits;
   private boolean[] erasureFlags;
 
-  public RSRawDecoder2(int numDataUnits, int numParityUnits) {
+  public RSRawDecoder(int numDataUnits, int numParityUnits) {
     super(numDataUnits, numParityUnits);
     if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
       throw new HadoopIllegalArgumentException(
@@ -64,7 +63,7 @@ public class RSRawDecoder2 extends AbstractRawErasureDecoder {
 
     int numAllUnits = getNumDataUnits() + numParityUnits;
     encodeMatrix = new byte[numAllUnits * getNumDataUnits()];
-    RSUtil2.genCauchyMatrix(encodeMatrix, numAllUnits, getNumDataUnits());
+    RSUtil.genCauchyMatrix(encodeMatrix, numAllUnits, getNumDataUnits());
     if (isAllowingVerboseDump()) {
       DumpUtil.dumpMatrix(encodeMatrix, numDataUnits, numAllUnits);
     }
@@ -79,7 +78,7 @@ public class RSRawDecoder2 extends AbstractRawErasureDecoder {
     for (int i = 0; i < getNumDataUnits(); i++) {
       realInputs[i] = inputs[validIndexes[i]];
     }
-    RSUtil2.encodeData(gfTables, realInputs, outputs);
+    RSUtil.encodeData(gfTables, realInputs, outputs);
   }
 
   @Override
@@ -94,7 +93,7 @@ public class RSRawDecoder2 extends AbstractRawErasureDecoder {
       realInputs[i] = inputs[validIndexes[i]];
       realInputOffsets[i] = inputOffsets[validIndexes[i]];
     }
-    RSUtil2.encodeData(gfTables, dataLen, realInputs, realInputOffsets,
+    RSUtil.encodeData(gfTables, dataLen, realInputs, realInputOffsets,
             outputs, outputOffsets);
   }
 
@@ -131,7 +130,7 @@ public class RSRawDecoder2 extends AbstractRawErasureDecoder {
 
     generateDecodeMatrix(erasedIndexes);
 
-    RSUtil2.initTables(getNumDataUnits(), erasedIndexes.length,
+    RSUtil.initTables(getNumDataUnits(), erasedIndexes.length,
         decodeMatrix, 0, gfTables);
     if (isAllowingVerboseDump()) {
       System.out.println(DumpUtil.bytesToHex(gfTables, -1));

+ 6 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder2.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java

@@ -21,7 +21,6 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil2;
 
 import java.nio.ByteBuffer;
 
@@ -32,7 +31,7 @@ import java.nio.ByteBuffer;
  * from HDFS-RAID, and also compatible with the native/ISA-L coder.
  */
 @InterfaceAudience.Private
-public class RSRawEncoder2 extends AbstractRawErasureEncoder {
+public class RSRawEncoder extends AbstractRawErasureEncoder {
   // relevant to schema and won't change during encode calls.
   private byte[] encodeMatrix;
   /**
@@ -41,7 +40,7 @@ public class RSRawEncoder2 extends AbstractRawErasureEncoder {
    */
   private byte[] gfTables;
 
-  public RSRawEncoder2(int numDataUnits, int numParityUnits) {
+  public RSRawEncoder(int numDataUnits, int numParityUnits) {
     super(numDataUnits, numParityUnits);
 
     if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
@@ -50,12 +49,12 @@ public class RSRawEncoder2 extends AbstractRawErasureEncoder {
     }
 
     encodeMatrix = new byte[getNumAllUnits() * numDataUnits];
-    RSUtil2.genCauchyMatrix(encodeMatrix, getNumAllUnits(), numDataUnits);
+    RSUtil.genCauchyMatrix(encodeMatrix, getNumAllUnits(), numDataUnits);
     if (isAllowingVerboseDump()) {
       DumpUtil.dumpMatrix(encodeMatrix, numDataUnits, getNumAllUnits());
     }
     gfTables = new byte[getNumAllUnits() * numDataUnits * 32];
-    RSUtil2.initTables(numDataUnits, numParityUnits, encodeMatrix,
+    RSUtil.initTables(numDataUnits, numParityUnits, encodeMatrix,
         numDataUnits * numDataUnits, gfTables);
     if (isAllowingVerboseDump()) {
       System.out.println(DumpUtil.bytesToHex(gfTables, -1));
@@ -64,13 +63,13 @@ public class RSRawEncoder2 extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    RSUtil2.encodeData(gfTables, inputs, outputs);
+    RSUtil.encodeData(gfTables, inputs, outputs);
   }
 
   @Override
   protected void doEncode(byte[][] inputs, int[] inputOffsets,
                           int dataLen, byte[][] outputs, int[] outputOffsets) {
-    RSUtil2.encodeData(gfTables, dataLen, inputs, inputOffsets, outputs,
+    RSUtil.encodeData(gfTables, dataLen, inputs, inputOffsets, outputs,
         outputOffsets);
   }
 }

+ 4 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory2.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java

@@ -20,18 +20,18 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 import org.apache.hadoop.classification.InterfaceAudience;
 
 /**
- * A raw coder factory for raw Reed-Solomon coder in Java.
+ * A raw coder factory for the new raw Reed-Solomon coder in Java.
  */
 @InterfaceAudience.Private
-public class RSRawErasureCoderFactory2 implements RawErasureCoderFactory {
+public class RSRawErasureCoderFactory implements RawErasureCoderFactory {
 
   @Override
   public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits) {
-    return new RSRawEncoder2(numDataUnits, numParityUnits);
+    return new RSRawEncoder(numDataUnits, numParityUnits);
   }
 
   @Override
   public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) {
-    return new RSRawDecoder2(numDataUnits, numParityUnits);
+    return new RSRawDecoder(numDataUnits, numParityUnits);
   }
 }

+ 147 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java

@@ -19,11 +19,16 @@ package org.apache.hadoop.io.erasurecode.rawcoder.util;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 
+import java.nio.ByteBuffer;
+
 /**
- * Utilities for implementing Reed-Solomon code, used by RS coder.
+ * Utilities for implementing Reed-Solomon code, used by RS coder. Some of the
+ * codes are borrowed from ISA-L implementation (C or ASM codes).
  */
 @InterfaceAudience.Private
-public class RSUtil {
+public final class RSUtil {
+
+  private RSUtil(){}
 
   // We always use the byte system (with symbol size 8, field size 256,
   // primitive polynomial 285, and primitive root 2).
@@ -39,4 +44,144 @@ public class RSUtil {
     return primitivePower;
   }
 
+  public static void initTables(int k, int rows, byte[] codingMatrix,
+      int matrixOffset, byte[] gfTables) {
+    int i, j;
+
+    int offset = 0, idx = matrixOffset;
+    for (i = 0; i < rows; i++) {
+      for (j = 0; j < k; j++) {
+        GF256.gfVectMulInit(codingMatrix[idx++], gfTables, offset);
+        offset += 32;
+      }
+    }
+  }
+
+  /**
+   * Ported from Intel ISA-L library.
+   */
+  public static void genCauchyMatrix(byte[] a, int m, int k) {
+    // Identity matrix in high position
+    for (int i = 0; i < k; i++) {
+      a[k * i + i] = 1;
+    }
+
+    // For the rest choose 1/(i + j) | i != j
+    int pos = k * k;
+    for (int i = k; i < m; i++) {
+      for (int j = 0; j < k; j++) {
+        a[pos++] = GF256.gfInv((byte) (i ^ j));
+      }
+    }
+  }
+
+  /**
+   * Encode a group of inputs data and generate the outputs. It's also used for
+   * decoding because, in this implementation, encoding and decoding are
+   * unified.
+   *
+   * The algorithm is ported from Intel ISA-L library for compatible. It
+   * leverages Java auto-vectorization support for performance.
+   */
+  public static void encodeData(byte[] gfTables, int dataLen, byte[][] inputs,
+      int[] inputOffsets, byte[][] outputs,
+      int[] outputOffsets) {
+    int numInputs = inputs.length;
+    int numOutputs = outputs.length;
+    int l, i, j, iPos, oPos;
+    byte[] input, output;
+    byte s;
+    final int times = dataLen / 8;
+    final int extra = dataLen - dataLen % 8;
+    byte[] tableLine;
+
+    for (l = 0; l < numOutputs; l++) {
+      output = outputs[l];
+
+      for (j = 0; j < numInputs; j++) {
+        input = inputs[j];
+        iPos = inputOffsets[j];
+        oPos = outputOffsets[l];
+
+        s = gfTables[j * 32 + l * numInputs * 32 + 1];
+        tableLine = GF256.gfMulTab()[s & 0xff];
+
+        /**
+         * Purely for performance, assuming we can use 8 bytes in the SIMD
+         * instruction. Subject to be improved.
+         */
+        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
+          output[oPos + 0] ^= tableLine[0xff & input[iPos + 0]];
+          output[oPos + 1] ^= tableLine[0xff & input[iPos + 1]];
+          output[oPos + 2] ^= tableLine[0xff & input[iPos + 2]];
+          output[oPos + 3] ^= tableLine[0xff & input[iPos + 3]];
+          output[oPos + 4] ^= tableLine[0xff & input[iPos + 4]];
+          output[oPos + 5] ^= tableLine[0xff & input[iPos + 5]];
+          output[oPos + 6] ^= tableLine[0xff & input[iPos + 6]];
+          output[oPos + 7] ^= tableLine[0xff & input[iPos + 7]];
+        }
+
+        /**
+         * For the left bytes, do it one by one.
+         */
+        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
+          output[oPos] ^= tableLine[0xff & input[iPos]];
+        }
+      }
+    }
+  }
+
+  /**
+   * See above. Try to use the byte[] version when possible.
+   */
+  public static void encodeData(byte[] gfTables, ByteBuffer[] inputs,
+      ByteBuffer[] outputs) {
+    int numInputs = inputs.length;
+    int numOutputs = outputs.length;
+    int dataLen = inputs[0].remaining();
+    int l, i, j, iPos, oPos;
+    ByteBuffer input, output;
+    byte s;
+    final int times = dataLen / 8;
+    final int extra = dataLen - dataLen % 8;
+    byte[] tableLine;
+
+    for (l = 0; l < numOutputs; l++) {
+      output = outputs[l];
+
+      for (j = 0; j < numInputs; j++) {
+        input = inputs[j];
+        iPos = input.position();
+        oPos = output.position();
+
+        s = gfTables[j * 32 + l * numInputs * 32 + 1];
+        tableLine = GF256.gfMulTab()[s & 0xff];
+
+        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
+          output.put(oPos + 0, (byte) (output.get(oPos + 0) ^
+              tableLine[0xff & input.get(iPos + 0)]));
+          output.put(oPos + 1, (byte) (output.get(oPos + 1) ^
+              tableLine[0xff & input.get(iPos + 1)]));
+          output.put(oPos + 2, (byte) (output.get(oPos + 2) ^
+              tableLine[0xff & input.get(iPos + 2)]));
+          output.put(oPos + 3, (byte) (output.get(oPos + 3) ^
+              tableLine[0xff & input.get(iPos + 3)]));
+          output.put(oPos + 4, (byte) (output.get(oPos + 4) ^
+              tableLine[0xff & input.get(iPos + 4)]));
+          output.put(oPos + 5, (byte) (output.get(oPos + 5) ^
+              tableLine[0xff & input.get(iPos + 5)]));
+          output.put(oPos + 6, (byte) (output.get(oPos + 6) ^
+              tableLine[0xff & input.get(iPos + 6)]));
+          output.put(oPos + 7, (byte) (output.get(oPos + 7) ^
+              tableLine[0xff & input.get(iPos + 7)]));
+        }
+
+        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
+          output.put(oPos, (byte) (output.get(oPos) ^
+              tableLine[0xff & input.get(iPos)]));
+        }
+      }
+    }
+  }
+
 }

+ 0 - 172
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil2.java

@@ -1,172 +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.util;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-import java.nio.ByteBuffer;
-
-/**
- * Utilities for implementing Reed-Solomon code, used by RS2 coder. Some of the
- * codes are borrowed from ISA-L implementation (C or ASM codes).
- */
-@InterfaceAudience.Private
-public final class RSUtil2 {
-
-  private RSUtil2() { }
-
-  public static void initTables(int k, int rows, byte[] codingMatrix,
-                                int matrixOffset, byte[] gfTables) {
-    int i, j;
-
-    int offset = 0, idx = matrixOffset;
-    for (i = 0; i < rows; i++) {
-      for (j = 0; j < k; j++) {
-        GF256.gfVectMulInit(codingMatrix[idx++], gfTables, offset);
-        offset += 32;
-      }
-    }
-  }
-
-  /**
-   * Ported from Intel ISA-L library.
-   */
-  public static void genCauchyMatrix(byte[] a, int m, int k) {
-    // Identity matrix in high position
-    for (int i = 0; i < k; i++) {
-      a[k * i + i] = 1;
-    }
-
-    // For the rest choose 1/(i + j) | i != j
-    int pos = k * k;
-    for (int i = k; i < m; i++) {
-      for (int j = 0; j < k; j++) {
-        a[pos++] = GF256.gfInv((byte) (i ^ j));
-      }
-    }
-  }
-
-  /**
-   * Encode a group of inputs data and generate the outputs. It's also used for
-   * decoding because, in this implementation, encoding and decoding are
-   * unified.
-   *
-   * The algorithm is ported from Intel ISA-L library for compatible. It
-   * leverages Java auto-vectorization support for performance.
-   */
-  public static void encodeData(byte[] gfTables, int dataLen, byte[][] inputs,
-                                int[] inputOffsets, byte[][] outputs,
-                                int[] outputOffsets) {
-    int numInputs = inputs.length;
-    int numOutputs = outputs.length;
-    int l, i, j, iPos, oPos;
-    byte[] input, output;
-    byte s;
-    final int times = dataLen / 8;
-    final int extra = dataLen - dataLen % 8;
-    byte[] tableLine;
-
-    for (l = 0; l < numOutputs; l++) {
-      output = outputs[l];
-
-      for (j = 0; j < numInputs; j++) {
-        input = inputs[j];
-        iPos = inputOffsets[j];
-        oPos = outputOffsets[l];
-
-        s = gfTables[j * 32 + l * numInputs * 32 + 1];
-        tableLine = GF256.gfMulTab()[s & 0xff];
-
-        /**
-         * Purely for performance, assuming we can use 8 bytes in the SIMD
-         * instruction. Subject to be improved.
-         */
-        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
-          output[oPos + 0] ^= tableLine[0xff & input[iPos + 0]];
-          output[oPos + 1] ^= tableLine[0xff & input[iPos + 1]];
-          output[oPos + 2] ^= tableLine[0xff & input[iPos + 2]];
-          output[oPos + 3] ^= tableLine[0xff & input[iPos + 3]];
-          output[oPos + 4] ^= tableLine[0xff & input[iPos + 4]];
-          output[oPos + 5] ^= tableLine[0xff & input[iPos + 5]];
-          output[oPos + 6] ^= tableLine[0xff & input[iPos + 6]];
-          output[oPos + 7] ^= tableLine[0xff & input[iPos + 7]];
-        }
-
-        /**
-         * For the left bytes, do it one by one.
-         */
-        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
-          output[oPos] ^= tableLine[0xff & input[iPos]];
-        }
-      }
-    }
-  }
-
-  /**
-   * See above. Try to use the byte[] version when possible.
-   */
-  public static void encodeData(byte[] gfTables, ByteBuffer[] inputs,
-                                ByteBuffer[] outputs) {
-    int numInputs = inputs.length;
-    int numOutputs = outputs.length;
-    int dataLen = inputs[0].remaining();
-    int l, i, j, iPos, oPos;
-    ByteBuffer input, output;
-    byte s;
-    final int times = dataLen / 8;
-    final int extra = dataLen - dataLen % 8;
-    byte[] tableLine;
-
-    for (l = 0; l < numOutputs; l++) {
-      output = outputs[l];
-
-      for (j = 0; j < numInputs; j++) {
-        input = inputs[j];
-        iPos = input.position();
-        oPos = output.position();
-
-        s = gfTables[j * 32 + l * numInputs * 32 + 1];
-        tableLine = GF256.gfMulTab()[s & 0xff];
-
-        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
-          output.put(oPos + 0, (byte) (output.get(oPos + 0) ^
-              tableLine[0xff & input.get(iPos + 0)]));
-          output.put(oPos + 1, (byte) (output.get(oPos + 1) ^
-              tableLine[0xff & input.get(iPos + 1)]));
-          output.put(oPos + 2, (byte) (output.get(oPos + 2) ^
-              tableLine[0xff & input.get(iPos + 2)]));
-          output.put(oPos + 3, (byte) (output.get(oPos + 3) ^
-              tableLine[0xff & input.get(iPos + 3)]));
-          output.put(oPos + 4, (byte) (output.get(oPos + 4) ^
-              tableLine[0xff & input.get(iPos + 4)]));
-          output.put(oPos + 5, (byte) (output.get(oPos + 5) ^
-              tableLine[0xff & input.get(iPos + 5)]));
-          output.put(oPos + 6, (byte) (output.get(oPos + 6) ^
-              tableLine[0xff & input.get(iPos + 6)]));
-          output.put(oPos + 7, (byte) (output.get(oPos + 7) ^
-              tableLine[0xff & input.get(iPos + 7)]));
-        }
-
-        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
-          output.put(oPos, (byte) (output.get(oPos) ^
-              tableLine[0xff & input.get(iPos)]));
-        }
-      }
-    }
-  }
-}

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.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.RSRawErasureCoderFactoryLegacy;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -51,7 +51,7 @@ public class TestHHXORErasureCoder extends TestHHErasureCoderBase {
      */
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        RSRawErasureCoderFactoryLegacy.class.getCanonicalName());
+        RSRawErasureCoderFactory.class.getCanonicalName());
     prepare(conf, 10, 4, new int[]{0}, new int[0]);
 
     testCoding(true);

+ 4 - 4
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.RSRawErasureCoderFactoryLegacy;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -58,20 +58,20 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
      */
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        RSRawErasureCoderFactoryLegacy.class.getCanonicalName());
+        RSRawErasureCoderFactory.class.getCanonicalName());
     prepare(conf, 10, 4, new int[]{0}, new int[0]);
 
     testCoding(true);
     testCoding(true);
   }
-  
+
   @Test
   public void testCodingDirectBuffer_10x4_erasing_p1() {
     prepare(null, 10, 4, new int[]{}, new int[]{1});
     testCoding(true);
     testCoding(true);
   }
-  
+
   @Test
   public void testCodingDirectBuffer_10x4_erasing_d2() {
     prepare(null, 10, 4, new int[] {2}, new int[] {});

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder2.java → hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java

@@ -22,12 +22,12 @@ import org.junit.Before;
 /**
  * Test the new raw Reed-solomon coder implemented in Java.
  */
-public class TestRSRawCoder2 extends TestRSRawCoderBase {
+public class TestRSRawCoder extends TestRSRawCoderBase {
 
   @Before
   public void setup() {
-    this.encoderClass = RSRawEncoder2.class;
-    this.decoderClass = RSRawDecoder2.class;
+    this.encoderClass = RSRawEncoder.class;
+    this.decoderClass = RSRawDecoder.class;
     setAllowDump(false);
   }
 }