HDFS-6735. A minor optimization to avoid pread() be blocked by read() inside the same DFSInputStream (Lars Hofhansl via stack)

This commit is contained in:
stack 2014-12-02 20:54:03 -08:00 committed by stack
parent 92ce6eda92
commit 7caa3bc98e
6 changed files with 217 additions and 155 deletions

View File

@ -422,6 +422,9 @@ Release 2.7.0 - UNRELEASED
HDFS-7446. HDFS inotify should have the ability to determine what txid it HDFS-7446. HDFS inotify should have the ability to determine what txid it
has read up to (cmccabe) has read up to (cmccabe)
HDFS-6735. A minor optimization to avoid pread() be blocked by read()
inside the same DFSInputStream (Lars Hofhansl via stack)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES

View File

@ -207,4 +207,13 @@
<Bug pattern="NP_LOAD_OF_KNOWN_NULL_VALUE" /> <Bug pattern="NP_LOAD_OF_KNOWN_NULL_VALUE" />
</Match> </Match>
<!--
We use a separate lock to guard cachingStrategy in order to separate
locks for p-reads from seek + read invocations.
-->
<Match>
<Class name="org.apache.hadoop.hdfs.DFSInputStream" />
<Field name="cachingStrategy" />
<Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match>
</FindBugsFilter> </FindBugsFilter>

View File

