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

[23/50] [abbrv] hadoop git commit: HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding (Kai Zheng via umamahesh)

HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding (Kai Zheng via umamahesh)


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

Branch: refs/heads/HDFS-7285
Commit: 257d22d00c41103101a5a0ea539c7922b9ea7b80
Parents: e892417
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Thu Jan 29 14:15:13 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 23 11:06:44 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  4 +
 .../apache/hadoop/io/erasurecode/ECChunk.java   | 82 +++++++++++++++++
 .../rawcoder/AbstractRawErasureCoder.java       | 63 +++++++++++++
 .../rawcoder/AbstractRawErasureDecoder.java     | 93 ++++++++++++++++++++
 .../rawcoder/AbstractRawErasureEncoder.java     | 93 ++++++++++++++++++++
 .../erasurecode/rawcoder/RawErasureCoder.java   | 78 ++++++++++++++++
 .../erasurecode/rawcoder/RawErasureDecoder.java | 55 ++++++++++++
 .../erasurecode/rawcoder/RawErasureEncoder.java | 54 ++++++++++++
 8 files changed, 522 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/257d22d0/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
new file mode 100644
index 0000000..8ce5a89
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -0,0 +1,4 @@
+  BREAKDOWN OF HADOOP-11264 SUBTASKS AND RELATED JIRAS (Common part of HDFS-7285)
+
+    HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding
+    (Kai Zheng via umamahesh)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/257d22d0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
new file mode 100644
index 0000000..f84eb11
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A wrapper for ByteBuffer or bytes array for an erasure code chunk.
+ */
+public class ECChunk {
+
+  private ByteBuffer chunkBuffer;
+
+  /**
+   * Wrapping a ByteBuffer
+   * @param buffer
+   */
+  public ECChunk(ByteBuffer buffer) {
+    this.chunkBuffer = buffer;
+  }
+
+  /**
+   * Wrapping a bytes array
+   * @param buffer
+   */
+  public ECChunk(byte[] buffer) {
+    this.chunkBuffer = ByteBuffer.wrap(buffer);
+  }
+
+  /**
+   * Convert to ByteBuffer
+   * @return ByteBuffer
+   */
+  public ByteBuffer getBuffer() {
+    return chunkBuffer;
+  }
+
+  /**
+   * Convert an array of this chunks to an array of ByteBuffers
+   * @param chunks
+   * @return an array of ByteBuffers
+   */
+  public static ByteBuffer[] toBuffers(ECChunk[] chunks) {
+    ByteBuffer[] buffers = new ByteBuffer[chunks.length];
+
+    for (int i = 0; i < chunks.length; i++) {
+      buffers[i] = chunks[i].getBuffer();
+    }
+
+    return buffers;
+  }
+
+  /**
+   * Convert an array of this chunks to an array of byte array
+   * @param chunks
+   * @return an array of byte array
+   */
+  public static byte[][] toArray(ECChunk[] chunks) {
+    byte[][] bytesArr = new byte[chunks.length][];
+
+    for (int i = 0; i < chunks.length; i++) {
+      bytesArr[i] = chunks[i].getBuffer().array();
+    }
+
+    return bytesArr;
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/257d22d0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java
new file mode 100644
index 0000000..4613b25
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.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 java.nio.ByteBuffer;
+
+/**
+ * An abstract raw erasure decoder that's to be inherited by new decoders.
+ *
+ * It implements the {@link RawErasureDecoder} interface.
+ */
+public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder
+    implements RawErasureDecoder {
+
+  @Override
+  public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
+                     ByteBuffer[] outputs) {
+    if (erasedIndexes.length == 0) {
+      return;
+    }
+
+    doDecode(inputs, erasedIndexes, outputs);
+  }
+
+  /**
+   * Perform the real decoding using ByteBuffer
+   * @param inputs
+   * @param erasedIndexes
+   * @param outputs
+   */
+  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;
+    }
+
+    doDecode(inputs, erasedIndexes, outputs);
+  }
+
+  /**
+   * Perform the real decoding using bytes array
+   * @param inputs
+   * @param erasedIndexes
+   * @param outputs
+   */
+  protected abstract void doDecode(byte[][] inputs, int[] erasedIndexes,
+                                   byte[][] outputs);
+
+  @Override
+  public void decode(ECChunk[] inputs, int[] erasedIndexes,
+                     ECChunk[] outputs) {
+    doDecode(inputs, erasedIndexes, outputs);
+  }
+
+  /**
+   * Perform the real decoding using chunks
+   * @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);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/257d22d0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java
new file mode 100644
index 0000000..4feaf39
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.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 java.nio.ByteBuffer;
+
+/**
+ * An abstract raw erasure encoder that's to be inherited by new encoders.
+ *
+ * It implements the {@link RawErasureEncoder} interface.
+ */
+public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder
+    implements RawErasureEncoder {
+
+  @Override
+  public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+    assert (inputs.length == getNumDataUnits());
+    assert (outputs.length == getNumParityUnits());
+
+    doEncode(inputs, outputs);
+  }
+
+  /**
+   * Perform the real encoding work using ByteBuffer
+   * @param inputs
+   * @param outputs
+   */
+  protected abstract void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs);
+
+  @Override
+  public void encode(byte[][] inputs, byte[][] outputs) {
+    assert (inputs.length == getNumDataUnits());
+    assert (outputs.length == getNumParityUnits());
+
+    doEncode(inputs, outputs);
+  }
+
+  /**
+   * Perform the real encoding work using bytes array
+   * @param inputs
+   * @param outputs
+   */
+  protected abstract void doEncode(byte[][] inputs, byte[][] outputs);
+
+  @Override
+  public void encode(ECChunk[] inputs, ECChunk[] outputs) {
+    assert (inputs.length == getNumDataUnits());
+    assert (outputs.length == getNumParityUnits());
+
+    doEncode(inputs, outputs);
+  }
+
+  /**
+   * Perform the real encoding work using chunks.
+   * @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);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/257d22d0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
new file mode 100644
index 0000000..91a9abf
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.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;
+
+/**
+ * RawErasureCoder is a common interface for {@link RawErasureEncoder} and
+ * {@link RawErasureDecoder} as both encoder and decoder share some properties.
+ *
+ * RawErasureCoder is part of ErasureCodec framework, where ErasureCoder is
+ * used to encode/decode a group of blocks (BlockGroup) according to the codec
+ * specific BlockGroup layout and logic. An ErasureCoder extracts chunks of
+ * data from the blocks and can employ various low level RawErasureCoders to
+ * perform encoding/decoding against the chunks.
+ *
+ * To distinguish from ErasureCoder, here RawErasureCoder is used to mean the
+ * low level constructs, since it only takes care of the math calculation with
+ * a group of byte buffers.
+ */
+public interface RawErasureCoder {
+
+  /**
+   * Initialize with the important parameters for the code.
+   * @param numDataUnits how many data inputs for the coding
+   * @param numParityUnits how many parity outputs the coding generates
+   * @param chunkSize the size of the input/output buffer
+   */
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize);
+
+  /**
+   * The number of data input units for the coding. A unit can be a byte,
+   * chunk or buffer or even a block.
+   * @return count of data input units
+   */
+  public int getNumDataUnits();
+
+  /**
+   * The number of parity output units for the coding. A unit can be a byte,
+   * chunk, buffer or even a block.
+   * @return count of parity output units
+   */
+  public int getNumParityUnits();
+
+  /**
+   * Chunk buffer size for the input/output
+   * @return chunk buffer size
+   */
+  public int getChunkSize();
+
+  /**
+   * Tell if native or off-heap buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, either on heap or off heap.
+   * It will return false by default.
+   * @return true if native buffer is preferred for performance consideration,
+   * otherwise false.
+   */
+  public boolean preferNativeBuffer();
+
+  /**
+   * Should be called when release this coder. Good chance to release encoding
+   * or decoding buffers
+   */
+  public void release();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/257d22d0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
new file mode 100644
index 0000000..1358b7d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.ECChunk;
+
+import java.nio.ByteBuffer;
+
+/**
+ * RawErasureDecoder performs decoding given chunks of input data and generates
+ * missing data that corresponds to an erasure code scheme, like XOR and
+ * Reed-Solomon.
+ *
+ * It extends the {@link RawErasureCoder} interface.
+ */
+public interface RawErasureDecoder extends RawErasureCoder {
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void decode(ByteBuffer[] inputs, int[] erasedIndexes,
+                     ByteBuffer[] outputs);
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs);
+
+  /**
+   * Decode with inputs and erasedIndexes, generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/257d22d0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java
new file mode 100644
index 0000000..974f86c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java
@@ -0,0 +1,54 @@
+/**
+ * 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 java.nio.ByteBuffer;
+
+/**
+ * RawErasureEncoder performs encoding given chunks of input data and generates
+ * parity outputs that corresponds to an erasure code scheme, like XOR and
+ * Reed-Solomon.
+ *
+ * It extends the {@link RawErasureCoder} interface.
+ */
+public interface RawErasureEncoder extends RawErasureCoder {
+
+  /**
+   * Encode with inputs and generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs);
+
+  /**
+   * Encode with inputs and generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void encode(byte[][] inputs, byte[][] outputs);
+
+  /**
+   * Encode with inputs and generates outputs
+   * @param inputs
+   * @param outputs
+   */
+  public void encode(ECChunk[] inputs, ECChunk[] outputs);
+
+}