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 we...@apache.org on 2016/10/25 23:58:22 UTC

hadoop git commit: HADOOP-11798. Native raw erasure coder in XOR codes. Contributed by SammiChen.

Repository: hadoop
Updated Branches:
  refs/heads/trunk 084bdab15 -> d88dca844


HADOOP-11798. Native raw erasure coder in XOR codes. Contributed by SammiChen.


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

Branch: refs/heads/trunk
Commit: d88dca844a8437397959f28dadf1294e6db31df9
Parents: 084bdab
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Oct 25 16:53:53 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Oct 25 16:53:53 2016 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/pom.xml     |  4 +
 .../hadoop-common/src/CMakeLists.txt            |  4 +-
 .../rawcoder/NativeXORRawDecoder.java           | 59 ++++++++++++++
 .../rawcoder/NativeXORRawEncoder.java           | 60 ++++++++++++++
 .../NativeXORRawErasureCoderFactory.java        | 39 ++++++++++
 .../src/main/native/native.vcxproj              |  6 +-
 .../hadoop/io/erasurecode/jni_xor_decoder.c     | 80 +++++++++++++++++++
 .../hadoop/io/erasurecode/jni_xor_encoder.c     | 82 ++++++++++++++++++++
 .../rawcoder/TestNativeXORRawCoder.java         | 36 +++++++++
 .../erasurecode/rawcoder/TestXORRawCoder.java   | 38 +--------
 .../rawcoder/TestXORRawCoderBase.java           | 59 ++++++++++++++
 .../rawcoder/TestXORRawCoderInteroperable1.java | 36 +++++++++
 .../rawcoder/TestXORRawCoderInteroperable2.java | 37 +++++++++
 13 files changed, 501 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 54d1cdd..fd9b7cd 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -640,6 +640,8 @@
                     <javahClassName>org.apache.hadoop.io.erasurecode.ErasureCodeNative</javahClassName>
                     <javahClassName>org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawEncoder</javahClassName>
                     <javahClassName>org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawDecoder</javahClassName>
+                    <javahClassName>org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawEncoder</javahClassName>
+                    <javahClassName>org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawDecoder</javahClassName>
                     <javahClassName>org.apache.hadoop.crypto.OpensslCipher</javahClassName>
                     <javahClassName>org.apache.hadoop.crypto.random.OpensslSecureRandom</javahClassName>
                     <javahClassName>org.apache.hadoop.util.NativeCrc32</javahClassName>
@@ -780,6 +782,8 @@
                     <javahClassName>org.apache.hadoop.io.erasurecode.ErasureCodeNative</javahClassName>
                     <javahClassName>org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawEncoder</javahClassName>
                     <javahClassName>org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawDecoder</javahClassName>
+                    <javahClassName>org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawEncoder</javahClassName>
+                    <javahClassName>org.apache.hadoop.io.erasurecode.rawcoder.NativeXORRawDecoder</javahClassName>
                     <javahClassName>org.apache.hadoop.crypto.OpensslCipher</javahClassName>
                     <javahClassName>org.apache.hadoop.crypto.random.OpensslSecureRandom</javahClassName>
                     <javahClassName>org.apache.hadoop.util.NativeCrc32</javahClassName>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/CMakeLists.txt b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
index 8026eb5..8317a46 100644
--- a/hadoop-common-project/hadoop-common/src/CMakeLists.txt
+++ b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
@@ -113,7 +113,9 @@ if (ISAL_LIBRARY)
         ${SRC}/io/erasurecode/jni_erasure_code_native.c
         ${SRC}/io/erasurecode/jni_common.c
         ${SRC}/io/erasurecode/jni_rs_encoder.c
