|
@@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
|
|
import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
|
|
|
|
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
import org.apache.htrace.Span;
|
|
import org.apache.htrace.Span;
|
|
import org.apache.htrace.Trace;
|
|
import org.apache.htrace.Trace;
|
|
@@ -50,7 +51,7 @@ import java.util.concurrent.Future;
|
|
*
|
|
*
|
|
* | <- Striped Block Group -> |
|
|
* | <- Striped Block Group -> |
|
|
* blk_0 blk_1 blk_2 <- A striped block group has
|
|
* blk_0 blk_1 blk_2 <- A striped block group has
|
|
- * | | | {@link #groupSize} blocks
|
|
|
|
|
|
+ * | | | {@link #dataBlkNum} blocks
|
|
* v v v
|
|
* v v v
|
|
* +------+ +------+ +------+
|
|
* +------+ +------+ +------+
|
|
* |cell_0| |cell_1| |cell_2| <- The logical read order should be
|
|
* |cell_0| |cell_1| |cell_2| <- The logical read order should be
|
|
@@ -72,7 +73,7 @@ import java.util.concurrent.Future;
|
|
public class DFSStripedInputStream extends DFSInputStream {
|
|
public class DFSStripedInputStream extends DFSInputStream {
|
|
/**
|
|
/**
|
|
* This method plans the read portion from each block in the stripe
|
|
* This method plans the read portion from each block in the stripe
|
|
- * @param groupSize The size / width of the striping group
|
|
|
|
|
|
+ * @param dataBlkNum The number of data blocks in the striping group
|
|
* @param cellSize The size of each striping cell
|
|
* @param cellSize The size of each striping cell
|
|
* @param startInBlk Starting offset in the striped block
|
|
* @param startInBlk Starting offset in the striped block
|
|
* @param len Length of the read request
|
|
* @param len Length of the read request
|
|
@@ -81,29 +82,29 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
* for an individual block in the group
|
|
* for an individual block in the group
|
|
*/
|
|
*/
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
- static ReadPortion[] planReadPortions(final int groupSize,
|
|
|
|
|
|
+ static ReadPortion[] planReadPortions(final int dataBlkNum,
|
|
final int cellSize, final long startInBlk, final int len, int bufOffset) {
|
|
final int cellSize, final long startInBlk, final int len, int bufOffset) {
|
|
- ReadPortion[] results = new ReadPortion[groupSize];
|
|
|
|
- for (int i = 0; i < groupSize; i++) {
|
|
|
|
|
|
+ ReadPortion[] results = new ReadPortion[dataBlkNum];
|
|
|
|
+ for (int i = 0; i < dataBlkNum; i++) {
|
|
results[i] = new ReadPortion();
|
|
results[i] = new ReadPortion();
|
|
}
|
|
}
|
|
|
|
|
|
// cellIdxInBlk is the index of the cell in the block
|
|
// cellIdxInBlk is the index of the cell in the block
|
|
// E.g., cell_3 is the 2nd cell in blk_0
|
|
// E.g., cell_3 is the 2nd cell in blk_0
|
|
- int cellIdxInBlk = (int) (startInBlk / (cellSize * groupSize));
|
|
|
|
|
|
+ int cellIdxInBlk = (int) (startInBlk / (cellSize * dataBlkNum));
|
|
|
|
|
|
// blkIdxInGroup is the index of the block in the striped block group
|
|
// blkIdxInGroup is the index of the block in the striped block group
|
|
// E.g., blk_2 is the 3rd block in the group
|
|
// E.g., blk_2 is the 3rd block in the group
|
|
- final int blkIdxInGroup = (int) (startInBlk / cellSize % groupSize);
|
|
|
|
|
|
+ final int blkIdxInGroup = (int) (startInBlk / cellSize % dataBlkNum);
|
|
results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk +
|
|
results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk +
|
|
startInBlk % cellSize;
|
|
startInBlk % cellSize;
|
|
boolean crossStripe = false;
|
|
boolean crossStripe = false;
|
|
- for (int i = 1; i < groupSize; i++) {
|
|
|
|
- if (blkIdxInGroup + i >= groupSize && !crossStripe) {
|
|
|
|
|
|
+ for (int i = 1; i < dataBlkNum; i++) {
|
|
|
|
+ if (blkIdxInGroup + i >= dataBlkNum && !crossStripe) {
|
|
cellIdxInBlk++;
|
|
cellIdxInBlk++;
|
|
crossStripe = true;
|
|
crossStripe = true;
|
|
}
|
|
}
|
|
- results[(blkIdxInGroup + i) % groupSize].startOffsetInBlock =
|
|
|
|
|
|
+ results[(blkIdxInGroup + i) % dataBlkNum].startOffsetInBlock =
|
|
cellSize * cellIdxInBlk;
|
|
cellSize * cellIdxInBlk;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -112,57 +113,21 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
results[blkIdxInGroup].lengths.add(firstCellLen);
|
|
results[blkIdxInGroup].lengths.add(firstCellLen);
|
|
results[blkIdxInGroup].readLength += firstCellLen;
|
|
results[blkIdxInGroup].readLength += firstCellLen;
|
|
|
|
|
|
- int i = (blkIdxInGroup + 1) % groupSize;
|
|
|
|
|
|
+ int i = (blkIdxInGroup + 1) % dataBlkNum;
|
|
for (int done = firstCellLen; done < len; done += cellSize) {
|
|
for (int done = firstCellLen; done < len; done += cellSize) {
|
|
ReadPortion rp = results[i];
|
|
ReadPortion rp = results[i];
|
|
rp.offsetsInBuf.add(done + bufOffset);
|
|
rp.offsetsInBuf.add(done + bufOffset);
|
|
final int readLen = Math.min(len - done, cellSize);
|
|
final int readLen = Math.min(len - done, cellSize);
|
|
rp.lengths.add(readLen);
|
|
rp.lengths.add(readLen);
|
|
rp.readLength += readLen;
|
|
rp.readLength += readLen;
|
|
- i = (i + 1) % groupSize;
|
|
|
|
|
|
+ i = (i + 1) % dataBlkNum;
|
|
}
|
|
}
|
|
return results;
|
|
return results;
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * This method parses a striped block group into individual blocks.
|
|
|
|
- *
|
|
|
|
- * @param bg The striped block group
|
|
|
|
- * @param dataBlkNum the number of data blocks
|
|
|
|
- * @return An array containing the blocks in the group
|
|
|
|
- */
|
|
|
|
- @VisibleForTesting
|
|
|
|
- static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg,
|
|
|
|
- int dataBlkNum, int cellSize) {
|
|
|
|
- int locatedBGSize = bg.getBlockIndices().length;
|
|
|
|
- // TODO not considering missing blocks for now, only identify data blocks
|
|
|
|
- LocatedBlock[] lbs = new LocatedBlock[dataBlkNum];
|
|
|
|
- for (short i = 0; i < locatedBGSize; i++) {
|
|
|
|
- final int idx = bg.getBlockIndices()[i];
|
|
|
|
- if (idx < dataBlkNum && lbs[idx] == null) {
|
|
|
|
- lbs[idx] = constructInternalBlock(bg, i, cellSize, idx);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- return lbs;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private static LocatedBlock constructInternalBlock(LocatedStripedBlock bg,
|
|
|
|
- int idxInReturnedLocs, int cellSize, int idxInBlockGroup) {
|
|
|
|
- final ExtendedBlock blk = new ExtendedBlock(bg.getBlock());
|
|
|
|
- blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup);
|
|
|
|
- // TODO: fix the numBytes computation
|
|
|
|
-
|
|
|
|
- return new LocatedBlock(blk,
|
|
|
|
- new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
|
|
|
|
- new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
|
|
|
|
- new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
|
|
|
|
- bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(),
|
|
|
|
- null);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
-
|
|
|
|
private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
|
|
private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
|
|
- private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS;
|
|
|
|
|
|
+ private final short dataBlkNum = HdfsConstants.NUM_DATA_BLOCKS;
|
|
|
|
+ private final short parityBlkNum = HdfsConstants.NUM_PARITY_BLOCKS;
|
|
|
|
|
|
DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum)
|
|
DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum)
|
|
throws IOException {
|
|
throws IOException {
|
|
@@ -199,7 +164,7 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
"LocatedStripedBlock for a striped file";
|
|
"LocatedStripedBlock for a striped file";
|
|
|
|
|
|
int idx = (int) (((blkStartOffset - lb.getStartOffset()) / cellSize)
|
|
int idx = (int) (((blkStartOffset - lb.getStartOffset()) / cellSize)
|
|
- % groupSize);
|
|
|
|
|
|
+ % dataBlkNum);
|
|
// If indexing information is returned, iterate through the index array
|
|
// If indexing information is returned, iterate through the index array
|
|
// to find the entry for position idx in the group
|
|
// to find the entry for position idx in the group
|
|
LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
|
|
LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
|
|
@@ -213,7 +178,8 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
DFSClient.LOG.debug("getBlockAt for striped blocks, offset="
|
|
DFSClient.LOG.debug("getBlockAt for striped blocks, offset="
|
|
+ blkStartOffset + ". Obtained block " + lb + ", idx=" + idx);
|
|
+ blkStartOffset + ". Obtained block " + lb + ", idx=" + idx);
|
|
}
|
|
}
|
|
- return constructInternalBlock(lsb, i, cellSize, idx);
|
|
|
|
|
|
+ return StripedBlockUtil.constructInternalBlock(lsb, i, cellSize,
|
|
|
|
+ dataBlkNum, idx);
|
|
}
|
|
}
|
|
|
|
|
|
private LocatedBlock getBlockGroupAt(long offset) throws IOException {
|
|
private LocatedBlock getBlockGroupAt(long offset) throws IOException {
|
|
@@ -240,13 +206,14 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
LocatedStripedBlock blockGroup = (LocatedStripedBlock) block;
|
|
LocatedStripedBlock blockGroup = (LocatedStripedBlock) block;
|
|
|
|
|
|
// Planning the portion of I/O for each shard
|
|
// Planning the portion of I/O for each shard
|
|
- ReadPortion[] readPortions = planReadPortions(groupSize, cellSize, start,
|
|
|
|
|
|
+ ReadPortion[] readPortions = planReadPortions(dataBlkNum, cellSize, start,
|
|
len, offset);
|
|
len, offset);
|
|
|
|
|
|
// Parse group to get chosen DN location
|
|
// Parse group to get chosen DN location
|
|
- LocatedBlock[] blks = parseStripedBlockGroup(blockGroup, groupSize, cellSize);
|
|
|
|
|
|
+ LocatedBlock[] blks = StripedBlockUtil.
|
|
|
|
+ parseStripedBlockGroup(blockGroup, cellSize, dataBlkNum, parityBlkNum);
|
|
|
|
|
|
- for (short i = 0; i < groupSize; i++) {
|
|
|
|
|
|
+ for (short i = 0; i < dataBlkNum; i++) {
|
|
ReadPortion rp = readPortions[i];
|
|
ReadPortion rp = readPortions[i];
|
|
if (rp.readLength <= 0) {
|
|
if (rp.readLength <= 0) {
|
|
continue;
|
|
continue;
|