Browse Source

HADOOP-11920. Refactor some codes for erasure coders. Contributed by Kai Zheng.

Zhe Zhang 10 năm trước cách đây
mục cha
commit
09c3a375ba
14 tập tin đã thay đổi với 156 bổ sung139 xóa
  1. 2 0
      hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
  2. 0 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
  4. 1 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
  5. 2 38
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
  6. 62 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
  7. 32 22
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
  8. 23 29
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
  9. 4 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
  10. 15 9
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
  11. 6 18
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
  12. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
  13. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
  14. 2 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java

+ 2 - 0
hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt

@@ -46,3 +46,5 @@
     HADOOP-11841. Remove unused ecschema-def.xml files.  (szetszwo)
 
     HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng via Zhe Zhang)
+
+    HADOOP-11920. Refactor some codes for erasure coders. (Kai Zheng via Zhe Zhang)

+ 0 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -143,10 +143,6 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   /** Supported erasure codec classes */
   public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs";
 
-  /** Use XOR raw coder when possible for the RS codec */
-  public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY =
-      "io.erasurecode.codec.rs.usexor";
-
   /** Raw coder factory for the RS codec */
   public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY =
       "io.erasurecode.codec.rs.rawcoder";

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java

@@ -71,7 +71,7 @@ public class ECChunk {
    * @param chunks
    * @return an array of byte array
    */
-  public static byte[][] toArray(ECChunk[] chunks) {
+  public static byte[][] toArrays(ECChunk[] chunks) {
     byte[][] bytesArr = new byte[chunks.length][];
 
     ByteBuffer buffer;

+ 1 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java

@@ -90,11 +90,7 @@ public abstract class AbstractErasureCoder
       throw new RuntimeException("Failed to create raw coder", e);
     }
 
-    if (fact != null) {
-      return isEncoder ? fact.createEncoder() : fact.createDecoder();
-    }
-
-    return null;
+    return isEncoder ? fact.createEncoder() : fact.createDecoder();
   }
 
   @Override

+ 2 - 38
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java

@@ -17,13 +17,11 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
-import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 
 /**
  * Reed-Solomon erasure decoder that decodes a block group.
@@ -32,38 +30,14 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
  */
 public class RSErasureDecoder extends AbstractErasureDecoder {
   private RawErasureDecoder rsRawDecoder;
-  private RawErasureDecoder xorRawDecoder;
-  private boolean useXorWhenPossible = true;
 
   @Override
-  public void setConf(Configuration conf) {
-    super.setConf(conf);
-
-    if (conf != null) {
-      this.useXorWhenPossible = conf.getBoolean(
-          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, true);
-    }
-  }
-
-    @Override
   protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
 
-    RawErasureDecoder rawDecoder;
-
     ECBlock[] inputBlocks = getInputBlocks(blockGroup);
     ECBlock[] outputBlocks = getOutputBlocks(blockGroup);
 
-    /**
-     * Optimization: according to some benchmark, when only one block is erased
-     * and to be recovering, the most simple XOR scheme can be much efficient.
-     * We will have benchmark tests to verify this opt is effect or not.
-     */
-    if (outputBlocks.length == 1 && useXorWhenPossible) {
-      rawDecoder = checkCreateXorRawDecoder();
-    } else {
-      rawDecoder = checkCreateRSRawDecoder();
-    }
-
+    RawErasureDecoder rawDecoder = checkCreateRSRawDecoder();
     return new ErasureDecodingStep(inputBlocks,
         getErasedIndexes(inputBlocks), outputBlocks, rawDecoder);
   }
@@ -81,19 +55,9 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
     return rsRawDecoder;
   }
 
