HADOOP-11646. Erasure Coder API for encoding and decoding of block group ( Contributed by Kai Zheng )

This commit is contained in:
Vinayakumar B 2015-03-09 12:32:26 +05:30 committed by Zhe Zhang
parent 39a0a85fb7
commit 8f89d7489d
18 changed files with 1241 additions and 1 deletions

View File

@ -15,4 +15,6 @@
HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng
( Kai Zheng )
HADOOP-11646. Erasure Coder API for encoding and decoding of block group
( Kai Zheng via vinayakumarb )

View File

@ -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.
*/
package org.apache.hadoop.io.erasurecode;
/**
* A wrapper of block level data source/output that {@link ECChunk}s can be
* extracted from. For HDFS, it can be an HDFS block (250MB). Note it only cares
* about erasure coding specific logic thus avoids coupling with any HDFS block
* details. We can have something like HdfsBlock extend it.
*/
public class ECBlock {
private boolean isParity;
private boolean isErased;
/**
* A default constructor. isParity and isErased are false by default.
*/
public ECBlock() {
this(false, false);
}
/**
* A constructor specifying isParity and isErased.
* @param isParity
* @param isErased
*/
public ECBlock(boolean isParity, boolean isErased) {
this.isParity = isParity;
this.isErased = isErased;
}
/**
* Set true if it's for a parity block.
* @param isParity
*/
public void setParity(boolean isParity) {
this.isParity = isParity;
}
/**
* Set true if the block is missing.
* @param isMissing
*/
public void setErased(boolean isMissing) {
this.isErased = isMissing;
}
/**
*
* @return true if it's parity block, otherwise false
*/
public boolean isParity() {
return isParity;
}
/**
*
* @return true if it's missing or corrupt due to erasure, otherwise false
*/
public boolean isErased() {
return isErased;
}
}

View File

@ -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;
/**
* A group of blocks or {@link ECBlock} incurred in an erasure coding task.
*/
public class ECBlockGroup {
private ECBlock[] dataBlocks;
private ECBlock[] parityBlocks;
/**
* A constructor specifying data blocks and parity blocks.
* @param dataBlocks
* @param parityBlocks
*/
public ECBlockGroup(ECBlock[] dataBlocks, ECBlock[] parityBlocks) {
this.dataBlocks = dataBlocks;
this.parityBlocks = parityBlocks;
}
/**
* Get data blocks
* @return data blocks
*/
public ECBlock[] getDataBlocks() {
return dataBlocks;
}
/**
* Get parity blocks
* @return parity blocks
*/
public ECBlock[] getParityBlocks() {
return parityBlocks;
}
/**
* Any erased data block?
* @return true if any erased data block, false otherwise
*/
public boolean anyErasedDataBlock() {
for (int i = 0; i < dataBlocks.length; ++i) {
if (dataBlocks[i].isErased()) {
return true;
}
}
return false;
}
/**
* Any erased parity block?
* @return true if any erased parity block, false otherwise
*/
public boolean anyErasedParityBlock() {
for (int i = 0; i < parityBlocks.length; ++i) {
if (parityBlocks[i].isErased()) {
return true;
}
}
return false;
}
}

View File

@ -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.coder;
/**
* A common class of basic facilities to be shared by encoder and decoder
*
* It implements the {@link ErasureCoder} interface.
*/
public abstract class AbstractErasureCoder implements ErasureCoder {
private int numDataUnits;
private int numParityUnits;
private int chunkSize;
@Override
public void initialize(int numDataUnits, int numParityUnits,
int chunkSize) {
this.numDataUnits = numDataUnits;
this.numParityUnits = numParityUnits;
this.chunkSize = chunkSize;
}
@Override
public int getNumDataUnits() {
return numDataUnits;
}
@Override
public int getNumParityUnits() {
return numParityUnits;
}
@Override
public int getChunkSize() {
return chunkSize;
}
@Override
public boolean preferNativeBuffer() {
return false;
}
@Override
public void release() {
// Nothing to do by default
}
}

View File

