|
@@ -92,17 +92,32 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
private final DFSClient dfsClient;
|
|
|
private boolean closed = false;
|
|
|
private final String src;
|
|
|
- private BlockReader blockReader = null;
|
|
|
private final boolean verifyChecksum;
|
|
|
- private LocatedBlocks locatedBlocks = null;
|
|
|
- private long lastBlockBeingWrittenLength = 0;
|
|
|
- private FileEncryptionInfo fileEncryptionInfo = null;
|
|
|
+
|
|
|
+ // state by stateful read only:
|
|
|
+ // (protected by lock on this)
|
|
|
+ /////
|
|
|
private DatanodeInfo currentNode = null;
|
|
|
private LocatedBlock currentLocatedBlock = null;
|
|
|
private long pos = 0;
|
|
|
private long blockEnd = -1;
|
|
|
+ private BlockReader blockReader = null;
|
|
|
+ ////
|
|
|
+
|
|
|
+ // state shared by stateful and positional read:
|
|
|
+ // (protected by lock on infoLock)
|
|
|
+ ////
|
|
|
+ private LocatedBlocks locatedBlocks = null;
|
|
|
+ private long lastBlockBeingWrittenLength = 0;
|
|
|
+ private FileEncryptionInfo fileEncryptionInfo = null;
|
|
|
private CachingStrategy cachingStrategy;
|
|
|
+ ////
|
|
|
+
|
|
|
private final ReadStatistics readStatistics = new ReadStatistics();
|
|
|
+ // lock for state shared between read and pread
|
|
|
+ // Note: Never acquire a lock on <this> with this lock held to avoid deadlocks
|
|
|
+ // (it's OK to acquire this lock when the lock on <this> is held)
|
|
|
+ private final Object infoLock = new Object();
|
|
|
|
|
|
/**
|
|
|
* Track the ByteBuffers that we have handed out to readers.
|
|
@@ -226,35 +241,38 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
this.dfsClient = dfsClient;
|
|
|
this.verifyChecksum = verifyChecksum;
|
|
|
this.src = src;
|
|
|
- this.cachingStrategy =
|
|
|
- dfsClient.getDefaultReadCachingStrategy();
|
|
|
+ synchronized (infoLock) {
|
|
|
+ this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy();
|
|
|
+ }
|
|
|
openInfo();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Grab the open-file info from namenode
|
|
|
*/
|
|
|
- synchronized void openInfo() throws IOException, UnresolvedLinkException {
|
|
|
- lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
|
|
|
- int retriesForLastBlockLength = dfsClient.getConf().retryTimesForGetLastBlockLength;
|
|
|
- while (retriesForLastBlockLength > 0) {
|
|
|
- // Getting last block length as -1 is a special case. When cluster
|
|
|
- // restarts, DNs may not report immediately. At this time partial block
|
|
|
- // locations will not be available with NN for getting the length. Lets
|
|
|
- // retry for 3 times to get the length.
|
|
|
- if (lastBlockBeingWrittenLength == -1) {
|
|
|
- DFSClient.LOG.warn("Last block locations not available. "
|
|
|
- + "Datanodes might not have reported blocks completely."
|
|
|
- + " Will retry for " + retriesForLastBlockLength + " times");
|
|
|
- waitFor(dfsClient.getConf().retryIntervalForGetLastBlockLength);
|
|
|
- lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
|
|
|
- } else {
|
|
|
- break;
|
|
|
+ void openInfo() throws IOException, UnresolvedLinkException {
|
|
|
+ synchronized(infoLock) {
|
|
|
+ lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
|
|
|
+ int retriesForLastBlockLength = dfsClient.getConf().retryTimesForGetLastBlockLength;
|
|
|
+ while (retriesForLastBlockLength > 0) {
|
|
|
+ // Getting last block length as -1 is a special case. When cluster
|
|
|
+ // restarts, DNs may not report immediately. At this time partial block
|
|
|
+ // locations will not be available with NN for getting the length. Lets
|
|
|
+ // retry for 3 times to get the length.
|
|
|
+ if (lastBlockBeingWrittenLength == -1) {
|
|
|
+ DFSClient.LOG.warn("Last block locations not available. "
|
|
|
+ + "Datanodes might not have reported blocks completely."
|
|
|
+ + " Will retry for " + retriesForLastBlockLength + " times");
|
|
|
+ waitFor(dfsClient.getConf().retryIntervalForGetLastBlockLength);
|
|
|
+ lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
|
|
|
+ } else {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ retriesForLastBlockLength--;
|
|
|
+ }
|
|
|
+ if (retriesForLastBlockLength == 0) {
|
|
|
+ throw new IOException("Could not obtain the last block locations.");
|
|
|
}
|
|
|
- retriesForLastBlockLength--;
|
|
|
- }
|
|
|
- if (retriesForLastBlockLength == 0) {
|
|
|
- throw new IOException("Could not obtain the last block locations.");
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -306,7 +324,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
|
|
|
fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo();
|
|
|
|
|
|
- currentNode = null;
|
|
|
return lastBlockBeingWrittenLength;
|
|
|
}
|
|
|
|
|
@@ -359,21 +376,25 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
throw new IOException("Cannot obtain block length for " + locatedblock);
|
|
|
}
|
|
|
|
|
|
- public synchronized long getFileLength() {
|
|
|
- return locatedBlocks == null? 0:
|
|
|
- locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
|
|
|
+ public long getFileLength() {
|
|
|
+ synchronized(infoLock) {
|
|
|
+ return locatedBlocks == null? 0:
|
|
|
+ locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
// Short circuit local reads are forbidden for files that are
|
|
|
// under construction. See HDFS-2757.
|
|
|
- synchronized boolean shortCircuitForbidden() {
|
|
|
- return locatedBlocks.isUnderConstruction();
|
|
|
+ boolean shortCircuitForbidden() {
|
|
|
+ synchronized(infoLock) {
|
|
|
+ return locatedBlocks.isUnderConstruction();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Returns the datanode from which the stream is currently reading.
|
|
|
*/
|
|
|
- public DatanodeInfo getCurrentDatanode() {
|
|
|
+ public synchronized DatanodeInfo getCurrentDatanode() {
|
|
|
return currentNode;
|
|
|
}
|
|
|
|
|
@@ -403,59 +424,67 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
* @return located block
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- private synchronized LocatedBlock getBlockAt(long offset,
|
|
|
+ private LocatedBlock getBlockAt(long offset,
|
|
|
boolean updatePosition) throws IOException {
|
|
|
- assert (locatedBlocks != null) : "locatedBlocks is null";
|
|
|
+ synchronized(infoLock) {
|
|
|
+ assert (locatedBlocks != null) : "locatedBlocks is null";
|
|
|
|
|
|
- final LocatedBlock blk;
|
|
|
+ final LocatedBlock blk;
|
|
|
|
|
|
- //check offset
|
|
|
- if (offset < 0 || offset >= getFileLength()) {
|
|
|
- throw new IOException("offset < 0 || offset >= getFileLength(), offset="
|
|
|
- + offset
|
|
|
- + ", updatePosition=" + updatePosition
|
|
|
- + ", locatedBlocks=" + locatedBlocks);
|
|
|
- }
|
|
|
- else if (offset >= locatedBlocks.getFileLength()) {
|
|
|
- // offset to the portion of the last block,
|
|
|
- // which is not known to the name-node yet;
|
|
|
- // getting the last block
|
|
|
- blk = locatedBlocks.getLastLocatedBlock();
|
|
|
- }
|
|
|
- else {
|
|
|
- // search cached blocks first
|
|
|
- int targetBlockIdx = locatedBlocks.findBlock(offset);
|
|
|
- if (targetBlockIdx < 0) { // block is not cached
|
|
|
- targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
|
|
|
- // fetch more blocks
|
|
|
- final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
|
|
|
- assert (newBlocks != null) : "Could not find target position " + offset;
|
|
|
- locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
|
|
|
+ //check offset
|
|
|
+ if (offset < 0 || offset >= getFileLength()) {
|
|
|
+ throw new IOException("offset < 0 || offset >= getFileLength(), offset="
|
|
|
+ + offset
|
|
|
+ + ", updatePosition=" + updatePosition
|
|
|
+ + ", locatedBlocks=" + locatedBlocks);
|
|
|
+ }
|
|
|
+ else if (offset >= locatedBlocks.getFileLength()) {
|
|
|
+ // offset to the portion of the last block,
|
|
|
+ // which is not known to the name-node yet;
|
|
|
+ // getting the last block
|
|
|
+ blk = locatedBlocks.getLastLocatedBlock();
|
|
|
+ }
|
|
|
+ else {
|
|
|
+ // search cached blocks first
|
|
|
+ int targetBlockIdx = locatedBlocks.findBlock(offset);
|
|
|
+ if (targetBlockIdx < 0) { // block is not cached
|
|
|
+ targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
|
|
|
+ // fetch more blocks
|
|
|
+ final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
|
|
|
+ assert (newBlocks != null) : "Could not find target position " + offset;
|
|
|
+ locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
|
|
|
+ }
|
|
|
+ blk = locatedBlocks.get(targetBlockIdx);
|
|
|
}
|
|
|
- blk = locatedBlocks.get(targetBlockIdx);
|
|
|
- }
|
|
|
|
|
|
- // update current position
|
|
|
- if (updatePosition) {
|
|
|
- pos = offset;
|
|
|
- blockEnd = blk.getStartOffset() + blk.getBlockSize() - 1;
|
|
|
- currentLocatedBlock = blk;
|
|
|
+ // update current position
|
|
|
+ if (updatePosition) {
|
|
|
+ // synchronized not strictly needed, since we only get here
|
|
|
+ // from synchronized caller methods
|
|
|
+ synchronized(this) {
|
|
|
+ pos = offset;
|
|
|
+ blockEnd = blk.getStartOffset() + blk.getBlockSize() - 1;
|
|
|
+ currentLocatedBlock = blk;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return blk;
|
|
|
}
|
|
|
- return blk;
|
|
|
}
|
|
|
|
|
|
/** Fetch a block from namenode and cache it */
|
|
|
- private synchronized void fetchBlockAt(long offset) throws IOException {
|
|
|
- int targetBlockIdx = locatedBlocks.findBlock(offset);
|
|
|
- if (targetBlockIdx < 0) { // block is not cached
|
|
|
- targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
|
|
|
- }
|
|
|
- // fetch blocks
|
|
|
- final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
|
|
|
- if (newBlocks == null) {
|
|
|
- throw new IOException("Could not find target position " + offset);
|
|
|
+ private void fetchBlockAt(long offset) throws IOException {
|
|
|
+ synchronized(infoLock) {
|
|
|
+ int targetBlockIdx = locatedBlocks.findBlock(offset);
|
|
|
+ if (targetBlockIdx < 0) { // block is not cached
|
|
|
+ targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
|
|
|
+ }
|
|
|
+ // fetch blocks
|
|
|
+ final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
|
|
|
+ if (newBlocks == null) {
|
|
|
+ throw new IOException("Could not find target position " + offset);
|
|
|
+ }
|
|
|
+ locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
|
|
|
}
|
|
|
- locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -467,7 +496,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
* @return consequent segment of located blocks
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- private synchronized List<LocatedBlock> getBlockRange(long offset,
|
|
|
+ private List<LocatedBlock> getBlockRange(long offset,
|
|
|
long length) throws IOException {
|
|
|
// getFileLength(): returns total file length
|
|
|
// locatedBlocks.getFileLength(): returns length of completed blocks
|
|
@@ -475,26 +504,27 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
throw new IOException("Offset: " + offset +
|
|
|
" exceeds file length: " + getFileLength());
|
|
|
}
|
|
|
+ synchronized(infoLock) {
|
|
|
+ final List<LocatedBlock> blocks;
|
|
|
+ final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
|
|
|
+ final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk;
|
|
|
+ final boolean readLengthPastCompleteBlk = offset + length > lengthOfCompleteBlk;
|
|
|
|
|
|
- final List<LocatedBlock> blocks;
|
|
|
- final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
|
|
|
- final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk;
|
|
|
- final boolean readLengthPastCompleteBlk = offset + length > lengthOfCompleteBlk;
|
|
|
+ if (readOffsetWithinCompleteBlk) {
|
|
|
+ //get the blocks of finalized (completed) block range
|
|
|
+ blocks = getFinalizedBlockRange(offset,
|
|
|
+ Math.min(length, lengthOfCompleteBlk - offset));
|
|
|
+ } else {
|
|
|
+ blocks = new ArrayList<LocatedBlock>(1);
|
|
|
+ }
|
|
|
|
|
|
- if (readOffsetWithinCompleteBlk) {
|
|
|
- //get the blocks of finalized (completed) block range
|
|
|
- blocks = getFinalizedBlockRange(offset,
|
|
|
- Math.min(length, lengthOfCompleteBlk - offset));
|
|
|
- } else {
|
|
|
- blocks = new ArrayList<LocatedBlock>(1);
|
|
|
- }
|
|
|
+ // get the blocks from incomplete block range
|
|
|
+ if (readLengthPastCompleteBlk) {
|
|
|
+ blocks.add(locatedBlocks.getLastLocatedBlock());
|
|
|
+ }
|
|
|
|
|
|
- // get the blocks from incomplete block range
|
|
|
- if (readLengthPastCompleteBlk) {
|
|
|
- blocks.add(locatedBlocks.getLastLocatedBlock());
|
|
|
+ return blocks;
|
|
|
}
|
|
|
-
|
|
|
- return blocks;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -502,35 +532,37 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
* Includes only the complete blocks.
|
|
|
* Fetch them from the namenode if not cached.
|
|
|
*/
|
|
|
- private synchronized List<LocatedBlock> getFinalizedBlockRange(
|
|
|
+ private List<LocatedBlock> getFinalizedBlockRange(
|
|
|
long offset, long length) throws IOException {
|
|
|
- assert (locatedBlocks != null) : "locatedBlocks is null";
|
|
|
- List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
|
|
|
- // search cached blocks first
|
|
|
- int blockIdx = locatedBlocks.findBlock(offset);
|
|
|
- if (blockIdx < 0) { // block is not cached
|
|
|
- blockIdx = LocatedBlocks.getInsertIndex(blockIdx);
|
|
|
- }
|
|
|
- long remaining = length;
|
|
|
- long curOff = offset;
|
|
|
- while(remaining > 0) {
|
|
|
- LocatedBlock blk = null;
|
|
|
- if(blockIdx < locatedBlocks.locatedBlockCount())
|
|
|
- blk = locatedBlocks.get(blockIdx);
|
|
|
- if (blk == null || curOff < blk.getStartOffset()) {
|
|
|
- LocatedBlocks newBlocks;
|
|
|
- newBlocks = dfsClient.getLocatedBlocks(src, curOff, remaining);
|
|
|
- locatedBlocks.insertRange(blockIdx, newBlocks.getLocatedBlocks());
|
|
|
- continue;
|
|
|
+ synchronized(infoLock) {
|
|
|
+ assert (locatedBlocks != null) : "locatedBlocks is null";
|
|
|
+ List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
|
|
|
+ // search cached blocks first
|
|
|
+ int blockIdx = locatedBlocks.findBlock(offset);
|
|
|
+ if (blockIdx < 0) { // block is not cached
|
|
|
+ blockIdx = LocatedBlocks.getInsertIndex(blockIdx);
|
|
|
+ }
|
|
|
+ long remaining = length;
|
|
|
+ long curOff = offset;
|
|
|
+ while(remaining > 0) {
|
|
|
+ LocatedBlock blk = null;
|
|
|
+ if(blockIdx < locatedBlocks.locatedBlockCount())
|
|
|
+ blk = locatedBlocks.get(blockIdx);
|
|
|
+ if (blk == null || curOff < blk.getStartOffset()) {
|
|
|
+ LocatedBlocks newBlocks;
|
|
|
+ newBlocks = dfsClient.getLocatedBlocks(src, curOff, remaining);
|
|
|
+ locatedBlocks.insertRange(blockIdx, newBlocks.getLocatedBlocks());
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ assert curOff >= blk.getStartOffset() : "Block not found";
|
|
|
+ blockRange.add(blk);
|
|
|
+ long bytesRead = blk.getStartOffset() + blk.getBlockSize() - curOff;
|
|
|
+ remaining -= bytesRead;
|
|
|
+ curOff += bytesRead;
|
|
|
+ blockIdx++;
|
|
|
}
|
|
|
- assert curOff >= blk.getStartOffset() : "Block not found";
|
|
|
- blockRange.add(blk);
|
|
|
- long bytesRead = blk.getStartOffset() + blk.getBlockSize() - curOff;
|
|
|
- remaining -= bytesRead;
|
|
|
- curOff += bytesRead;
|
|
|
- blockIdx++;
|
|
|
- }
|
|
|
- return blockRange;
|
|
|
+ return blockRange;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -573,6 +605,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
try {
|
|
|
ExtendedBlock blk = targetBlock.getBlock();
|
|
|
Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
|
|
|
+ CachingStrategy curCachingStrategy;
|
|
|
+ boolean shortCircuitForbidden;
|
|
|
+ synchronized(infoLock) {
|
|
|
+ curCachingStrategy = cachingStrategy;
|
|
|
+ shortCircuitForbidden = shortCircuitForbidden();
|
|
|
+ }
|
|
|
blockReader = new BlockReaderFactory(dfsClient.getConf()).
|
|
|
setInetSocketAddress(targetAddr).
|
|
|
setRemotePeerFactory(dfsClient).
|
|
@@ -585,8 +623,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
setVerifyChecksum(verifyChecksum).
|
|
|
setClientName(dfsClient.clientName).
|
|
|
setLength(blk.getNumBytes() - offsetIntoBlock).
|
|
|
- setCachingStrategy(cachingStrategy).
|
|
|
- setAllowShortCircuitLocalReads(!shortCircuitForbidden()).
|
|
|
+ setCachingStrategy(curCachingStrategy).
|
|
|
+ setAllowShortCircuitLocalReads(!shortCircuitForbidden).
|
|
|
setClientCacheContext(dfsClient.getClientContext()).
|
|
|
setUserGroupInformation(dfsClient.ugi).
|
|
|
setConfiguration(dfsClient.getConfiguration()).
|
|
@@ -782,7 +820,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
|
|
|
+ private synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
|
|
|
dfsClient.checkOpen();
|
|
|
if (closed) {
|
|
|
throw new IOException("Stream closed");
|
|
@@ -800,9 +838,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
currentNode = blockSeekTo(pos);
|
|
|
}
|
|
|
int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
|
|
|
- if (locatedBlocks.isLastBlockComplete()) {
|
|
|
- realLen = (int) Math.min(realLen,
|
|
|
- locatedBlocks.getFileLength() - pos);
|
|
|
+ synchronized(infoLock) {
|
|
|
+ if (locatedBlocks.isLastBlockComplete()) {
|
|
|
+ realLen = (int) Math.min(realLen,
|
|
|
+ locatedBlocks.getFileLength() - pos);
|
|
|
+ }
|
|
|
}
|
|
|
int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
|
|
|
|
|
@@ -1055,8 +1095,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
// start of the loop.
|
|
|
CachingStrategy curCachingStrategy;
|
|
|
boolean allowShortCircuitLocalReads;
|
|
|
- synchronized (this) {
|
|
|
- block = getBlockAt(block.getStartOffset(), false);
|
|
|
+ block = getBlockAt(block.getStartOffset(), false);
|
|
|
+ synchronized(infoLock) {
|
|
|
curCachingStrategy = cachingStrategy;
|
|
|
allowShortCircuitLocalReads = !shortCircuitForbidden();
|
|
|
}
|
|
@@ -1488,7 +1528,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
* Same as {@link #seekToNewSource(long)} except that it does not exclude
|
|
|
* the current datanode and might connect to the same node.
|
|
|
*/
|
|
|
- private synchronized boolean seekToBlockSource(long targetPos)
|
|
|
+ private boolean seekToBlockSource(long targetPos)
|
|
|
throws IOException {
|
|
|
currentNode = blockSeekTo(targetPos);
|
|
|
return true;
|
|
@@ -1575,11 +1615,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
return new ReadStatistics(readStatistics);
|
|
|
}
|
|
|
|
|
|
- public synchronized FileEncryptionInfo getFileEncryptionInfo() {
|
|
|
- return fileEncryptionInfo;
|
|
|
+ public FileEncryptionInfo getFileEncryptionInfo() {
|
|
|
+ synchronized(infoLock) {
|
|
|
+ return fileEncryptionInfo;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- private synchronized void closeCurrentBlockReader() {
|
|
|
+ private void closeCurrentBlockReader() {
|
|
|
if (blockReader == null) return;
|
|
|
// Close the current block reader so that the new caching settings can
|
|
|
// take effect immediately.
|
|
@@ -1594,18 +1636,20 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
@Override
|
|
|
public synchronized void setReadahead(Long readahead)
|
|
|
throws IOException {
|
|
|
- this.cachingStrategy =
|
|
|
- new CachingStrategy.Builder(this.cachingStrategy).
|
|
|
- setReadahead(readahead).build();
|
|
|
+ synchronized (infoLock) {
|
|
|
+ this.cachingStrategy =
|
|
|
+ new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
|
|
|
+ }
|
|
|
closeCurrentBlockReader();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public synchronized void setDropBehind(Boolean dropBehind)
|
|
|
throws IOException {
|
|
|
- this.cachingStrategy =
|
|
|
- new CachingStrategy.Builder(this.cachingStrategy).
|
|
|
- setDropBehind(dropBehind).build();
|
|
|
+ synchronized (infoLock) {
|
|
|
+ this.cachingStrategy =
|
|
|
+ new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
|
|
|
+ }
|
|
|
closeCurrentBlockReader();
|
|
|
}
|
|
|
|