Return-Path: X-Original-To: apmail-hadoop-common-commits-archive@www.apache.org Delivered-To: apmail-hadoop-common-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 53CC417DE7 for ; Mon, 6 Apr 2015 17:39:09 +0000 (UTC) Received: (qmail 72560 invoked by uid 500); 6 Apr 2015 17:38:16 -0000 Delivered-To: apmail-hadoop-common-commits-archive@hadoop.apache.org Received: (qmail 72249 invoked by uid 500); 6 Apr 2015 17:38:16 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: common-dev@hadoop.apache.org Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 71690 invoked by uid 99); 6 Apr 2015 17:38:15 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 06 Apr 2015 17:38:15 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 3C754E182E; Mon, 6 Apr 2015 17:38:01 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: zhz@apache.org To: common-commits@hadoop.apache.org Date: Mon, 06 Apr 2015 17:38:15 -0000 Message-Id: <6beadcbff13e4d23803d59d66241f6b0@git.apache.org> In-Reply-To: <374e05a227944c769d59d36f99a37f26@git.apache.org> References: <374e05a227944c769d59d36f99a37f26@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [15/51] [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/7175f1cc Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7175f1cc Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7175f1cc Branch: refs/heads/HDFS-7285 Commit: 7175f1cc0b9c8eb88cf7b0285c127e8c11045245 Parents: dd5fd7b Author: Uma Maheswara Rao G Authored: Thu Jan 29 14:15:13 2015 +0530 Committer: Zhe Zhang Committed: Mon Apr 6 10:01:05 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/7175f1cc/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/7175f1cc/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/7175f1cc/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/7175f1cc/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/7175f1cc/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/7175f1cc/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/7175f1cc/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/7175f1cc/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); + +}