@ -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.coder;
import org.apache.hadoop.io.erasurecode.ECBlock;
/**
* Abstract class for common facilities shared by {@link ErasureEncodingStep}
* and {@link ErasureDecodingStep}.
*
* It implements {@link ErasureEncodingStep}.
*/
public abstract class AbstractErasureCodingStep implements ErasureCodingStep {
private ECBlock[] inputBlocks;
private ECBlock[] outputBlocks;
/**
* Constructor given input blocks and output blocks.
* @param inputBlocks
* @param outputBlocks
*/
public AbstractErasureCodingStep(ECBlock[] inputBlocks,
ECBlock[] outputBlocks) {
this.inputBlocks = inputBlocks;
this.outputBlocks = outputBlocks;
}
@Override
public ECBlock[] getInputBlocks() {
return inputBlocks;
}
@Override
public ECBlock[] getOutputBlocks() {
return outputBlocks;
}
@Override
public void finish() {
// NOOP by default
}
}

View File

@ -0,0 +1,152 @@
/**
* 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.coder;
import org.apache.hadoop.io.erasurecode.ECBlock;
import org.apache.hadoop.io.erasurecode.ECBlockGroup;
/**
* An abstract erasure decoder that's to be inherited by new decoders.
*
* It implements the {@link ErasureDecoder} interface.
*/
public abstract class AbstractErasureDecoder extends AbstractErasureCoder
implements ErasureDecoder {
@Override
public ErasureCodingStep decode(ECBlockGroup blockGroup) {
return performDecoding(blockGroup);
}
/**
* Perform decoding against a block blockGroup.
* @param blockGroup
* @return decoding step for caller to do the real work
*/
protected abstract ErasureCodingStep performDecoding(ECBlockGroup blockGroup);
/**
* We have all the data blocks and parity blocks as input blocks for
* recovering by default. It's codec specific
* @param blockGroup
* @return
*/
protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
ECBlock[] inputBlocks = new ECBlock[getNumParityUnits()
+ getNumDataUnits()];
System.arraycopy(blockGroup.getParityBlocks(), 0, inputBlocks, 0,
getNumParityUnits());
System.arraycopy(blockGroup.getDataBlocks(), 0, inputBlocks,
getNumParityUnits(), getNumDataUnits());
return inputBlocks;
}
/**
* Which blocks were erased ? We only care data blocks here. Sub-classes can
* override this behavior.
* @param blockGroup
* @return output blocks to recover
*/
protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
ECBlock[] outputBlocks = new ECBlock[
getNumErasedBlocks(blockGroup.getDataBlocks())];
int idx = 0;
for (int i = 0; i < getNumDataUnits(); i++) {
if (blockGroup.getDataBlocks()[i].isErased()) {
outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
}
}
return outputBlocks;
}
/**
* Get the number of erased blocks in the block group.
* @param blockGroup
* @return number of erased blocks
*/
protected int getNumErasedBlocks(ECBlockGroup blockGroup) {
int num = getNumErasedBlocks(blockGroup.getParityBlocks());
num += getNumErasedBlocks(blockGroup.getDataBlocks());
return num;
}
/**
* Find out how many blocks are erased.
* @param inputBlocks all the input blocks
* @return number of erased blocks
*/
protected static int getNumErasedBlocks(ECBlock[] inputBlocks) {
int numErased = 0;
for (int i = 0; i < inputBlocks.length; i++) {
if (inputBlocks[i].isErased()) {
numErased ++;
}
}
return numErased;
}
/**
* Get indexes of erased blocks from inputBlocks
* @param inputBlocks
* @return indexes of erased blocks from inputBlocks
*/
protected int[] getErasedIndexes(ECBlock[] inputBlocks) {
int numErased = getNumErasedBlocks(inputBlocks);
if (numErased == 0) {
return new int[0];
}
int[] erasedIndexes = new int[numErased];
int i = 0, j = 0;
for (; i < inputBlocks.length && j < erasedIndexes.length; i++) {
if (inputBlocks[i].isErased()) {
erasedIndexes[j++] = i;
}
}
return erasedIndexes;
}
/**
* Get erased input blocks from inputBlocks
* @param inputBlocks
* @return an array of erased blocks from inputBlocks
*/
protected ECBlock[] getErasedBlocks(ECBlock[] inputBlocks) {
int numErased = getNumErasedBlocks(inputBlocks);
if (numErased == 0) {
return new ECBlock[0];
}
ECBlock[] erasedBlocks = new ECBlock[numErased];
int i = 0, j = 0;
for (; i < inputBlocks.length && j < erasedBlocks.length; i++) {
if (inputBlocks[i].isErased()) {
erasedBlocks[j++] = inputBlocks[i];
}
}
return erasedBlocks;
}
}