-  private RawErasureDecoder checkCreateXorRawDecoder() {
-    if (xorRawDecoder == null) {
-      xorRawDecoder = new XORRawDecoder();
-      xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize());
-    }
-    return xorRawDecoder;
-  }
-
   @Override
   public void release() {
-    if (xorRawDecoder != null) {
-      xorRawDecoder.release();
-    } else if (rsRawDecoder != null) {
+    if (rsRawDecoder != null) {
       rsRawDecoder.release();
     }
   }

+ 62 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java

@@ -19,6 +19,9 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 
 import org.apache.hadoop.conf.Configured;
 
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
 /**
  * A common class of basic facilities to be shared by encoder and decoder
  *
@@ -27,6 +30,9 @@ import org.apache.hadoop.conf.Configured;
 public abstract class AbstractRawErasureCoder
     extends Configured implements RawErasureCoder {
 
+  // Hope to reset coding buffers a little faster using it
+  private byte[] zeroChunkBytes;
+
   private int numDataUnits;
   private int numParityUnits;
   private int chunkSize;
@@ -37,6 +43,8 @@ public abstract class AbstractRawErasureCoder
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
     this.chunkSize = chunkSize;
+
+    zeroChunkBytes = new byte[chunkSize]; // With ZERO by default
   }
 
   @Override
@@ -55,7 +63,7 @@ public abstract class AbstractRawErasureCoder
   }
 
   @Override
-  public boolean preferNativeBuffer() {
+  public boolean preferDirectBuffer() {
     return false;
   }
 
@@ -63,4 +71,57 @@ public abstract class AbstractRawErasureCoder
   public void release() {
     // Nothing to do by default
   }
+
+  /**
+   * Convert an array of heap ByteBuffers to an array of byte array.
+   * @param buffers
+   * @return an array of byte array
+   */
+  protected static byte[][] toArrays(ByteBuffer[] buffers) {
+    byte[][] bytesArr = new byte[buffers.length][];
+
+    ByteBuffer buffer;
+    for (int i = 0; i < buffers.length; i++) {
+      buffer = buffers[i];
+      if (buffer == null) {
+        bytesArr[i] = null;
+        continue;
+      }
+
+      if (buffer.hasArray()) {
+        bytesArr[i] = buffer.array();
+      } else {
+        throw new IllegalArgumentException("Invalid ByteBuffer passed, " +
+            "expecting heap buffer");
+      }
+    }
+
+    return bytesArr;
+  }
+
+  /**
+   * Ensure the buffer (either input or output) ready to read or write with ZERO
+   * bytes fully in chunkSize.
+   * @param buffer
+   * @return the buffer itself
+   */
+  protected ByteBuffer resetBuffer(ByteBuffer buffer) {
+    buffer.clear();
+    buffer.put(zeroChunkBytes);
+    buffer.position(0);
+
+    return buffer;
+  }
+
+  /**
+   * Ensure the buffer (either input or output) ready to read or write with ZERO
+   * bytes fully in chunkSize.
+   * @param buffer bytes array buffer
+   * @return the buffer itself
+   */
+  protected byte[] resetBuffer(byte[] buffer) {
+    System.arraycopy(zeroChunkBytes, 0, buffer, 0, buffer.length);
+
+    return buffer;
+  }
 }

+ 32 - 22
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java

@@ -32,27 +32,30 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   @Override
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs) {
-    if (erasedIndexes.length == 0) {
-      return;
-    }
+    checkParameters(inputs, erasedIndexes, outputs);
 
-    doDecode(inputs, erasedIndexes, outputs);
+    boolean hasArray = inputs[0].hasArray();
+    if (hasArray) {
+      byte[][] newInputs = toArrays(inputs);
+      byte[][] newOutputs = toArrays(outputs);
+      doDecode(newInputs, erasedIndexes, newOutputs);
+    } else {
+      doDecode(inputs, erasedIndexes, outputs);
+    }
   }
 
   /**
-   * Perform the real decoding using ByteBuffer
-   * @param inputs
+   * Perform the real decoding using Direct ByteBuffer.
+   * @param inputs Direct ByteBuffers expected
    * @param erasedIndexes
-   * @param outputs
+   * @param outputs Direct ByteBuffers expected
    */
   protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
                                    ByteBuffer[] outputs);
 
   @Override
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) {
-    if (erasedIndexes.length == 0) {
-      return;
-    }
+    checkParameters(inputs, erasedIndexes, outputs);
 
     doDecode(inputs, erasedIndexes, outputs);
   }
@@ -69,25 +72,32 @@ public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
   @Override
   public void decode(ECChunk[] inputs, int[] erasedIndexes,
                      ECChunk[] outputs) {
-    doDecode(inputs, erasedIndexes, outputs);
+    ByteBuffer[] newInputs = ECChunk.toBuffers(inputs);
+    ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs);
+    decode(newInputs, erasedIndexes, newOutputs);
   }
 
   /**
-   * Perform the real decoding using chunks
+   * Check and validate decoding parameters, throw exception accordingly. The
+   * checking assumes it's a MDS code. Other code  can override this.
    * @param inputs
    * @param erasedIndexes
    * @param outputs
    */
