|
@@ -41,7 +41,28 @@ import java.util.concurrent.Future;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
/**
|
|
|
- * Utility class for analyzing striped block groups
|
|
|
+ * When accessing a file in striped layout, operations on logical byte ranges
|
|
|
+ * in the file need to be mapped to physical byte ranges on block files stored
|
|
|
+ * on DataNodes. This utility class facilities this mapping by defining and
|
|
|
+ * exposing a number of striping-related concepts. The most basic ones are
|
|
|
+ * illustrated in the following diagram. Unless otherwise specified, all
|
|
|
+ * range-related calculations are inclusive (the end offset of the previous
|
|
|
+ * range should be 1 byte lower than the start offset of the next one).
|
|
|
+ *
|
|
|
+ * | <---- Block Group ----> | <- Block Group: logical unit composing
|
|
|
+ * | | striped HDFS files.
|
|
|
+ * blk_0 blk_1 blk_2 <- Internal Blocks: each internal block
|
|
|
+ * | | | represents a physically stored local
|
|
|
+ * v v v block file
|
|
|
+ * +------+ +------+ +------+
|
|
|
+ * |cell_0| |cell_1| |cell_2| <- {@link StripingCell} represents the
|
|
|
+ * +------+ +------+ +------+ logical order that a Block Group should
|
|
|
+ * |cell_3| |cell_4| |cell_5| be accessed: cell_0, cell_1, ...
|
|
|
+ * +------+ +------+ +------+
|
|
|
+ * |cell_6| |cell_7| |cell_8|
|
|
|
+ * +------+ +------+ +------+
|
|
|
+ * |cell_9|
|
|
|
+ * +------+ <- A cell contains cellSize bytes of data
|
|
|
*/
|
|
|
@InterfaceAudience.Private
|
|
|
public class StripedBlockUtil {
|
|
@@ -103,31 +124,6 @@ public class StripedBlockUtil {
|
|
|
cellSize, dataBlkNum, idxInBlockGroup));
|
|
|
return block;
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * This method creates an internal {@link ExtendedBlock} at the given index
|
|
|
- * of a block group, for both data and parity block.
|
|
|
- */
|
|
|
- public static ExtendedBlock constructStripedBlock(ExtendedBlock blockGroup,
|
|
|
- int cellSize, int dataBlkNum, int idxInBlockGroup) {
|
|
|
- ExtendedBlock block = new ExtendedBlock(blockGroup);
|
|
|
- block.setBlockId(blockGroup.getBlockId() + idxInBlockGroup);
|
|
|
- block.setNumBytes(getStripedBlockLength(blockGroup.getNumBytes(), cellSize,
|
|
|
- dataBlkNum, idxInBlockGroup));
|
|
|
- return block;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Returns an internal block length at the given index of a block group,
|
|
|
- * for both data and parity block.
|
|
|
- */
|
|
|
- public static long getStripedBlockLength(long numBytes, int cellSize,
|
|
|
- int dataBlkNum, int idxInBlockGroup) {
|
|
|
- // parity block length is the same as the first striped block length.
|
|
|
- return StripedBlockUtil.getInternalBlockLength(
|
|
|
- numBytes, cellSize, dataBlkNum,
|
|
|
- idxInBlockGroup < dataBlkNum ? idxInBlockGroup : 0);
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Get the size of an internal block at the given index of a block group
|
|
@@ -157,7 +153,7 @@ public class StripedBlockUtil {
|
|
|
return (numStripes - 1L)*cellSize
|
|
|
+ lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private static int lastCellSize(int size, int cellSize, int numDataBlocks,
|
|
|
int i) {
|
|
|
if (i < numDataBlocks) {
|
|
@@ -183,60 +179,6 @@ public class StripedBlockUtil {
|
|
|
+ offsetInBlk % cellSize; // partial cell
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * This method plans the read portion from each block in the stripe
|
|
|
- * @param dataBlkNum The number of data blocks in the striping group
|
|
|
- * @param cellSize The size of each striping cell
|
|
|
- * @param startInBlk Starting offset in the striped block
|
|
|
- * @param len Length of the read request
|
|
|
- * @param bufOffset Initial offset in the result buffer
|
|
|
- * @return array of {@link ReadPortion}, each representing the portion of I/O
|
|
|
- * for an individual block in the group
|
|
|
- */
|
|
|
- @VisibleForTesting
|
|
|
- public static ReadPortion[] planReadPortions(final int dataBlkNum,
|
|
|
- final int cellSize, final long startInBlk, final int len, int bufOffset) {
|
|
|
- ReadPortion[] results = new ReadPortion[dataBlkNum];
|
|
|
- for (int i = 0; i < dataBlkNum; i++) {
|
|
|
- results[i] = new ReadPortion();
|
|
|
- }
|
|
|
-
|
|
|
- // cellIdxInBlk is the index of the cell in the block
|
|
|
- // E.g., cell_3 is the 2nd cell in blk_0
|
|
|
- int cellIdxInBlk = (int) (startInBlk / (cellSize * dataBlkNum));
|
|
|
-
|
|
|
- // blkIdxInGroup is the index of the block in the striped block group
|
|
|
- // E.g., blk_2 is the 3rd block in the group
|
|
|
- final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum);
|
|
|
- results[blkIdxInGroup].setStartOffsetInBlock(cellSize * cellIdxInBlk +
|
|
|
- startInBlk % cellSize);
|
|
|
- boolean crossStripe = false;
|
|
|
- for (int i = 1; i < dataBlkNum; i++) {
|
|
|
- if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) {
|
|
|
- cellIdxInBlk++;
|
|
|
- crossStripe = true;
|
|
|
- }
|
|
|
- results[(blkIdxInGroup + i) % dataBlkNum].setStartOffsetInBlock(
|
|
|
- cellSize * cellIdxInBlk);
|
|
|
- }
|
|
|
-
|
|
|
- int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len);
|
|
|
- results[blkIdxInGroup].offsetsInBuf.add(bufOffset);
|
|
|
- results[blkIdxInGroup].lengths.add(firstCellLen);
|
|
|
- results[blkIdxInGroup].addReadLength(firstCellLen);
|
|
|
-
|
|
|
- int i = (blkIdxInGroup + 1) % dataBlkNum;
|
|
|
- for (int done = firstCellLen; done < len; done += cellSize) {
|
|
|
- ReadPortion rp = results[i];
|
|
|
- rp.offsetsInBuf.add(done + bufOffset);
|
|
|
- final int readLen = Math.min(len - done, cellSize);
|
|
|
- rp.lengths.add(readLen);
|
|
|
- rp.addReadLength(readLen);
|
|
|
- i = (i + 1) % dataBlkNum;
|
|
|
- }
|
|
|
- return results;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Get the next completed striped read task
|
|
|
*
|
|
@@ -360,84 +302,167 @@ public class StripedBlockUtil {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * This method divides a requested byte range into an array of
|
|
|
- * {@link AlignedStripe}
|
|
|
- *
|
|
|
+ * This method divides a requested byte range into an array of inclusive
|
|
|
+ * {@link AlignedStripe}.
|
|
|
+ * @param ecSchema The codec schema for the file, which carries the numbers
|
|
|
+ * of data / parity blocks, as well as cell size
|
|
|
+ * @param blockGroup The striped block group
|
|
|
+ * @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
|
|
|
- * TODO: cleanup and get rid of planReadPortions
|
|
|
+ * At most 5 stripes will be generated from each logical range, as
|
|
|
+ * demonstrated in the header of {@link AlignedStripe}.
|
|
|
*/
|
|
|
public static AlignedStripe[] divideByteRangeIntoStripes (
|
|
|
- ECSchema ecSchema, LocatedStripedBlock blockGroup, long start, long end,
|
|
|
- byte[] buf, int offsetInBuf) {
|
|
|
+ ECSchema ecSchema, LocatedStripedBlock blockGroup,
|
|
|
+ long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf,
|
|
|
+ int offsetInBuf) {
|
|
|
// TODO: change ECSchema naming to use cell size instead of chunk size
|
|
|
|
|
|
// Step 0: analyze range and calculate basic parameters
|
|
|
int cellSize = ecSchema.getChunkSize();
|
|
|
int dataBlkNum = ecSchema.getNumDataUnits();
|
|
|
- int len = (int) (end - start + 1);
|
|
|
- int firstCellIdxInBG = (int) (start / cellSize);
|
|
|
- int lastCellIdxInBG = (int) (end / cellSize);
|
|
|
- int firstCellSize = Math.min(cellSize - (int) (start % cellSize), len);
|
|
|
- long firstCellOffsetInBlk = firstCellIdxInBG / dataBlkNum * cellSize +
|
|
|
- start % cellSize;
|
|
|
- int lastCellSize = lastCellIdxInBG == firstCellIdxInBG ?
|
|
|
- firstCellSize : (int) (end % cellSize) + 1;
|
|
|
-
|
|
|
- // Step 1: get the unmerged ranges on each internal block
|
|
|
- // TODO: StripingCell should carry info on size and start offset (HDFS-8320)
|
|
|
- VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema,
|
|
|
- firstCellIdxInBG, lastCellIdxInBG, firstCellSize, firstCellOffsetInBlk,
|
|
|
- lastCellSize);
|
|
|
-
|
|
|
- // Step 2: merge into at most 5 stripes
|
|
|
+
|
|
|
+ // Step 1: map the byte range to StripingCells
|
|
|
+ StripingCell[] cells = getStripingCellsOfByteRange(ecSchema, blockGroup,
|
|
|
+ rangeStartInBlockGroup, rangeEndInBlockGroup);
|
|
|
+
|
|
|
+ // Step 2: get the unmerged ranges on each internal block
|
|
|
+ VerticalRange[] ranges = getRangesForInternalBlocks(ecSchema, cells);
|
|
|
+
|
|
|
+ // Step 3: merge into at most 5 stripes
|
|
|
AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecSchema, ranges);
|
|
|
|
|
|
- // Step 3: calculate each chunk's position in destination buffer
|
|
|
- calcualteChunkPositionsInBuf(ecSchema, blockGroup, buf, offsetInBuf,
|
|
|
- firstCellIdxInBG, lastCellIdxInBG, firstCellSize, firstCellOffsetInBlk,
|
|
|
- lastCellSize, stripes);
|
|
|
+ // Step 4: calculate each chunk's position in destination buffer
|
|
|
+ calcualteChunkPositionsInBuf(ecSchema, stripes, cells, buf, offsetInBuf);
|
|
|
|
|
|
- // Step 4: prepare ALLZERO blocks
|
|
|
+ // Step 5: prepare ALLZERO blocks
|
|
|
prepareAllZeroChunks(blockGroup, buf, stripes, cellSize, dataBlkNum);
|
|
|
|
|
|
return stripes;
|
|
|
}
|
|
|
|
|
|
- private static VerticalRange[] getRangesForInternalBlocks (ECSchema ecSchema,
|
|
|
- int firstCellIdxInBG, int lastCellIdxInBG, int firstCellSize,
|
|
|
- long firstCellOffsetInBlk, int lastCellSize) {
|
|
|
+ /**
|
|
|
+ * Map the logical byte range to a set of inclusive {@link StripingCell}
|
|
|
+ * instances, each representing the overlap of the byte range to a cell
|
|
|
+ * used by {@link DFSStripedOutputStream} in encoding
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ private static StripingCell[] getStripingCellsOfByteRange(ECSchema ecSchema,
|
|
|
+ LocatedStripedBlock blockGroup,
|
|
|
+ long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
|
|
|
+ Preconditions.checkArgument(
|
|
|
+ rangeStartInBlockGroup <= rangeEndInBlockGroup &&
|
|
|
+ rangeEndInBlockGroup < blockGroup.getBlockSize());
|
|
|
int cellSize = ecSchema.getChunkSize();
|
|
|
- int dataBlkNum = ecSchema.getNumDataUnits();
|
|
|
+ int len = (int) (rangeEndInBlockGroup - rangeStartInBlockGroup + 1);
|
|
|
+ int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
|
|
|
+ int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize);
|
|
|
+ int numCells = lastCellIdxInBG - firstCellIdxInBG + 1;
|
|
|
+ StripingCell[] cells = new StripingCell[numCells];
|
|
|
+ cells[0] = new StripingCell(ecSchema, firstCellIdxInBG);
|
|
|
+ cells[numCells - 1] = new StripingCell(ecSchema, lastCellIdxInBG);
|
|
|
+
|
|
|
+ cells[0].offset = (int) (rangeStartInBlockGroup % cellSize);
|
|
|
+ cells[0].size =
|
|
|
+ Math.min(cellSize - (int) (rangeStartInBlockGroup % cellSize), len);
|
|
|
+ if (lastCellIdxInBG != firstCellIdxInBG) {
|
|
|
+ cells[numCells - 1].size = (int) (rangeEndInBlockGroup % cellSize) + 1;
|
|
|
+ }
|
|
|
+
|
|
|
+ for (int i = 1; i < numCells - 1; i++) {
|
|
|
+ cells[i] = new StripingCell(ecSchema, i + firstCellIdxInBG);
|
|
|
+ }
|
|
|
+
|
|
|
+ return cells;
|
|
|
+ }
|
|
|
|
|
|
+ /**
|
|
|
+ * Given a logical start offset in a block group, calculate the physical
|
|
|
+ * start offset into each stored internal block.
|
|
|
+ */
|
|
|
+ public static long[] getStartOffsetsForInternalBlocks(
|
|
|
+ ECSchema ecSchema, LocatedStripedBlock blockGroup,
|
|
|
+ long rangeStartInBlockGroup) {
|
|
|
+ Preconditions.checkArgument(
|
|
|
+ rangeStartInBlockGroup < blockGroup.getBlockSize());
|
|
|
+ int dataBlkNum = ecSchema.getNumDataUnits();
|
|
|
+ int parityBlkNum = ecSchema.getNumParityUnits();
|
|
|
+ int cellSize = ecSchema.getChunkSize();
|
|
|
+ long[] startOffsets = new long[dataBlkNum + parityBlkNum];
|
|
|
+ Arrays.fill(startOffsets, -1L);
|
|
|
+ int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
|
|
|
StripingCell firstCell = new StripingCell(ecSchema, firstCellIdxInBG);
|
|
|
- StripingCell lastCell = new StripingCell(ecSchema, lastCellIdxInBG);
|
|
|
+ firstCell.offset = (int) (rangeStartInBlockGroup % cellSize);
|
|
|
+ startOffsets[firstCell.idxInStripe] =
|
|
|
+ firstCell.idxInInternalBlk * cellSize + firstCell.offset;
|
|
|
+ long earliestStart = startOffsets[firstCell.idxInStripe];
|
|
|
+ for (int i = 1; i < dataBlkNum; i++) {
|
|
|
+ int idx = firstCellIdxInBG + i;
|
|
|
+ if (idx * cellSize >= blockGroup.getBlockSize()) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ StripingCell cell = new StripingCell(ecSchema, idx);
|
|
|
+ startOffsets[cell.idxInStripe] = cell.idxInInternalBlk * cellSize;
|
|
|
+ if (startOffsets[cell.idxInStripe] < earliestStart) {
|
|
|
+ earliestStart = startOffsets[cell.idxInStripe];
|
|
|
+ }
|
|
|
+ }
|
|
|
+ for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) {
|
|
|
+ startOffsets[i] = earliestStart;
|
|
|
+ }
|
|
|
+ return startOffsets;
|
|
|
+ }
|
|
|
|
|
|
- VerticalRange ranges[] = new VerticalRange[dataBlkNum];
|
|
|
- ranges[firstCell.idxInStripe] =
|
|
|
- new VerticalRange(firstCellOffsetInBlk, firstCellSize);
|
|
|
- for (int i = firstCellIdxInBG + 1; i < lastCellIdxInBG; i++) {
|
|
|
+ /**
|
|
|
+ * Given a logical byte range, mapped to each {@link StripingCell}, calculate
|
|
|
+ * the physical byte range (inclusive) on each stored internal block.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ private static VerticalRange[] getRangesForInternalBlocks(ECSchema ecSchema,
|
|
|
+ StripingCell[] cells) {
|
|
|
+ int cellSize = ecSchema.getChunkSize();
|
|
|
+ int dataBlkNum = ecSchema.getNumDataUnits();
|
|
|
+ int parityBlkNum = ecSchema.getNumParityUnits();
|
|
|
+
|
|
|
+ VerticalRange ranges[] = new VerticalRange[dataBlkNum + parityBlkNum];
|
|
|
+
|
|
|
+ long earliestStart = Long.MAX_VALUE;
|
|
|
+ long latestEnd = -1;
|
|
|
+ for (StripingCell cell : cells) {
|
|
|
// iterate through all cells and update the list of StripeRanges
|
|
|
- StripingCell cell = new StripingCell(ecSchema, i);
|
|
|
if (ranges[cell.idxInStripe] == null) {
|
|
|
ranges[cell.idxInStripe] = new VerticalRange(
|
|
|
- cell.idxInInternalBlk * cellSize, cellSize);
|
|
|
+ cell.idxInInternalBlk * cellSize + cell.offset, cell.size);
|
|
|
} else {
|
|
|
- ranges[cell.idxInStripe].spanInBlock += cellSize;
|
|
|
+ ranges[cell.idxInStripe].spanInBlock += cell.size;
|
|
|
+ }
|
|
|
+ VerticalRange range = ranges[cell.idxInStripe];
|
|
|
+ if (range.offsetInBlock < earliestStart) {
|
|
|
+ earliestStart = range.offsetInBlock;
|
|
|
+ }
|
|
|
+ if (range.offsetInBlock + range.spanInBlock - 1 > latestEnd) {
|
|
|
+ latestEnd = range.offsetInBlock + range.spanInBlock - 1;
|
|
|
}
|
|
|
}
|
|
|
- if (ranges[lastCell.idxInStripe] == null) {
|
|
|
- ranges[lastCell.idxInStripe] = new VerticalRange(
|
|
|
- lastCell.idxInInternalBlk * cellSize, lastCellSize);
|
|
|
- } else if (lastCell.idxInBlkGroup != firstCell.idxInBlkGroup) {
|
|
|
- ranges[lastCell.idxInStripe].spanInBlock += lastCellSize;
|
|
|
+
|
|
|
+ // Each parity block should be fetched at maximum range of all data blocks
|
|
|
+ for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) {
|
|
|
+ ranges[i] = new VerticalRange(earliestStart,
|
|
|
+ latestEnd - earliestStart + 1);
|
|
|
}
|
|
|
|
|
|
return ranges;
|
|
|
}
|
|
|
|
|
|
- private static AlignedStripe[] mergeRangesForInternalBlocks(ECSchema ecSchema,
|
|
|
- VerticalRange[] ranges) {
|
|
|
+ /**
|
|
|
+ * Merge byte ranges on each internal block into a set of inclusive
|
|
|
+ * {@link AlignedStripe} instances.
|
|
|
+ */
|
|
|
+ private static AlignedStripe[] mergeRangesForInternalBlocks(
|
|
|
+ ECSchema ecSchema, VerticalRange[] ranges) {
|
|
|
int dataBlkNum = ecSchema.getNumDataUnits();
|
|
|
int parityBlkNum = ecSchema.getNumParityUnits();
|
|
|
List<AlignedStripe> stripes = new ArrayList<>();
|
|
@@ -461,12 +486,8 @@ public class StripedBlockUtil {
|
|
|
}
|
|
|
|
|
|
private static void calcualteChunkPositionsInBuf(ECSchema ecSchema,
|
|
|
- LocatedStripedBlock blockGroup, byte[] buf, int offsetInBuf,
|
|
|
- int firstCellIdxInBG, int lastCellIdxInBG, int firstCellSize,
|
|
|
- long firstCellOffsetInBlk, int lastCellSize, AlignedStripe[] stripes) {
|
|
|
- int cellSize = ecSchema.getChunkSize();
|
|
|
- int dataBlkNum = ecSchema.getNumDataUnits();
|
|
|
- // Step 3: calculate each chunk's position in destination buffer
|
|
|
+ AlignedStripe[] stripes, StripingCell[] cells, byte[] buf,
|
|
|
+ int offsetInBuf) {
|
|
|
/**
|
|
|
* | <--------------- AlignedStripe --------------->|
|
|
|
*
|
|
@@ -484,20 +505,11 @@ public class StripedBlockUtil {
|
|
|
*
|
|
|
* Cell indexing convention defined in {@link StripingCell}
|
|
|
*/
|
|
|
+ int cellSize = ecSchema.getChunkSize();
|
|
|
int done = 0;
|
|
|
- for (int i = firstCellIdxInBG; i <= lastCellIdxInBG; i++) {
|
|
|
- StripingCell cell = new StripingCell(ecSchema, i);
|
|
|
- long cellStart = i == firstCellIdxInBG ?
|
|
|
- firstCellOffsetInBlk : cell.idxInInternalBlk * cellSize;
|
|
|
- int cellLen;
|
|
|
- if (i == firstCellIdxInBG) {
|
|
|
- cellLen = firstCellSize;
|
|
|
- } else if (i == lastCellIdxInBG) {
|
|
|
- cellLen = lastCellSize;
|
|
|
- } else {
|
|
|
- cellLen = cellSize;
|
|
|
- }
|
|
|
- long cellEnd = cellStart + cellLen - 1;
|
|
|
+ for (StripingCell cell : cells) {
|
|
|
+ long cellStart = cell.idxInInternalBlk * cellSize + cell.offset;
|
|
|
+ long cellEnd = cellStart + cell.size - 1;
|
|
|
for (AlignedStripe s : stripes) {
|
|
|
long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1;
|
|
|
long overlapStart = Math.max(cellStart, s.getOffsetInBlock());
|
|
@@ -514,10 +526,14 @@ public class StripedBlockUtil {
|
|
|
add((int)(offsetInBuf + done + overlapStart - cellStart));
|
|
|
s.chunks[cell.idxInStripe].lengthsInBuf.add(overLapLen);
|
|
|
}
|
|
|
- done += cellLen;
|
|
|
+ done += cell.size;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * If a {@link StripingChunk} maps to a byte range beyond an internal block's
|
|
|
+ * size, the chunk should be treated as zero bytes in decoding.
|
|
|
+ */
|
|
|
private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup,
|
|
|
byte[] buf, AlignedStripe[] stripes, int cellSize, int dataBlkNum) {
|
|
|
for (AlignedStripe s : stripes) {
|
|
@@ -534,51 +550,13 @@ public class StripedBlockUtil {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * This class represents the portion of I/O associated with each block in the
|
|
|
- * striped block group.
|
|
|
- * TODO: consolidate ReadPortion with AlignedStripe
|
|
|
- */
|
|
|
- public static class ReadPortion {
|
|
|
- private long startOffsetInBlock = 0;
|
|
|
- private int readLength = 0;
|
|
|
- public final List<Integer> offsetsInBuf = new ArrayList<>();
|
|
|
- public final List<Integer> lengths = 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 int[] getLengths() {
|
|
|
- int[] lens = new int[this.lengths.size()];
|
|
|
- for (int i = 0; i < lens.length; i++) {
|
|
|
- lens[i] = this.lengths.get(i);
|
|
|
- }
|
|
|
- return lens;
|
|
|
- }
|
|
|
-
|
|
|
- public long getStartOffsetInBlock() {
|
|
|
- return startOffsetInBlock;
|
|
|
- }
|
|
|
-
|
|
|
- public int getReadLength() {
|
|
|
- return readLength;
|
|
|
- }
|
|
|
-
|
|
|
- public void setStartOffsetInBlock(long startOffsetInBlock) {
|
|
|
- this.startOffsetInBlock = startOffsetInBlock;
|
|
|
- }
|
|
|
-
|
|
|
- void addReadLength(int extraLength) {
|
|
|
- this.readLength += extraLength;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * The unit of encoding used in {@link DFSStripedOutputStream}
|
|
|
+ * Cell is the unit of encoding used in {@link DFSStripedOutputStream}. This
|
|
|
+ * size impacts how a logical offset in the file or block group translates
|
|
|
+ * to physical byte offset in a stored internal block. The StripingCell util
|
|
|
+ * class facilitates this calculation. Each StripingCell is inclusive with
|
|
|
+ * its start and end offsets -- e.g., the end logical offset of cell_0_0_0
|
|
|
+ * should be 1 byte lower than the start logical offset of cell_1_0_1.
|
|
|
+ *
|
|
|
* | <------- Striped Block Group -------> |
|
|
|
* blk_0 blk_1 blk_2
|
|
|
* | | |
|
|
@@ -586,43 +564,57 @@ public class StripedBlockUtil {
|
|
|
* +----------+ +----------+ +----------+
|
|
|
* |cell_0_0_0| |cell_1_0_1| |cell_2_0_2|
|
|
|
* +----------+ +----------+ +----------+
|
|
|
- * |cell_3_1_0| |cell_4_1_1| |cell_5_1_2| <- {@link idxInBlkGroup} = 5
|
|
|
- * +----------+ +----------+ +----------+ {@link idxInInternalBlk} = 1
|
|
|
- * {@link idxInStripe} = 2
|
|
|
+ * |cell_3_1_0| |cell_4_1_1| |cell_5_1_2| <- {@link #idxInBlkGroup} = 5
|
|
|
+ * +----------+ +----------+ +----------+ {@link #idxInInternalBlk} = 1
|
|
|
+ * {@link #idxInStripe} = 2
|
|
|
* A StripingCell is a special instance of {@link StripingChunk} whose offset
|
|
|
* and size align with the cell used when writing data.
|
|
|
* TODO: consider parity cells
|
|
|
*/
|
|
|
- public static class StripingCell {
|
|
|
+ @VisibleForTesting
|
|
|
+ static class StripingCell {
|
|
|
public final ECSchema schema;
|
|
|
/** Logical order in a block group, used when doing I/O to a block group */
|
|
|
- public final int idxInBlkGroup;
|
|
|
- public final int idxInInternalBlk;
|
|
|
- public final int idxInStripe;
|
|
|
+ final int idxInBlkGroup;
|
|
|
+ final int idxInInternalBlk;
|
|
|
+ final int idxInStripe;
|
|
|
+ /**
|
|
|
+ * When a logical byte range is mapped to a set of cells, it might
|
|
|
+ * partially overlap with the first and last cells. This field and the
|
|
|
+ * {@link #size} variable represent the start offset and size of the
|
|
|
+ * overlap.
|
|
|
+ */
|
|
|
+ int offset;
|
|
|
+ int size;
|
|
|
|
|
|
- public StripingCell(ECSchema ecSchema, int idxInBlkGroup) {
|
|
|
+ StripingCell(ECSchema ecSchema, int idxInBlkGroup) {
|
|
|
this.schema = ecSchema;
|
|
|
this.idxInBlkGroup = idxInBlkGroup;
|
|
|
this.idxInInternalBlk = idxInBlkGroup / ecSchema.getNumDataUnits();
|
|
|
this.idxInStripe = idxInBlkGroup -
|
|
|
this.idxInInternalBlk * ecSchema.getNumDataUnits();
|
|
|
+ this.offset = 0;
|
|
|
+ this.size = ecSchema.getChunkSize();
|
|
|
}
|
|
|
|
|
|
- public StripingCell(ECSchema ecSchema, int idxInInternalBlk,
|
|
|
+ StripingCell(ECSchema ecSchema, int idxInInternalBlk,
|
|
|
int idxInStripe) {
|
|
|
this.schema = ecSchema;
|
|
|
this.idxInInternalBlk = idxInInternalBlk;
|
|
|
this.idxInStripe = idxInStripe;
|
|
|
this.idxInBlkGroup =
|
|
|
idxInInternalBlk * ecSchema.getNumDataUnits() + idxInStripe;
|
|
|
+ this.offset = 0;
|
|
|
+ this.size = ecSchema.getChunkSize();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Given a requested byte range on a striped block group, an AlignedStripe
|
|
|
- * represents a {@link VerticalRange} that is aligned with both the byte range
|
|
|
- * and boundaries of all internal blocks. As illustrated in the diagram, any
|
|
|
- * given byte range on a block group leads to 1~5 AlignedStripe's.
|
|
|
+ * represents an inclusive {@link VerticalRange} that is aligned with both
|
|
|
+ * the byte range and boundaries of all internal blocks. As illustrated in
|
|
|
+ * the diagram, any given byte range on a block group leads to 1~5
|
|
|
+ * AlignedStripe's.
|
|
|
*
|
|
|
* |<-------- Striped Block Group -------->|
|
|
|
* blk_0 blk_1 blk_2 blk_3 blk_4
|
|
@@ -648,6 +640,7 @@ public class StripedBlockUtil {
|
|
|
*
|
|
|
* The coverage of an AlignedStripe on an internal block is represented as a
|
|
|
* {@link StripingChunk}.
|
|
|
+ *
|
|
|
* To simplify the logic of reading a logical byte range from a block group,
|
|
|
* a StripingChunk is either completely in the requested byte range or
|
|
|
* completely outside the requested byte range.
|
|
@@ -692,19 +685,19 @@ public class StripedBlockUtil {
|
|
|
|
|
|
/**
|
|
|
* A simple utility class representing an arbitrary vertical inclusive range
|
|
|
- * starting at {@link offsetInBlock} and lasting for {@link length} bytes in
|
|
|
- * an internal block. Note that VerticalRange doesn't necessarily align with
|
|
|
- * {@link StripingCell}.
|
|
|
+ * starting at {@link #offsetInBlock} and lasting for {@link #spanInBlock}
|
|
|
+ * bytes in an internal block. Note that VerticalRange doesn't necessarily
|
|
|
+ * align with {@link StripingCell}.
|
|
|
*
|
|
|
* |<- Striped Block Group ->|
|
|
|
* blk_0
|
|
|
* |
|
|
|
* v
|
|
|
* +-----+
|
|
|
- * |~~~~~| <-- {@link offsetInBlock}
|
|
|
+ * |~~~~~| <-- {@link #offsetInBlock}
|
|
|
* | | ^
|
|
|
* | | |
|
|
|
- * | | | {@link spanInBlock}
|
|
|
+ * | | | {@link #spanInBlock}
|
|
|
* | | v
|
|
|
* |~~~~~| ---
|
|
|
* | |
|
|
@@ -743,9 +736,9 @@ public class StripedBlockUtil {
|
|
|
* +---------+ +---------+ | +----+ +----+
|
|
|
* <----------- data blocks ------------> | <--- parity --->
|
|
|
*
|
|
|
- * The class also carries {@link buf}, {@link offsetsInBuf}, and
|
|
|
- * {@link lengthsInBuf} to define how read task for this chunk should deliver
|
|
|
- * the returned data.
|
|
|
+ * The class also carries {@link #buf}, {@link #offsetsInBuf}, and
|
|
|
+ * {@link #lengthsInBuf} to define how read task for this chunk should
|
|
|
+ * deliver the returned data.
|
|
|
*/
|
|
|
public static class StripingChunk {
|
|
|
/** Chunk has been successfully fetched */
|