@ -92,17 +92,32 @@ public class DFSInputStream extends FSInputStream
private final DFSClient dfsClient; private final DFSClient dfsClient;
private boolean closed = false; private boolean closed = false;
private final String src; private final String src;
private BlockReader blockReader = null;
private final boolean verifyChecksum; private final boolean verifyChecksum;
private LocatedBlocks locatedBlocks = null;
private long lastBlockBeingWrittenLength = 0; // state by stateful read only:
private FileEncryptionInfo fileEncryptionInfo = null; // (protected by lock on this)
/////
private DatanodeInfo currentNode = null; private DatanodeInfo currentNode = null;
private LocatedBlock currentLocatedBlock = null; private LocatedBlock currentLocatedBlock = null;
private long pos = 0; private long pos = 0;
private long blockEnd = -1; 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 CachingStrategy cachingStrategy;
////
private final ReadStatistics readStatistics = new ReadStatistics(); 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. * Track the ByteBuffers that we have handed out to readers.
@ -226,15 +241,17 @@ void addToDeadNodes(DatanodeInfo dnInfo) {
this.dfsClient = dfsClient; this.dfsClient = dfsClient;
this.verifyChecksum = verifyChecksum; this.verifyChecksum = verifyChecksum;
this.src = src; this.src = src;
this.cachingStrategy = synchronized (infoLock) {
dfsClient.getDefaultReadCachingStrategy(); this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy();
}
openInfo(); openInfo();
} }
/** /**
* Grab the open-file info from namenode * Grab the open-file info from namenode
*/ */
synchronized void openInfo() throws IOException, UnresolvedLinkException { void openInfo() throws IOException, UnresolvedLinkException {
synchronized(infoLock) {
lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength(); lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
int retriesForLastBlockLength = dfsClient.getConf().retryTimesForGetLastBlockLength; int retriesForLastBlockLength = dfsClient.getConf().retryTimesForGetLastBlockLength;
while (retriesForLastBlockLength > 0) { while (retriesForLastBlockLength > 0) {
@ -257,6 +274,7 @@ synchronized void openInfo() throws IOException, UnresolvedLinkException {
throw new IOException("Could not obtain the last block locations."); throw new IOException("Could not obtain the last block locations.");
} }
} }
}
private void waitFor(int waitTime) throws IOException { private void waitFor(int waitTime) throws IOException {
try { try {
@ -306,7 +324,6 @@ private long fetchLocatedBlocksAndGetLastBlockLength() throws IOException {
fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo(); fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo();
currentNode = null;
return lastBlockBeingWrittenLength; return lastBlockBeingWrittenLength;
} }
@ -359,21 +376,25 @@ private long readBlockLength(LocatedBlock locatedblock) throws IOException {
throw new IOException("Cannot obtain block length for " + locatedblock); throw new IOException("Cannot obtain block length for " + locatedblock);
} }
public synchronized long getFileLength() { public long getFileLength() {
synchronized(infoLock) {
return locatedBlocks == null? 0: return locatedBlocks == null? 0:
locatedBlocks.getFileLength() + lastBlockBeingWrittenLength; locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
} }
}
// Short circuit local reads are forbidden for files that are // Short circuit local reads are forbidden for files that are
// under construction. See HDFS-2757. // under construction. See HDFS-2757.
synchronized boolean shortCircuitForbidden() { boolean shortCircuitForbidden() {
synchronized(infoLock) {
return locatedBlocks.isUnderConstruction(); return locatedBlocks.isUnderConstruction();
} }
}
/** /**
* Returns the datanode from which the stream is currently reading. * Returns the datanode from which the stream is currently reading.
*/ */
public DatanodeInfo getCurrentDatanode() { public synchronized DatanodeInfo getCurrentDatanode() {
return currentNode; return currentNode;
} }
@ -403,8 +424,9 @@ public synchronized List<LocatedBlock> getAllBlocks() throws IOException {
* @return located block * @return located block
* @throws IOException * @throws IOException
*/ */
private synchronized LocatedBlock getBlockAt(long offset, private LocatedBlock getBlockAt(long offset,
boolean updatePosition) throws IOException { boolean updatePosition) throws IOException {
synchronized(infoLock) {
assert (locatedBlocks != null) : "locatedBlocks is null"; assert (locatedBlocks != null) : "locatedBlocks is null";
final LocatedBlock blk; final LocatedBlock blk;
@ -437,15 +459,21 @@ else if (offset >= locatedBlocks.getFileLength()) {
// update current position // update current position
if (updatePosition) { if (updatePosition) {
// synchronized not strictly needed, since we only get here
// from synchronized caller methods
synchronized(this) {
pos = offset; pos = offset;
blockEnd = blk.getStartOffset() + blk.getBlockSize() - 1; blockEnd = blk.getStartOffset() + blk.getBlockSize() - 1;
currentLocatedBlock = blk; currentLocatedBlock = blk;
} }
}
return blk; return blk;
} }
}
/** Fetch a block from namenode and cache it */ /** Fetch a block from namenode and cache it */
private synchronized void fetchBlockAt(long offset) throws IOException { private void fetchBlockAt(long offset) throws IOException {
synchronized(infoLock) {
int targetBlockIdx = locatedBlocks.findBlock(offset); int targetBlockIdx = locatedBlocks.findBlock(offset);
if (targetBlockIdx < 0) { // block is not cached if (targetBlockIdx < 0) { // block is not cached
targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx); targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
@ -457,6 +485,7 @@ private synchronized void fetchBlockAt(long offset) throws IOException {
} }
locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks()); locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
} }
}
/** /**
* Get blocks in the specified range. * Get blocks in the specified range.
@ -467,7 +496,7 @@ private synchronized void fetchBlockAt(long offset) throws IOException {
* @return consequent segment of located blocks * @return consequent segment of located blocks
* @throws IOException * @throws IOException
*/ */
private synchronized List<LocatedBlock> getBlockRange(long offset, private List<LocatedBlock> getBlockRange(long offset,
long length) throws IOException { long length) throws IOException {
// getFileLength(): returns total file length // getFileLength(): returns total file length
// locatedBlocks.getFileLength(): returns length of completed blocks // locatedBlocks.getFileLength(): returns length of completed blocks
@ -475,7 +504,7 @@ private synchronized List<LocatedBlock> getBlockRange(long offset,
throw new IOException("Offset: " + offset + throw new IOException("Offset: " + offset +
" exceeds file length: " + getFileLength()); " exceeds file length: " + getFileLength());
} }
synchronized(infoLock) {
final List<LocatedBlock> blocks; final List<LocatedBlock> blocks;
final long lengthOfCompleteBlk = locatedBlocks.getFileLength(); final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk; final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk;
@ -496,14 +525,16 @@ private synchronized List<LocatedBlock> getBlockRange(long offset,
return blocks; return blocks;
} }
}
/** /**
* Get blocks in the specified range. * Get blocks in the specified range.
* Includes only the complete blocks. * Includes only the complete blocks.
* Fetch them from the namenode if not cached. * Fetch them from the namenode if not cached.
*/ */
private synchronized List<LocatedBlock> getFinalizedBlockRange( private List<LocatedBlock> getFinalizedBlockRange(
long offset, long length) throws IOException { long offset, long length) throws IOException {
synchronized(infoLock) {
assert (locatedBlocks != null) : "locatedBlocks is null"; assert (locatedBlocks != null) : "locatedBlocks is null";
List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>(); List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
// search cached blocks first // search cached blocks first
@ -532,6 +563,7 @@ private synchronized List<LocatedBlock> getFinalizedBlockRange(
} }
return blockRange; return blockRange;
} }
}
/** /**
* Open a DataInputStream to a DataNode so that it can be read from. * Open a DataInputStream to a DataNode so that it can be read from.
@ -573,6 +605,12 @@ private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
try { try {
ExtendedBlock blk = targetBlock.getBlock(); ExtendedBlock blk = targetBlock.getBlock();
Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken(); Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
CachingStrategy curCachingStrategy;
boolean shortCircuitForbidden;
synchronized(infoLock) {
curCachingStrategy = cachingStrategy;
shortCircuitForbidden = shortCircuitForbidden();
}
blockReader = new BlockReaderFactory(dfsClient.getConf()). blockReader = new BlockReaderFactory(dfsClient.getConf()).
setInetSocketAddress(targetAddr). setInetSocketAddress(targetAddr).
setRemotePeerFactory(dfsClient). setRemotePeerFactory(dfsClient).
@ -585,8 +623,8 @@ private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
setVerifyChecksum(verifyChecksum). setVerifyChecksum(verifyChecksum).
setClientName(dfsClient.clientName). setClientName(dfsClient.clientName).
setLength(blk.getNumBytes() - offsetIntoBlock). setLength(blk.getNumBytes() - offsetIntoBlock).
setCachingStrategy(cachingStrategy). setCachingStrategy(curCachingStrategy).
setAllowShortCircuitLocalReads(!shortCircuitForbidden()). setAllowShortCircuitLocalReads(!shortCircuitForbidden).
setClientCacheContext(dfsClient.getClientContext()). setClientCacheContext(dfsClient.getClientContext()).
setUserGroupInformation(dfsClient.ugi). setUserGroupInformation(dfsClient.ugi).
setConfiguration(dfsClient.getConfiguration()). setConfiguration(dfsClient.getConfiguration()).
@ -782,7 +820,7 @@ private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
} }
} }
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(); dfsClient.checkOpen();
if (closed) { if (closed) {
throw new IOException("Stream closed"); throw new IOException("Stream closed");
@ -800,10 +838,12 @@ private int readWithStrategy(ReaderStrategy strategy, int off, int len) throws I
currentNode = blockSeekTo(pos); currentNode = blockSeekTo(pos);
} }
int realLen = (int) Math.min(len, (blockEnd - pos + 1L)); int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
synchronized(infoLock) {
if (locatedBlocks.isLastBlockComplete()) { if (locatedBlocks.isLastBlockComplete()) {
realLen = (int) Math.min(realLen, realLen = (int) Math.min(realLen,
locatedBlocks.getFileLength() - pos); locatedBlocks.getFileLength() - pos);
} }
}
int result = readBuffer(strategy, off, realLen, corruptedBlockMap); int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
if (result >= 0) { if (result >= 0) {
@ -1055,8 +1095,8 @@ private void actualGetFromOneDataNode(final DNAddrPair datanode,
// start of the loop. // start of the loop.
CachingStrategy curCachingStrategy; CachingStrategy curCachingStrategy;
boolean allowShortCircuitLocalReads; boolean allowShortCircuitLocalReads;
synchronized (this) {
block = getBlockAt(block.getStartOffset(), false); block = getBlockAt(block.getStartOffset(), false);
synchronized(infoLock) {
curCachingStrategy = cachingStrategy; curCachingStrategy = cachingStrategy;
allowShortCircuitLocalReads = !shortCircuitForbidden(); allowShortCircuitLocalReads = !shortCircuitForbidden();
} }
@ -1488,7 +1528,7 @@ public synchronized void seek(long targetPos) throws IOException {
* Same as {@link #seekToNewSource(long)} except that it does not exclude * Same as {@link #seekToNewSource(long)} except that it does not exclude
* the current datanode and might connect to the same node. * the current datanode and might connect to the same node.
*/ */
private synchronized boolean seekToBlockSource(long targetPos) private boolean seekToBlockSource(long targetPos)
throws IOException { throws IOException {
currentNode = blockSeekTo(targetPos); currentNode = blockSeekTo(targetPos);
return true; return true;
@ -1575,11 +1615,13 @@ public synchronized ReadStatistics getReadStatistics() {
return new ReadStatistics(readStatistics); return new ReadStatistics(readStatistics);
} }
public synchronized FileEncryptionInfo getFileEncryptionInfo() { public FileEncryptionInfo getFileEncryptionInfo() {
synchronized(infoLock) {
return fileEncryptionInfo; return fileEncryptionInfo;
} }
}
private synchronized void closeCurrentBlockReader() { private void closeCurrentBlockReader() {
if (blockReader == null) return; if (blockReader == null) return;
// Close the current block reader so that the new caching settings can // Close the current block reader so that the new caching settings can
// take effect immediately. // take effect immediately.
@ -1594,18 +1636,20 @@ private synchronized void closeCurrentBlockReader() {
@Override @Override
public synchronized void setReadahead(Long readahead) public synchronized void setReadahead(Long readahead)
throws IOException { throws IOException {
synchronized (infoLock) {
this.cachingStrategy = this.cachingStrategy =
new CachingStrategy.Builder(this.cachingStrategy). new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
setReadahead(readahead).build(); }
closeCurrentBlockReader(); closeCurrentBlockReader();
} }
@Override @Override
public synchronized void setDropBehind(Boolean dropBehind) public synchronized void setDropBehind(Boolean dropBehind)
throws IOException { throws IOException {
synchronized (infoLock) {
this.cachingStrategy = this.cachingStrategy =
new CachingStrategy.Builder(this.cachingStrategy). new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
setDropBehind(dropBehind).build(); }
closeCurrentBlockReader(); closeCurrentBlockReader();
} }

View File

@ -34,14 +34,17 @@ public class LocatedBlocks {
private final long fileLength; private final long fileLength;
private final List<LocatedBlock> blocks; // array of blocks with prioritized locations private final List<LocatedBlock> blocks; // array of blocks with prioritized locations
private final boolean underConstruction; private final boolean underConstruction;
private LocatedBlock lastLocatedBlock = null; private final LocatedBlock lastLocatedBlock;
private boolean isLastBlockComplete = false; private final boolean isLastBlockComplete;
private FileEncryptionInfo fileEncryptionInfo = null; private final FileEncryptionInfo fileEncryptionInfo;
public LocatedBlocks() { public LocatedBlocks() {
fileLength = 0; fileLength = 0;
blocks = null; blocks = null;
underConstruction = false; underConstruction = false;
lastLocatedBlock = null;
isLastBlockComplete = false;
fileEncryptionInfo = null;
} }
public LocatedBlocks(long flength, boolean isUnderConstuction, public LocatedBlocks(long flength, boolean isUnderConstuction,

View File

@ -400,7 +400,7 @@ private void ref(ShortCircuitReplica replica) {
lock.lock(); lock.lock();
try { try {
Preconditions.checkArgument(replica.refCount > 0, Preconditions.checkArgument(replica.refCount > 0,
"can't ref " + replica + " because its refCount reached " + "can't ref %s because its refCount reached %d", replica,
replica.refCount); replica.refCount);
Long evictableTimeNs = replica.getEvictableTimeNs(); Long evictableTimeNs = replica.getEvictableTimeNs();
replica.refCount++; replica.refCount++;
@ -456,14 +456,13 @@ void unref(ShortCircuitReplica replica) {
if (newRefCount == 0) { if (newRefCount == 0) {
// Close replica, since there are no remaining references to it. // Close replica, since there are no remaining references to it.
Preconditions.checkArgument(replica.purged, Preconditions.checkArgument(replica.purged,
"Replica " + replica + " reached a refCount of 0 without " + "Replica %s reached a refCount of 0 without being purged", replica);
"being purged");
replica.close(); replica.close();
} else if (newRefCount == 1) { } else if (newRefCount == 1) {
Preconditions.checkState(null == replica.getEvictableTimeNs(), Preconditions.checkState(null == replica.getEvictableTimeNs(),
"Replica " + replica + " had a refCount higher than 1, " + "Replica %s had a refCount higher than 1, " +
"but was still evictable (evictableTimeNs = " + "but was still evictable (evictableTimeNs = %d)",
replica.getEvictableTimeNs() + ")"); replica, replica.getEvictableTimeNs());
if (!replica.purged) { if (!replica.purged) {
// Add the replica to the end of an eviction list. // Add the replica to the end of an eviction list.
// Eviction lists are sorted by time. // Eviction lists are sorted by time.
@ -478,8 +477,8 @@ void unref(ShortCircuitReplica replica) {
} }
} else { } else {
Preconditions.checkArgument(replica.refCount >= 0, Preconditions.checkArgument(replica.refCount >= 0,
"replica's refCount went negative (refCount = " + "replica's refCount went negative (refCount = %d" +
replica.refCount + " for " + replica + ")"); " for %s)", replica.refCount, replica);
} }
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace(this + ": unref replica " + replica + LOG.trace(this + ": unref replica " + replica +
@ -602,7 +601,7 @@ private void removeEvictable(ShortCircuitReplica replica,
Preconditions.checkNotNull(evictableTimeNs); Preconditions.checkNotNull(evictableTimeNs);
ShortCircuitReplica removed = map.remove(evictableTimeNs); ShortCircuitReplica removed = map.remove(evictableTimeNs);
Preconditions.checkState(removed == replica, Preconditions.checkState(removed == replica,
"failed to make " + replica + " unevictable"); "failed to make %s unevictable", replica);
replica.setEvictableTimeNs(null); replica.setEvictableTimeNs(null);
} }
@ -859,7 +858,7 @@ ClientMmap getOrCreateClientMmap(ShortCircuitReplica replica,
Condition cond = (Condition)replica.mmapData; Condition cond = (Condition)replica.mmapData;
cond.awaitUninterruptibly(); cond.awaitUninterruptibly();
} else { } else {
Preconditions.checkState(false, "invalid mmapData type " + Preconditions.checkState(false, "invalid mmapData type %s",
replica.mmapData.getClass().getName()); replica.mmapData.getClass().getName());
} }
} }

View File

@ -243,19 +243,23 @@ void close() {
String suffix = ""; String suffix = "";
Preconditions.checkState(refCount == 0, Preconditions.checkState(refCount == 0,
"tried to close replica with refCount " + refCount + ": " + this); "tried to close replica with refCount %d: %s", refCount, this);
refCount = -1; refCount = -1;
Preconditions.checkState(purged, Preconditions.checkState(purged,
"tried to close unpurged replica " + this); "tried to close unpurged replica %s", this);
if (hasMmap()) { if (hasMmap()) {
munmap(); munmap();
if (LOG.isTraceEnabled()) {
suffix += " munmapped."; suffix += " munmapped.";
} }
}
IOUtils.cleanup(LOG, dataStream, metaStream); IOUtils.cleanup(LOG, dataStream, metaStream);
if (slot != null) { if (slot != null) {
cache.scheduleSlotReleaser(slot); cache.scheduleSlotReleaser(slot);
if (LOG.isTraceEnabled()) {
suffix += " scheduling " + slot + " for later release."; suffix += " scheduling " + slot + " for later release.";
} }
}
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace("closed " + this + suffix); LOG.trace("closed " + this + suffix);
} }