|
@@ -73,7 +73,8 @@ import java.util.concurrent.TimeUnit;
|
|
|
@InterfaceAudience.Private
|
|
|
public class StripedBlockUtil {
|
|
|
|
|
|
- public static final Logger LOG = LoggerFactory.getLogger(StripedBlockUtil.class);
|
|
|
+ public static final Logger LOG =
|
|
|
+ LoggerFactory.getLogger(StripedBlockUtil.class);
|
|
|
|
|
|
/**
|
|
|
* Parses a striped block group into individual blocks.
|
|
@@ -312,16 +313,17 @@ public class StripedBlockUtil {
|
|
|
* schedule a new fetch request with the decoding input buffer as transfer
|
|
|
* destination.
|
|
|
*/
|
|
|
- public static byte[][] initDecodeInputs(AlignedStripe alignedStripe,
|
|
|
+ public static ByteBuffer[] initDecodeInputs(AlignedStripe alignedStripe,
|
|
|
int dataBlkNum, int parityBlkNum) {
|
|
|
- byte[][] decodeInputs =
|
|
|
- new byte[dataBlkNum + parityBlkNum][(int) alignedStripe.getSpanInBlock()];
|
|
|
+ ByteBuffer[] decodeInputs = new ByteBuffer[dataBlkNum + parityBlkNum];
|
|
|
+ for (int i = 0; i < decodeInputs.length; i++) {
|
|
|
+ decodeInputs[i] = ByteBuffer.allocate(
|
|
|
+ (int) alignedStripe.getSpanInBlock());
|
|
|
+ }
|
|
|
// read the full data aligned stripe
|
|
|
for (int i = 0; i < dataBlkNum; i++) {
|
|
|
if (alignedStripe.chunks[i] == null) {
|
|
|
alignedStripe.chunks[i] = new StripingChunk(decodeInputs[i]);
|
|
|
- alignedStripe.chunks[i].addByteArraySlice(0,
|
|
|
- (int) alignedStripe.getSpanInBlock());
|
|
|
}
|
|
|
}
|
|
|
return decodeInputs;
|
|
@@ -334,14 +336,21 @@ public class StripedBlockUtil {
|
|
|
* When all pending requests have returned, this method should be called to
|
|
|
* finalize decode input buffers.
|
|
|
*/
|
|
|
- public static void finalizeDecodeInputs(final byte[][] decodeInputs,
|
|
|
+ public static void finalizeDecodeInputs(final ByteBuffer[] decodeInputs,
|
|
|
AlignedStripe alignedStripe) {
|
|
|
for (int i = 0; i < alignedStripe.chunks.length; i++) {
|
|
|
final StripingChunk chunk = alignedStripe.chunks[i];
|
|
|
if (chunk != null && chunk.state == StripingChunk.FETCHED) {
|
|
|
- chunk.copyTo(decodeInputs[i]);
|
|
|
+ if (chunk.useChunkBuffer()) {
|
|
|
+ chunk.getChunkBuffer().copyTo(decodeInputs[i]);
|
|
|
+ } else {
|
|
|
+ chunk.getByteBuffer().flip();
|
|
|
+ }
|
|
|
} else if (chunk != null && chunk.state == StripingChunk.ALLZERO) {
|
|
|
- Arrays.fill(decodeInputs[i], (byte) 0);
|
|
|
+ //ZERO it. Will be better handled in other following issue.
|
|
|
+ byte[] emptyBytes = new byte[decodeInputs[i].limit()];
|
|
|
+ decodeInputs[i].put(emptyBytes);
|
|
|
+ decodeInputs[i].flip();
|
|
|
} else {
|
|
|
decodeInputs[i] = null;
|
|
|
}
|
|
@@ -351,7 +360,7 @@ public class StripedBlockUtil {
|
|
|
/**
|
|
|
* Decode based on the given input buffers and erasure coding policy.
|
|
|
*/
|
|
|
- public static void decodeAndFillBuffer(final byte[][] decodeInputs,
|
|
|
+ public static void decodeAndFillBuffer(final ByteBuffer[] decodeInputs,
|
|
|
AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum,
|
|
|
RawErasureDecoder decoder) {
|
|
|
// Step 1: prepare indices and output buffers for missing data units
|
|
@@ -364,8 +373,11 @@ public class StripedBlockUtil {
|
|
|
}
|
|
|
}
|
|
|
decodeIndices = Arrays.copyOf(decodeIndices, pos);
|
|
|
- byte[][] decodeOutputs =
|
|
|
- new byte[decodeIndices.length][(int) alignedStripe.getSpanInBlock()];
|
|
|
+ ByteBuffer[] decodeOutputs = new ByteBuffer[decodeIndices.length];
|
|
|
+ for (int i = 0; i < decodeOutputs.length; i++) {
|
|
|
+ decodeOutputs[i] = ByteBuffer.allocate(
|
|
|
+ (int) alignedStripe.getSpanInBlock());
|
|
|
+ }
|
|
|
|
|
|
// Step 2: decode into prepared output buffers
|
|
|
decoder.decode(decodeInputs, decodeIndices, decodeOutputs);
|
|
@@ -374,8 +386,8 @@ public class StripedBlockUtil {
|
|
|
for (int i = 0; i < decodeIndices.length; i++) {
|
|
|
int missingBlkIdx = decodeIndices[i];
|
|
|
StripingChunk chunk = alignedStripe.chunks[missingBlkIdx];
|
|
|
- if (chunk.state == StripingChunk.MISSING) {
|
|
|
- chunk.copyFrom(decodeOutputs[i]);
|
|
|
+ if (chunk.state == StripingChunk.MISSING && chunk.useChunkBuffer()) {
|
|
|
+ chunk.getChunkBuffer().copyFrom(decodeOutputs[i]);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -402,7 +414,8 @@ public class StripedBlockUtil {
|
|
|
|
|
|
// Step 4: calculate each chunk's position in destination buffer. Since the
|
|
|
// whole read range is within a single stripe, the logic is simpler here.
|
|
|
- int bufOffset = (int) (rangeStartInBlockGroup % ((long) cellSize * dataBlkNum));
|
|
|
+ int bufOffset =
|
|
|
+ (int) (rangeStartInBlockGroup % ((long) cellSize * dataBlkNum));
|
|
|
for (StripingCell cell : cells) {
|
|
|
long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
|
|
|
long cellEnd = cellStart + cell.size - 1;
|
|
@@ -437,15 +450,14 @@ public class StripedBlockUtil {
|
|
|
* @param rangeStartInBlockGroup The byte range's start offset in block group
|
|
|
* @param rangeEndInBlockGroup The byte range's end offset in block group
|
|
|
* @param buf Destination buffer of the read operation for the byte range
|
|
|
- * @param offsetInBuf Start offset into the destination buffer
|
|
|
*
|
|
|
* At most 5 stripes will be generated from each logical range, as
|
|
|
* demonstrated in the header of {@link AlignedStripe}.
|
|
|
*/
|
|
|
- public static AlignedStripe[] divideByteRangeIntoStripes(ErasureCodingPolicy ecPolicy,
|
|
|
+ public static AlignedStripe[] divideByteRangeIntoStripes(
|
|
|
+ ErasureCodingPolicy ecPolicy,
|
|
|
int cellSize, LocatedStripedBlock blockGroup,
|
|
|
- long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf,
|
|
|
- int offsetInBuf) {
|
|
|
+ long rangeStartInBlockGroup, long rangeEndInBlockGroup, ByteBuffer buf) {
|
|
|
|
|
|
// Step 0: analyze range and calculate basic parameters
|
|
|
final int dataBlkNum = ecPolicy.getNumDataUnits();
|
|
@@ -462,7 +474,7 @@ public class StripedBlockUtil {
|
|
|
AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges);
|
|
|
|
|
|
// Step 4: calculate each chunk's position in destination buffer
|
|
|
- calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf, offsetInBuf);
|
|
|
+ calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf);
|
|
|
|
|
|
// Step 5: prepare ALLZERO blocks
|
|
|
prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum);
|
|
@@ -476,7 +488,8 @@ public class StripedBlockUtil {
|
|
|
* used by {@link DFSStripedOutputStream} in encoding
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
- private static StripingCell[] getStripingCellsOfByteRange(ErasureCodingPolicy ecPolicy,
|
|
|
+ private static StripingCell[] getStripingCellsOfByteRange(
|
|
|
+ ErasureCodingPolicy ecPolicy,
|
|
|
int cellSize, LocatedStripedBlock blockGroup,
|
|
|
long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
|
|
|
Preconditions.checkArgument(
|
|
@@ -511,7 +524,8 @@ public class StripedBlockUtil {
|
|
|
* the physical byte range (inclusive) on each stored internal block.
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
- private static VerticalRange[] getRangesForInternalBlocks(ErasureCodingPolicy ecPolicy,
|
|
|
+ private static VerticalRange[] getRangesForInternalBlocks(
|
|
|
+ ErasureCodingPolicy ecPolicy,
|
|
|
int cellSize, StripingCell[] cells) {
|
|
|
int dataBlkNum = ecPolicy.getNumDataUnits();
|
|
|
int parityBlkNum = ecPolicy.getNumParityUnits();
|
|
@@ -575,8 +589,7 @@ public class StripedBlockUtil {
|
|
|
}
|
|
|
|
|
|
private static void calcualteChunkPositionsInBuf(int cellSize,
|
|
|
- AlignedStripe[] stripes, StripingCell[] cells, byte[] buf,
|
|
|
- int offsetInBuf) {
|
|
|
+ AlignedStripe[] stripes, StripingCell[] cells, ByteBuffer buf) {
|
|
|
/**
|
|
|
* | <--------------- AlignedStripe --------------->|
|
|
|
*
|
|
@@ -598,6 +611,7 @@ public class StripedBlockUtil {
|
|
|
for (StripingCell cell : cells) {
|
|
|
long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
|
|
|
long cellEnd = cellStart + cell.size - 1;
|
|
|
+ StripingChunk chunk;
|
|
|
for (AlignedStripe s : stripes) {
|
|
|
long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1;
|
|
|
long overlapStart = Math.max(cellStart, s.getOffsetInBlock());
|
|
@@ -606,11 +620,13 @@ public class StripedBlockUtil {
|
|
|
if (overLapLen <= 0) {
|
|
|
continue;
|
|
|
}
|
|
|
- if (s.chunks[cell.idxInStripe] == null) {
|
|
|
- s.chunks[cell.idxInStripe] = new StripingChunk(buf);
|
|
|
+ chunk = s.chunks[cell.idxInStripe];
|
|
|
+ if (chunk == null) {
|
|
|
+ chunk = new StripingChunk();
|
|
|
+ s.chunks[cell.idxInStripe] = chunk;
|
|
|
}
|
|
|
- s.chunks[cell.idxInStripe].addByteArraySlice(
|
|
|
- (int)(offsetInBuf + done + overlapStart - cellStart), overLapLen);
|
|
|
+ chunk.getChunkBuffer().addSlice(buf,
|
|
|
+ (int) (done + overlapStart - cellStart), overLapLen);
|
|
|
}
|
|
|
done += cell.size;
|
|
|
}
|
|
@@ -833,88 +849,89 @@ public class StripedBlockUtil {
|
|
|
*/
|
|
|
public int state = REQUESTED;
|
|
|
|
|
|
- public final ChunkByteArray byteArray;
|
|
|
- public final ByteBuffer byteBuffer;
|
|
|
+ private final ChunkByteBuffer chunkBuffer;
|
|
|
+ private final ByteBuffer byteBuffer;
|
|
|
|
|
|
- public StripingChunk(byte[] buf) {
|
|
|
- this.byteArray = new ChunkByteArray(buf);
|
|
|
+ public StripingChunk() {
|
|
|
+ this.chunkBuffer = new ChunkByteBuffer();
|
|
|
byteBuffer = null;
|
|
|
}
|
|
|
|
|
|
public StripingChunk(ByteBuffer buf) {
|
|
|
- this.byteArray = null;
|
|
|
+ this.chunkBuffer = null;
|
|
|
this.byteBuffer = buf;
|
|
|
}
|
|
|
|
|
|
public StripingChunk(int state) {
|
|
|
- this.byteArray = null;
|
|
|
+ this.chunkBuffer = null;
|
|
|
this.byteBuffer = null;
|
|
|
this.state = state;
|
|
|
}
|
|
|
|
|
|
- public void addByteArraySlice(int offset, int length) {
|
|
|
- assert byteArray != null;
|
|
|
- byteArray.offsetsInBuf.add(offset);
|
|
|
- byteArray.lengthsInBuf.add(length);
|
|
|
+ public boolean useByteBuffer(){
|
|
|
+ return byteBuffer != null;
|
|
|
}
|
|
|
|
|
|
- void copyTo(byte[] target) {
|
|
|
- assert byteArray != null;
|
|
|
- byteArray.copyTo(target);
|
|
|
+ public boolean useChunkBuffer() {
|
|
|
+ return chunkBuffer != null;
|
|
|
}
|
|
|
|
|
|
- void copyFrom(byte[] src) {
|
|
|
- assert byteArray != null;
|
|
|
- byteArray.copyFrom(src);
|
|
|
+ public ByteBuffer getByteBuffer() {
|
|
|
+ assert byteBuffer != null;
|
|
|
+ return byteBuffer;
|
|
|
+ }
|
|
|
+
|
|
|
+ public ChunkByteBuffer getChunkBuffer() {
|
|
|
+ assert chunkBuffer != null;
|
|
|
+ return chunkBuffer;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public static class ChunkByteArray {
|
|
|
- private final byte[] buf;
|
|
|
- private final List<Integer> offsetsInBuf;
|
|
|
- private final List<Integer> lengthsInBuf;
|
|
|
+ /**
|
|
|
+ * A utility to manage ByteBuffer slices for a reader.
|
|
|
+ */
|
|
|
+ public static class ChunkByteBuffer {
|
|
|
+ private final List<ByteBuffer> slices;
|
|
|
|
|
|
- ChunkByteArray(byte[] buf) {
|
|
|
- this.buf = buf;
|
|
|
- this.offsetsInBuf = new ArrayList<>();
|
|
|
- this.lengthsInBuf = new ArrayList<>();
|
|
|
+ ChunkByteBuffer() {
|
|
|
+ this.slices = new ArrayList<>();
|
|
|
}
|
|
|
|
|
|
- public int[] getOffsets() {
|
|
|
- int[] offsets = new int[offsetsInBuf.size()];
|
|
|
- for (int i = 0; i < offsets.length; i++) {
|
|
|
- offsets[i] = offsetsInBuf.get(i);
|
|
|
- }
|
|
|
- return offsets;
|
|
|
+ public void addSlice(ByteBuffer buffer, int offset, int len) {
|
|
|
+ ByteBuffer tmp = buffer.duplicate();
|
|
|
+ tmp.position(buffer.position() + offset);
|
|
|
+ tmp.limit(buffer.position() + offset + len);
|
|
|
+ slices.add(tmp.slice());
|
|
|
}
|
|
|
|
|
|
- public int[] getLengths() {
|
|
|
- int[] lens = new int[this.lengthsInBuf.size()];
|
|
|
- for (int i = 0; i < lens.length; i++) {
|
|
|
- lens[i] = this.lengthsInBuf.get(i);
|
|
|
- }
|
|
|
- return lens;
|
|
|
+ public ByteBuffer getSlice(int i) {
|
|
|
+ return slices.get(i);
|
|
|
}
|
|
|
|
|
|
- public byte[] buf() {
|
|
|
- return buf;
|
|
|
+ public List<ByteBuffer> getSlices() {
|
|
|
+ return slices;
|
|
|
}
|
|
|
|
|
|
- void copyTo(byte[] target) {
|
|
|
- int posInBuf = 0;
|
|
|
- for (int i = 0; i < offsetsInBuf.size(); i++) {
|
|
|
- System.arraycopy(buf, offsetsInBuf.get(i),
|
|
|
- target, posInBuf, lengthsInBuf.get(i));
|
|
|
- posInBuf += lengthsInBuf.get(i);
|
|
|
+ /**
|
|
|
+ * Note: target will be ready-to-read state after the call.
|
|
|
+ */
|
|
|
+ void copyTo(ByteBuffer target) {
|
|
|
+ for (ByteBuffer slice : slices) {
|
|
|
+ slice.flip();
|
|
|
+ target.put(slice);
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
- void copyFrom(byte[] src) {
|
|
|
- int srcPos = 0;
|
|
|
- for (int j = 0; j < offsetsInBuf.size(); j++) {
|
|
|
- System.arraycopy(src, srcPos, buf, offsetsInBuf.get(j),
|
|
|
- lengthsInBuf.get(j));
|
|
|
- srcPos += lengthsInBuf.get(j);
|
|
|
+ target.flip();
|
|
|
+ }
|
|
|
+
|
|
|
+ void copyFrom(ByteBuffer src) {
|
|
|
+ ByteBuffer tmp;
|
|
|
+ int len;
|
|
|
+ for (ByteBuffer slice : slices) {
|
|
|
+ len = slice.remaining();
|
|
|
+ tmp = src.duplicate();
|
|
|
+ tmp.limit(tmp.position() + len);
|
|
|
+ slice.put(tmp);
|
|
|
+ src.position(src.position() + len);
|
|
|
}
|
|
|
}
|
|
|
}
|