HDFS-6416. Merging change r1597868 from trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1597870 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Brandon Li 2014-05-27 20:26:12 +00:00
parent 3bfc0088f4
commit 3bd21e8814
3 changed files with 14 additions and 9 deletions

View File

@ -54,6 +54,7 @@ import org.apache.hadoop.nfs.nfs3.response.WccData;
import org.apache.hadoop.oncrpc.XDR;
import org.apache.hadoop.oncrpc.security.VerifierNone;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
import org.jboss.netty.channel.Channel;
import com.google.common.annotations.VisibleForTesting;
@ -136,7 +137,7 @@ class OpenFileCtx {
this.channel = channel;
this.xid = xid;
this.preOpAttr = preOpAttr;
this.startTime = System.currentTimeMillis();
this.startTime = Time.monotonicNow();
}
@Override
@ -158,11 +159,11 @@ class OpenFileCtx {
private Daemon dumpThread;
private void updateLastAccessTime() {
lastAccessTime = System.currentTimeMillis();
lastAccessTime = Time.monotonicNow();
}
private boolean checkStreamTimeout(long streamTimeout) {
return System.currentTimeMillis() - lastAccessTime > streamTimeout;
return Time.monotonicNow() - lastAccessTime > streamTimeout;
}
long getLastAccessTime() {
@ -698,7 +699,7 @@ class OpenFileCtx {
+ " updating the mtime, then return success");
Nfs3FileAttributes postOpAttr = null;
try {
dfsClient.setTimes(path, System.currentTimeMillis(), -1);
dfsClient.setTimes(path, Time.monotonicNow(), -1);
postOpAttr = Nfs3Utils.getFileAttr(dfsClient, path, iug);
} catch (IOException e) {
LOG.info("Got error when processing perfect overwrite, path=" + path
@ -1007,7 +1008,7 @@ class OpenFileCtx {
if (LOG.isDebugEnabled()) {
LOG.debug("FileId: " + latestAttr.getFileId() + " Service time:"
+ (System.currentTimeMillis() - commit.getStartTime())
+ (Time.monotonicNow() - commit.getStartTime())
+ "ms. Sent response for commit:" + commit);
}
entry = pendingCommits.firstEntry();

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.nfs.nfs3.FileHandle;
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@ -99,7 +100,7 @@ class OpenFileCtxCache {
LOG.warn("No eviction candidate. All streams have pending work.");
return null;
} else {
long idleTime = System.currentTimeMillis()
long idleTime = Time.monotonicNow()
- idlest.getValue().getLastAccessTime();
if (idleTime < Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT) {
if (LOG.isDebugEnabled()) {
@ -250,7 +251,7 @@ class OpenFileCtxCache {
// Check if it can sleep
try {
long workedTime = System.currentTimeMillis() - lastWakeupTime;
long workedTime = Time.monotonicNow() - lastWakeupTime;
if (workedTime < rotation) {
if (LOG.isTraceEnabled()) {
LOG.trace("StreamMonitor can still have a sleep:"
@ -258,7 +259,7 @@ class OpenFileCtxCache {
}
Thread.sleep(rotation - workedTime);
}
lastWakeupTime = System.currentTimeMillis();
lastWakeupTime = Time.monotonicNow();
} catch (InterruptedException e) {
LOG.info("StreamMonitor got interrupted");
@ -267,4 +268,4 @@ class OpenFileCtxCache {
}
}
}
}
}

View File

@ -129,6 +129,9 @@ Release 2.5.0 - UNRELEASED
HDFS-6110 adding more slow action log in critical write path
(Liang Xie via stack)
HDFS-6416. Use Time#monotonicNow in OpenFileCtx and OpenFileCtxCatch to
avoid system clock bugs (Abhiraj Butala via brandonli)
OPTIMIZATIONS
HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)