|
@@ -43,6 +43,7 @@ import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe;
|
|
|
import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk;
|
|
|
import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
|
|
|
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.erasurecode.CodecUtil;
|
|
|
import org.apache.hadoop.io.erasurecode.ECSchema;
|
|
|
|
|
@@ -113,16 +114,43 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private final BlockReader[] blockReaders;
|
|
|
- /**
|
|
|
- * when initializing block readers, their starting offsets are set to the same
|
|
|
- * number: the smallest internal block offsets among all the readers. This is
|
|
|
- * because it is possible that for some internal blocks we have to read
|
|
|
- * "backwards" for decoding purpose. We thus use this offset array to track
|
|
|
- * offsets for all the block readers so that we can skip data if necessary.
|
|
|
- */
|
|
|
- private final long[] blockReaderOffsets;
|
|
|
- private final DatanodeInfo[] currentNodes;
|
|
|
+ private static class BlockReaderInfo {
|
|
|
+ final BlockReader reader;
|
|
|
+ final DatanodeInfo datanode;
|
|
|
+ /**
|
|
|
+ * when initializing block readers, their starting offsets are set to the same
|
|
|
+ * number: the smallest internal block offsets among all the readers. This is
|
|
|
+ * because it is possible that for some internal blocks we have to read
|
|
|
+ * "backwards" for decoding purpose. We thus use this offset array to track
|
|
|
+ * offsets for all the block readers so that we can skip data if necessary.
|
|
|
+ */
|
|
|
+ long blockReaderOffset;
|
|
|
+ LocatedBlock targetBlock;
|
|
|
+ /**
|
|
|
+ * We use this field to indicate whether we should use this reader. In case
|
|
|
+ * we hit any issue with this reader, we set this field to true and avoid
|
|
|
+ * using it for the next stripe.
|
|
|
+ */
|
|
|
+ boolean shouldSkip = false;
|
|
|
+
|
|
|
+ BlockReaderInfo(BlockReader reader, LocatedBlock targetBlock,
|
|
|
+ DatanodeInfo dn, long offset) {
|
|
|
+ this.reader = reader;
|
|
|
+ this.targetBlock = targetBlock;
|
|
|
+ this.datanode = dn;
|
|
|
+ this.blockReaderOffset = offset;
|
|
|
+ }
|
|
|
+
|
|
|
+ void setOffset(long offset) {
|
|
|
+ this.blockReaderOffset = offset;
|
|
|
+ }
|
|
|
+
|
|
|
+ void skip() {
|
|
|
+ this.shouldSkip = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private final BlockReaderInfo[] blockReaders;
|
|
|
private final int cellSize;
|
|
|
private final short dataBlkNum;
|
|
|
private final short parityBlkNum;
|
|
@@ -151,9 +179,7 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
dataBlkNum = (short) schema.getNumDataUnits();
|
|
|
parityBlkNum = (short) schema.getNumParityUnits();
|
|
|
groupSize = dataBlkNum + parityBlkNum;
|
|
|
- blockReaders = new BlockReader[groupSize];
|
|
|
- blockReaderOffsets = new long[groupSize];
|
|
|
- currentNodes = new DatanodeInfo[groupSize];
|
|
|
+ blockReaders = new BlockReaderInfo[groupSize];
|
|
|
curStripeRange = new StripeRange(0, 0);
|
|
|
readingService =
|
|
|
new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool());
|
|
@@ -218,18 +244,26 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
for (int i = 0; i < dataBlkNum; i++) {
|
|
|
LocatedBlock targetBlock = targetBlocks[i];
|
|
|
if (targetBlock != null) {
|
|
|
- DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null);
|
|
|
- if (retval != null) {
|
|
|
- currentNodes[i] = retval.info;
|
|
|
- blockReaders[i] = getBlockReaderWithRetry(targetBlock,
|
|
|
+ DNAddrPair dnInfo = getBestNodeDNAddrPair(targetBlock, null);
|
|
|
+ if (dnInfo != null) {
|
|
|
+ BlockReader reader = getBlockReaderWithRetry(targetBlock,
|
|
|
minOffset, targetBlock.getBlockSize() - minOffset,
|
|
|
- retval.addr, retval.storageType, retval.info, target, retry);
|
|
|
- blockReaderOffsets[i] = minOffset;
|
|
|
+ dnInfo.addr, dnInfo.storageType, dnInfo.info, target, retry);
|
|
|
+ if (reader != null) {
|
|
|
+ blockReaders[i] = new BlockReaderInfo(reader, targetBlock,
|
|
|
+ dnInfo.info, minOffset);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * @throws IOException only when failing to refetch block token, which happens
|
|
|
+ * when this client cannot get located block information from NameNode. This
|
|
|
+ * method returns null instead of throwing exception when failing to connect
|
|
|
+ * to the DataNode.
|
|
|
+ */
|
|
|
private BlockReader getBlockReaderWithRetry(LocatedBlock targetBlock,
|
|
|
long offsetInBlock, long length, InetSocketAddress targetAddr,
|
|
|
StorageType storageType, DatanodeInfo datanode, long offsetInFile,
|
|
@@ -275,21 +309,16 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
}
|
|
|
for (int i = 0; i < groupSize; i++) {
|
|
|
closeReader(i);
|
|
|
- currentNodes[i] = null;
|
|
|
+ blockReaders[i] = null;
|
|
|
}
|
|
|
blockEnd = -1;
|
|
|
}
|
|
|
|
|
|
private void closeReader(int index) {
|
|
|
if (blockReaders[index] != null) {
|
|
|
- try {
|
|
|
- blockReaders[index].close();
|
|
|
- } catch (IOException e) {
|
|
|
- DFSClient.LOG.error("error closing blockReader " + index, e);
|
|
|
- }
|
|
|
- blockReaders[index] = null;
|
|
|
+ IOUtils.cleanup(DFSClient.LOG, blockReaders[index].reader);
|
|
|
+ blockReaders[index].skip();
|
|
|
}
|
|
|
- blockReaderOffsets[index] = 0;
|
|
|
}
|
|
|
|
|
|
private long getOffsetInBlockGroup() {
|
|
@@ -323,16 +352,14 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(schema, cellSize,
|
|
|
blockGroup, offsetInBlockGroup,
|
|
|
offsetInBlockGroup + curStripeRange.length - 1, curStripeBuf);
|
|
|
- // TODO handle null elements in blks (e.g., NN does not know locations for
|
|
|
- // all the internal blocks)
|
|
|
final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(
|
|
|
blockGroup, cellSize, dataBlkNum, parityBlkNum);
|
|
|
// read the whole stripe
|
|
|
for (AlignedStripe stripe : stripes) {
|
|
|
// Parse group to get chosen DN location
|
|
|
StripeReader sreader = new StatefulStripeReader(readingService, stripe,
|
|
|
- blks);
|
|
|
- sreader.readStripe(blks, corruptedBlockMap);
|
|
|
+ blks, corruptedBlockMap);
|
|
|
+ sreader.readStripe();
|
|
|
}
|
|
|
curStripeBuf.position(stripeBufOffset);
|
|
|
curStripeBuf.limit(stripeLimit);
|
|
@@ -549,14 +576,13 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
blockGroup, start, end, buf, offset);
|
|
|
CompletionService<Void> readService = new ExecutorCompletionService<>(
|
|
|
dfsClient.getStripedReadsThreadPool());
|
|
|
- // TODO handle null elements in blks (e.g., NN does not know locations for
|
|
|
- // all the internal blocks)
|
|
|
final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(
|
|
|
blockGroup, cellSize, dataBlkNum, parityBlkNum);
|
|
|
for (AlignedStripe stripe : stripes) {
|
|
|
// Parse group to get chosen DN location
|
|
|
- StripeReader preader = new PositionStripeReader(readService, stripe);
|
|
|
- preader.readStripe(blks, corruptedBlockMap);
|
|
|
+ StripeReader preader = new PositionStripeReader(readService, stripe,
|
|
|
+ blks, corruptedBlockMap);
|
|
|
+ preader.readStripe();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -586,43 +612,89 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
final Map<Future<Void>, Integer> futures = new HashMap<>();
|
|
|
final AlignedStripe alignedStripe;
|
|
|
final CompletionService<Void> service;
|
|
|
+ final LocatedBlock[] targetBlocks;
|
|
|
+ final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap;
|
|
|
|
|
|
- StripeReader(CompletionService<Void> service, AlignedStripe alignedStripe) {
|
|
|
+ StripeReader(CompletionService<Void> service, AlignedStripe alignedStripe,
|
|
|
+ LocatedBlock[] targetBlocks,
|
|
|
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
|
|
|
this.service = service;
|
|
|
this.alignedStripe = alignedStripe;
|
|
|
+ this.targetBlocks = targetBlocks;
|
|
|
+ this.corruptedBlockMap = corruptedBlockMap;
|
|
|
}
|
|
|
|
|
|
- /** submit reading chunk task */
|
|
|
- abstract void readChunk(final CompletionService<Void> service,
|
|
|
- final LocatedBlock block, int chunkIndex,
|
|
|
- Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap);
|
|
|
+ abstract boolean readChunk(final CompletionService<Void> service,
|
|
|
+ final LocatedBlock block, int chunkIndex);
|
|
|
|
|
|
- /**
|
|
|
- * When seeing first missing block, initialize decode input buffers.
|
|
|
- * Also prepare the reading for data blocks outside of the reading range.
|
|
|
- */
|
|
|
- abstract void prepareDecodeInputs() throws IOException;
|
|
|
+ /** prepare all the data chunks */
|
|
|
+ abstract void prepareDecodeInputs();
|
|
|
|
|
|
- /**
|
|
|
- * Prepare reading for one more parity chunk.
|
|
|
- */
|
|
|
- abstract void prepareParityChunk() throws IOException;
|
|
|
+ /** prepare the parity chunk and block reader if necessary */
|
|
|
+ abstract boolean prepareParityChunk(int index) throws IOException;
|
|
|
|
|
|
abstract void decode();
|
|
|
|
|
|
abstract void updateState4SuccessRead(StripingChunkReadResult result);
|
|
|
|
|
|
+ private void checkMissingBlocks() throws IOException {
|
|
|
+ if (alignedStripe.missingChunksNum > parityBlkNum) {
|
|
|
+ clearFutures(futures.keySet());
|
|
|
+ throw new IOException(alignedStripe.missingChunksNum
|
|
|
+ + " missing blocks, the stripe is: " + alignedStripe);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * We need decoding. Thus go through all the data chunks and make sure we
|
|
|
+ * submit read requests for all of them.
|
|
|
+ */
|
|
|
+ private void readDataForDecoding() throws IOException {
|
|
|
+ prepareDecodeInputs();
|
|
|
+ for (int i = 0; i < dataBlkNum; i++) {
|
|
|
+ Preconditions.checkNotNull(alignedStripe.chunks[i]);
|
|
|
+ if (alignedStripe.chunks[i].state == StripingChunk.REQUESTED) {
|
|
|
+ if (!readChunk(service, targetBlocks[i], i)) {
|
|
|
+ alignedStripe.missingChunksNum++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ checkMissingBlocks();
|
|
|
+ }
|
|
|
+
|
|
|
+ void readParityChunks(int num) throws IOException {
|
|
|
+ for (int i = dataBlkNum, j = 0; i < dataBlkNum + parityBlkNum && j < num;
|
|
|
+ i++) {
|
|
|
+ if (alignedStripe.chunks[i] == null) {
|
|
|
+ if (prepareParityChunk(i) && readChunk(service, targetBlocks[i], i)) {
|
|
|
+ j++;
|
|
|
+ } else {
|
|
|
+ alignedStripe.missingChunksNum++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ checkMissingBlocks();
|
|
|
+ }
|
|
|
+
|
|
|
/** read the whole stripe. do decoding if necessary */
|
|
|
- void readStripe(LocatedBlock[] blocks,
|
|
|
- Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
|
|
|
- throws IOException {
|
|
|
- assert alignedStripe.getSpanInBlock() > 0;
|
|
|
- for (short i = 0; i < dataBlkNum; i++) {
|
|
|
- if (alignedStripe.chunks[i] != null
|
|
|
- && alignedStripe.chunks[i].state != StripingChunk.ALLZERO) {
|
|
|
- readChunk(service, blocks[i], i, corruptedBlockMap);
|
|
|
+ void readStripe() throws IOException {
|
|
|
+ for (int i = 0; i < dataBlkNum; i++) {
|
|
|
+ if (alignedStripe.chunks[i] != null &&
|
|
|
+ alignedStripe.chunks[i].state != StripingChunk.ALLZERO) {
|
|
|
+ if (!readChunk(service, targetBlocks[i], i)) {
|
|
|
+ alignedStripe.missingChunksNum++;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
+ // There are missing block locations at this stage. Thus we need to read
|
|
|
+ // the full stripe and one more parity block.
|
|
|
+ if (alignedStripe.missingChunksNum > 0) {
|
|
|
+ checkMissingBlocks();
|
|
|
+ readDataForDecoding();
|
|
|
+ // read parity chunks
|
|
|
+ readParityChunks(alignedStripe.missingChunksNum);
|
|
|
+ }
|
|
|
+ // TODO: for a full stripe we can start reading (dataBlkNum + 1) chunks
|
|
|
|
|
|
// Input buffers for potential decode operation, which remains null until
|
|
|
// first read failure
|
|
@@ -648,24 +720,15 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
}
|
|
|
} else {
|
|
|
returnedChunk.state = StripingChunk.MISSING;
|
|
|
- alignedStripe.missingChunksNum++;
|
|
|
- if (alignedStripe.missingChunksNum > parityBlkNum) {
|
|
|
- clearFutures(futures.keySet());
|
|
|
- throw new IOException("Too many blocks are missing: "
|
|
|
- + alignedStripe);
|
|
|
- }
|
|
|
-
|
|
|
- prepareDecodeInputs();
|
|
|
- prepareParityChunk();
|
|
|
// close the corresponding reader
|
|
|
closeReader(r.index);
|
|
|
|
|
|
- for (int i = 0; i < alignedStripe.chunks.length; i++) {
|
|
|
- StripingChunk chunk = alignedStripe.chunks[i];
|
|
|
- if (chunk != null && chunk.state == StripingChunk.REQUESTED) {
|
|
|
- readChunk(service, blocks[i], i, corruptedBlockMap);
|
|
|
- }
|
|
|
- }
|
|
|
+ final int missing = alignedStripe.missingChunksNum;
|
|
|
+ alignedStripe.missingChunksNum++;
|
|
|
+ checkMissingBlocks();
|
|
|
+
|
|
|
+ readDataForDecoding();
|
|
|
+ readParityChunks(alignedStripe.missingChunksNum - missing);
|
|
|
}
|
|
|
} catch (InterruptedException ie) {
|
|
|
String err = "Read request interrupted";
|
|
@@ -686,20 +749,24 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
private byte[][] decodeInputs = null;
|
|
|
|
|
|
PositionStripeReader(CompletionService<Void> service,
|
|
|
- AlignedStripe alignedStripe) {
|
|
|
- super(service, alignedStripe);
|
|
|
+ AlignedStripe alignedStripe, LocatedBlock[] targetBlocks,
|
|
|
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
|
|
|
+ super(service, alignedStripe, targetBlocks, corruptedBlockMap);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- void readChunk(final CompletionService<Void> service,
|
|
|
- final LocatedBlock block, int chunkIndex,
|
|
|
- Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
|
|
|
+ boolean readChunk(final CompletionService<Void> service,
|
|
|
+ final LocatedBlock block, int chunkIndex) {
|
|
|
+ final StripingChunk chunk = alignedStripe.chunks[chunkIndex];
|
|
|
+ if (block == null) {
|
|
|
+ chunk.state = StripingChunk.MISSING;
|
|
|
+ return false;
|
|
|
+ }
|
|
|
DatanodeInfo loc = block.getLocations()[0];
|
|
|
StorageType type = block.getStorageTypes()[0];
|
|
|
DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr(
|
|
|
loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())),
|
|
|
type);
|
|
|
- StripingChunk chunk = alignedStripe.chunks[chunkIndex];
|
|
|
chunk.state = StripingChunk.PENDING;
|
|
|
Callable<Void> readCallable = getFromOneDataNode(dnAddr,
|
|
|
block, alignedStripe.getOffsetInBlock(),
|
|
@@ -715,6 +782,7 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
+ alignedStripe.getSpanInBlock() - 1));
|
|
|
}
|
|
|
futures.put(getFromDNRequest, chunkIndex);
|
|
|
+ return true;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -728,18 +796,15 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- void prepareParityChunk() {
|
|
|
- for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) {
|
|
|
- if (alignedStripe.chunks[i] == null) {
|
|
|
- final int decodeIndex = convertIndex4Decode(i,
|
|
|
- dataBlkNum, parityBlkNum);
|
|
|
- alignedStripe.chunks[i] =
|
|
|
- new StripingChunk(decodeInputs[decodeIndex]);
|
|
|
- alignedStripe.chunks[i].addByteArraySlice(0,
|
|
|
- (int) alignedStripe.getSpanInBlock());
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
+ boolean prepareParityChunk(int index) {
|
|
|
+ Preconditions.checkState(index >= dataBlkNum &&
|
|
|
+ alignedStripe.chunks[index] == null);
|
|
|
+ final int decodeIndex = convertIndex4Decode(index, dataBlkNum,
|
|
|
+ parityBlkNum);
|
|
|
+ alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]);
|
|
|
+ alignedStripe.chunks[index].addByteArraySlice(0,
|
|
|
+ (int) alignedStripe.getSpanInBlock());
|
|
|
+ return true;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -753,39 +818,43 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
|
|
|
class StatefulStripeReader extends StripeReader {
|
|
|
ByteBuffer[] decodeInputs;
|
|
|
- final LocatedBlock[] targetBlocks;
|
|
|
|
|
|
StatefulStripeReader(CompletionService<Void> service,
|
|
|
- AlignedStripe alignedStripe, LocatedBlock[] targetBlocks) {
|
|
|
- super(service, alignedStripe);
|
|
|
- this.targetBlocks = targetBlocks;
|
|
|
+ AlignedStripe alignedStripe, LocatedBlock[] targetBlocks,
|
|
|
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
|
|
|
+ super(service, alignedStripe, targetBlocks, corruptedBlockMap);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- void readChunk(final CompletionService<Void> service,
|
|
|
- final LocatedBlock block, int chunkIndex, Map<ExtendedBlock,
|
|
|
- Set<DatanodeInfo>> corruptedBlockMap) {
|
|
|
- StripingChunk chunk = alignedStripe.chunks[chunkIndex];
|
|
|
+ boolean readChunk(final CompletionService<Void> service,
|
|
|
+ final LocatedBlock block, int chunkIndex) {
|
|
|
+ final StripingChunk chunk = alignedStripe.chunks[chunkIndex];
|
|
|
+ final BlockReaderInfo readerInfo = blockReaders[chunkIndex];
|
|
|
+ if (readerInfo == null || block == null || readerInfo.shouldSkip) {
|
|
|
+ chunk.state = StripingChunk.MISSING;
|
|
|
+ return false;
|
|
|
+ }
|
|
|
chunk.state = StripingChunk.PENDING;
|
|
|
ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer);
|
|
|
- Callable<Void> readCallable = readCell(blockReaders[chunkIndex],
|
|
|
- currentNodes[chunkIndex], blockReaderOffsets[chunkIndex],
|
|
|
+ Callable<Void> readCallable = readCell(readerInfo.reader,
|
|
|
+ readerInfo.datanode, readerInfo.blockReaderOffset,
|
|
|
alignedStripe.getOffsetInBlock(), strategy,
|
|
|
chunk.byteBuffer.remaining(), block.getBlock(), corruptedBlockMap);
|
|
|
Future<Void> request = readingService.submit(readCallable);
|
|
|
futures.put(request, chunkIndex);
|
|
|
+ return true;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
void updateState4SuccessRead(StripingChunkReadResult result) {
|
|
|
Preconditions.checkArgument(
|
|
|
result.state == StripingChunkReadResult.SUCCESSFUL);
|
|
|
- blockReaderOffsets[result.index] =
|
|
|
- alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock();
|
|
|
+ blockReaders[result.index].setOffset(alignedStripe.getOffsetInBlock()
|
|
|
+ + alignedStripe.getSpanInBlock());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- void prepareDecodeInputs() throws IOException {
|
|
|
+ void prepareDecodeInputs() {
|
|
|
if (decodeInputs == null) {
|
|
|
decodeInputs = new ByteBuffer[dataBlkNum + parityBlkNum];
|
|
|
ByteBuffer cur = curStripeBuf.duplicate();
|
|
@@ -799,52 +868,58 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
parityBlkNum);
|
|
|
decodeInputs[decodeIndex] = cur.slice();
|
|
|
if (alignedStripe.chunks[i] == null) {
|
|
|
- alignedStripe.chunks[i] =
|
|
|
- new StripingChunk(decodeInputs[decodeIndex]);
|
|
|
+ alignedStripe.chunks[i] = new StripingChunk(
|
|
|
+ decodeInputs[decodeIndex]);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- void prepareParityChunk() throws IOException {
|
|
|
- for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) {
|
|
|
- if (alignedStripe.chunks[i] == null) {
|
|
|
- final int decodeIndex = convertIndex4Decode(i, dataBlkNum,
|
|
|
- parityBlkNum);
|
|
|
- decodeInputs[decodeIndex] = ByteBuffer.allocateDirect(
|
|
|
- (int) alignedStripe.range.spanInBlock);
|
|
|
- alignedStripe.chunks[i] =
|
|
|
- new StripingChunk(decodeInputs[decodeIndex]);
|
|
|
- if (blockReaders[i] == null) {
|
|
|
- prepareParityBlockReader(i);
|
|
|
- }
|
|
|
- break;
|
|
|
- }
|
|
|
+ boolean prepareParityChunk(int index) throws IOException {
|
|
|
+ Preconditions.checkState(index >= dataBlkNum
|
|
|
+ && alignedStripe.chunks[index] == null);
|
|
|
+ if (blockReaders[index] != null && blockReaders[index].shouldSkip) {
|
|
|
+ alignedStripe.chunks[index] = new StripingChunk(StripingChunk.MISSING);
|
|
|
+ // we have failed the block reader before
|
|
|
+ return false;
|
|
|
}
|
|
|
+ final int decodeIndex = convertIndex4Decode(index, dataBlkNum,
|
|
|
+ parityBlkNum);
|
|
|
+ decodeInputs[decodeIndex] = ByteBuffer.allocateDirect(
|
|
|
+ (int) alignedStripe.range.spanInBlock);
|
|
|
+ alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]);
|
|
|
+ if (blockReaders[index] == null && !prepareParityBlockReader(index)) {
|
|
|
+ alignedStripe.chunks[index] = new StripingChunk(StripingChunk.MISSING);
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ return true;
|
|
|
}
|
|
|
|
|
|
- private void prepareParityBlockReader(int i) throws IOException {
|
|
|
+ private boolean prepareParityBlockReader(int i) throws IOException {
|
|
|
// prepare the block reader for the parity chunk
|
|
|
LocatedBlock targetBlock = targetBlocks[i];
|
|
|
if (targetBlock != null) {
|
|
|
final long offsetInBlock = alignedStripe.getOffsetInBlock();
|
|
|
- DNAddrPair retval = getBestNodeDNAddrPair(targetBlock, null);
|
|
|
- if (retval != null) {
|
|
|
- currentNodes[i] = retval.info;
|
|
|
- blockReaders[i] = getBlockReaderWithRetry(targetBlock,
|
|
|
+ DNAddrPair dnInfo = getBestNodeDNAddrPair(targetBlock, null);
|
|
|
+ if (dnInfo != null) {
|
|
|
+ BlockReader reader = getBlockReaderWithRetry(targetBlock,
|
|
|
offsetInBlock, targetBlock.getBlockSize() - offsetInBlock,
|
|
|
- retval.addr, retval.storageType, retval.info,
|
|
|
+ dnInfo.addr, dnInfo.storageType, dnInfo.info,
|
|
|
DFSStripedInputStream.this.getPos(), retry);
|
|
|
- blockReaderOffsets[i] = offsetInBlock;
|
|
|
+ if (reader != null) {
|
|
|
+ blockReaders[i] = new BlockReaderInfo(reader, targetBlock,
|
|
|
+ dnInfo.info, offsetInBlock);
|
|
|
+ return true;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
+ return false;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
void decode() {
|
|
|
- // TODO no copy for data chunks. this depends on HADOOP-12047 for some
|
|
|
- // decoders to work
|
|
|
+ // TODO no copy for data chunks. this depends on HADOOP-12047
|
|
|
final int span = (int) alignedStripe.getSpanInBlock();
|
|
|
for (int i = 0; i < alignedStripe.chunks.length; i++) {
|
|
|
final int decodeIndex = convertIndex4Decode(i,
|