|
@@ -1016,7 +1016,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
}
|
|
|
deadNodes.clear(); //2nd option is to remove only nodes[blockId]
|
|
|
openInfo();
|
|
|
- block = getBlockAt(block.getStartOffset());
|
|
|
+ block = refreshLocatedBlock(block);
|
|
|
failures++;
|
|
|
}
|
|
|
}
|
|
@@ -1088,15 +1088,15 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
return errMsgr.toString();
|
|
|
}
|
|
|
|
|
|
- protected void fetchBlockByteRange(long blockStartOffset, long start, long end,
|
|
|
+ protected void fetchBlockByteRange(LocatedBlock block, long start, long end,
|
|
|
byte[] buf, int offset,
|
|
|
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
|
|
|
throws IOException {
|
|
|
- LocatedBlock block = getBlockAt(blockStartOffset);
|
|
|
+ block = refreshLocatedBlock(block);
|
|
|
while (true) {
|
|
|
DNAddrPair addressPair = chooseDataNode(block, null);
|
|
|
try {
|
|
|
- actualGetFromOneDataNode(addressPair, blockStartOffset, start, end,
|
|
|
+ actualGetFromOneDataNode(addressPair, block, start, end,
|
|
|
buf, offset, corruptedBlockMap);
|
|
|
return;
|
|
|
} catch (IOException e) {
|
|
@@ -1107,7 +1107,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
}
|
|
|
|
|
|
private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
|
|
|
- final long blockStartOffset, final long start, final long end,
|
|
|
+ final LocatedBlock block, final long start, final long end,
|
|
|
final ByteBuffer bb,
|
|
|
final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
|
|
|
final int hedgedReadId) {
|
|
@@ -1120,7 +1120,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
TraceScope scope =
|
|
|
Trace.startSpan("hedgedRead" + hedgedReadId, parentSpan);
|
|
|
try {
|
|
|
- actualGetFromOneDataNode(datanode, blockStartOffset, start, end, buf,
|
|
|
+ actualGetFromOneDataNode(datanode, block, start, end, buf,
|
|
|
offset, corruptedBlockMap);
|
|
|
return bb;
|
|
|
} finally {
|
|
@@ -1134,18 +1134,18 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
* Used when reading contiguous blocks
|
|
|
*/
|
|
|
private void actualGetFromOneDataNode(final DNAddrPair datanode,
|
|
|
- long blockStartOffset, final long start, final long end, byte[] buf,
|
|
|
+ LocatedBlock block, final long start, final long end, byte[] buf,
|
|
|
int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
|
|
|
throws IOException {
|
|
|
final int length = (int) (end - start + 1);
|
|
|
- actualGetFromOneDataNode(datanode, blockStartOffset, start, end, buf,
|
|
|
+ actualGetFromOneDataNode(datanode, block, start, end, buf,
|
|
|
new int[]{offset}, new int[]{length}, corruptedBlockMap);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Read data from one DataNode.
|
|
|
* @param datanode the datanode from which to read data
|
|
|
- * @param blockStartOffset starting offset in the file
|
|
|
+ * @param block the located block containing the requested data
|
|
|
* @param startInBlk the startInBlk offset of the block
|
|
|
* @param endInBlk the endInBlk offset of the block
|
|
|
* @param buf the given byte array into which the data is read
|
|
@@ -1157,7 +1157,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
* block replica
|
|
|
*/
|
|
|
void actualGetFromOneDataNode(final DNAddrPair datanode,
|
|
|
- long blockStartOffset, final long startInBlk, final long endInBlk,
|
|
|
+ LocatedBlock block, final long startInBlk, final long endInBlk,
|
|
|
byte[] buf, int[] offsets, int[] lengths,
|
|
|
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
|
|
|
throws IOException {
|
|
@@ -1171,7 +1171,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
// cached block locations may have been updated by chooseDataNode()
|
|
|
// or fetchBlockAt(). Always get the latest list of locations at the
|
|
|
// start of the loop.
|
|
|
- LocatedBlock block = getBlockAt(blockStartOffset);
|
|
|
+ block = refreshLocatedBlock(block);
|
|
|
BlockReader reader = null;
|
|
|
try {
|
|
|
DFSClientFaultInjector.get().fetchFromDatanodeException();
|
|
@@ -1227,6 +1227,17 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Refresh cached block locations.
|
|
|
+ * @param block The currently cached block locations
|
|
|
+ * @return Refreshed block locations
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ protected LocatedBlock refreshLocatedBlock(LocatedBlock block)
|
|
|
+ throws IOException {
|
|
|
+ return getBlockAt(block.getStartOffset());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* This method verifies that the read portions are valid and do not overlap
|
|
|
* with each other.
|
|
@@ -1250,7 +1261,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
* 'hedged' read if the first read is taking longer than configured amount of
|
|
|
* time. We then wait on which ever read returns first.
|
|
|
*/
|
|
|
- private void hedgedFetchBlockByteRange(long blockStartOffset, long start,
|
|
|
+ private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
|
|
|
long end, byte[] buf, int offset,
|
|
|
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
|
|
|
throws IOException {
|
|
@@ -1263,7 +1274,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
ByteBuffer bb = null;
|
|
|
int len = (int) (end - start + 1);
|
|
|
int hedgedReadId = 0;
|
|
|
- LocatedBlock block = getBlockAt(blockStartOffset);
|
|
|
+ block = refreshLocatedBlock(block);
|
|
|
while (true) {
|
|
|
// see HDFS-6591, this metric is used to verify/catch unnecessary loops
|
|
|
hedgedReadOpsLoopNumForTesting++;
|
|
@@ -1275,7 +1286,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
chosenNode = chooseDataNode(block, ignored);
|
|
|
bb = ByteBuffer.wrap(buf, offset, len);
|
|
|
Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
|
|
|
- chosenNode, block.getStartOffset(), start, end, bb,
|
|
|
+ chosenNode, block, start, end, bb,
|
|
|
corruptedBlockMap, hedgedReadId++);
|
|
|
Future<ByteBuffer> firstRequest = hedgedService
|
|
|
.submit(getFromDataNodeCallable);
|
|
@@ -1312,7 +1323,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
}
|
|
|
bb = ByteBuffer.allocate(len);
|
|
|
Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
|
|
|
- chosenNode, block.getStartOffset(), start, end, bb,
|
|
|
+ chosenNode, block, start, end, bb,
|
|
|
corruptedBlockMap, hedgedReadId++);
|
|
|
Future<ByteBuffer> oneMoreRequest = hedgedService
|
|
|
.submit(getFromDataNodeCallable);
|
|
@@ -1466,12 +1477,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
|
|
|
try {
|
|
|
if (dfsClient.isHedgedReadsEnabled() && !blk.isStriped()) {
|
|
|
- hedgedFetchBlockByteRange(blk.getStartOffset(), targetStart,
|
|
|
+ hedgedFetchBlockByteRange(blk, targetStart,
|
|
|
targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
|
|
|
} else {
|
|
|
- fetchBlockByteRange(blk.getStartOffset(), targetStart,
|
|
|
- targetStart + bytesToRead - 1, buffer, offset,
|
|
|
- corruptedBlockMap);
|
|
|
+ fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
|
|
|
+ buffer, offset, corruptedBlockMap);
|
|
|
}
|
|
|
} finally {
|
|
|
// Check and report if any block replicas are corrupted.
|