View File

@ -0,0 +1,50 @@
/**
* 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.coder;
import org.apache.hadoop.io.erasurecode.ECBlock;
import org.apache.hadoop.io.erasurecode.ECBlockGroup;
/**
* An abstract erasure encoder that's to be inherited by new encoders.
*
* It implements the {@link ErasureEncoder} interface.
*/
public abstract class AbstractErasureEncoder extends AbstractErasureCoder
implements ErasureEncoder {
@Override
public ErasureCodingStep encode(ECBlockGroup blockGroup) {
return performEncoding(blockGroup);
}
/**
* Perform encoding against a block group.
* @param blockGroup
* @return encoding step for caller to do the real work
*/
protected abstract ErasureCodingStep performEncoding(ECBlockGroup blockGroup);
protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
return blockGroup.getDataBlocks();
}
protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
return blockGroup.getParityBlocks();
}
}

View File

@ -0,0 +1,77 @@
/**
* 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.coder;
/**
* An erasure coder to perform encoding or decoding given a group. Generally it
* involves calculating necessary internal steps according to codec logic. For
* each step,it calculates necessary input blocks to read chunks from and output
* parity blocks to write parity chunks into from the group. It also takes care
* of appropriate raw coder to use for the step. And encapsulates all the
* necessary info (input blocks, output blocks and raw coder) into a step
* represented by {@link ErasureCodingStep}. ErasureCoder callers can use the
* step to do the real work with retrieved input and output chunks.
*
* Note, currently only one coding step is supported. Will support complex cases
* of multiple coding steps.
*
*/
public interface ErasureCoder {
/**
* 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();
/**
* Release the resources if any. Good chance to invoke RawErasureCoder#release.
*/
public void release();
}

View File

@ -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.coder;
import org.apache.hadoop.io.erasurecode.ECBlock;
import org.apache.hadoop.io.erasurecode.ECChunk;
/**
* Erasure coding step that's involved in encoding/decoding of a block group.
*/
public interface ErasureCodingStep {
/**
* Input blocks of readable data involved in this step, may be data blocks
* or parity blocks.
* @return input blocks
*/
public ECBlock[] getInputBlocks();
/**
* Output blocks of writable buffers involved in this step, may be data
* blocks or parity blocks.
* @return output blocks
*/
public ECBlock[] getOutputBlocks();
/**
* Perform encoding or decoding given the input chunks, and generated results
* will be written to the output chunks.
* @param inputChunks
* @param outputChunks
*/
public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks);
/**
* Notify erasure coder that all the chunks of input blocks are processed so
* the coder can be able to update internal states, considering next step.
*/
public void finish();
}

View File

@ -0,0 +1,41 @@
/**
* 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.coder;
import org.apache.hadoop.io.erasurecode.ECBlockGroup;
/**
* Erasure decoder interface to perform decoding given a block group.
*
* It extends {@link ErasureCoder}.
*/
public interface ErasureDecoder extends ErasureCoder {
/**
* Perform the decoding given a blockGroup. By default it will try the best to
* attempt to recover all the missing blocks according to the codec logic.
*
* Note, currently only one coding step is supported. Will support complex
* cases of multiple coding steps.
*
* @param blockGroup
*/
public ErasureCodingStep decode(ECBlockGroup blockGroup);
}

View File

@ -0,0 +1,52 @@
/**
* 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.coder;
import org.apache.hadoop.io.erasurecode.ECBlock;
import org.apache.hadoop.io.erasurecode.ECChunk;
import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
/**
* Erasure decoding step, a wrapper of all the necessary information to perform
* a decoding step involved in the whole process of decoding a block group.
*/
public class ErasureDecodingStep extends AbstractErasureCodingStep {
private int[] erasedIndexes;
private RawErasureDecoder rawDecoder;
/**
* The constructor with all the necessary info.
* @param inputBlocks
* @param erasedIndexes the indexes of erased blocks in inputBlocks array
* @param outputBlocks
* @param rawDecoder
*/
public ErasureDecodingStep(ECBlock[] inputBlocks, int[] erasedIndexes,
ECBlock[] outputBlocks,
RawErasureDecoder rawDecoder) {
super(inputBlocks, outputBlocks);
this.erasedIndexes = erasedIndexes;
this.rawDecoder = rawDecoder;
}
@Override
public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) {
rawDecoder.decode(inputChunks, erasedIndexes, outputChunks);
}
}

