|
@@ -228,7 +228,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
dfsClient.getConf().shortCircuitStreamsCacheSize,
|
|
dfsClient.getConf().shortCircuitStreamsCacheSize,
|
|
dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
|
|
dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
|
|
this.cachingStrategy =
|
|
this.cachingStrategy =
|
|
- dfsClient.getDefaultReadCachingStrategy().duplicate();
|
|
|
|
|
|
+ dfsClient.getDefaultReadCachingStrategy();
|
|
openInfo();
|
|
openInfo();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -574,7 +574,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
|
|
Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
|
|
blockReader = getBlockReader(targetAddr, chosenNode, src, blk,
|
|
blockReader = getBlockReader(targetAddr, chosenNode, src, blk,
|
|
accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
|
|
accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
|
|
- buffersize, verifyChecksum, dfsClient.clientName);
|
|
|
|
|
|
+ buffersize, verifyChecksum, dfsClient.clientName, cachingStrategy);
|
|
if(connectFailedOnce) {
|
|
if(connectFailedOnce) {
|
|
DFSClient.LOG.info("Successfully connected to " + targetAddr +
|
|
DFSClient.LOG.info("Successfully connected to " + targetAddr +
|
|
" for " + blk);
|
|
" for " + blk);
|
|
@@ -928,7 +928,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
// cached block locations may have been updated by chooseDataNode()
|
|
// cached block locations may have been updated by chooseDataNode()
|
|
// or fetchBlockAt(). Always get the latest list of locations at the
|
|
// or fetchBlockAt(). Always get the latest list of locations at the
|
|
// start of the loop.
|
|
// start of the loop.
|
|
- block = getBlockAt(block.getStartOffset(), false);
|
|
|
|
|
|
+ CachingStrategy curCachingStrategy;
|
|
|
|
+ synchronized (this) {
|
|
|
|
+ block = getBlockAt(block.getStartOffset(), false);
|
|
|
|
+ curCachingStrategy = cachingStrategy;
|
|
|
|
+ }
|
|
DNAddrPair retval = chooseDataNode(block);
|
|
DNAddrPair retval = chooseDataNode(block);
|
|
DatanodeInfo chosenNode = retval.info;
|
|
DatanodeInfo chosenNode = retval.info;
|
|
InetSocketAddress targetAddr = retval.addr;
|
|
InetSocketAddress targetAddr = retval.addr;
|
|
@@ -940,7 +944,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
int len = (int) (end - start + 1);
|
|
int len = (int) (end - start + 1);
|
|
reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(),
|
|
reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(),
|
|
blockToken, start, len, buffersize, verifyChecksum,
|
|
blockToken, start, len, buffersize, verifyChecksum,
|
|
- dfsClient.clientName);
|
|
|
|
|
|
+ dfsClient.clientName, curCachingStrategy);
|
|
int nread = reader.readAll(buf, offset, len);
|
|
int nread = reader.readAll(buf, offset, len);
|
|
if (nread != len) {
|
|
if (nread != len) {
|
|
throw new IOException("truncated return from reader.read(): " +
|
|
throw new IOException("truncated return from reader.read(): " +
|
|
@@ -1053,6 +1057,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
* @param bufferSize The IO buffer size (not the client buffer size)
|
|
* @param bufferSize The IO buffer size (not the client buffer size)
|
|
* @param verifyChecksum Whether to verify checksum
|
|
* @param verifyChecksum Whether to verify checksum
|
|
* @param clientName Client name
|
|
* @param clientName Client name
|
|
|
|
+ * @param CachingStrategy caching strategy to use
|
|
* @return New BlockReader instance
|
|
* @return New BlockReader instance
|
|
*/
|
|
*/
|
|
protected BlockReader getBlockReader(InetSocketAddress dnAddr,
|
|
protected BlockReader getBlockReader(InetSocketAddress dnAddr,
|
|
@@ -1064,7 +1069,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
long len,
|
|
long len,
|
|
int bufferSize,
|
|
int bufferSize,
|
|
boolean verifyChecksum,
|
|
boolean verifyChecksum,
|
|
- String clientName)
|
|
|
|
|
|
+ String clientName,
|
|
|
|
+ CachingStrategy curCachingStrategy)
|
|
throws IOException {
|
|
throws IOException {
|
|
// Firstly, we check to see if we have cached any file descriptors for
|
|
// Firstly, we check to see if we have cached any file descriptors for
|
|
// local blocks. If so, we can just re-use those file descriptors.
|
|
// local blocks. If so, we can just re-use those file descriptors.
|
|
@@ -1084,7 +1090,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
setBlockMetadataHeader(BlockMetadataHeader.
|
|
setBlockMetadataHeader(BlockMetadataHeader.
|
|
preadHeader(fis[1].getChannel())).
|
|
preadHeader(fis[1].getChannel())).
|
|
setFileInputStreamCache(fileInputStreamCache).
|
|
setFileInputStreamCache(fileInputStreamCache).
|
|
- setCachingStrategy(cachingStrategy).
|
|
|
|
|
|
+ setCachingStrategy(curCachingStrategy).
|
|
build();
|
|
build();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1119,7 +1125,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
dfsClient.getConf(), file, block, blockToken, startOffset,
|
|
dfsClient.getConf(), file, block, blockToken, startOffset,
|
|
len, verifyChecksum, clientName, peer, chosenNode,
|
|
len, verifyChecksum, clientName, peer, chosenNode,
|
|
dsFactory, peerCache, fileInputStreamCache,
|
|
dsFactory, peerCache, fileInputStreamCache,
|
|
- allowShortCircuitLocalReads, cachingStrategy);
|
|
|
|
|
|
+ allowShortCircuitLocalReads, curCachingStrategy);
|
|
return reader;
|
|
return reader;
|
|
} catch (IOException ex) {
|
|
} catch (IOException ex) {
|
|
DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
|
|
DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
|
|
@@ -1142,7 +1148,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
dfsClient.getConf(), file, block, blockToken, startOffset,
|
|
dfsClient.getConf(), file, block, blockToken, startOffset,
|
|
len, verifyChecksum, clientName, peer, chosenNode,
|
|
len, verifyChecksum, clientName, peer, chosenNode,
|
|
dsFactory, peerCache, fileInputStreamCache,
|
|
dsFactory, peerCache, fileInputStreamCache,
|
|
- allowShortCircuitLocalReads, cachingStrategy);
|
|
|
|
|
|
+ allowShortCircuitLocalReads, curCachingStrategy);
|
|
return reader;
|
|
return reader;
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
DFSClient.LOG.warn("failed to connect to " + domSock, e);
|
|
DFSClient.LOG.warn("failed to connect to " + domSock, e);
|
|
@@ -1166,7 +1172,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
dfsClient.getConf(), file, block, blockToken, startOffset,
|
|
dfsClient.getConf(), file, block, blockToken, startOffset,
|
|
len, verifyChecksum, clientName, peer, chosenNode,
|
|
len, verifyChecksum, clientName, peer, chosenNode,
|
|
dsFactory, peerCache, fileInputStreamCache, false,
|
|
dsFactory, peerCache, fileInputStreamCache, false,
|
|
- cachingStrategy);
|
|
|
|
|
|
+ curCachingStrategy);
|
|
return reader;
|
|
return reader;
|
|
} catch (IOException ex) {
|
|
} catch (IOException ex) {
|
|
DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
|
|
DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
|
|
@@ -1186,7 +1192,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
dfsClient.getConf(), file, block, blockToken, startOffset,
|
|
dfsClient.getConf(), file, block, blockToken, startOffset,
|
|
len, verifyChecksum, clientName, peer, chosenNode,
|
|
len, verifyChecksum, clientName, peer, chosenNode,
|
|
dsFactory, peerCache, fileInputStreamCache, false,
|
|
dsFactory, peerCache, fileInputStreamCache, false,
|
|
- cachingStrategy);
|
|
|
|
|
|
+ curCachingStrategy);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
@@ -1460,14 +1466,18 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
@Override
|
|
@Override
|
|
public synchronized void setReadahead(Long readahead)
|
|
public synchronized void setReadahead(Long readahead)
|
|
throws IOException {
|
|
throws IOException {
|
|
- this.cachingStrategy.setReadahead(readahead);
|
|
|
|
|
|
+ this.cachingStrategy =
|
|
|
|
+ new CachingStrategy.Builder(this.cachingStrategy).
|
|
|
|
+ setReadahead(readahead).build();
|
|
closeCurrentBlockReader();
|
|
closeCurrentBlockReader();
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public synchronized void setDropBehind(Boolean dropBehind)
|
|
public synchronized void setDropBehind(Boolean dropBehind)
|
|
throws IOException {
|
|
throws IOException {
|
|
- this.cachingStrategy.setDropBehind(dropBehind);
|
|
|
|
|
|
+ this.cachingStrategy =
|
|
|
|
+ new CachingStrategy.Builder(this.cachingStrategy).
|
|
|
|
+ setDropBehind(dropBehind).build();
|
|
closeCurrentBlockReader();
|
|
closeCurrentBlockReader();
|
|
}
|
|
}
|
|
|
|
|