-        ${SRC}/io/erasurecode/jni_rs_decoder.c)
+        ${SRC}/io/erasurecode/jni_rs_decoder.c
+        ${SRC}/io/erasurecode/jni_xor_encoder.c
+        ${SRC}/io/erasurecode/jni_xor_decoder.c)
 
         add_executable(erasure_code_test
         ${SRC}/io/erasurecode/isal_load.c

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/NativeXORRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/NativeXORRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/NativeXORRawDecoder.java
new file mode 100644
index 0000000..b6b1673
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/NativeXORRawDecoder.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A XOR raw decoder using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeXORRawDecoder extends AbstractNativeRawDecoder {
+
+  static {
+    ErasureCodeNative.checkNativeCodeLoaded();
+  }
+
+  public NativeXORRawDecoder(ErasureCoderOptions coderOptions) {
+    super(coderOptions);
+    initImpl(coderOptions.getNumDataUnits(), coderOptions.getNumParityUnits());
+  }
+
+  @Override
+  protected void performDecodeImpl(ByteBuffer[] inputs, int[] inputOffsets,
+      int dataLen, int[] erased, ByteBuffer[] outputs, int[] outputOffsets) {
+    decodeImpl(inputs, inputOffsets, dataLen, erased, outputs, outputOffsets);
+  }
+
+  @Override
+  public void release() {
+    destroyImpl();
+  }
+
+  private native void initImpl(int numDataUnits, int numParityUnits);
+
+  private native void decodeImpl(
+      ByteBuffer[] inputs, int[] inputOffsets, int dataLen, int[] erased,
+      ByteBuffer[] outputs, int[] outputOffsets);
+
+  private native void destroyImpl();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/NativeXORRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/NativeXORRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/NativeXORRawEncoder.java
new file mode 100644
index 0000000..9b4b449
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/NativeXORRawEncoder.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
+import org.apache.hadoop.io.erasurecode.ErasureCoderOptions;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A XOR raw encoder using Intel ISA-L library.
+ */
+@InterfaceAudience.Private
+public class NativeXORRawEncoder extends AbstractNativeRawEncoder {
+
+  static {
+    ErasureCodeNative.checkNativeCodeLoaded();
+  }
+
+  public NativeXORRawEncoder(ErasureCoderOptions coderOptions) {
+    super(coderOptions);
+    initImpl(coderOptions.getNumDataUnits(), coderOptions.getNumParityUnits());
+  }
+
+  @Override
+  protected void performEncodeImpl(
+      ByteBuffer[] inputs, int[] inputOffsets, int dataLen,
+      ByteBuffer[] outputs, int[] outputOffsets) {
+    encodeImpl(inputs, inputOffsets, dataLen, outputs, outputOffsets);
+  }
+
+  @Override
+  public void release() {
+    destroyImpl();
+  }
+
+  private native void initImpl(int numDataUnits, int numParityUnits);
+
+  private native void encodeImpl(ByteBuffer[] inputs, int[] inputOffsets,
+                                 int dataLen, ByteBuffer[] outputs,
+                                 int[] outputOffsets);
+
+  private native void destroyImpl();
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj b/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
index 2274c41..1119069 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
+++ b/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
@@ -156,14 +156,14 @@
     <ClCompile Include="src\org\apache\hadoop\util\bulk_crc32.c" />
     <ClCompile Include="src\org\apache\hadoop\util\NativeCodeLoader.c">
       <AdditionalOptions Condition="'$(SnappyEnabled)' == 'true'">/D HADOOP_SNAPPY_LIBRARY=L\"snappy.dll\"</AdditionalOptions>
-      <AdditionalOptions Condition="'$(IsalEnabled)' == 'true'">/D HADOOP_ISAL_LIBRARY=\"isa-l.dll\"</AdditionalOptions>
+      <AdditionalOptions Condition="'$(IsalEnabled)' == 'true'">/D HADOOP_ISAL_LIBRARY=L\"isa-l.dll\"</AdditionalOptions>
     </ClCompile>
     <ClCompile Include="src\org\apache\hadoop\util\NativeCrc32.c" />
     <ClCompile Include="src\org\apache\hadoop\yarn\server\nodemanager\windows_secure_container_executor.c">
       <AdditionalIncludeDirectories>src\org\apache\hadoop\io\nativeio;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
     </ClCompile>
     <ClCompile Include="src\org\apache\hadoop\io\erasurecode\isal_load.c" Condition="'$(IsalEnabled)' == 'true'">
-      <AdditionalOptions>/D HADOOP_ISAL_LIBRARY=\"isa-l.dll\"</AdditionalOptions>
+      <AdditionalOptions>/D HADOOP_ISAL_LIBRARY=L\"isa-l.dll\"</AdditionalOptions>
     </ClCompile>
     <ClCompile Include="src\org\apache\hadoop\io\erasurecode\erasure_code.c" Condition="'$(IsalEnabled)' == 'true'"/>
     <ClCompile Include="src\org\apache\hadoop\io\erasurecode\gf_util.c" Condition="'$(IsalEnabled)' == 'true'"/>
@@ -173,6 +173,8 @@
     <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_common.c" Condition="'$(IsalEnabled)' == 'true'"/>
     <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_rs_encoder.c" Condition="'$(IsalEnabled)' == 'true'"/>
     <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_rs_decoder.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_xor_encoder.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_xor_decoder.c" Condition="'$(IsalEnabled)' == 'true'"/>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\src\org\apache\hadoop\util\crc32c_tables.h" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_decoder.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_decoder.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_decoder.c
new file mode 100644
index 0000000..ff0d3c5
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_decoder.c
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <jni.h>
+
+#include "org_apache_hadoop.h"
+#include "erasure_code.h"
+#include "gf_util.h"
+#include "jni_common.h"
+#include "org_apache_hadoop_io_erasurecode_rawcoder_NativeXORRawDecoder.h"
+
+typedef struct _XOREncoder {
+  IsalCoder isalCoder;
+  unsigned char* inputs[MMAX];
+  unsigned char* outputs[1];
+} XORDecoder;
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeXORRawDecoder_initImpl(
+  JNIEnv *env, jobject thiz, jint numDataUnits, jint numParityUnits) {
+  XORDecoder* xorDecoder =
+                           (XORDecoder*)malloc(sizeof(XORDecoder));
+  memset(xorDecoder, 0, sizeof(*xorDecoder));
+  initCoder(&xorDecoder->isalCoder, numDataUnits, numParityUnits);
+
+  setCoder(env, thiz, &xorDecoder->isalCoder);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeXORRawDecoder_decodeImpl(
+  JNIEnv *env, jobject thiz, jobjectArray inputs, jintArray inputOffsets,
+  jint dataLen, jintArray erasedIndexes, jobjectArray outputs,
+                                                    jintArray outputOffsets) {
+  int i, j, numDataUnits, numParityUnits, chunkSize;
+  XORDecoder* xorDecoder;
+
+  xorDecoder = (XORDecoder*)getCoder(env, thiz);
+  numDataUnits = ((IsalCoder*)xorDecoder)->numDataUnits;
+  numParityUnits = ((IsalCoder*)xorDecoder)->numParityUnits;
+  chunkSize = (int)dataLen;
+
+  getInputs(env, inputs, inputOffsets, xorDecoder->inputs,
+                                               numDataUnits + numParityUnits);
+  getOutputs(env, outputs, outputOffsets, xorDecoder->outputs, numParityUnits);
+
+  for (i = 0; i < numDataUnits + numParityUnits; i++) {
+    if (xorDecoder->inputs[i] == NULL) {
+      continue;
+    }
+    for (j = 0; j < chunkSize; j++) {
+      xorDecoder->outputs[0][j] ^= xorDecoder->inputs[i][j];
+    }
+  }
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeXORRawDecoder_destroyImpl
+  (JNIEnv *env, jobject thiz){
+  XORDecoder* xorDecoder = (XORDecoder*)getCoder(env, thiz);
+  free(xorDecoder);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_encoder.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_encoder.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_encoder.c
new file mode 100644
index 0000000..3cfa01c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_xor_encoder.c
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <jni.h>
+
+#include "org_apache_hadoop.h"
+#include "erasure_code.h"
+#include "gf_util.h"
+#include "jni_common.h"
+#include "org_apache_hadoop_io_erasurecode_rawcoder_NativeXORRawEncoder.h"
+
+typedef struct _XOREncoder {
+  IsalCoder isalCoder;
+  unsigned char* inputs[MMAX];
+  unsigned char* outputs[1];
+} XOREncoder;
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeXORRawEncoder_initImpl
+  (JNIEnv *env, jobject thiz, jint numDataUnits, jint numParityUnits) {
+  XOREncoder* xorEncoder =
+                           (XOREncoder*)malloc(sizeof(XOREncoder));
+  memset(xorEncoder, 0, sizeof(*xorEncoder));
+  initCoder(&xorEncoder->isalCoder, numDataUnits, numParityUnits);
+
+  setCoder(env, thiz, &xorEncoder->isalCoder);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeXORRawEncoder_encodeImpl(
+  JNIEnv *env, jobject thiz, jobjectArray inputs, jintArray inputOffsets,
+  jint dataLen, jobjectArray outputs, jintArray outputOffsets) {
+
+  int i, j, numDataUnits, numParityUnits, chunkSize;
+  XOREncoder* xorEncoder;
+
+  xorEncoder = (XOREncoder*)getCoder(env, thiz);
+  numDataUnits = ((IsalCoder*)xorEncoder)->numDataUnits;
+  numParityUnits = ((IsalCoder*)xorEncoder)->numParityUnits;
+  chunkSize = (int)dataLen;
+
+  getInputs(env, inputs, inputOffsets, xorEncoder->inputs, numDataUnits);
+  getOutputs(env, outputs, outputOffsets, xorEncoder->outputs, numParityUnits);
+
+  // Get the first buffer's data.
+  for (j = 0; j < chunkSize; j++) {
+    xorEncoder->outputs[0][j] = xorEncoder->inputs[0][j];
+  }
+
+  // XOR with everything else.
+  for (i = 1; i < numDataUnits; i++) {
+    for (j = 0; j < chunkSize; j++) {
+      xorEncoder->outputs[0][j] ^= xorEncoder->inputs[i][j];
+    }
+  }
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeXORRawEncoder_destroyImpl
+  (JNIEnv *env, jobject thiz) {
+  XOREncoder* xorEncoder = (XOREncoder*)getCoder(env, thiz);
+  free(xorEncoder);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestNativeXORRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestNativeXORRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestNativeXORRawCoder.java
new file mode 100644
index 0000000..ba4b7b4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestNativeXORRawCoder.java
@@ -0,0 +1,36 @@
+/**
+ * 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.ErasureCodeNative;
+import org.junit.Assume;
+import org.junit.Before;
+
+/**
+ * Test NativeXOR encoding and decoding.
+ */
+public class TestNativeXORRawCoder extends TestXORRawCoderBase {
+
+  @Before
+  public void setup() {
+    Assume.assumeTrue(ErasureCodeNative.isNativeCodeLoaded());
+    this.encoderClass = NativeXORRawEncoder.class;
+    this.decoderClass = NativeXORRawDecoder.class;
+    setAllowDump(true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/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
index 48463ad..aae3a04 100644
--- 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
@@ -18,49 +18,15 @@
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
 import org.junit.Before;
-import org.junit.Test;
 
 /**
- * Test XOR encoding and decoding.
+ * Test pure Java XOR encoding and decoding.
  */
-public class TestXORRawCoder extends TestRawCoderBase {
+public class TestXORRawCoder extends TestXORRawCoderBase {
 
   @Before
   public void setup() {
     this.encoderClass = XORRawEncoder.class;
     this.decoderClass = XORRawDecoder.class;
   }
-
-  @Test
-  public void testCoding_10x1_erasing_d0() {
-    prepare(null, 10, 1, new int[] {0}, new int[0]);
-    testCodingDoMixAndTwice();
-  }
-
-  @Test
-  public void testCoding_10x1_erasing_p0() {
-    prepare(null, 10, 1, new int[0], new int[] {0});
-    testCodingDoMixAndTwice();
-  }
-
-  @Test
-  public void testCoding_10x1_erasing_d5() {
-    prepare(null, 10, 1, new int[]{5}, new int[0]);
-    testCodingDoMixAndTwice();
-  }
-
-  @Test
-  public void testCodingNegative_10x1_erasing_too_many() {
-    prepare(null, 10, 1, new int[]{2}, new int[]{0});
-    testCodingWithErasingTooMany();
-  }
-
-  @Test
-  public void testCodingNegative_10x1_erasing_d5() {
-    prepare(null, 10, 1, new int[]{5}, new int[0]);
-    testCodingWithBadInput(true);
-    testCodingWithBadOutput(false);
-    testCodingWithBadInput(true);
-    testCodingWithBadOutput(false);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderBase.java
new file mode 100644
index 0000000..27de379
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderBase.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.junit.Test;
+
+/**
+ * Test base for raw XOR coders.
+ */
+public abstract class TestXORRawCoderBase extends TestRawCoderBase {
+
+  @Test
+  public void testCoding_10x1_erasing_d0() {
+    prepare(null, 10, 1, new int[] {0}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_10x1_erasing_p0() {
+    prepare(null, 10, 1, new int[0], new int[] {0});
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCoding_10x1_erasing_d5() {
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
+    testCodingDoMixAndTwice();
+  }
+
+  @Test
+  public void testCodingNegative_10x1_erasing_too_many() {
+    prepare(null, 10, 1, new int[]{2}, new int[]{0});
+    testCodingWithErasingTooMany();
+  }
+
+  @Test
+  public void testCodingNegative_10x1_erasing_d5() {
+    prepare(null, 10, 1, new int[]{5}, new int[0]);
+    testCodingWithBadInput(true);
+    testCodingWithBadOutput(false);
+    testCodingWithBadInput(true);
+    testCodingWithBadOutput(false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderInteroperable1.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderInteroperable1.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderInteroperable1.java
new file mode 100644
index 0000000..c5a809c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderInteroperable1.java
@@ -0,0 +1,36 @@
+/**
+ * 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.ErasureCodeNative;
+import org.junit.Assume;
+import org.junit.Before;
+
+/**
+ * Test raw XOR coder implemented in Java.
+ */
+public class TestXORRawCoderInteroperable1 extends TestXORRawCoderBase {
+
+  @Before
+  public void setup() {
+    Assume.assumeTrue(ErasureCodeNative.isNativeCodeLoaded());
+    this.encoderClass = XORRawEncoder.class;
+    this.decoderClass = NativeXORRawDecoder.class;
+    setAllowDump(true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d88dca84/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderInteroperable2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderInteroperable2.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderInteroperable2.java
new file mode 100644
index 0000000..5345eb6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoderInteroperable2.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.ErasureCodeNative;
+import org.junit.Assume;
+import org.junit.Before;
+
+/**
+ * Test raw XOR coder implemented in Java.
+ */
+public class TestXORRawCoderInteroperable2 extends TestXORRawCoderBase {
+
+  @Before
+  public void setup() {
+    Assume.assumeTrue(ErasureCodeNative.isNativeCodeLoaded());
+    this.encoderClass = NativeXORRawEncoder.class;
+    this.decoderClass = XORRawDecoder.class;
+    setAllowDump(true);
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org