View File

@ -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.coder;
import org.apache.hadoop.io.erasurecode.ECBlockGroup;
/**
* Erasure encoder interface to perform encoding given a block group.
*
* It extends {@link ErasureCoder}.
*/
public interface ErasureEncoder extends ErasureCoder {
/**
* Calculate the encoding steps given a block blockGroup.
*
* Note, currently only one coding step is supported. Will support complex
* cases of multiple coding steps.
*
* @param blockGroup
*/
public ErasureCodingStep encode(ECBlockGroup blockGroup);
}

View File

@ -0,0 +1,49 @@
/**
* 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.coder;
import org.apache.hadoop.io.erasurecode.ECBlock;
import org.apache.hadoop.io.erasurecode.ECChunk;
import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
/**
* Erasure encoding step, a wrapper of all the necessary information to perform
* an encoding step involved in the whole process of encoding a block group.
*/
public class ErasureEncodingStep extends AbstractErasureCodingStep {
private RawErasureEncoder rawEncoder;
/**
* The constructor with all the necessary info.
* @param inputBlocks
* @param outputBlocks
* @param rawEncoder
*/
public ErasureEncodingStep(ECBlock[] inputBlocks, ECBlock[] outputBlocks,
RawErasureEncoder rawEncoder) {
super(inputBlocks, outputBlocks);
this.rawEncoder = rawEncoder;
}
@Override
public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) {
rawEncoder.encode(inputChunks, outputChunks);
}
}

View File

@ -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.coder;
import org.apache.hadoop.io.erasurecode.ECBlock;
import org.apache.hadoop.io.erasurecode.ECBlockGroup;
import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
/**
* Xor erasure decoder that decodes a block group.
*
* It implements {@link ErasureDecoder}.
*/
public class XorErasureDecoder extends AbstractErasureDecoder {
@Override
protected ErasureCodingStep performDecoding(final ECBlockGroup blockGroup) {
// May be configured
RawErasureDecoder rawDecoder = new XorRawDecoder();
rawDecoder.initialize(getNumDataUnits(),
getNumParityUnits(), getChunkSize());
ECBlock[] inputBlocks = getInputBlocks(blockGroup);
return new ErasureDecodingStep(inputBlocks,
getErasedIndexes(inputBlocks),
getOutputBlocks(blockGroup), rawDecoder);
}
/**
* Which blocks were erased ? For XOR it's simple we only allow and return one
* erased block, either data or parity.
* @param blockGroup
* @return output blocks to recover
*/
@Override
protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
/**
* If more than one blocks (either data or parity) erased, then it's not
* edible to recover. We don't have the check here since it will be done
* by upper level: ErasreCoder call can be avoid if not possible to recover
* at all.
*/
int erasedNum = getNumErasedBlocks(blockGroup);
ECBlock[] outputBlocks = new ECBlock[erasedNum];
int idx = 0;
for (int i = 0; i < getNumParityUnits(); i++) {
if (blockGroup.getParityBlocks()[i].isErased()) {
outputBlocks[idx++] = blockGroup.getParityBlocks()[i];
}
}
for (int i = 0; i < getNumDataUnits(); i++) {
if (blockGroup.getDataBlocks()[i].isErased()) {
outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
}
}
return outputBlocks;
}
}

View File

@ -0,0 +1,45 @@
/**
* 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.coder;
import org.apache.hadoop.io.erasurecode.ECBlock;
import org.apache.hadoop.io.erasurecode.ECBlockGroup;
import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
import org.apache.hadoop.io.erasurecode.rawcoder.XorRawEncoder;
/**
* Xor erasure encoder that encodes a block group.
*
* It implements {@link ErasureEncoder}.
*/
public class XorErasureEncoder extends AbstractErasureEncoder {
@Override
protected ErasureCodingStep performEncoding(final ECBlockGroup blockGroup) {
// May be configured
RawErasureEncoder rawEncoder = new XorRawEncoder();
rawEncoder.initialize(getNumDataUnits(),
getNumParityUnits(), getChunkSize());
ECBlock[] inputBlocks = getInputBlocks(blockGroup);
return new ErasureEncodingStep(inputBlocks,
getOutputBlocks(blockGroup), rawEncoder);
}
}

View File

