|
@@ -31,14 +31,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
|
import org.apache.hadoop.io.ByteBufferPool;
|
|
|
|
|
|
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.convertIndex4Decode;
|
|
|
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.divideByteRangeIntoStripes;
|
|
|
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.finalizeDecodeInputs;
|
|
|
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.decodeAndFillBuffer;
|
|
|
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getNextCompletedStripedRead;
|
|
|
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getStartOffsetsForInternalBlocks;
|
|
|
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.initDecodeInputs;
|
|
|
-import static org.apache.hadoop.hdfs.util.StripedBlockUtil.parseStripedBlockGroup;
|
|
|
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;
|
|
@@ -48,10 +40,6 @@ import org.apache.hadoop.io.erasurecode.CodecUtil;
|
|
|
import org.apache.hadoop.io.erasurecode.ECSchema;
|
|
|
|
|
|
import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
|
|
|
-import org.apache.hadoop.net.NetUtils;
|
|
|
-import org.apache.htrace.Span;
|
|
|
-import org.apache.htrace.Trace;
|
|
|
-import org.apache.htrace.TraceScope;
|
|
|
|
|
|
import java.io.EOFException;
|
|
|
import java.io.IOException;
|
|
@@ -166,7 +154,6 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
*/
|
|
|
private StripeRange curStripeRange;
|
|
|
private final CompletionService<Void> readingService;
|
|
|
- private ReaderRetryPolicy retry;
|
|
|
|
|
|
DFSStripedInputStream(DFSClient dfsClient, String src,
|
|
|
boolean verifyChecksum, ECSchema schema, int cellSize,
|
|
@@ -198,18 +185,6 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
curStripeRange = new StripeRange(0, 0);
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public synchronized int read(final ByteBuffer buf) throws IOException {
|
|
|
- ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
|
|
|
- TraceScope scope =
|
|
|
- dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src);
|
|
|
- try {
|
|
|
- return readWithStrategy(byteBufferReader, 0, buf.remaining());
|
|
|
- } finally {
|
|
|
- scope.close();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* When seeking into a new block group, create blockReader for each internal
|
|
|
* block in the group.
|
|
@@ -229,33 +204,6 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
this.blockEnd = targetBlockGroup.getStartOffset() +
|
|
|
targetBlockGroup.getBlockSize() - 1;
|
|
|
currentLocatedBlock = targetBlockGroup;
|
|
|
-
|
|
|
- final long offsetIntoBlockGroup = getOffsetInBlockGroup();
|
|
|
- LocatedBlock[] targetBlocks = parseStripedBlockGroup(
|
|
|
- targetBlockGroup, cellSize, dataBlkNum, parityBlkNum);
|
|
|
- // The purpose is to get start offset into each block.
|
|
|
- long[] offsetsForInternalBlocks = getStartOffsetsForInternalBlocks(schema,
|
|
|
- cellSize, targetBlockGroup, offsetIntoBlockGroup);
|
|
|
- Preconditions.checkState(offsetsForInternalBlocks.length ==
|
|
|
- dataBlkNum + parityBlkNum);
|
|
|
- long minOffset = offsetsForInternalBlocks[dataBlkNum];
|
|
|
-
|
|
|
- retry = new ReaderRetryPolicy();
|
|
|
- for (int i = 0; i < dataBlkNum; i++) {
|
|
|
- LocatedBlock targetBlock = targetBlocks[i];
|
|
|
- if (targetBlock != null) {
|
|
|
- DNAddrPair dnInfo = getBestNodeDNAddrPair(targetBlock, null);
|
|
|
- if (dnInfo != null) {
|
|
|
- BlockReader reader = getBlockReaderWithRetry(targetBlock,
|
|
|
- minOffset, targetBlock.getBlockSize() - minOffset,
|
|
|
- dnInfo.addr, dnInfo.storageType, dnInfo.info, target, retry);
|
|
|
- if (reader != null) {
|
|
|
- blockReaders[i] = new BlockReaderInfo(reader, targetBlock,
|
|
|
- dnInfo.info, minOffset);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -308,16 +256,16 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
return;
|
|
|
}
|
|
|
for (int i = 0; i < groupSize; i++) {
|
|
|
- closeReader(i);
|
|
|
+ closeReader(blockReaders[i]);
|
|
|
blockReaders[i] = null;
|
|
|
}
|
|
|
blockEnd = -1;
|
|
|
}
|
|
|
|
|
|
- private void closeReader(int index) {
|
|
|
- if (blockReaders[index] != null) {
|
|
|
- IOUtils.cleanup(DFSClient.LOG, blockReaders[index].reader);
|
|
|
- blockReaders[index].skip();
|
|
|
+ private void closeReader(BlockReaderInfo readerInfo) {
|
|
|
+ if (readerInfo != null) {
|
|
|
+ IOUtils.cleanup(DFSClient.LOG, readerInfo.reader);
|
|
|
+ readerInfo.skip();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -358,17 +306,17 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
for (AlignedStripe stripe : stripes) {
|
|
|
// Parse group to get chosen DN location
|
|
|
StripeReader sreader = new StatefulStripeReader(readingService, stripe,
|
|
|
- blks, corruptedBlockMap);
|
|
|
+ blks, blockReaders, corruptedBlockMap);
|
|
|
sreader.readStripe();
|
|
|
}
|
|
|
curStripeBuf.position(stripeBufOffset);
|
|
|
curStripeBuf.limit(stripeLimit);
|
|
|
}
|
|
|
|
|
|
- private Callable<Void> readCell(final BlockReader reader,
|
|
|
+ private Callable<Void> readCells(final BlockReader reader,
|
|
|
final DatanodeInfo datanode, final long currentReaderOffset,
|
|
|
- final long targetReaderOffset, final ByteBufferStrategy strategy,
|
|
|
- final int targetLength, final ExtendedBlock currentBlock,
|
|
|
+ final long targetReaderOffset, final ByteBufferStrategy[] strategies,
|
|
|
+ final ExtendedBlock currentBlock,
|
|
|
final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
|
|
|
return new Callable<Void>() {
|
|
|
@Override
|
|
@@ -386,27 +334,31 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
skipped == targetReaderOffset - currentReaderOffset);
|
|
|
}
|
|
|
int result = 0;
|
|
|
- while (result < targetLength) {
|
|
|
- int ret = readToBuffer(reader, datanode, strategy, currentBlock,
|
|
|
+ for (ByteBufferStrategy strategy : strategies) {
|
|
|
+ result += readToBuffer(reader, datanode, strategy, currentBlock,
|
|
|
corruptedBlockMap);
|
|
|
- if (ret < 0) {
|
|
|
- throw new IOException("Unexpected EOS from the reader");
|
|
|
- }
|
|
|
- result += ret;
|
|
|
}
|
|
|
- updateReadStatistics(readStatistics, targetLength, reader);
|
|
|
return null;
|
|
|
}
|
|
|
};
|
|
|
}
|
|
|
|
|
|
private int readToBuffer(BlockReader blockReader,
|
|
|
- DatanodeInfo currentNode, ByteBufferStrategy readerStrategy,
|
|
|
+ DatanodeInfo currentNode, ByteBufferStrategy strategy,
|
|
|
ExtendedBlock currentBlock,
|
|
|
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
|
|
|
throws IOException {
|
|
|
+ final int targetLength = strategy.buf.remaining();
|
|
|
+ int length = 0;
|
|
|
try {
|
|
|
- return readerStrategy.doRead(blockReader, 0, 0);
|
|
|
+ while (length < targetLength) {
|
|
|
+ int ret = strategy.doRead(blockReader, 0, 0);
|
|
|
+ if (ret < 0) {
|
|
|
+ throw new IOException("Unexpected EOS from the reader");
|
|
|
+ }
|
|
|
+ length += ret;
|
|
|
+ }
|
|
|
+ return length;
|
|
|
} catch (ChecksumException ce) {
|
|
|
DFSClient.LOG.warn("Found Checksum error for "
|
|
|
+ currentBlock + " from " + currentNode
|
|
@@ -572,61 +524,49 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
// Refresh the striped block group
|
|
|
LocatedStripedBlock blockGroup = getBlockGroupAt(block.getStartOffset());
|
|
|
|
|
|
- AlignedStripe[] stripes = divideByteRangeIntoStripes(schema, cellSize,
|
|
|
- blockGroup, start, end, buf, offset);
|
|
|
+ AlignedStripe[] stripes = StripedBlockUtil.divideByteRangeIntoStripes(
|
|
|
+ schema, cellSize, blockGroup, start, end, buf, offset);
|
|
|
CompletionService<Void> readService = new ExecutorCompletionService<>(
|
|
|
dfsClient.getStripedReadsThreadPool());
|
|
|
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,
|
|
|
- blks, corruptedBlockMap);
|
|
|
- preader.readStripe();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private Callable<Void> getFromOneDataNode(final DNAddrPair datanode,
|
|
|
- final LocatedBlock block, final long start, final long end,
|
|
|
- final byte[] buf, final int[] offsets, final int[] lengths,
|
|
|
- final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
|
|
|
- final int hedgedReadId) {
|
|
|
- final Span parentSpan = Trace.currentSpan();
|
|
|
- return new Callable<Void>() {
|
|
|
- @Override
|
|
|
- public Void call() throws Exception {
|
|
|
- TraceScope scope =
|
|
|
- Trace.startSpan("Parallel reading " + hedgedReadId, parentSpan);
|
|
|
- try {
|
|
|
- actualGetFromOneDataNode(datanode, block, start,
|
|
|
- end, buf, offsets, lengths, corruptedBlockMap);
|
|
|
- } finally {
|
|
|
- scope.close();
|
|
|
- }
|
|
|
- return null;
|
|
|
+ final BlockReaderInfo[] preaderInfos = new BlockReaderInfo[groupSize];
|
|
|
+ try {
|
|
|
+ for (AlignedStripe stripe : stripes) {
|
|
|
+ // Parse group to get chosen DN location
|
|
|
+ StripeReader preader = new PositionStripeReader(readService, stripe,
|
|
|
+ blks, preaderInfos, corruptedBlockMap);
|
|
|
+ preader.readStripe();
|
|
|
}
|
|
|
- };
|
|
|
+ } finally {
|
|
|
+ for (BlockReaderInfo preaderInfo : preaderInfos) {
|
|
|
+ closeReader(preaderInfo);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * The reader for reading a complete {@link AlignedStripe}. Note that an
|
|
|
+ * {@link AlignedStripe} may cross multiple stripes with cellSize width.
|
|
|
+ */
|
|
|
private abstract class StripeReader {
|
|
|
final Map<Future<Void>, Integer> futures = new HashMap<>();
|
|
|
final AlignedStripe alignedStripe;
|
|
|
final CompletionService<Void> service;
|
|
|
final LocatedBlock[] targetBlocks;
|
|
|
final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap;
|
|
|
+ final BlockReaderInfo[] readerInfos;
|
|
|
|
|
|
StripeReader(CompletionService<Void> service, AlignedStripe alignedStripe,
|
|
|
- LocatedBlock[] targetBlocks,
|
|
|
+ LocatedBlock[] targetBlocks, BlockReaderInfo[] readerInfos,
|
|
|
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
|
|
|
this.service = service;
|
|
|
this.alignedStripe = alignedStripe;
|
|
|
this.targetBlocks = targetBlocks;
|
|
|
+ this.readerInfos = readerInfos;
|
|
|
this.corruptedBlockMap = corruptedBlockMap;
|
|
|
}
|
|
|
|
|
|
- abstract boolean readChunk(final CompletionService<Void> service,
|
|
|
- final LocatedBlock block, int chunkIndex);
|
|
|
-
|
|
|
/** prepare all the data chunks */
|
|
|
abstract void prepareDecodeInputs();
|
|
|
|
|
@@ -635,7 +575,12 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
|
|
|
abstract void decode();
|
|
|
|
|
|
- abstract void updateState4SuccessRead(StripingChunkReadResult result);
|
|
|
+ void updateState4SuccessRead(StripingChunkReadResult result) {
|
|
|
+ Preconditions.checkArgument(
|
|
|
+ result.state == StripingChunkReadResult.SUCCESSFUL);
|
|
|
+ readerInfos[result.index].setOffset(alignedStripe.getOffsetInBlock()
|
|
|
+ + alignedStripe.getSpanInBlock());
|
|
|
+ }
|
|
|
|
|
|
private void checkMissingBlocks() throws IOException {
|
|
|
if (alignedStripe.missingChunksNum > parityBlkNum) {
|
|
@@ -654,7 +599,7 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
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)) {
|
|
|
+ if (!readChunk(targetBlocks[i], i)) {
|
|
|
alignedStripe.missingChunksNum++;
|
|
|
}
|
|
|
}
|
|
@@ -666,7 +611,7 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
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)) {
|
|
|
+ if (prepareParityChunk(i) && readChunk(targetBlocks[i], i)) {
|
|
|
j++;
|
|
|
} else {
|
|
|
alignedStripe.missingChunksNum++;
|
|
@@ -676,12 +621,75 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
checkMissingBlocks();
|
|
|
}
|
|
|
|
|
|
+ boolean createBlockReader(LocatedBlock block, int chunkIndex)
|
|
|
+ throws IOException {
|
|
|
+ DNAddrPair dnInfo = getBestNodeDNAddrPair(block, null);
|
|
|
+ if (dnInfo != null) {
|
|
|
+ BlockReader reader = getBlockReaderWithRetry(block,
|
|
|
+ alignedStripe.getOffsetInBlock(),
|
|
|
+ block.getBlockSize() - alignedStripe.getOffsetInBlock(),
|
|
|
+ dnInfo.addr, dnInfo.storageType, dnInfo.info,
|
|
|
+ block.getStartOffset(), new ReaderRetryPolicy());
|
|
|
+ if (reader != null) {
|
|
|
+ readerInfos[chunkIndex] = new BlockReaderInfo(reader, block,
|
|
|
+ dnInfo.info, alignedStripe.getOffsetInBlock());
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) {
|
|
|
+ if (chunk.byteBuffer != null) {
|
|
|
+ ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer);
|
|
|
+ return new ByteBufferStrategy[]{strategy};
|
|
|
+ } else {
|
|
|
+ ByteBufferStrategy[] strategies =
|
|
|
+ new ByteBufferStrategy[chunk.byteArray.getOffsets().length];
|
|
|
+ for (int i = 0; i < strategies.length; i++) {
|
|
|
+ ByteBuffer buffer = ByteBuffer.wrap(chunk.byteArray.buf(),
|
|
|
+ chunk.byteArray.getOffsets()[i], chunk.byteArray.getLengths()[i]);
|
|
|
+ strategies[i] = new ByteBufferStrategy(buffer);
|
|
|
+ }
|
|
|
+ return strategies;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean readChunk(final LocatedBlock block, int chunkIndex)
|
|
|
+ throws IOException {
|
|
|
+ final StripingChunk chunk = alignedStripe.chunks[chunkIndex];
|
|
|
+ if (block == null) {
|
|
|
+ chunk.state = StripingChunk.MISSING;
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ if (readerInfos[chunkIndex] == null) {
|
|
|
+ if (!createBlockReader(block, chunkIndex)) {
|
|
|
+ chunk.state = StripingChunk.MISSING;
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ } else if (readerInfos[chunkIndex].shouldSkip) {
|
|
|
+ chunk.state = StripingChunk.MISSING;
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ chunk.state = StripingChunk.PENDING;
|
|
|
+ Callable<Void> readCallable = readCells(readerInfos[chunkIndex].reader,
|
|
|
+ readerInfos[chunkIndex].datanode,
|
|
|
+ readerInfos[chunkIndex].blockReaderOffset,
|
|
|
+ alignedStripe.getOffsetInBlock(), getReadStrategies(chunk),
|
|
|
+ block.getBlock(), corruptedBlockMap);
|
|
|
+
|
|
|
+ Future<Void> request = service.submit(readCallable);
|
|
|
+ futures.put(request, chunkIndex);
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
/** read the whole stripe. do decoding if necessary */
|
|
|
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)) {
|
|
|
+ if (!readChunk(targetBlocks[i], i)) {
|
|
|
alignedStripe.missingChunksNum++;
|
|
|
}
|
|
|
}
|
|
@@ -700,8 +708,8 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
// first read failure
|
|
|
while (!futures.isEmpty()) {
|
|
|
try {
|
|
|
- StripingChunkReadResult r = getNextCompletedStripedRead(service,
|
|
|
- futures, 0);
|
|
|
+ StripingChunkReadResult r = StripedBlockUtil
|
|
|
+ .getNextCompletedStripedRead(service, futures, 0);
|
|
|
if (DFSClient.LOG.isDebugEnabled()) {
|
|
|
DFSClient.LOG.debug("Read task returned: " + r + ", for stripe "
|
|
|
+ alignedStripe);
|
|
@@ -721,7 +729,7 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
} else {
|
|
|
returnedChunk.state = StripingChunk.MISSING;
|
|
|
// close the corresponding reader
|
|
|
- closeReader(r.index);
|
|
|
+ closeReader(readerInfos[r.index]);
|
|
|
|
|
|
final int missing = alignedStripe.missingChunksNum;
|
|
|
alignedStripe.missingChunksNum++;
|
|
@@ -750,48 +758,17 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
|
|
|
PositionStripeReader(CompletionService<Void> service,
|
|
|
AlignedStripe alignedStripe, LocatedBlock[] targetBlocks,
|
|
|
+ BlockReaderInfo[] readerInfos,
|
|
|
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
|
|
|
- super(service, alignedStripe, targetBlocks, corruptedBlockMap);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- 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);
|
|
|
- chunk.state = StripingChunk.PENDING;
|
|
|
- Callable<Void> readCallable = getFromOneDataNode(dnAddr,
|
|
|
- block, alignedStripe.getOffsetInBlock(),
|
|
|
- alignedStripe.getOffsetInBlock() + alignedStripe.getSpanInBlock() - 1,
|
|
|
- chunk.byteArray.buf(), chunk.byteArray.getOffsets(),
|
|
|
- chunk.byteArray.getLengths(), corruptedBlockMap, chunkIndex);
|
|
|
- Future<Void> getFromDNRequest = service.submit(readCallable);
|
|
|
- if (DFSClient.LOG.isDebugEnabled()) {
|
|
|
- DFSClient.LOG.debug("Submitting striped read request for " + chunkIndex
|
|
|
- + ". Info of the block: " + block + ", offset in block is "
|
|
|
- + alignedStripe.getOffsetInBlock() + ", end is "
|
|
|
- + (alignedStripe.getOffsetInBlock()
|
|
|
- + alignedStripe.getSpanInBlock() - 1));
|
|
|
- }
|
|
|
- futures.put(getFromDNRequest, chunkIndex);
|
|
|
- return true;
|
|
|
+ super(service, alignedStripe, targetBlocks, readerInfos,
|
|
|
+ corruptedBlockMap);
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- void updateState4SuccessRead(StripingChunkReadResult r) {}
|
|
|
-
|
|
|
@Override
|
|
|
void prepareDecodeInputs() {
|
|
|
if (decodeInputs == null) {
|
|
|
- decodeInputs = initDecodeInputs(alignedStripe, dataBlkNum, parityBlkNum);
|
|
|
+ decodeInputs = StripedBlockUtil.initDecodeInputs(alignedStripe,
|
|
|
+ dataBlkNum, parityBlkNum);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -799,8 +776,8 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
boolean prepareParityChunk(int index) {
|
|
|
Preconditions.checkState(index >= dataBlkNum &&
|
|
|
alignedStripe.chunks[index] == null);
|
|
|
- final int decodeIndex = convertIndex4Decode(index, dataBlkNum,
|
|
|
- parityBlkNum);
|
|
|
+ final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index,
|
|
|
+ dataBlkNum, parityBlkNum);
|
|
|
alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]);
|
|
|
alignedStripe.chunks[index].addByteArraySlice(0,
|
|
|
(int) alignedStripe.getSpanInBlock());
|
|
@@ -809,10 +786,10 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
|
|
|
@Override
|
|
|
void decode() {
|
|
|
- finalizeDecodeInputs(decodeInputs, dataBlkNum, parityBlkNum,
|
|
|
- alignedStripe);
|
|
|
- decodeAndFillBuffer(decodeInputs, alignedStripe, dataBlkNum,
|
|
|
- parityBlkNum, decoder);
|
|
|
+ StripedBlockUtil.finalizeDecodeInputs(decodeInputs, dataBlkNum,
|
|
|
+ parityBlkNum, alignedStripe);
|
|
|
+ StripedBlockUtil.decodeAndFillBuffer(decodeInputs, alignedStripe,
|
|
|
+ dataBlkNum, parityBlkNum, decoder);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -821,36 +798,10 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
|
|
|
StatefulStripeReader(CompletionService<Void> service,
|
|
|
AlignedStripe alignedStripe, LocatedBlock[] targetBlocks,
|
|
|
+ BlockReaderInfo[] readerInfos,
|
|
|
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
|
|
|
- super(service, alignedStripe, targetBlocks, corruptedBlockMap);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- 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(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);
|
|
|
- blockReaders[result.index].setOffset(alignedStripe.getOffsetInBlock()
|
|
|
- + alignedStripe.getSpanInBlock());
|
|
|
+ super(service, alignedStripe, targetBlocks, readerInfos,
|
|
|
+ corruptedBlockMap);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -864,8 +815,8 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
int pos = (int) (range.offsetInBlock % cellSize + cellSize * i);
|
|
|
cur.position(pos);
|
|
|
cur.limit((int) (pos + range.spanInBlock));
|
|
|
- final int decodeIndex = convertIndex4Decode(i, dataBlkNum,
|
|
|
- parityBlkNum);
|
|
|
+ final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
|
|
|
+ dataBlkNum, parityBlkNum);
|
|
|
decodeInputs[decodeIndex] = cur.slice();
|
|
|
if (alignedStripe.chunks[i] == null) {
|
|
|
alignedStripe.chunks[i] = new StripingChunk(
|
|
@@ -884,45 +835,20 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
// we have failed the block reader before
|
|
|
return false;
|
|
|
}
|
|
|
- final int decodeIndex = convertIndex4Decode(index, dataBlkNum,
|
|
|
- parityBlkNum);
|
|
|
+ final int decodeIndex = StripedBlockUtil.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 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 dnInfo = getBestNodeDNAddrPair(targetBlock, null);
|
|
|
- if (dnInfo != null) {
|
|
|
- BlockReader reader = getBlockReaderWithRetry(targetBlock,
|
|
|
- offsetInBlock, targetBlock.getBlockSize() - offsetInBlock,
|
|
|
- dnInfo.addr, dnInfo.storageType, dnInfo.info,
|
|
|
- DFSStripedInputStream.this.getPos(), retry);
|
|
|
- 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
|
|
|
final int span = (int) alignedStripe.getSpanInBlock();
|
|
|
for (int i = 0; i < alignedStripe.chunks.length; i++) {
|
|
|
- final int decodeIndex = convertIndex4Decode(i,
|
|
|
+ final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i,
|
|
|
dataBlkNum, parityBlkNum);
|
|
|
if (alignedStripe.chunks[i] != null &&
|
|
|
alignedStripe.chunks[i].state == StripingChunk.ALLZERO) {
|
|
@@ -941,7 +867,7 @@ public class DFSStripedInputStream extends DFSInputStream {
|
|
|
for (int i = 0; i < alignedStripe.chunks.length; i++) {
|
|
|
if (alignedStripe.chunks[i] != null &&
|
|
|
alignedStripe.chunks[i].state == StripingChunk.MISSING) {
|
|
|
- decodeIndices[pos++] = convertIndex4Decode(i,
|
|
|
+ decodeIndices[pos++] = StripedBlockUtil.convertIndex4Decode(i,
|
|
|
dataBlkNum, parityBlkNum);
|
|
|
}
|
|
|
}
|