-  protected void doDecode(ECChunk[] inputs, int[] erasedIndexes,
-                          ECChunk[] outputs) {
-    if (inputs[0].getBuffer().hasArray()) {
-      byte[][] inputBytesArr = ECChunk.toArray(inputs);
-      byte[][] outputBytesArr = ECChunk.toArray(outputs);
-      doDecode(inputBytesArr, erasedIndexes, outputBytesArr);
-    } else {
-      ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
-      ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
-      doDecode(inputBuffers, erasedIndexes, outputBuffers);
+  protected void checkParameters(Object[] inputs, int[] erasedIndexes,
+                                 Object[] outputs) {
+    if (inputs.length != getNumParityUnits() + getNumDataUnits()) {
+      throw new IllegalArgumentException("Invalid inputs length");
+    }
+
+    if (erasedIndexes.length != outputs.length) {
+      throw new IllegalArgumentException(
+          "erasedIndexes and outputs mismatch in length");
+    }
+
+    if (erasedIndexes.length > getNumParityUnits()) {
+      throw new IllegalArgumentException(
+          "Too many erased, not recoverable");
     }
   }
 }

+ 23 - 29
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java

@@ -31,23 +31,28 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
 
   @Override
   public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    assert (inputs.length == getNumDataUnits());
-    assert (outputs.length == getNumParityUnits());
+    checkParameters(inputs, outputs);
 
-    doEncode(inputs, outputs);
+    boolean hasArray = inputs[0].hasArray();
+    if (hasArray) {
+      byte[][] newInputs = toArrays(inputs);
+      byte[][] newOutputs = toArrays(outputs);
+      doEncode(newInputs, newOutputs);
+    } else {
+      doEncode(inputs, outputs);
+    }
   }
 
   /**
-   * Perform the real encoding work using ByteBuffer
-   * @param inputs
-   * @param outputs
+   * Perform the real encoding work using direct ByteBuffer
+   * @param inputs Direct ByteBuffers expected
+   * @param outputs Direct ByteBuffers expected
    */
   protected abstract void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs);
 
   @Override
   public void encode(byte[][] inputs, byte[][] outputs) {
-    assert (inputs.length == getNumDataUnits());
-    assert (outputs.length == getNumParityUnits());
+    checkParameters(inputs, outputs);
 
     doEncode(inputs, outputs);
   }
@@ -61,33 +66,22 @@ public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
 
   @Override
   public void encode(ECChunk[] inputs, ECChunk[] outputs) {
-    assert (inputs.length == getNumDataUnits());
-    assert (outputs.length == getNumParityUnits());
-
-    doEncode(inputs, outputs);
+    ByteBuffer[] newInputs = ECChunk.toBuffers(inputs);
+    ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs);
+    encode(newInputs, newOutputs);
   }
 
   /**
-   * Perform the real encoding work using chunks.
+   * Check and validate decoding parameters, throw exception accordingly.
    * @param inputs
    * @param outputs
    */
