HDFS-5657. Merging change r1551691 from trunk
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1551705 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
88d09130e8
commit
c143486c17
|
@ -85,6 +85,7 @@ class OpenFileCtx {
|
||||||
private volatile boolean activeState;
|
private volatile boolean activeState;
|
||||||
// The stream write-back status. True means one thread is doing write back.
|
// The stream write-back status. True means one thread is doing write back.
|
||||||
private volatile boolean asyncStatus;
|
private volatile boolean asyncStatus;
|
||||||
|
private volatile long asyncWriteBackStartOffset;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The current offset of the file in HDFS. All the content before this offset
|
* The current offset of the file in HDFS. All the content before this offset
|
||||||
|
@ -209,6 +210,7 @@ class OpenFileCtx {
|
||||||
updateLastAccessTime();
|
updateLastAccessTime();
|
||||||
activeState = true;
|
activeState = true;
|
||||||
asyncStatus = false;
|
asyncStatus = false;
|
||||||
|
asyncWriteBackStartOffset = 0;
|
||||||
dumpOut = null;
|
dumpOut = null;
|
||||||
raf = null;
|
raf = null;
|
||||||
nonSequentialWriteInMemory = new AtomicLong(0);
|
nonSequentialWriteInMemory = new AtomicLong(0);
|
||||||
|
@ -582,6 +584,7 @@ class OpenFileCtx {
|
||||||
+ nextOffset.get());
|
+ nextOffset.get());
|
||||||
}
|
}
|
||||||
asyncStatus = true;
|
asyncStatus = true;
|
||||||
|
asyncWriteBackStartOffset = writeCtx.getOffset();
|
||||||
asyncDataService.execute(new AsyncDataService.WriteBackTask(this));
|
asyncDataService.execute(new AsyncDataService.WriteBackTask(this));
|
||||||
} else {
|
} else {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
|
@ -911,9 +914,11 @@ class OpenFileCtx {
|
||||||
/** Invoked by AsynDataService to write back to HDFS */
|
/** Invoked by AsynDataService to write back to HDFS */
|
||||||
void executeWriteBack() {
|
void executeWriteBack() {
|
||||||
Preconditions.checkState(asyncStatus,
|
Preconditions.checkState(asyncStatus,
|
||||||
"The openFileCtx has false async status");
|
"openFileCtx has false asyncStatus, fileId:" + latestAttr.getFileid());
|
||||||
|
final long startOffset = asyncWriteBackStartOffset;
|
||||||
try {
|
try {
|
||||||
while (activeState) {
|
while (activeState) {
|
||||||
|
// asyncStatus could be changed to false in offerNextToWrite()
|
||||||
WriteCtx toWrite = offerNextToWrite();
|
WriteCtx toWrite = offerNextToWrite();
|
||||||
if (toWrite != null) {
|
if (toWrite != null) {
|
||||||
// Do the write
|
// Do the write
|
||||||
|
@ -929,8 +934,18 @@ class OpenFileCtx {
|
||||||
+ latestAttr.getFileId());
|
+ latestAttr.getFileId());
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
// make sure we reset asyncStatus to false
|
// Make sure to reset asyncStatus to false unless a race happens
|
||||||
|
synchronized (this) {
|
||||||
|
if (startOffset == asyncWriteBackStartOffset) {
|
||||||
asyncStatus = false;
|
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.");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -392,6 +392,8 @@ Release 2.3.0 - UNRELEASED
|
||||||
|
|
||||||
HDFS-5666. Fix inconsistent synchronization in BPOfferService (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
|
Release 2.2.0 - 2013-10-13
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
Loading…
Reference in New Issue