HDFS-5657. race condition causes writeback state error in NFS gateway. Contributed by Brandon Li
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1551691 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
991c453ca3
commit
5792d59da3
@ -85,6 +85,7 @@ static enum COMMIT_STATUS {
|
||||
private volatile boolean activeState;
|
||||
// The stream write-back status. True means one thread is doing write back.
|
||||
private volatile boolean asyncStatus;
|
||||
private volatile long asyncWriteBackStartOffset;
|
||||
|
||||
/**
|
||||
* The current offset of the file in HDFS. All the content before this offset
|
||||
@ -209,6 +210,7 @@ private long updateNonSequentialWriteInMemory(long count) {
|
||||
updateLastAccessTime();
|
||||
activeState = true;
|
||||
asyncStatus = false;
|
||||
asyncWriteBackStartOffset = 0;
|
||||
dumpOut = null;
|
||||
raf = null;
|
||||
nonSequentialWriteInMemory = new AtomicLong(0);
|
||||
@ -580,6 +582,7 @@ private synchronized boolean checkAndStartWrite(
|
||||
+ nextOffset.get());
|
||||
}
|
||||
asyncStatus = true;
|
||||
asyncWriteBackStartOffset = writeCtx.getOffset();
|
||||
asyncDataService.execute(new AsyncDataService.WriteBackTask(this));
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
@ -903,9 +906,11 @@ private synchronized WriteCtx offerNextToWrite() {
|
||||
/** Invoked by AsynDataService to write back to HDFS */
|
||||
void executeWriteBack() {
|
||||
Preconditions.checkState(asyncStatus,
|
||||
"The openFileCtx has false async status");
|
||||
"openFileCtx has false asyncStatus, fileId:" + latestAttr.getFileid());
|
||||
final long startOffset = asyncWriteBackStartOffset;
|
||||
try {
|
||||
while (activeState) {
|
||||
// asyncStatus could be changed to false in offerNextToWrite()
|
||||
WriteCtx toWrite = offerNextToWrite();
|
||||
if (toWrite != null) {
|
||||
// Do the write
|
||||
@ -921,8 +926,18 @@ void executeWriteBack() {
|
||||
+ latestAttr.getFileId());
|
||||
}
|
||||
} finally {
|
||||
// make sure we reset asyncStatus to false
|
||||
asyncStatus = false;
|
||||
// Make sure to reset asyncStatus to false unless a race happens
|
||||
synchronized (this) {
|
||||
if (startOffset == asyncWriteBackStartOffset) {
|
||||
asyncStatus = false;
|
||||
} else {
|
||||
LOG.info("Another asyn task is already started before this one"
|
||||
+ " is finalized. fileId:" + latestAttr.getFileid()
|
||||
+ " asyncStatus:" + asyncStatus + " original startOffset:"
|
||||
+ startOffset + " new startOffset:" + asyncWriteBackStartOffset
|
||||
+ ". Won't change asyncStatus here.");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -1177,4 +1192,4 @@ public String toString() {
|
||||
return String.format("activeState: %b asyncStatus: %b nextOffset: %d",
|
||||
activeState, asyncStatus, nextOffset.get());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -976,6 +976,8 @@ Release 2.3.0 - UNRELEASED
|
||||
HDFS-4201. NPE in BPServiceActor#sendHeartBeat. (jxiang via cmccabe)
|
||||
|
||||
HDFS-5666. Fix inconsistent synchronization in BPOfferService (jxiang via cmccabe)
|
||||
|
||||
HDFS-5657. race condition causes writeback state error in NFS gateway (brandonli)
|
||||
|
||||
Release 2.2.0 - 2013-10-13
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user