-  protected void doEncode(ECChunk[] inputs, ECChunk[] outputs) {
-    /**
-     * Note callers may pass byte array, or ByteBuffer via ECChunk according
-     * to how ECChunk is created. Some implementations of coder use byte array
-     * (ex: pure Java), some use native ByteBuffer (ex: ISA-L), all for the
-     * better performance.
-     */
-    if (inputs[0].getBuffer().hasArray()) {
-      byte[][] inputBytesArr = ECChunk.toArray(inputs);
-      byte[][] outputBytesArr = ECChunk.toArray(outputs);
-      doEncode(inputBytesArr, outputBytesArr);
-    } else {
-      ByteBuffer[] inputBuffers = ECChunk.toBuffers(inputs);
-      ByteBuffer[] outputBuffers = ECChunk.toBuffers(outputs);
-      doEncode(inputBuffers, outputBuffers);
+  protected void checkParameters(Object[] inputs, Object[] outputs) {
+    if (inputs.length != getNumDataUnits()) {
+      throw new IllegalArgumentException("Invalid inputs length");
+    }
+    if (outputs.length != getNumParityUnits()) {
+      throw new IllegalArgumentException("Invalid outputs length");
     }
   }
-
 }

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

@@ -64,13 +64,13 @@ public interface RawErasureCoder extends Configurable {
   public int getChunkSize();
 
   /**
-   * Tell if native or off-heap buffer is preferred or not. It's for callers to
-   * decide how to allocate coding chunk buffers, either on heap or off heap.
-   * It will return false by default.
+   * Tell if direct buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, using DirectByteBuffer or
+   * bytes array. It will return false by default.
    * @return true if native buffer is preferred for performance consideration,
    * otherwise false.
    */
-  public boolean preferNativeBuffer();
+  public boolean preferDirectBuffer();
 
   /**
    * Should be called when release this coder. Good chance to release encoding

+ 15 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java

@@ -31,24 +31,30 @@ import java.nio.ByteBuffer;
 public interface RawErasureDecoder extends RawErasureCoder {
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs
-   * @param inputs
-   * @param outputs
+   * Decode with inputs and erasedIndexes, generates outputs.
+   * @param inputs inputs to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes
    */
   public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
                      ByteBuffer[] outputs);
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs
-   * @param inputs
-   * @param outputs
+   * Decode with inputs and erasedIndexes, generates outputs.
+   * @param inputs inputs to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes
    */
   public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs);
 
   /**
-   * Decode with inputs and erasedIndexes, generates outputs
-   * @param inputs
-   * @param outputs
+   * Decode with inputs and erasedIndexes, generates outputs.
+   * @param inputs inputs to read data from
+   * @param erasedIndexes indexes of erased units in the inputs array
+   * @param outputs outputs to write into for data generated according to
+   *                erasedIndexes
    */
   public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs);
 

+ 6 - 18
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java

@@ -27,17 +27,11 @@ public class XORRawDecoder extends AbstractRawErasureDecoder {
   @Override
   protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
                           ByteBuffer[] outputs) {
-    assert(erasedIndexes.length == outputs.length);
-    assert(erasedIndexes.length <= 1);
+    resetBuffer(outputs[0]);
 
-    int bufSize = inputs[0].remaining();
+    int bufSize = getChunkSize();
     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.
@@ -52,19 +46,13 @@ public class XORRawDecoder extends AbstractRawErasureDecoder {
   }
 
   @Override
-  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
-                          byte[][] outputs) {
-    assert(erasedIndexes.length == outputs.length);
-    assert(erasedIndexes.length <= 1);
+  protected void doDecode(byte[][] inputs,
+                          int[] erasedIndexes, byte[][] outputs) {
+    resetBuffer(outputs[0]);
 
-    int bufSize = inputs[0].length;
+    int bufSize = getChunkSize();
     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.

+ 4 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java

@@ -26,8 +26,9 @@ public class XORRawEncoder extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    int bufSize = inputs[0].remaining();
+    resetBuffer(outputs[0]);
 
+    int bufSize = getChunkSize();
     // Get the first buffer's data.
     for (int j = 0; j < bufSize; j++) {
       outputs[0].put(j, inputs[0].get(j));
@@ -43,8 +44,9 @@ public class XORRawEncoder extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(byte[][] inputs, byte[][] outputs) {
-    int bufSize = inputs[0].length;
+    resetBuffer(outputs[0]);
 
+    int bufSize = getChunkSize();
     // Get the first buffer's data.
     for (int j = 0; j < bufSize; j++) {
       outputs[0][j] = inputs[0][j];

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java

@@ -75,8 +75,8 @@ public abstract class TestCoderBase {
    */
   protected void compareAndVerify(ECChunk[] erasedChunks,
                                   ECChunk[] recoveredChunks) {
-    byte[][] erased = ECChunk.toArray(erasedChunks);
-    byte[][] recovered = ECChunk.toArray(recoveredChunks);
+    byte[][] erased = ECChunk.toArrays(erasedChunks);
+    byte[][] recovered = ECChunk.toArrays(recoveredChunks);
     boolean result = Arrays.deepEquals(erased, recovered);
     assertTrue("Decoding and comparing failed.", result);
   }

+ 2 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java

@@ -56,14 +56,12 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
      * This tests if the two configuration items work or not.
      */
     Configuration conf = new Configuration();
-    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        RSRawErasureCoderFactory.class.getCanonicalName());
-    conf.setBoolean(
-        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false);
+    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, RSRawErasureCoderFactory.class.getCanonicalName());
 
     prepare(conf, 10, 4, new int[]{0});
 
     testCoding(true);
+    testCoding(true);
   }
 
   @Test