@ -71,7 +71,7 @@ public interface RawErasureCoder {
public boolean preferNativeBuffer();
/**
* Should be called when release this blockcoder. Good chance to release encoding
* Should be called when release this coder. Good chance to release encoding
* or decoding buffers
*/
public void release();

View File

@ -0,0 +1,266 @@
/**
* 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.coder;
import org.apache.hadoop.io.erasurecode.ECBlock;
import org.apache.hadoop.io.erasurecode.ECChunk;
import org.apache.hadoop.io.erasurecode.ECBlockGroup;
import org.apache.hadoop.io.erasurecode.TestCoderBase;
/**
* Erasure coder test base with utilities.
*/
public abstract class TestErasureCoderBase extends TestCoderBase {
protected Class<? extends ErasureEncoder> encoderClass;
protected Class<? extends ErasureDecoder> decoderClass;
protected int numChunksInBlock = 16;
/**
* It's just a block for this test purpose. We don't use HDFS block here
* at all for simple.
*/
protected static class TestBlock extends ECBlock {
private ECChunk[] chunks;
// For simple, just assume the block have the chunks already ready.
// In practice we need to read/write chunks from/to the block via file IO.
public TestBlock(ECChunk[] chunks) {
this.chunks = chunks;
}
}
/**
* Generating source data, encoding, recovering and then verifying.
* RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
* it supports two kinds of ByteBuffers, one is array backed, the other is
* direct ByteBuffer. Have usingDirectBuffer to indicate which case to test.
* @param usingDirectBuffer
*/
protected void testCoding(boolean usingDirectBuffer) {
this.usingDirectBuffer = usingDirectBuffer;
ErasureEncoder encoder = createEncoder();
// Generate data and encode
ECBlockGroup blockGroup = prepareBlockGroupForEncoding();
// Backup all the source chunks for later recovering because some coders
// may affect the source data.
TestBlock[] clonedDataBlocks = cloneBlocksWithData((TestBlock[])
blockGroup.getDataBlocks());
// Make a copy of a strip for later comparing
TestBlock[] toEraseBlocks = copyDataBlocksToErase(clonedDataBlocks);
ErasureCodingStep codingStep = encoder.encode(blockGroup);
performCodingStep(codingStep);
// Erase the copied sources
eraseSomeDataBlocks(clonedDataBlocks);
//Decode
blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
ErasureDecoder decoder = createDecoder();
codingStep = decoder.decode(blockGroup);
performCodingStep(codingStep);
//Compare
compareAndVerify(toEraseBlocks, codingStep.getOutputBlocks());
}
/**
* This is typically how a coding step should be performed.
* @param codingStep
*/
private void performCodingStep(ErasureCodingStep codingStep) {
// Pretend that we're opening these input blocks and output blocks.
ECBlock[] inputBlocks = codingStep.getInputBlocks();
ECBlock[] outputBlocks = codingStep.getOutputBlocks();
// We allocate input and output chunks accordingly.
ECChunk[] inputChunks = new ECChunk[inputBlocks.length];
ECChunk[] outputChunks = new ECChunk[outputBlocks.length];
for (int i = 0; i < numChunksInBlock; ++i) {
// Pretend that we're reading input chunks from input blocks.
for (int j = 0; j < inputBlocks.length; ++j) {
inputChunks[j] = ((TestBlock) inputBlocks[j]).chunks[i];
}
// Pretend that we allocate and will write output results to the blocks.
for (int j = 0; j < outputBlocks.length; ++j) {
outputChunks[j] = allocateOutputChunk();
((TestBlock) outputBlocks[j]).chunks[i] = outputChunks[j];
}
// Given the input chunks and output chunk buffers, just call it !
codingStep.performCoding(inputChunks, outputChunks);
}
codingStep.finish();
}
/**
* Compare and verify if recovered blocks data are the same with the erased
* blocks data.
* @param erasedBlocks
* @param recoveredBlocks
*/
protected void compareAndVerify(ECBlock[] erasedBlocks,
ECBlock[] recoveredBlocks) {
for (int i = 0; i < erasedBlocks.length; ++i) {
compareAndVerify(((TestBlock) erasedBlocks[i]).chunks,
((TestBlock) recoveredBlocks[i]).chunks);
}
}
/**
* Create erasure encoder for test.
* @return
*/
private ErasureEncoder createEncoder() {
ErasureEncoder encoder;
try {
encoder = encoderClass.newInstance();
} catch (Exception e) {
throw new RuntimeException("Failed to create encoder", e);
}
encoder.initialize(numDataUnits, numParityUnits, chunkSize);
return encoder;
}
/**
* Create the erasure decoder for the test.
* @return
*/
private ErasureDecoder createDecoder() {
ErasureDecoder decoder;
try {
decoder = decoderClass.newInstance();
} catch (Exception e) {
throw new RuntimeException("Failed to create decoder", e);
}
decoder.initialize(numDataUnits, numParityUnits, chunkSize);
return decoder;
}
/**
* Prepare a block group for encoding.
* @return
*/
protected ECBlockGroup prepareBlockGroupForEncoding() {
ECBlock[] dataBlocks = new TestBlock[numDataUnits];
ECBlock[] parityBlocks = new TestBlock[numParityUnits];
for (int i = 0; i < numDataUnits; i++) {
dataBlocks[i] = generateDataBlock();
}
for (int i = 0; i < numParityUnits; i++) {
parityBlocks[i] = allocateOutputBlock();
}
return new ECBlockGroup(dataBlocks, parityBlocks);
}
/**
* Generate random data and return a data block.
* @return
*/
protected ECBlock generateDataBlock() {
ECChunk[] chunks = new ECChunk[numChunksInBlock];
for (int i = 0; i < numChunksInBlock; ++i) {
chunks[i] = generateDataChunk();
}
return new TestBlock(chunks);
}
/**
* Copy those data blocks that's to be erased for later comparing and
* verifying.
* @param dataBlocks
* @return
*/
protected TestBlock[] copyDataBlocksToErase(TestBlock[] dataBlocks) {
TestBlock[] copiedBlocks = new TestBlock[erasedDataIndexes.length];
for (int i = 0; i < erasedDataIndexes.length; ++i) {
copiedBlocks[i] = cloneBlockWithData(dataBlocks[erasedDataIndexes[i]]);
}
return copiedBlocks;
}
/**
* Allocate an output block. Note the chunk buffer will be allocated by the
* up caller when performing the coding step.
* @return
*/
protected TestBlock allocateOutputBlock() {
ECChunk[] chunks = new ECChunk[numChunksInBlock];
return new TestBlock(chunks);
}
/**
* Clone blocks with data copied along with, avoiding affecting the original
* blocks.
* @param blocks
* @return
*/
protected static TestBlock[] cloneBlocksWithData(TestBlock[] blocks) {
TestBlock[] results = new TestBlock[blocks.length];
for (int i = 0; i < blocks.length; ++i) {
results[i] = cloneBlockWithData(blocks[i]);
}
return results;
}
/**
* Clone exactly a block, avoiding affecting the original block.
* @param block
* @return a new block
*/
protected static TestBlock cloneBlockWithData(TestBlock block) {
ECChunk[] newChunks = cloneChunksWithData(block.chunks);
return new TestBlock(newChunks);
}
/**
* Erase some data blocks specified by the indexes from the data blocks.
* @param dataBlocks
*/
protected void eraseSomeDataBlocks(TestBlock[] dataBlocks) {
for (int i = 0; i < erasedDataIndexes.length; ++i) {
eraseDataFromBlock(dataBlocks, erasedDataIndexes[i]);
}
}
/**
* Erase data from a block specified by erased index.
* @param blocks
* @param erasedIndex
*/
protected void eraseDataFromBlock(TestBlock[] blocks, int erasedIndex) {
TestBlock theBlock = blocks[erasedIndex];
eraseDataFromChunks(theBlock.chunks);
theBlock.setErased(true);
}
}

View File

@ -0,0 +1,50 @@
/**
* 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.coder;
import org.junit.Before;
import org.junit.Test;
/**
* Test XOR encoding and decoding.
*/
public class TestXorCoder extends TestErasureCoderBase {
@Before
public void setup() {
this.encoderClass = XorErasureEncoder.class;
this.decoderClass = XorErasureDecoder.class;
this.numDataUnits = 10;
this.numParityUnits = 1;
this.erasedDataIndexes = new int[] {0};
this.numChunksInBlock = 10;
}
@Test
public void testCodingNoDirectBuffer() {
testCoding(false);
}
@Test
public void testCodingDirectBuffer() {
testCoding(true);
}
}