HDFS-3641. Move server Util time methods to common and use now instead of System#currentTimeMillis. Contributed by Eli Collins

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1360896 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Eli Collins 2012-07-12 19:59:20 +00:00
parent 790e970931
commit dce88a6dbc
122 changed files with 556 additions and 421 deletions

View File

@ -27,6 +27,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.Time;
/**
* A daemon thread that waits for the next file system to renew.
@ -62,7 +63,7 @@ public class DelegationTokenRenewer<T extends FileSystem & DelegationTokenRenewe
/** Get the delay until this event should happen. */
@Override
public long getDelay(final TimeUnit unit) {
final long millisLeft = renewalTime - System.currentTimeMillis();
final long millisLeft = renewalTime - Time.now();
return unit.convert(millisLeft, TimeUnit.MILLISECONDS);
}
@ -92,7 +93,7 @@ public class DelegationTokenRenewer<T extends FileSystem & DelegationTokenRenewe
* @param newTime the new time
*/
private void updateRenewalTime() {
renewalTime = RENEW_CYCLE + System.currentTimeMillis();
renewalTime = RENEW_CYCLE + Time.now();
}
/**

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.Time;
/** Provides a <i>trash</i> feature. Files are moved to a user's trash
* directory, a subdirectory of their home directory named ".Trash". Files are
@ -136,7 +137,7 @@ public class TrashPolicyDefault extends TrashPolicy {
String orig = trashPath.toString();
while(fs.exists(trashPath)) {
trashPath = new Path(orig + System.currentTimeMillis());
trashPath = new Path(orig + Time.now());
}
if (fs.rename(path, trashPath)) // move to current trash
@ -187,7 +188,7 @@ public class TrashPolicyDefault extends TrashPolicy {
return;
}
long now = System.currentTimeMillis();
long now = Time.now();
for (int i = 0; i < dirs.length; i++) {
Path path = dirs[i].getPath();
String dir = path.toUri().getPath();
@ -248,7 +249,7 @@ public class TrashPolicyDefault extends TrashPolicy {
public void run() {
if (emptierInterval == 0)
return; // trash disabled
long now = System.currentTimeMillis();
long now = Time.now();
long end;
while (true) {
end = ceiling(now, emptierInterval);
@ -259,7 +260,7 @@ public class TrashPolicyDefault extends TrashPolicy {
}
try {
now = System.currentTimeMillis();
now = Time.now();
if (now >= end) {
FileStatus[] homes = null;

View File

@ -55,6 +55,7 @@ import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time;
/**
* ViewFileSystem (extends the FileSystem interface) implements a client-side
@ -146,7 +147,7 @@ public class ViewFileSystem extends FileSystem {
*/
public ViewFileSystem() throws IOException {
ugi = UserGroupInformation.getCurrentUser();
creationTime = System.currentTimeMillis();
creationTime = Time.now();
}
/**

View File

@ -55,6 +55,7 @@ import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time;
/**
@ -192,7 +193,7 @@ public class ViewFs extends AbstractFileSystem {
ViewFs(final URI theUri, final Configuration conf) throws IOException,
URISyntaxException {
super(theUri, FsConstants.VIEWFS_SCHEME, false, -1);
creationTime = System.currentTimeMillis();
creationTime = Time.now();
ugi = UserGroupInformation.getCurrentUser();
config = conf;
// Now build client side view (i.e. client side mount table) from config.

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.NativeCodeLoader;
import org.apache.hadoop.util.MergeSort;
import org.apache.hadoop.util.PriorityQueue;
import org.apache.hadoop.util.Time;
/**
* <code>SequenceFile</code>s are flat files consisting of binary key/value
@ -835,7 +836,7 @@ public class SequenceFile {
{
try {
MessageDigest digester = MessageDigest.getInstance("MD5");
long time = System.currentTimeMillis();
long time = Time.now();
digester.update((new UID()+"@"+time).getBytes());
sync = digester.digest();
} catch (Exception e) {

View File

@ -75,6 +75,7 @@ import org.apache.hadoop.security.token.TokenInfo;
import org.apache.hadoop.security.token.TokenSelector;
import org.apache.hadoop.util.ProtoUtil;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Time;
/** A client for an IPC service. IPC calls take a single {@link Writable} as a
* parameter, and return a {@link Writable} as their value. A service runs on
@ -316,7 +317,7 @@ public class Client {
/** Update lastActivity with the current time. */
private void touch() {
lastActivity.set(System.currentTimeMillis());
lastActivity.set(Time.now());
}
/**
@ -762,7 +763,7 @@ public class Client {
private synchronized boolean waitForWork() {
if (calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
long timeout = maxIdleTime-
(System.currentTimeMillis()-lastActivity.get());
(Time.now()-lastActivity.get());
if (timeout>0) {
try {
wait(timeout);
@ -792,7 +793,7 @@ public class Client {
* since last I/O activity is equal to or greater than the ping interval
*/
private synchronized void sendPing() throws IOException {
long curTime = System.currentTimeMillis();
long curTime = Time.now();
if ( curTime - lastActivity.get() >= pingInterval) {
lastActivity.set(curTime);
synchronized (out) {

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.ProtoUtil;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingService;
@ -185,7 +186,7 @@ public class ProtobufRpcEngine implements RpcEngine {
throws ServiceException {
long startTime = 0;
if (LOG.isDebugEnabled()) {
startTime = System.currentTimeMillis();
startTime = Time.now();
}
HadoopRpcRequestProto rpcRequest = constructRpcRequest(method, args);
@ -198,7 +199,7 @@ public class ProtobufRpcEngine implements RpcEngine {
}
if (LOG.isDebugEnabled()) {
long callTime = System.currentTimeMillis() - startTime;
long callTime = Time.now() - startTime;
LOG.debug("Call: " + method.getName() + " " + callTime);
}
@ -426,10 +427,10 @@ public class ProtobufRpcEngine implements RpcEngine {
.mergeFrom(rpcRequest.getRequest()).build();
Message result;
try {
long startTime = System.currentTimeMillis();
long startTime = Time.now();
server.rpcDetailedMetrics.init(protocolImpl.protocolClass);
result = service.callBlockingMethod(methodDescriptor, null, param);
int processingTime = (int) (System.currentTimeMillis() - startTime);
int processingTime = (int) (Time.now() - startTime);
int qTime = (int) (startTime - receiveTime);
if (LOG.isDebugEnabled()) {
LOG.info("Served: " + methodName + " queueTime= " + qTime +

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Time;
import com.google.protobuf.BlockingService;
@ -369,7 +370,7 @@ public class RPC {
int rpcTimeout,
RetryPolicy connectionRetryPolicy,
long timeout) throws IOException {
long startTime = System.currentTimeMillis();
long startTime = Time.now();
IOException ioe;
while (true) {
try {
@ -387,7 +388,7 @@ public class RPC {
ioe = nrthe;
}
// check if timed out
if (System.currentTimeMillis()-timeout >= startTime) {
if (Time.now()-timeout >= startTime) {
throw ioe;
}

View File

@ -95,6 +95,7 @@ import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.ProtoUtil;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
@ -411,7 +412,7 @@ public abstract class Server {
this.callId = id;
this.rpcRequest = param;
this.connection = connection;
this.timestamp = System.currentTimeMillis();
this.timestamp = Time.now();
this.rpcResponse = null;
this.rpcKind = kind;
}
@ -561,7 +562,7 @@ public abstract class Server {
*/
private void cleanupConnections(boolean force) {
if (force || numConnections > thresholdIdleConnections) {
long currentTime = System.currentTimeMillis();
long currentTime = Time.now();
if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
return;
}
@ -597,7 +598,7 @@ public abstract class Server {
}
else i++;
}
lastCleanupRunTime = System.currentTimeMillis();
lastCleanupRunTime = Time.now();
}
}
@ -682,7 +683,7 @@ public abstract class Server {
try {
reader.startAdd();
SelectionKey readKey = reader.registerChannel(channel);
c = new Connection(readKey, channel, System.currentTimeMillis());
c = new Connection(readKey, channel, Time.now());
readKey.attach(c);
synchronized (connectionList) {
connectionList.add(numConnections, c);
@ -704,7 +705,7 @@ public abstract class Server {
if (c == null) {
return;
}
c.setLastContact(System.currentTimeMillis());
c.setLastContact(Time.now());
try {
count = c.readAndProcess();
@ -726,7 +727,7 @@ public abstract class Server {
c = null;
}
else {
c.setLastContact(System.currentTimeMillis());
c.setLastContact(Time.now());
}
}
@ -805,7 +806,7 @@ public abstract class Server {
LOG.info(getName() + ": doAsyncWrite threw exception " + e);
}
}
long now = System.currentTimeMillis();
long now = Time.now();
if (now < lastPurgeTime + PURGE_INTERVAL) {
continue;
}
@ -951,7 +952,7 @@ public abstract class Server {
if (inHandler) {
// set the serve time when the response has to be sent later
call.timestamp = System.currentTimeMillis();
call.timestamp = Time.now();
incPending();
try {

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.*;
@ -218,13 +219,13 @@ public class WritableRpcEngine implements RpcEngine {
throws Throwable {
long startTime = 0;
if (LOG.isDebugEnabled()) {
startTime = System.currentTimeMillis();
startTime = Time.now();
}
ObjectWritable value = (ObjectWritable)
client.call(RPC.RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId);
if (LOG.isDebugEnabled()) {
long callTime = System.currentTimeMillis() - startTime;
long callTime = Time.now() - startTime;
LOG.debug("Call: " + method.getName() + " " + callTime);
}
return value.get();
@ -464,7 +465,7 @@ public class WritableRpcEngine implements RpcEngine {
// Invoke the protocol method
long startTime = System.currentTimeMillis();
long startTime = Time.now();
Method method =
protocolImpl.protocolClass.getMethod(call.getMethodName(),
call.getParameterClasses());
@ -472,7 +473,7 @@ public class WritableRpcEngine implements RpcEngine {
server.rpcDetailedMetrics.init(protocolImpl.protocolClass);
Object value =
method.invoke(protocolImpl.protocolImpl, call.getParameters());
int processingTime = (int) (System.currentTimeMillis() - startTime);
int processingTime = (int) (Time.now() - startTime);
int qTime = (int) (startTime-receivedTime);
if (LOG.isDebugEnabled()) {
LOG.debug("Served: " + call.getMethodName() +

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.metrics2.MetricsFilter;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.MetricsTag;
import org.apache.hadoop.metrics2.lib.Interns;
import org.apache.hadoop.util.Time;
class MetricsRecordBuilderImpl extends MetricsRecordBuilder {
private final MetricsCollector parent;
@ -44,7 +45,7 @@ class MetricsRecordBuilderImpl extends MetricsRecordBuilder {
MetricsFilter rf, MetricsFilter mf,
boolean acceptable) {
this.parent = parent;
timestamp = System.currentTimeMillis();
timestamp = Time.now();
recInfo = info;
metrics = Lists.newArrayList();
tags = Lists.newArrayList();

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import static org.apache.hadoop.metrics2.util.Contracts.*;
import org.apache.hadoop.metrics2.MetricsFilter;
import org.apache.hadoop.metrics2.MetricsSink;
import org.apache.hadoop.util.Time;
/**
* An adapter class for metrics sink and associated filters
@ -158,7 +159,7 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
}
if (ts > 0) {
sink.flush();
latency.add(System.currentTimeMillis() - ts);
latency.add(Time.now() - ts);
}
LOG.debug("Done");
}

View File

@ -40,6 +40,8 @@ import org.apache.hadoop.metrics2.MetricsSource;
import org.apache.hadoop.metrics2.MetricsTag;
import static org.apache.hadoop.metrics2.impl.MetricsConfig.*;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.util.Time;
import static org.apache.hadoop.metrics2.util.Contracts.*;
/**
@ -152,9 +154,9 @@ class MetricsSourceAdapter implements DynamicMBean {
private void updateJmxCache() {
boolean getAllMetrics = false;
synchronized(this) {
if (System.currentTimeMillis() - jmxCacheTS >= jmxCacheTTL) {
if (Time.now() - jmxCacheTS >= jmxCacheTTL) {
// temporarilly advance the expiry while updating the cache
jmxCacheTS = System.currentTimeMillis() + jmxCacheTTL;
jmxCacheTS = Time.now() + jmxCacheTTL;
if (lastRecs == null) {
getAllMetrics = true;
}
@ -175,7 +177,7 @@ class MetricsSourceAdapter implements DynamicMBean {
if (oldCacheSize < newCacheSize) {
updateInfoCache();
}
jmxCacheTS = System.currentTimeMillis();
jmxCacheTS = Time.now();
lastRecs = null; // in case regular interval update is not running
}
}

View File

@ -60,6 +60,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
import org.apache.hadoop.metrics2.lib.MetricsSourceBuilder;
import org.apache.hadoop.metrics2.lib.MutableStat;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.util.Time;
/**
* A base class for metrics system singletons
@ -372,10 +373,10 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
private void snapshotMetrics(MetricsSourceAdapter sa,
MetricsBufferBuilder bufferBuilder) {
long startTime = System.currentTimeMillis();
long startTime = Time.now();
bufferBuilder.add(sa.name(), sa.getMetrics(collector, false));
collector.clear();
snapshotStat.add(System.currentTimeMillis() - startTime);
snapshotStat.add(Time.now() - startTime);
LOG.debug("Snapshotted source "+ sa.name());
}
@ -386,9 +387,9 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
synchronized void publishMetrics(MetricsBuffer buffer) {
int dropped = 0;
for (MetricsSinkAdapter sa : sinks.values()) {
long startTime = System.currentTimeMillis();
long startTime = Time.now();
dropped += sa.putMetrics(buffer, logicalTime) ? 0 : 1;
publishStat.add(System.currentTimeMillis() - startTime);
publishStat.add(Time.now() - startTime);
}
droppedPubAll.incr(dropped);
}

View File

@ -34,6 +34,7 @@ import java.util.LinkedList;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
/**
* This supports input and output streams for a socket channels.
@ -194,7 +195,7 @@ abstract class SocketIOWithTimeout {
}
long timeoutLeft = timeout;
long endTime = (timeout > 0) ? (System.currentTimeMillis() + timeout): 0;
long endTime = (timeout > 0) ? (Time.now() + timeout): 0;
while (true) {
// we might have to call finishConnect() more than once
@ -209,7 +210,7 @@ abstract class SocketIOWithTimeout {
if (ret == 0 ||
(timeout > 0 &&
(timeoutLeft = (endTime - System.currentTimeMillis())) <= 0)) {
(timeoutLeft = (endTime - Time.now())) <= 0)) {
throw new SocketTimeoutException(
timeoutExceptionString(channel, timeout,
SelectionKey.OP_CONNECT));
@ -329,7 +330,7 @@ abstract class SocketIOWithTimeout {
try {
while (true) {
long start = (timeout == 0) ? 0 : System.currentTimeMillis();
long start = (timeout == 0) ? 0 : Time.now();
key = channel.register(info.selector, ops);
ret = info.selector.select(timeout);
@ -342,7 +343,7 @@ abstract class SocketIOWithTimeout {
* unknown reasons. So select again if required.
*/
if (timeout > 0) {
timeout -= System.currentTimeMillis() - start;
timeout -= Time.now() - start;
if (timeout <= 0) {
return 0;
}
@ -414,7 +415,7 @@ abstract class SocketIOWithTimeout {
selInfo = queue.removeLast();
}
trimIdleSelectors(System.currentTimeMillis());
trimIdleSelectors(Time.now());
return selInfo;
}
@ -425,7 +426,7 @@ abstract class SocketIOWithTimeout {
* @param info
*/
private synchronized void release(SelectorInfo info) {
long now = System.currentTimeMillis();
long now = Time.now();
trimIdleSelectors(now);
info.lastActivityTime = now;
info.queue.addLast(info);

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Time;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -75,7 +76,7 @@ public class Groups {
public List<String> getGroups(String user) throws IOException {
// Return cached value if available
CachedGroups groups = userToGroupsMap.get(user);
long now = System.currentTimeMillis();
long now = Time.now();
// if cache has a value and it hasn't expired
if (groups != null && (groups.getTimestamp() + cacheTimeout > now)) {
if(LOG.isDebugEnabled()) {
@ -134,7 +135,7 @@ public class Groups {
*/
CachedGroups(List<String> groups) {
this.groups = groups;
this.timestamp = System.currentTimeMillis();
this.timestamp = Time.now();
}
/**

View File

@ -64,6 +64,7 @@ import org.apache.hadoop.security.authentication.util.KerberosUtil;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.Time;
/**
* User and group information for Hadoop.
@ -710,7 +711,7 @@ public class UserGroupInformation {
long nextRefresh = getRefreshTime(tgt);
while (true) {
try {
long now = System.currentTimeMillis();
long now = Time.now();
if(LOG.isDebugEnabled()) {
LOG.debug("Current time is " + now);
LOG.debug("Next refresh is " + nextRefresh);
@ -772,15 +773,15 @@ public class UserGroupInformation {
try {
login = newLoginContext(HadoopConfiguration.KEYTAB_KERBEROS_CONFIG_NAME,
subject, new HadoopConfiguration());
start = System.currentTimeMillis();
start = Time.now();
login.login();
metrics.loginSuccess.add(System.currentTimeMillis() - start);
metrics.loginSuccess.add(Time.now() - start);
loginUser = new UserGroupInformation(subject);
loginUser.setLogin(login);
loginUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
} catch (LoginException le) {
if (start > 0) {
metrics.loginFailure.add(System.currentTimeMillis() - start);
metrics.loginFailure.add(Time.now() - start);
}
throw new IOException("Login failure for " + user + " from keytab " +
path, le);
@ -800,7 +801,7 @@ public class UserGroupInformation {
|| !isKeytab)
return;
KerberosTicket tgt = getTGT();
if (tgt != null && System.currentTimeMillis() < getRefreshTime(tgt)) {
if (tgt != null && Time.now() < getRefreshTime(tgt)) {
return;
}
reloginFromKeytab();
@ -824,7 +825,7 @@ public class UserGroupInformation {
!isKeytab)
return;
long now = System.currentTimeMillis();
long now = Time.now();
if (!hasSufficientTimeElapsed(now)) {
return;
}
@ -856,14 +857,14 @@ public class UserGroupInformation {
HadoopConfiguration.KEYTAB_KERBEROS_CONFIG_NAME, getSubject(),
new HadoopConfiguration());
LOG.info("Initiating re-login for " + keytabPrincipal);
start = System.currentTimeMillis();
start = Time.now();
login.login();
metrics.loginSuccess.add(System.currentTimeMillis() - start);
metrics.loginSuccess.add(Time.now() - start);
setLogin(login);
}
} catch (LoginException le) {
if (start > 0) {
metrics.loginFailure.add(System.currentTimeMillis() - start);
metrics.loginFailure.add(Time.now() - start);
}
throw new IOException("Login failure for " + keytabPrincipal +
" from keytab " + keytabFile, le);
@ -889,7 +890,7 @@ public class UserGroupInformation {
if (login == null) {
throw new IOException("login must be done first");
}
long now = System.currentTimeMillis();
long now = Time.now();
if (!hasSufficientTimeElapsed(now)) {
return;
}
@ -944,9 +945,9 @@ public class UserGroupInformation {
HadoopConfiguration.KEYTAB_KERBEROS_CONFIG_NAME, subject,
new HadoopConfiguration());
start = System.currentTimeMillis();
start = Time.now();
login.login();
metrics.loginSuccess.add(System.currentTimeMillis() - start);
metrics.loginSuccess.add(Time.now() - start);
UserGroupInformation newLoginUser = new UserGroupInformation(subject);
newLoginUser.setLogin(login);
newLoginUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
@ -954,7 +955,7 @@ public class UserGroupInformation {
return newLoginUser;
} catch (LoginException le) {
if (start > 0) {
metrics.loginFailure.add(System.currentTimeMillis() - start);
metrics.loginFailure.add(Time.now() - start);
}
throw new IOException("Login failure for " + user + " from keytab " +
path, le);

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.security.HadoopKerberosName;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
import com.google.common.base.Preconditions;
@ -165,7 +166,7 @@ extends AbstractDelegationTokenIdentifier>
synchronized (this) {
removeExpiredKeys();
/* set final expiry date for retiring currentKey */
currentKey.setExpiryDate(System.currentTimeMillis() + tokenMaxLifetime);
currentKey.setExpiryDate(Time.now() + tokenMaxLifetime);
/*
* currentKey might have been removed by removeExpiredKeys(), if
* updateMasterKey() isn't called at expected interval. Add it back to
@ -177,7 +178,7 @@ extends AbstractDelegationTokenIdentifier>
}
private synchronized void removeExpiredKeys() {
long now = System.currentTimeMillis();
long now = Time.now();
for (Iterator<Map.Entry<Integer, DelegationKey>> it = allKeys.entrySet()
.iterator(); it.hasNext();) {
Map.Entry<Integer, DelegationKey> e = it.next();
@ -191,7 +192,7 @@ extends AbstractDelegationTokenIdentifier>
protected synchronized byte[] createPassword(TokenIdent identifier) {
LOG.info("Creating password for identifier: "+identifier);
int sequenceNum;
long now = System.currentTimeMillis();
long now = Time.now();
sequenceNum = ++delegationTokenSequenceNumber;
identifier.setIssueDate(now);
identifier.setMaxDate(now + tokenMaxLifetime);
@ -211,7 +212,7 @@ extends AbstractDelegationTokenIdentifier>
throw new InvalidToken("token (" + identifier.toString()
+ ") can't be found in cache");
}
long now = System.currentTimeMillis();
long now = Time.now();
if (info.getRenewDate() < now) {
throw new InvalidToken("token (" + identifier.toString() + ") is expired");
}
@ -243,7 +244,7 @@ extends AbstractDelegationTokenIdentifier>
*/
public synchronized long renewToken(Token<TokenIdent> token,
String renewer) throws InvalidToken, IOException {
long now = System.currentTimeMillis();
long now = Time.now();
ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
DataInputStream in = new DataInputStream(buf);
TokenIdent id = createIdentifier();
@ -353,7 +354,7 @@ extends AbstractDelegationTokenIdentifier>
/** Remove expired delegation tokens from cache */
private synchronized void removeExpiredToken() {
long now = System.currentTimeMillis();
long now = Time.now();
Iterator<DelegationTokenInformation> i = currentTokens.values().iterator();
while (i.hasNext()) {
long renewDate = i.next().getRenewDate();
@ -399,7 +400,7 @@ extends AbstractDelegationTokenIdentifier>
/ (60 * 1000) + " min(s)");
try {
while (running) {
long now = System.currentTimeMillis();
long now = Time.now();
if (lastMasterKeyUpdate + keyUpdateInterval < now) {
try {
rollMasterKey();

View File

@ -126,12 +126,12 @@ public class AsyncDiskService {
public synchronized boolean awaitTermination(long milliseconds)
throws InterruptedException {
long end = System.currentTimeMillis() + milliseconds;
long end = Time.now() + milliseconds;
for (Map.Entry<String, ThreadPoolExecutor> e:
executors.entrySet()) {
ThreadPoolExecutor executor = e.getValue();
if (!executor.awaitTermination(
Math.max(end - System.currentTimeMillis(), 0),
Math.max(end - Time.now(), 0),
TimeUnit.MILLISECONDS)) {
LOG.warn("AsyncDiskService awaitTermination timeout.");
return false;

View File

@ -205,7 +205,7 @@ public class ReflectionUtils {
boolean dumpStack = false;
if (log.isInfoEnabled()) {
synchronized (ReflectionUtils.class) {
long now = System.currentTimeMillis();
long now = Time.now();
if (now - previousLogTime >= minInterval * 1000) {
previousLogTime = now;
dumpStack = true;

View File

@ -124,7 +124,7 @@ abstract public class Shell {
/** check to see if a command needs to be executed and execute if needed */
protected void run() throws IOException {
if (lastTime + interval > System.currentTimeMillis())
if (lastTime + interval > Time.now())
return;
exitCode = 0; // reset for next run
runCommand();
@ -223,7 +223,7 @@ abstract public class Shell {
LOG.warn("Error while closing the error stream", ioe);
}
process.destroy();
lastTime = System.currentTimeMillis();
lastTime = Time.now();
}
}

View File

@ -35,10 +35,10 @@ public class ThreadUtil {
* @param millis the number of milliseconds for the current thread to sleep
*/
public static void sleepAtLeastIgnoreInterrupts(long millis) {
long start = System.currentTimeMillis();
while (System.currentTimeMillis() - start < millis) {
long start = Time.now();
while (Time.now() - start < millis) {
long timeToSleep = millis -
(System.currentTimeMillis() - start);
(Time.now() - start);
try {
Thread.sleep(timeToSleep);
} catch (InterruptedException ie) {

View File

@ -0,0 +1,52 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.util;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Utility methods for getting the time and computing intervals.
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Unstable
public final class Time {
/**
* Current system time. Do not use this to calculate a duration or interval
* to sleep, because it will be broken by settimeofday. Instead, use
* monotonicNow.
* @return current time in msec.
*/
public static long now() {
return System.currentTimeMillis();
}
/**
* Current time from some arbitrary time base in the past, counting in
* milliseconds, and not affected by settimeofday or similar system clock
* changes. This is appropriate to use when computing how much longer to
* wait for an interval to expire.
* @return a monotonic clock that counts in milliseconds.
*/
public static long monotonicNow() {
final long NANOSECONDS_PER_MILLISECOND = 1000000;
return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
}
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.conf;
import org.apache.hadoop.util.Time;
import org.junit.Test;
import org.junit.Before;
import static org.junit.Assert.*;
@ -295,8 +296,8 @@ public class TestReconfiguration {
}
dummy.reconfigureProperty(PROP1, VAL2);
long endWait = System.currentTimeMillis() + 2000;
while (dummyThread.isAlive() && System.currentTimeMillis() < endWait) {
long endWait = Time.now() + 2000;
while (dummyThread.isAlive() && Time.now() < endWait) {
try {
Thread.sleep(50);
} catch (InterruptedException ignore) {

View File

@ -32,6 +32,7 @@ import java.util.Set;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.Time;
/**
* This class tests commands from Trash.
@ -600,7 +601,7 @@ public class TestTrash extends TestCase {
writeFile(fs, myFile, 10);
start = System.currentTimeMillis();
start = Time.now();
try {
retVal = shell.run(args);
@ -612,7 +613,7 @@ public class TestTrash extends TestCase {
assertTrue(retVal == 0);
long iterTime = System.currentTimeMillis() - start;
long iterTime = Time.now() - start;
// take median of the first 10 runs
if(i<10) {
if(i==0) {

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.Options.CreateOpts;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@ -121,7 +122,7 @@ public class LoadGenerator extends Configured implements Tool {
private double [] writeProbs = {0.3333};
private volatile int currentIndex = 0;
long totalTime = 0;
private long startTime = System.currentTimeMillis()+10000;
private long startTime = Time.now()+10000;
final static private int BLOCK_SIZE = 10;
private ArrayList<String> files = new ArrayList<String>(); // a table of file names
private ArrayList<String> dirs = new ArrayList<String>(); // a table of directory names
@ -232,9 +233,9 @@ public class LoadGenerator extends Configured implements Tool {
* the entire file */
private void read() throws IOException {
String fileName = files.get(r.nextInt(files.size()));
long startTime = System.currentTimeMillis();
long startTime = Time.now();
InputStream in = fc.open(new Path(fileName));
executionTime[OPEN] += (System.currentTimeMillis()-startTime);
executionTime[OPEN] += (Time.now()-startTime);
totalNumOfOps[OPEN]++;
while (in.read(buffer) != -1) {}
in.close();
@ -254,9 +255,9 @@ public class LoadGenerator extends Configured implements Tool {
double fileSize = 0;
while ((fileSize = r.nextGaussian()+2)<=0) {}
genFile(file, (long)(fileSize*BLOCK_SIZE));
long startTime = System.currentTimeMillis();
long startTime = Time.now();
fc.delete(file, true);
executionTime[DELETE] += (System.currentTimeMillis()-startTime);
executionTime[DELETE] += (Time.now()-startTime);
totalNumOfOps[DELETE]++;
}
@ -265,9 +266,9 @@ public class LoadGenerator extends Configured implements Tool {
*/
private void list() throws IOException {
String dirName = dirs.get(r.nextInt(dirs.size()));
long startTime = System.currentTimeMillis();
long startTime = Time.now();
fc.listStatus(new Path(dirName));
executionTime[LIST] += (System.currentTimeMillis()-startTime);
executionTime[LIST] += (Time.now()-startTime);
totalNumOfOps[LIST]++;
}
}
@ -435,7 +436,7 @@ public class LoadGenerator extends Configured implements Tool {
}
if (r==null) {
r = new Random(System.currentTimeMillis()+hostHashCode);
r = new Random(Time.now()+hostHashCode);
}
return initFileDirTables();
@ -571,7 +572,7 @@ public class LoadGenerator extends Configured implements Tool {
*/
private void barrier() {
long sleepTime;
while ((sleepTime = startTime - System.currentTimeMillis()) > 0) {
while ((sleepTime = startTime - Time.now()) > 0) {
try {
Thread.sleep(sleepTime);
} catch (InterruptedException ex) {
@ -583,20 +584,20 @@ public class LoadGenerator extends Configured implements Tool {
* The file is filled with 'a'.
*/
private void genFile(Path file, long fileSize) throws IOException {
long startTime = System.currentTimeMillis();
long startTime = Time.now();
FSDataOutputStream out = fc.create(file,
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
CreateOpts.createParent(), CreateOpts.bufferSize(4096),
CreateOpts.repFac((short) 3));
executionTime[CREATE] += (System.currentTimeMillis()-startTime);
executionTime[CREATE] += (Time.now()-startTime);
totalNumOfOps[CREATE]++;
for (long i=0; i<fileSize; i++) {
out.writeByte('a');
}
startTime = System.currentTimeMillis();
startTime = Time.now();
out.close();
executionTime[WRITE_CLOSE] += (System.currentTimeMillis()-startTime);
executionTime[WRITE_CLOSE] += (Time.now()-startTime);
totalNumOfOps[WRITE_CLOSE]++;
}

View File

@ -39,6 +39,7 @@ import java.util.TreeSet;
import java.util.Map.Entry;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.Time;
/**
* <p>
@ -59,7 +60,7 @@ class InMemoryNativeFileSystemStore implements NativeFileSystemStore {
}
public void storeEmptyFile(String key) throws IOException {
metadataMap.put(key, new FileMetadata(key, 0, System.currentTimeMillis()));
metadataMap.put(key, new FileMetadata(key, 0, Time.now()));
dataMap.put(key, new byte[0]);
}
@ -81,7 +82,7 @@ class InMemoryNativeFileSystemStore implements NativeFileSystemStore {
}
}
metadataMap.put(key,
new FileMetadata(key, file.length(), System.currentTimeMillis()));
new FileMetadata(key, file.length(), Time.now()));
dataMap.put(key, out.toByteArray());
}

View File

@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.server.ZooKeeperServer;
@ -36,7 +37,7 @@ public abstract class ActiveStandbyElectorTestUtil {
public static void waitForActiveLockData(TestContext ctx,
ZooKeeperServer zks, String parentDir, byte[] activeData)
throws Exception {
long st = System.currentTimeMillis();
long st = Time.now();
long lastPrint = st;
while (true) {
if (ctx != null) {
@ -51,17 +52,17 @@ public abstract class ActiveStandbyElectorTestUtil {
Arrays.equals(activeData, data)) {
return;
}
if (System.currentTimeMillis() > lastPrint + LOG_INTERVAL_MS) {
if (Time.now() > lastPrint + LOG_INTERVAL_MS) {
LOG.info("Cur data: " + StringUtils.byteToHexString(data));
lastPrint = System.currentTimeMillis();
lastPrint = Time.now();
}
} catch (NoNodeException nne) {
if (activeData == null) {
return;
}
if (System.currentTimeMillis() > lastPrint + LOG_INTERVAL_MS) {
if (Time.now() > lastPrint + LOG_INTERVAL_MS) {
LOG.info("Cur data: no node");
lastPrint = System.currentTimeMillis();
lastPrint = Time.now();
}
}
Thread.sleep(50);

View File

@ -32,6 +32,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.util.Time;
import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.TestableZooKeeper;
import org.apache.zookeeper.WatchedEvent;
@ -111,11 +112,11 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
return connected;
}
synchronized void waitForConnected(long timeout) throws InterruptedException, TimeoutException {
long expire = System.currentTimeMillis() + timeout;
long expire = Time.now() + timeout;
long left = timeout;
while(!connected && left > 0) {
wait(left);
left = expire - System.currentTimeMillis();
left = expire - Time.now();
}
if (!connected) {
throw new TimeoutException("Did not connect");
@ -123,11 +124,11 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
}
}
synchronized void waitForDisconnected(long timeout) throws InterruptedException, TimeoutException {
long expire = System.currentTimeMillis() + timeout;
long expire = Time.now() + timeout;
long left = timeout;
while(connected && left > 0) {
wait(left);
left = expire - System.currentTimeMillis();
left = expire - Time.now();
}
if (connected) {
throw new TimeoutException("Did not disconnect");
@ -248,7 +249,7 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
}
public static boolean waitForServerUp(String hp, long timeout) {
long start = System.currentTimeMillis();
long start = Time.now();
while (true) {
try {
// if there are multiple hostports, just take the first one
@ -263,7 +264,7 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
LOG.info("server " + hp + " not up " + e);
}
if (System.currentTimeMillis() > start + timeout) {
if (Time.now() > start + timeout) {
break;
}
try {
@ -275,7 +276,7 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
return false;
}
public static boolean waitForServerDown(String hp, long timeout) {
long start = System.currentTimeMillis();
long start = Time.now();
while (true) {
try {
HostPort hpobj = parseHostPortList(hp).get(0);
@ -284,7 +285,7 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
return true;
}
if (System.currentTimeMillis() > start + timeout) {
if (Time.now() > start + timeout) {
break;
}
try {

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.HealthMonitor.Callback;
import org.apache.hadoop.ha.HealthMonitor.State;
import org.apache.hadoop.util.Time;
import org.junit.Before;
import org.junit.Test;
@ -136,8 +137,8 @@ public class TestHealthMonitor {
private void waitForState(HealthMonitor hm, State state)
throws InterruptedException {
long st = System.currentTimeMillis();
while (System.currentTimeMillis() - st < 2000) {
long st = Time.now();
while (Time.now() - st < 2000) {
if (hm.getHealthState() == state) {
return;
}

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
import org.apache.hadoop.ha.HealthMonitor.State;
import org.apache.hadoop.ha.MiniZKFCCluster.DummyZKFC;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Level;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooKeeper;
@ -394,9 +395,9 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
// Ask it to cede active for 3 seconds. It should respond promptly
// (i.e. the RPC itself should not take 3 seconds!)
ZKFCProtocol proxy = zkfc.getLocalTarget().getZKFCProxy(conf, 5000);
long st = System.currentTimeMillis();
long st = Time.now();
proxy.cedeActive(3000);
long et = System.currentTimeMillis();
long et = Time.now();
assertTrue("RPC to cedeActive took " + (et - st) + " ms",
et - st < 1000);
@ -408,7 +409,7 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
// After the prescribed 3 seconds, should go into STANDBY state,
// since the other node in the cluster would have taken ACTIVE.
cluster.waitForElectorState(0, ActiveStandbyElector.State.STANDBY);
long et2 = System.currentTimeMillis();
long et2 = Time.now();
assertTrue("Should take ~3 seconds to rejoin. Only took " + (et2 - et) +
"ms before rejoining.",
et2 - et > 2800);

View File

@ -21,6 +21,7 @@ import java.util.Random;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.Time;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -61,11 +62,11 @@ public class TestZKFailoverControllerStress extends ClientBaseWithFixes {
@Test(timeout=(STRESS_RUNTIME_SECS + EXTRA_TIMEOUT_SECS) * 1000)
public void testExpireBackAndForth() throws Exception {
cluster.start();
long st = System.currentTimeMillis();
long st = Time.now();
long runFor = STRESS_RUNTIME_SECS * 1000;
int i = 0;
while (System.currentTimeMillis() - st < runFor) {
while (Time.now() - st < runFor) {
// flip flop the services back and forth
int from = i % 2;
int to = (i + 1) % 2;
@ -87,11 +88,11 @@ public class TestZKFailoverControllerStress extends ClientBaseWithFixes {
@Test(timeout=(STRESS_RUNTIME_SECS + EXTRA_TIMEOUT_SECS) * 1000)
public void testRandomExpirations() throws Exception {
cluster.start();
long st = System.currentTimeMillis();
long st = Time.now();
long runFor = STRESS_RUNTIME_SECS * 1000;
Random r = new Random();
while (System.currentTimeMillis() - st < runFor) {
while (Time.now() - st < runFor) {
cluster.getTestContext().checkException();
int targetIdx = r.nextInt(2);
ActiveStandbyElector target = cluster.getElector(targetIdx);
@ -125,8 +126,8 @@ public class TestZKFailoverControllerStress extends ClientBaseWithFixes {
// setting up the mock.
cluster.start();
long st = System.currentTimeMillis();
while (System.currentTimeMillis() - st < runFor) {
long st = Time.now();
while (Time.now() - st < runFor) {
cluster.getTestContext().checkException();
serverFactory.closeAll();
Thread.sleep(50);

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner.Entry;
import org.apache.hadoop.util.Time;
public class TestTFileSeqFileComparison extends TestCase {
MyOptions options;
@ -86,12 +87,12 @@ public class TestTFileSeqFileComparison extends TestCase {
}
public void startTime() throws IOException {
startTimeEpoch = System.currentTimeMillis();
startTimeEpoch = Time.now();
System.out.println(formatTime() + " Started timing.");
}
public void stopTime() throws IOException {
finishTimeEpoch = System.currentTimeMillis();
finishTimeEpoch = Time.now();
System.out.println(formatTime() + " Stopped timing.");
}
@ -111,7 +112,7 @@ public class TestTFileSeqFileComparison extends TestCase {
}
public String formatTime() {
return formatTime(System.currentTimeMillis());
return formatTime(Time.now());
}
private interface KVAppendable {

View File

@ -20,6 +20,8 @@ import java.io.IOException;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import org.apache.hadoop.util.Time;
/**
* this class is a time class to
* measure to measure the time
@ -31,11 +33,11 @@ public class Timer {
private DateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
public void startTime() throws IOException {
startTimeEpoch = System.currentTimeMillis();
startTimeEpoch = Time.now();
}
public void stopTime() throws IOException {
finishTimeEpoch = System.currentTimeMillis();
finishTimeEpoch = Time.now();
}
public long getIntervalMillis() throws IOException {
@ -56,7 +58,7 @@ public class Timer {
}
public String formatCurrentTime() {
return formatTime(System.currentTimeMillis());
return formatTime(Time.now());
}
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.NativeCodeLoader;
import org.apache.hadoop.util.Time;
public class TestNativeIO {
static final Log LOG = LogFactory.getLog(TestNativeIO.class);
@ -88,8 +89,8 @@ public class TestNativeIO {
for (int i = 0; i < 10; i++) {
Thread statter = new Thread() {
public void run() {
long et = System.currentTimeMillis() + 5000;
while (System.currentTimeMillis() < et) {
long et = Time.now() + 5000;
while (Time.now() < et) {
try {
NativeIO.Stat stat = NativeIO.fstat(fos.getFD());
assertEquals(System.getProperty("user.name"), stat.getOwner());

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.security.token.TokenInfo;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
import org.apache.hadoop.security.token.delegation.TestDelegationToken.TestDelegationTokenIdentifier;
import org.apache.hadoop.security.token.delegation.TestDelegationToken.TestDelegationTokenSecretManager;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
@ -186,10 +187,10 @@ public class MiniRPCBenchmark {
throws IOException {
MiniProtocol client = null;
try {
long start = System.currentTimeMillis();
long start = Time.now();
client = (MiniProtocol) RPC.getProxy(MiniProtocol.class,
MiniProtocol.versionID, addr, conf);
long end = System.currentTimeMillis();
long end = Time.now();
return end - start;
} finally {
RPC.stopProxy(client);
@ -231,7 +232,7 @@ public class MiniRPCBenchmark {
final Configuration conf, final InetSocketAddress addr) throws IOException {
MiniProtocol client = null;
try {
long start = System.currentTimeMillis();
long start = Time.now();
try {
client = currentUgi.doAs(new PrivilegedExceptionAction<MiniProtocol>() {
public MiniProtocol run() throws IOException {
@ -242,7 +243,7 @@ public class MiniRPCBenchmark {
} catch (InterruptedException e) {
e.printStackTrace();
}
long end = System.currentTimeMillis();
long end = Time.now();
return end - start;
} finally {
RPC.stopProxy(client);

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.log;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Appender;
import org.apache.log4j.Category;
import org.apache.log4j.Level;
@ -63,7 +64,7 @@ public class TestLog4Json extends TestCase {
new NoRouteToHostException("that box caught fire 3 years ago");
ThrowableInformation ti = new ThrowableInformation(e);
Log4Json l4j = new Log4Json();
long timeStamp = System.currentTimeMillis();
long timeStamp = Time.now();
String outcome = l4j.toJson(new StringWriter(),
"testException",
timeStamp,
@ -82,7 +83,7 @@ public class TestLog4Json extends TestCase {
Exception ioe = new IOException("Datacenter problems", e);
ThrowableInformation ti = new ThrowableInformation(ioe);
Log4Json l4j = new Log4Json();
long timeStamp = System.currentTimeMillis();
long timeStamp = Time.now();
String outcome = l4j.toJson(new StringWriter(),
"testNestedException",
timeStamp,

View File

@ -25,6 +25,7 @@ import javax.naming.NameNotFoundException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.Time;
import org.junit.Test;
import static org.junit.Assert.*;
@ -57,9 +58,9 @@ public class TestDNS {
String hostname1 = DNS.getDefaultHost(DEFAULT);
assertNotNull(hostname1);
String hostname2 = DNS.getDefaultHost(DEFAULT);
long t1 = System.currentTimeMillis();
long t1 = Time.now();
String hostname3 = DNS.getDefaultHost(DEFAULT);
long t2 = System.currentTimeMillis();
long t2 = Time.now();
assertEquals(hostname3, hostname2);
assertEquals(hostname2, hostname1);
long interval = t2 - t1;

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.MultithreadedTestUtil;
import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
import org.apache.hadoop.util.Time;
import org.junit.Test;
import static org.junit.Assert.*;
@ -59,7 +60,7 @@ public class TestSocketIOWithTimeout {
byte buf[] = new byte[4192];
while (true) {
long start = System.currentTimeMillis();
long start = Time.now();
try {
if (in != null) {
in.read(buf);
@ -67,7 +68,7 @@ public class TestSocketIOWithTimeout {
out.write(buf);
}
} catch (SocketTimeoutException e) {
long diff = System.currentTimeMillis() - start;
long diff = Time.now() - start;
LOG.info("Got SocketTimeoutException as expected after " +
diff + " millis : " + e.getMessage());
assertTrue(Math.abs(expectedTimeout - diff) <=

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.DelegationTokenInformation;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.junit.Test;
import static org.junit.Assert.*;
@ -188,7 +189,7 @@ public class TestDelegationToken {
}
}, AccessControlException.class);
long time = dtSecretManager.renewToken(token, "JobTracker");
assertTrue("renew time is in future", time > System.currentTimeMillis());
assertTrue("renew time is in future", time > Time.now());
TestDelegationTokenIdentifier identifier =
new TestDelegationTokenIdentifier();
byte[] tokenId = token.getIdentifier();

View File

@ -31,6 +31,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Layout;
import org.apache.log4j.Logger;
import org.apache.log4j.WriterAppender;
@ -94,7 +95,7 @@ public abstract class GenericTestUtils {
int checkEveryMillis, int waitForMillis)
throws TimeoutException, InterruptedException
{
long st = System.currentTimeMillis();
long st = Time.now();
do {
boolean result = check.get();
if (result) {
@ -102,7 +103,7 @@ public abstract class GenericTestUtils {
}
Thread.sleep(checkEveryMillis);
} while (System.currentTimeMillis() - st < waitForMillis);
} while (Time.now() - st < waitForMillis);
throw new TimeoutException("Timed out waiting for condition");
}

View File

@ -22,6 +22,7 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.Time;
/**
* A utility to easily test threaded/synchronized code.
@ -109,10 +110,10 @@ public abstract class MultithreadedTestUtil {
* have thrown up an error.
*/
public synchronized void waitFor(long millis) throws Exception {
long endTime = System.currentTimeMillis() + millis;
long endTime = Time.now() + millis;
while (shouldRun() &&
finishedThreads.size() < testThreads.size()) {
long left = endTime - System.currentTimeMillis();
long left = endTime - Time.now();
if (left <= 0) break;
checkException();
wait(left);

View File

@ -26,6 +26,7 @@ import org.junit.Test;
import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
import org.apache.hadoop.util.Time;
public class TestMultithreadedTestUtil {
@ -47,9 +48,9 @@ public class TestMultithreadedTestUtil {
}
assertEquals(0, threadsRun.get());
ctx.startThreads();
long st = System.currentTimeMillis();
long st = Time.now();
ctx.waitFor(30000);
long et = System.currentTimeMillis();
long et = Time.now();
// All threads should have run
assertEquals(3, threadsRun.get());
@ -69,7 +70,7 @@ public class TestMultithreadedTestUtil {
}
});
ctx.startThreads();
long st = System.currentTimeMillis();
long st = Time.now();
try {
ctx.waitFor(30000);
fail("waitFor did not throw");
@ -77,7 +78,7 @@ public class TestMultithreadedTestUtil {
// expected
assertEquals(FAIL_MSG, rte.getCause().getMessage());
}
long et = System.currentTimeMillis();
long et = Time.now();
// Test shouldn't have waited the full 30 seconds, since
// the thread throws faster than that
assertTrue("Test took " + (et - st) + "ms",
@ -94,7 +95,7 @@ public class TestMultithreadedTestUtil {
}
});
ctx.startThreads();
long st = System.currentTimeMillis();
long st = Time.now();
try {
ctx.waitFor(30000);
fail("waitFor did not throw");
@ -102,7 +103,7 @@ public class TestMultithreadedTestUtil {
// expected
assertEquals("my ioe", rte.getCause().getMessage());
}
long et = System.currentTimeMillis();
long et = Time.now();
// Test shouldn't have waited the full 30 seconds, since
// the thread throws faster than that
assertTrue("Test took " + (et - st) + "ms",
@ -121,10 +122,10 @@ public class TestMultithreadedTestUtil {
}
});
ctx.startThreads();
long st = System.currentTimeMillis();
long st = Time.now();
ctx.waitFor(3000);
ctx.stop();
long et = System.currentTimeMillis();
long et = Time.now();
long elapsed = et - st;
// Test should have waited just about 3 seconds

View File

@ -55,7 +55,7 @@ public class TestShell extends TestCase {
testInterval(Long.MIN_VALUE / 60000); // test a negative interval
testInterval(0L); // test a zero interval
testInterval(10L); // interval equal to 10mins
testInterval(System.currentTimeMillis() / 60000 + 60); // test a very big interval
testInterval(Time.now() / 60000 + 60); // test a very big interval
}
/**

View File

@ -22,6 +22,7 @@ import org.apache.hadoop.lib.server.BaseService;
import org.apache.hadoop.lib.server.ServiceException;
import org.apache.hadoop.lib.service.Instrumentation;
import org.apache.hadoop.lib.service.Scheduler;
import org.apache.hadoop.util.Time;
import org.json.simple.JSONAware;
import org.json.simple.JSONObject;
import org.json.simple.JSONStreamAware;
@ -164,10 +165,10 @@ public class InstrumentationService extends BaseService implements Instrumentati
throw new IllegalStateException("Cron already used");
}
if (start == 0) {
start = System.currentTimeMillis();
start = Time.now();
lapStart = start;
} else if (lapStart == 0) {
lapStart = System.currentTimeMillis();
lapStart = Time.now();
}
return this;
}
@ -177,7 +178,7 @@ public class InstrumentationService extends BaseService implements Instrumentati
throw new IllegalStateException("Cron already used");
}
if (lapStart > 0) {
own += System.currentTimeMillis() - lapStart;
own += Time.now() - lapStart;
lapStart = 0;
}
return this;
@ -185,7 +186,7 @@ public class InstrumentationService extends BaseService implements Instrumentati
void end() {
stop();
total = System.currentTimeMillis() - start;
total = Time.now() - start;
}
}

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.lib.server.ServiceException;
import org.apache.hadoop.lib.service.Instrumentation;
import org.apache.hadoop.lib.service.Scheduler;
import org.apache.hadoop.lib.util.Check;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -59,11 +60,11 @@ public class SchedulerService extends BaseService implements Scheduler {
@Override
public void destroy() {
try {
long limit = System.currentTimeMillis() + 30 * 1000;
long limit = Time.now() + 30 * 1000;
scheduler.shutdownNow();
while (!scheduler.awaitTermination(1000, TimeUnit.MILLISECONDS)) {
LOG.debug("Waiting for scheduler to shutdown");
if (System.currentTimeMillis() > limit) {
if (Time.now() > limit) {
LOG.warn("Gave up waiting for scheduler to shutdown");
break;
}

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.test.HTestCase;
import org.apache.hadoop.test.TestDir;
import org.apache.hadoop.test.TestDirHelper;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.json.simple.JSONObject;
import org.json.simple.parser.JSONParser;
import org.junit.Test;
@ -50,26 +51,26 @@ public class TestInstrumentationService extends HTestCase {
Assert.assertEquals(cron.lapStart, 0);
Assert.assertEquals(cron.own, 0);
Assert.assertEquals(cron.total, 0);
long begin = System.currentTimeMillis();
long begin = Time.now();
Assert.assertEquals(cron.start(), cron);
Assert.assertEquals(cron.start(), cron);
Assert.assertEquals(cron.start, begin, 20);
Assert.assertEquals(cron.start, cron.lapStart);
sleep(100);
Assert.assertEquals(cron.stop(), cron);
long end = System.currentTimeMillis();
long end = Time.now();
long delta = end - begin;
Assert.assertEquals(cron.own, delta, 20);
Assert.assertEquals(cron.total, 0);
Assert.assertEquals(cron.lapStart, 0);
sleep(100);
long reStart = System.currentTimeMillis();
long reStart = Time.now();
cron.start();
Assert.assertEquals(cron.start, begin, 20);
Assert.assertEquals(cron.lapStart, reStart, 20);
sleep(100);
cron.stop();
long reEnd = System.currentTimeMillis();
long reEnd = Time.now();
delta += reEnd - reStart;
Assert.assertEquals(cron.own, delta, 20);
Assert.assertEquals(cron.total, 0);
@ -109,22 +110,22 @@ public class TestInstrumentationService extends HTestCase {
long avgOwn;
cron.start();
ownStart = System.currentTimeMillis();
ownStart = Time.now();
totalStart = ownStart;
ownDelta = 0;
sleep(100);
cron.stop();
ownEnd = System.currentTimeMillis();
ownEnd = Time.now();
ownDelta += ownEnd - ownStart;
sleep(100);
cron.start();
ownStart = System.currentTimeMillis();
ownStart = Time.now();
sleep(100);
cron.stop();
ownEnd = System.currentTimeMillis();
ownEnd = Time.now();
ownDelta += ownEnd - ownStart;
totalEnd = ownEnd;
totalDelta = totalEnd - totalStart;
@ -142,22 +143,22 @@ public class TestInstrumentationService extends HTestCase {
cron = new InstrumentationService.Cron();
cron.start();
ownStart = System.currentTimeMillis();
ownStart = Time.now();
totalStart = ownStart;
ownDelta = 0;
sleep(200);
cron.stop();
ownEnd = System.currentTimeMillis();
ownEnd = Time.now();
ownDelta += ownEnd - ownStart;
sleep(200);
cron.start();
ownStart = System.currentTimeMillis();
ownStart = Time.now();
sleep(200);
cron.stop();
ownEnd = System.currentTimeMillis();
ownEnd = Time.now();
ownDelta += ownEnd - ownStart;
totalEnd = ownEnd;
totalDelta = totalEnd - totalStart;
@ -178,22 +179,22 @@ public class TestInstrumentationService extends HTestCase {
cron = new InstrumentationService.Cron();
cron.start();
ownStart = System.currentTimeMillis();
ownStart = Time.now();
totalStart = ownStart;
ownDelta = 0;
sleep(300);
cron.stop();
ownEnd = System.currentTimeMillis();
ownEnd = Time.now();
ownDelta += ownEnd - ownStart;
sleep(300);
cron.start();
ownStart = System.currentTimeMillis();
ownStart = Time.now();
sleep(300);
cron.stop();
ownEnd = System.currentTimeMillis();
ownEnd = Time.now();
ownDelta += ownEnd - ownStart;
totalEnd = ownEnd;
totalDelta = totalEnd - totalStart;

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.test;
import junit.framework.Assert;
import org.apache.hadoop.util.Time;
import org.junit.Rule;
import org.junit.rules.MethodRule;
@ -142,18 +144,18 @@ public abstract class HTestCase {
* to <code>true</code>.
*/
protected long waitFor(int timeout, boolean failIfTimeout, Predicate predicate) {
long started = System.currentTimeMillis();
long mustEnd = System.currentTimeMillis() + (long) (getWaitForRatio() * timeout);
long started = Time.now();
long mustEnd = Time.now() + (long) (getWaitForRatio() * timeout);
long lastEcho = 0;
try {
long waiting = mustEnd - System.currentTimeMillis();
long waiting = mustEnd - Time.now();
System.out.println(MessageFormat.format("Waiting up to [{0}] msec", waiting));
boolean eval;
while (!(eval = predicate.evaluate()) && System.currentTimeMillis() < mustEnd) {
if ((System.currentTimeMillis() - lastEcho) > 5000) {
waiting = mustEnd - System.currentTimeMillis();
while (!(eval = predicate.evaluate()) && Time.now() < mustEnd) {
if ((Time.now() - lastEcho) > 5000) {
waiting = mustEnd - Time.now();
System.out.println(MessageFormat.format("Waiting up to [{0}] msec", waiting));
lastEcho = System.currentTimeMillis();
lastEcho = Time.now();
}
Thread.sleep(100);
}
@ -164,7 +166,7 @@ public abstract class HTestCase {
System.out.println(MessageFormat.format("Waiting timed out after [{0}] msec", timeout));
}
}
return (eval) ? System.currentTimeMillis() - started : -1;
return (eval) ? Time.now() - started : -1;
} catch (Exception ex) {
throw new RuntimeException(ex);
}

View File

@ -22,6 +22,7 @@ import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.Time;
import org.junit.Test;
import org.mortbay.jetty.Server;
import org.mortbay.jetty.servlet.Context;
@ -73,13 +74,13 @@ public class TestHFSTestCase extends HFSTestCase {
@Test
public void waitFor() {
long start = System.currentTimeMillis();
long start = Time.now();
long waited = waitFor(1000, new Predicate() {
public boolean evaluate() throws Exception {
return true;
}
});
long end = System.currentTimeMillis();
long end = Time.now();
Assert.assertEquals(waited, 0, 50);
Assert.assertEquals(end - start - waited, 0, 50);
}
@ -87,13 +88,13 @@ public class TestHFSTestCase extends HFSTestCase {
@Test
public void waitForTimeOutRatio1() {
setWaitForRatio(1);
long start = System.currentTimeMillis();
long start = Time.now();
long waited = waitFor(200, new Predicate() {
public boolean evaluate() throws Exception {
return false;
}
});
long end = System.currentTimeMillis();
long end = Time.now();
Assert.assertEquals(waited, -1);
Assert.assertEquals(end - start, 200, 50);
}
@ -101,13 +102,13 @@ public class TestHFSTestCase extends HFSTestCase {
@Test
public void waitForTimeOutRatio2() {
setWaitForRatio(2);
long start = System.currentTimeMillis();
long start = Time.now();
long waited = waitFor(200, new Predicate() {
public boolean evaluate() throws Exception {
return false;
}
});
long end = System.currentTimeMillis();
long end = Time.now();
Assert.assertEquals(waited, -1);
Assert.assertEquals(end - start, 200 * getWaitForRatio(), 50 * getWaitForRatio());
}
@ -115,18 +116,18 @@ public class TestHFSTestCase extends HFSTestCase {
@Test
public void sleepRatio1() {
setWaitForRatio(1);
long start = System.currentTimeMillis();
long start = Time.now();
sleep(100);
long end = System.currentTimeMillis();
long end = Time.now();
Assert.assertEquals(end - start, 100, 50);
}
@Test
public void sleepRatio2() {
setWaitForRatio(1);
long start = System.currentTimeMillis();
long start = Time.now();
sleep(100);
long end = System.currentTimeMillis();
long end = Time.now();
Assert.assertEquals(end - start, 100 * getWaitForRatio(), 50 * getWaitForRatio());
}

View File

@ -19,6 +19,8 @@
package org.apache.hadoop.test;
import junit.framework.Assert;
import org.apache.hadoop.util.Time;
import org.junit.Test;
import org.mortbay.jetty.Server;
import org.mortbay.jetty.servlet.Context;
@ -58,13 +60,13 @@ public class TestHTestCase extends HTestCase {
@Test
public void waitFor() {
long start = System.currentTimeMillis();
long start = Time.now();
long waited = waitFor(1000, new Predicate() {
public boolean evaluate() throws Exception {
return true;
}
});
long end = System.currentTimeMillis();
long end = Time.now();
Assert.assertEquals(waited, 0, 50);
Assert.assertEquals(end - start - waited, 0, 50);
}
@ -72,13 +74,13 @@ public class TestHTestCase extends HTestCase {
@Test
public void waitForTimeOutRatio1() {
setWaitForRatio(1);
long start = System.currentTimeMillis();
long start = Time.now();
long waited = waitFor(200, new Predicate() {
public boolean evaluate() throws Exception {
return false;
}
});
long end = System.currentTimeMillis();
long end = Time.now();
Assert.assertEquals(waited, -1);
Assert.assertEquals(end - start, 200, 50);
}
@ -86,13 +88,13 @@ public class TestHTestCase extends HTestCase {
@Test
public void waitForTimeOutRatio2() {
setWaitForRatio(2);
long start = System.currentTimeMillis();
long start = Time.now();
long waited = waitFor(200, new Predicate() {
public boolean evaluate() throws Exception {
return false;
}
});
long end = System.currentTimeMillis();
long end = Time.now();
Assert.assertEquals(waited, -1);
Assert.assertEquals(end - start, 200 * getWaitForRatio(), 50 * getWaitForRatio());
}
@ -100,18 +102,18 @@ public class TestHTestCase extends HTestCase {
@Test
public void sleepRatio1() {
setWaitForRatio(1);
long start = System.currentTimeMillis();
long start = Time.now();
sleep(100);
long end = System.currentTimeMillis();
long end = Time.now();
Assert.assertEquals(end - start, 100, 50);
}
@Test
public void sleepRatio2() {
setWaitForRatio(1);
long start = System.currentTimeMillis();
long start = Time.now();
sleep(100);
long end = System.currentTimeMillis();
long end = Time.now();
Assert.assertEquals(end - start, 100 * getWaitForRatio(), 50 * getWaitForRatio());
}

View File

@ -125,6 +125,9 @@ Release 2.0.1-alpha - UNRELEASED
HDFS-3582. Hook daemon process exit for testing. (eli)
HDFS-3641. Move server Util time methods to common and use now
instead of System#currentTimeMillis. (eli)
OPTIMIZATIONS
HDFS-2982. Startup performance suffers when there are many edit log

View File

@ -139,6 +139,7 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
@ -520,7 +521,7 @@ public class DFSClient implements java.io.Closeable {
if (filesBeingWritten.isEmpty()) {
return;
}
lastLeaseRenewal = System.currentTimeMillis();
lastLeaseRenewal = Time.now();
}
}
@ -537,7 +538,7 @@ public class DFSClient implements java.io.Closeable {
return true;
} catch (IOException e) {
// Abort if the lease has already expired.
final long elapsed = System.currentTimeMillis() - getLastLeaseRenewal();
final long elapsed = Time.now() - getLastLeaseRenewal();
if (elapsed > HdfsConstants.LEASE_SOFTLIMIT_PERIOD) {
LOG.warn("Failed to renew lease for " + clientName + " for "
+ (elapsed/1000) + " seconds (>= soft-limit ="
@ -635,7 +636,7 @@ public class DFSClient implements java.io.Closeable {
* @see ClientProtocol#getServerDefaults()
*/
public FsServerDefaults getServerDefaults() throws IOException {
long now = System.currentTimeMillis();
long now = Time.now();
if (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD) {
serverDefaults = namenode.getServerDefaults();
serverDefaultsLastUpdate = now;

View File

@ -74,6 +74,7 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time;
/****************************************************************
@ -396,7 +397,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
* and closes them. Any error recovery is also done by this thread.
*/
public void run() {
long lastPacket = System.currentTimeMillis();
long lastPacket = Time.now();
while (!streamerClosed && dfsClient.clientRunning) {
// if the Responder encountered an error, shutdown Responder
@ -420,7 +421,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
synchronized (dataQueue) {
// wait for a packet to be sent.
long now = System.currentTimeMillis();
long now = Time.now();
while ((!streamerClosed && !hasError && dfsClient.clientRunning
&& dataQueue.size() == 0 &&
(stage != BlockConstructionStage.DATA_STREAMING ||
@ -435,7 +436,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
} catch (InterruptedException e) {
}
doSleep = false;
now = System.currentTimeMillis();
now = Time.now();
}
if (streamerClosed || hasError || !dfsClient.clientRunning) {
continue;
@ -518,7 +519,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
errorIndex = 0;
throw e;
}
lastPacket = System.currentTimeMillis();
lastPacket = Time.now();
if (one.isHeartbeatPacket()) { //heartbeat packet
}
@ -981,7 +982,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
errorIndex = -1;
success = false;
long startTime = System.currentTimeMillis();
long startTime = Time.now();
DatanodeInfo[] excluded = excludedNodes.toArray(
new DatanodeInfo[excludedNodes.size()]);
block = oldBlock;
@ -1107,7 +1108,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
long sleeptime = 400;
while (true) {
long localstart = System.currentTimeMillis();
long localstart = Time.now();
while (true) {
try {
return dfsClient.namenode.addBlock(src, dfsClient.clientName, block, excludedNodes);
@ -1130,9 +1131,9 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
} else {
--retries;
DFSClient.LOG.info("Exception while adding a block", e);
if (System.currentTimeMillis() - localstart > 5000) {
if (Time.now() - localstart > 5000) {
DFSClient.LOG.info("Waiting for replication for "
+ (System.currentTimeMillis() - localstart) / 1000
+ (Time.now() - localstart) / 1000
+ " seconds");
}
try {
@ -1727,14 +1728,14 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
// should be called holding (this) lock since setTestFilename() may
// be called during unit tests
private void completeFile(ExtendedBlock last) throws IOException {
long localstart = System.currentTimeMillis();
long localstart = Time.now();
boolean fileComplete = false;
while (!fileComplete) {
fileComplete = dfsClient.namenode.complete(src, dfsClient.clientName, last);
if (!fileComplete) {
if (!dfsClient.clientRunning ||
(dfsClient.hdfsTimeout > 0 &&
localstart + dfsClient.hdfsTimeout < System.currentTimeMillis())) {
localstart + dfsClient.hdfsTimeout < Time.now())) {
String msg = "Unable to close file because dfsclient " +
" was unable to contact the HDFS servers." +
" clientRunning " + dfsClient.clientRunning +
@ -1744,7 +1745,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
}
try {
Thread.sleep(400);
if (System.currentTimeMillis() - localstart > 5000) {
if (Time.now() - localstart > 5000) {
DFSClient.LOG.info("Could not complete file " + src + " retrying...");
}
} catch (InterruptedException ie) {

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.web.URLUtils;
import org.apache.hadoop.util.Time;
/**
* An implementation of a protocol for accessing filesystems over HTTPS. The
@ -164,8 +165,7 @@ public class HsftpFileSystem extends HftpFileSystem {
final int warnDays = ExpWarnDays;
if (warnDays > 0) { // make sure only check once
ExpWarnDays = 0;
long expTimeThreshold = warnDays * MM_SECONDS_PER_DAY
+ System.currentTimeMillis();
long expTimeThreshold = warnDays * MM_SECONDS_PER_DAY + Time.now();
X509Certificate[] clientCerts = (X509Certificate[]) conn
.getLocalCertificates();
if (clientCerts != null) {
@ -175,7 +175,7 @@ public class HsftpFileSystem extends HftpFileSystem {
StringBuilder sb = new StringBuilder();
sb.append("\n Client certificate "
+ cert.getSubjectX500Principal().getName());
int dayOffSet = (int) ((expTime - System.currentTimeMillis()) / MM_SECONDS_PER_DAY);
int dayOffSet = (int) ((expTime - Time.now()) / MM_SECONDS_PER_DAY);
sb.append(" have " + dayOffSet + " days to expire");
LOG.warn(sb.toString());
}

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
/**
* <p>
@ -279,7 +280,7 @@ class LeaseRenewer {
/** Is the empty period longer than the grace period? */
private synchronized boolean isRenewerExpired() {
return emptyTime != Long.MAX_VALUE
&& System.currentTimeMillis() - emptyTime > gracePeriod;
&& Time.now() - emptyTime > gracePeriod;
}
synchronized void put(final String src, final DFSOutputStream out,
@ -339,7 +340,7 @@ class LeaseRenewer {
}
}
//discover the first time that all file-being-written maps are empty.
emptyTime = System.currentTimeMillis();
emptyTime = Time.now();
}
}
}
@ -354,7 +355,7 @@ class LeaseRenewer {
}
if (emptyTime == Long.MAX_VALUE) {
//discover the first time that the client list is empty.
emptyTime = System.currentTimeMillis();
emptyTime = Time.now();
}
}
@ -427,10 +428,10 @@ class LeaseRenewer {
* when the lease period is half over.
*/
private void run(final int id) throws InterruptedException {
for(long lastRenewed = System.currentTimeMillis();
for(long lastRenewed = Time.now();
clientsRunning() && !Thread.interrupted();
Thread.sleep(getSleepPeriod())) {
final long elapsed = System.currentTimeMillis() - lastRenewed;
final long elapsed = Time.now() - lastRenewed;
if (elapsed >= getRenewalTime()) {
try {
renew();
@ -438,7 +439,7 @@ class LeaseRenewer {
LOG.debug("Lease renewer daemon for " + clientsString()
+ " with renew id " + id + " executed");
}
lastRenewed = System.currentTimeMillis();
lastRenewed = Time.now();
} catch (SocketTimeoutException ie) {
LOG.warn("Failed to renew lease for " + clientsString() + " for "
+ (elapsed/1000) + " seconds. Aborting ...", ie);

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@ -137,10 +138,10 @@ public class BlockTokenSecretManager extends
* more.
*/
setSerialNo(serialNo + 1);
currentKey = new BlockKey(serialNo, System.currentTimeMillis() + 2
currentKey = new BlockKey(serialNo, Time.now() + 2
* keyUpdateInterval + tokenLifetime, generateSecret());
setSerialNo(serialNo + 1);
nextKey = new BlockKey(serialNo, System.currentTimeMillis() + 3
nextKey = new BlockKey(serialNo, Time.now() + 3
* keyUpdateInterval + tokenLifetime, generateSecret());
allKeys.put(currentKey.getKeyId(), currentKey);
allKeys.put(nextKey.getKeyId(), nextKey);
@ -157,7 +158,7 @@ public class BlockTokenSecretManager extends
}
private synchronized void removeExpiredKeys() {
long now = System.currentTimeMillis();
long now = Time.now();
for (Iterator<Map.Entry<Integer, BlockKey>> it = allKeys.entrySet()
.iterator(); it.hasNext();) {
Map.Entry<Integer, BlockKey> e = it.next();
@ -207,15 +208,15 @@ public class BlockTokenSecretManager extends
removeExpiredKeys();
// set final expiry date of retiring currentKey
allKeys.put(currentKey.getKeyId(), new BlockKey(currentKey.getKeyId(),
System.currentTimeMillis() + keyUpdateInterval + tokenLifetime,
Time.now() + keyUpdateInterval + tokenLifetime,
currentKey.getKey()));
// update the estimated expiry date of new currentKey
currentKey = new BlockKey(nextKey.getKeyId(), System.currentTimeMillis()
currentKey = new BlockKey(nextKey.getKeyId(), Time.now()
+ 2 * keyUpdateInterval + tokenLifetime, nextKey.getKey());
allKeys.put(currentKey.getKeyId(), currentKey);
// generate a new nextKey
setSerialNo(serialNo + 1);
nextKey = new BlockKey(serialNo, System.currentTimeMillis() + 3
nextKey = new BlockKey(serialNo, Time.now() + 3
* keyUpdateInterval + tokenLifetime, generateSecret());
allKeys.put(nextKey.getKeyId(), nextKey);
return true;
@ -290,7 +291,7 @@ public class BlockTokenSecretManager extends
}
private static boolean isExpired(long expiryDate) {
return System.currentTimeMillis() > expiryDate;
return Time.now() > expiryDate;
}
/**
@ -335,7 +336,7 @@ public class BlockTokenSecretManager extends
}
if (key == null)
throw new IllegalStateException("currentKey hasn't been initialized.");
identifier.setExpiryDate(System.currentTimeMillis() + tokenLifetime);
identifier.setExpiryDate(Time.now() + tokenLifetime);
identifier.setKeyId(key.getKeyId());
if (LOG.isDebugEnabled()) {
LOG.debug("Generating block token for " + identifier.toString());

View File

@ -72,6 +72,7 @@ import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@ -709,7 +710,7 @@ public class Balancer {
*/
private static final long MAX_ITERATION_TIME = 20*60*1000L; //20 mins
private void dispatchBlocks() {
long startTime = Util.now();
long startTime = Time.now();
this.blocksToReceive = 2*scheduledSize;
boolean isTimeUp = false;
while(!isTimeUp && scheduledSize>0 &&
@ -738,7 +739,7 @@ public class Balancer {
}
// check if time is up or not
if (Util.now()-startTime > MAX_ITERATION_TIME) {
if (Time.now()-startTime > MAX_ITERATION_TIME) {
isTimeUp = true;
continue;
}
@ -1143,7 +1144,7 @@ public class Balancer {
* move blocks in current window to old window.
*/
private static class MovedBlocks {
private long lastCleanupTime = System.currentTimeMillis();
private long lastCleanupTime = Time.now();
final private static int CUR_WIN = 0;
final private static int OLD_WIN = 1;
final private static int NUM_WINS = 2;
@ -1174,7 +1175,7 @@ public class Balancer {
/* remove old blocks */
synchronized private void cleanup() {
long curTime = System.currentTimeMillis();
long curTime = Time.now();
// check if old win is older than winWidth
if (lastCleanupTime + WIN_WIDTH <= curTime) {
// purge the old window
@ -1471,7 +1472,7 @@ public class Balancer {
/** Parse arguments and then run Balancer */
@Override
public int run(String[] args) {
final long startTime = Util.now();
final long startTime = Time.now();
final Configuration conf = getConf();
WIN_WIDTH = conf.getLong(
DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY,
@ -1489,7 +1490,7 @@ public class Balancer {
System.out.println(e + ". Exiting ...");
return ReturnStatus.INTERRUPTED.code;
} finally {
System.out.println("Balancing took " + time2Str(Util.now()-startTime));
System.out.println("Balancing took " + time2Str(Time.now()-startTime));
}
}

View File

@ -72,6 +72,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@ -1457,7 +1458,7 @@ public class BlockManager {
public void processReport(final DatanodeID nodeID, final String poolId,
final BlockListAsLongs newReport) throws IOException {
namesystem.writeLock();
final long startTime = Util.now(); //after acquiring write lock
final long startTime = Time.now(); //after acquiring write lock
final long endTime;
try {
final DatanodeDescriptor node = datanodeManager.getDatanode(nodeID);
@ -1496,7 +1497,7 @@ public class BlockManager {
}
} finally {
endTime = Util.now();
endTime = Time.now();
namesystem.writeUnlock();
}

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import java.util.ArrayList;
import java.util.Collection;

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.util.LightWeightHashSet;
import org.apache.hadoop.util.Time;
/**
* This class extends the DatanodeInfo class with ephemeral information (eg
@ -310,7 +311,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
this.dfsUsed = dfsUsed;
this.remaining = remaining;
this.blockPoolUsed = blockPoolUsed;
this.lastUpdate = System.currentTimeMillis();
this.lastUpdate = Time.now();
this.xceiverCount = xceiverCount;
this.volumeFailures = volFailures;
this.heartbeatedSinceFailover = true;

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import java.io.IOException;
import java.io.PrintWriter;
@ -70,6 +70,7 @@ import org.apache.hadoop.net.ScriptBasedMapping;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.HostsFileReader;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.net.InetAddresses;
@ -320,7 +321,7 @@ public class DatanodeManager {
/** Is the datanode dead? */
boolean isDatanodeDead(DatanodeDescriptor node) {
return (node.getLastUpdate() <
(Util.now() - heartbeatExpireInterval));
(Time.now() - heartbeatExpireInterval));
}
/** Add a datanode. */

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
/**
* Manage the heartbeats received from datanodes.
@ -247,7 +248,7 @@ class HeartbeatManager implements DatanodeStatistics {
public void run() {
while(namesystem.isRunning()) {
try {
final long now = Util.now();
final long now = Time.now();
if (lastHeartbeatCheck + heartbeatRecheckInterval < now) {
heartbeatCheck();
lastHeartbeatCheck = now;

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import java.io.PrintWriter;
import java.sql.Time;

View File

@ -33,29 +33,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
public final class Util {
private final static Log LOG = LogFactory.getLog(Util.class.getName());
/**
* Current system time. Do not use this to calculate a duration or interval
* to sleep, because it will be broken by settimeofday. Instead, use
* monotonicNow.
* @return current time in msec.
*/
public static long now() {
return System.currentTimeMillis();
}
/**
* Current time from some arbitrary time base in the past, counting in
* milliseconds, and not affected by settimeofday or similar system clock
* changes. This is appropriate to use when computing how much longer to
* wait for an interval to expire.
* @return a monotonic clock that counts in milliseconds.
*/
public static long monotonicNow() {
final long NANOSECONDS_PER_MILLISECOND = 1000000;
return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
}
/**
* Interprets the passed string as a URI. In case of error it
* assumes the specified string is a file.

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs.server.datanode;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import java.io.IOException;
import java.net.InetSocketAddress;
@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.VersionInfo;
import org.apache.hadoop.util.VersionUtil;
@ -226,7 +227,7 @@ class BPServiceActor implements Runnable {
*/
void scheduleBlockReport(long delay) {
if (delay > 0) { // send BR after random delay
lastBlockReport = System.currentTimeMillis()
lastBlockReport = Time.now()
- ( dnConf.blockReportInterval - DFSUtil.getRandom().nextInt((int)(delay)));
} else { // send at next heartbeat
lastBlockReport = lastHeartbeat - dnConf.blockReportInterval;
@ -561,7 +562,7 @@ class BPServiceActor implements Runnable {
// or work arrives, and then iterate again.
//
long waitTime = dnConf.heartBeatInterval -
(System.currentTimeMillis() - lastHeartbeat);
(Time.now() - lastHeartbeat);
synchronized(pendingIncrementalBR) {
if (waitTime > 0 && pendingReceivedRequests == 0) {
try {

View File

@ -49,6 +49,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Time;
/**
* Scans the block files under a block pool and verifies that the
@ -90,7 +91,7 @@ class BlockPoolSliceScanner {
private long totalTransientErrors = 0;
private final AtomicInteger totalBlocksScannedInLastRun = new AtomicInteger(); // Used for test only
private long currentPeriodStart = System.currentTimeMillis();
private long currentPeriodStart = Time.now();
private long bytesLeft = 0; // Bytes to scan in this period
private long totalBytesToScan = 0;
@ -224,7 +225,7 @@ class BlockPoolSliceScanner {
long period = Math.min(scanPeriod,
Math.max(blockMap.size(),1) * 600 * 1000L);
int periodInt = Math.abs((int)period);
return System.currentTimeMillis() - scanPeriod +
return Time.now() - scanPeriod +
DFSUtil.getRandom().nextInt(periodInt);
}
@ -281,7 +282,7 @@ class BlockPoolSliceScanner {
info = new BlockScanInfo(block);
}
long now = System.currentTimeMillis();
long now = Time.now();
info.lastScanType = type;
info.lastScanTime = now;
info.lastScanOk = scanOk;
@ -358,7 +359,7 @@ class BlockPoolSliceScanner {
}
private synchronized void adjustThrottler() {
long timeLeft = currentPeriodStart+scanPeriod - System.currentTimeMillis();
long timeLeft = currentPeriodStart+scanPeriod - Time.now();
long bw = Math.max(bytesLeft*1000/timeLeft, MIN_SCAN_RATE);
throttler.setBandwidth(Math.min(bw, MAX_SCAN_RATE));
}
@ -481,7 +482,7 @@ class BlockPoolSliceScanner {
private boolean assignInitialVerificationTimes() {
//First updates the last verification times from the log file.
if (verificationLog != null) {
long now = System.currentTimeMillis();
long now = Time.now();
RollingLogs.LineIterator logIterator = null;
try {
logIterator = verificationLog.logs.iterator(false);
@ -529,7 +530,7 @@ class BlockPoolSliceScanner {
// Initially spread the block reads over half of scan period
// so that we don't keep scanning the blocks too quickly when restarted.
long verifyInterval = Math.min(scanPeriod/(2L * numBlocks), 10*60*1000L);
long lastScanTime = System.currentTimeMillis() - scanPeriod;
long lastScanTime = Time.now() - scanPeriod;
if (!blockInfoSet.isEmpty()) {
BlockScanInfo info;
@ -556,7 +557,7 @@ class BlockPoolSliceScanner {
// reset the byte counts :
bytesLeft = totalBytesToScan;
currentPeriodStart = System.currentTimeMillis();
currentPeriodStart = Time.now();
}
void scanBlockPoolSlice() {
@ -571,7 +572,7 @@ class BlockPoolSliceScanner {
scan();
} finally {
totalBlocksScannedInLastRun.set(processedBlocks.size());
lastScanTime.set(System.currentTimeMillis());
lastScanTime.set(Time.now());
}
}
@ -584,7 +585,7 @@ class BlockPoolSliceScanner {
while (datanode.shouldRun && !Thread.interrupted()
&& datanode.isBPServiceAlive(blockPoolId)) {
long now = System.currentTimeMillis();
long now = Time.now();
synchronized (this) {
if ( now >= (currentPeriodStart + scanPeriod)) {
startNewPeriod();
@ -642,7 +643,7 @@ class BlockPoolSliceScanner {
int total = blockInfoSet.size();
long now = System.currentTimeMillis();
long now = Time.now();
Date date = new Date();

View File

@ -165,6 +165,7 @@ import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.ServicePlugin;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.VersionInfo;
import org.mortbay.util.ajax.JSON;
@ -972,7 +973,7 @@ public class DataNode extends Configured
int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
return "DS-" + rand + "-" + ip + "-" + port + "-"
+ System.currentTimeMillis();
+ Time.now();
}
/** Ensure the authentication method is kerberos */

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_ACCESS_TOKEN;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
import java.io.BufferedInputStream;

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
/**
* Periodically scans the data directories for block and block metadata files.
@ -239,7 +240,7 @@ public class DirectoryScanner implements Runnable {
void start() {
shouldRun = true;
long offset = DFSUtil.getRandom().nextInt((int) (scanPeriodMsecs/1000L)) * 1000L; //msec
long firstScanTime = System.currentTimeMillis() + offset;
long firstScanTime = Time.now() + offset;
LOG.info("Periodic Directory Tree Verification scan starting at "
+ firstScanTime + " with interval " + scanPeriodMsecs);
masterThread.scheduleAtFixedRate(this, offset, scanPeriodMsecs,

View File

@ -78,6 +78,7 @@ import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Time;
/**************************************************
* FSDataset manages a set of data blocks. Each block
@ -1142,7 +1143,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
}
// Otherwise remove blocks for the failed volumes
long mlsec = System.currentTimeMillis();
long mlsec = Time.now();
synchronized (this) {
for (FsVolumeImpl fv: failedVols) {
for (String bpid : fv.getBlockPoolList()) {
@ -1161,7 +1162,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
}
}
} // end of sync
mlsec = System.currentTimeMillis() - mlsec;
mlsec = Time.now() - mlsec;
LOG.warn("Removed " + removedBlocks + " out of " + totalBlocks +
"(took " + mlsec + " millisecs)");

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import java.io.File;
import java.io.IOException;

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.io.Closeable;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import java.io.Closeable;
import java.io.FileNotFoundException;

View File

@ -17,7 +17,8 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import java.net.URI;
import java.io.IOException;
import java.util.ArrayList;

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import java.io.File;
import java.io.FilterInputStream;

View File

@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.Storage.StorageState;
import org.apache.hadoop.hdfs.server.common.Util;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.util.Canceler;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil;
@ -660,7 +661,7 @@ public class FSImage implements Closeable {
final long checkpointTxnCount = conf.getLong(
DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY,
DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);
long checkpointAge = System.currentTimeMillis() - imageFile.lastModified();
long checkpointAge = Time.now() - imageFile.lastModified();
return (checkpointAge > checkpointPeriod * 1000) ||
(numEditsLoaded > checkpointTxnCount);
@ -757,7 +758,7 @@ public class FSImage implements Closeable {
saver.save(newFile, compression);
MD5FileUtils.saveMD5File(dstFile, saver.getSavedDigest());
storage.setMostRecentCheckpointInfo(txid, Util.now());
storage.setMostRecentCheckpointInfo(txid, Time.now());
}
/**
@ -1072,7 +1073,7 @@ public class FSImage implements Closeable {
// advertise it as such to other checkpointers
// from now on
if (txid > storage.getMostRecentCheckpointTxId()) {
storage.setMostRecentCheckpointInfo(txid, Util.now());
storage.setMostRecentCheckpointInfo(txid, Time.now());
}
}

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import java.io.DataInputStream;
import java.io.DataOutputStream;

View File

@ -62,7 +62,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SUPPORT_APPEND_KEY;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import java.io.BufferedWriter;
import java.io.ByteArrayInputStream;
@ -190,6 +190,7 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.VersionInfo;
import org.mortbay.util.ajax.JSON;
@ -5396,7 +5397,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
private long getLastContact(DatanodeDescriptor alivenode) {
return (System.currentTimeMillis() - alivenode.getLastUpdate())/1000;
return (Time.now() - alivenode.getLastUpdate())/1000;
}
private long getDfsUsed(DatanodeDescriptor alivenode) {

View File

@ -39,7 +39,7 @@ import org.apache.hadoop.util.Daemon;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
/**
* LeaseManager does the lease housekeeping for writing on files.

View File

@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.util.AtomicFileOutputStream;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.DNS;
import org.apache.hadoop.util.Time;
import com.google.common.base.Preconditions;
import com.google.common.annotations.VisibleForTesting;
@ -1023,7 +1024,7 @@ public class NNStorage extends Storage implements Closeable {
}
int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
String bpid = "BP-" + rand + "-"+ ip + "-" + System.currentTimeMillis();
String bpid = "BP-" + rand + "-"+ ip + "-" + Time.now();
return bpid;
}

View File

@ -55,6 +55,7 @@ import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
@ -182,7 +183,7 @@ public class NamenodeFsck {
* Check files on DFS, starting from the indicated path.
*/
public void fsck() {
final long startTime = System.currentTimeMillis();
final long startTime = Time.now();
try {
String msg = "FSCK started by " + UserGroupInformation.getCurrentUser()
+ " from " + remoteAddress + " for path " + path + " at " + new Date();
@ -207,7 +208,7 @@ public class NamenodeFsck {
out.println(" Number of racks:\t\t" + networktopology.getNumOfRacks());
out.println("FSCK ended at " + new Date() + " in "
+ (System.currentTimeMillis() - startTime + " milliseconds"));
+ (Time.now() - startTime + " milliseconds"));
// If there were internal errors during the fsck operation, we want to
// return FAILURE_STATUS, even if those errors were not immediately
@ -233,7 +234,7 @@ public class NamenodeFsck {
String errMsg = "Fsck on path '" + path + "' " + FAILURE_STATUS;
LOG.warn(errMsg, e);
out.println("FSCK ended at " + new Date() + " in "
+ (System.currentTimeMillis() - startTime + " milliseconds"));
+ (Time.now() - startTime + " milliseconds"));
out.println(e.getMessage());
out.print("\n\n" + errMsg);
} finally {

View File

@ -58,6 +58,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.ServletUtil;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.VersionInfo;
import org.znerd.xmlenc.XMLOutputter;
@ -487,7 +488,7 @@ class NamenodeJspHelper {
long decommRequestTime = d.decommissioningStatus.getStartTime();
long timestamp = d.getLastUpdate();
long currentTime = System.currentTimeMillis();
long currentTime = Time.now();
long hoursSinceDecommStarted = (currentTime - decommRequestTime)/3600000;
long remainderMinutes = ((currentTime - decommRequestTime)/60000) % 60;
out.print("<td class=\"lastcontact\"> "
@ -534,7 +535,7 @@ class NamenodeJspHelper {
String adminState = d.getAdminState().toString();
long timestamp = d.getLastUpdate();
long currentTime = System.currentTimeMillis();
long currentTime = Time.now();
long bpUsed = d.getBlockPoolUsed();
String percentBpUsed = StringUtils.limitDecimalTo2(d

View File

@ -73,6 +73,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
@ -99,7 +100,7 @@ public class SecondaryNameNode implements Runnable {
public static final Log LOG =
LogFactory.getLog(SecondaryNameNode.class.getName());
private final long starttime = System.currentTimeMillis();
private final long starttime = Time.now();
private volatile long lastCheckpointTime = 0;
private String fsName;
@ -315,7 +316,7 @@ public class SecondaryNameNode implements Runnable {
if(UserGroupInformation.isSecurityEnabled())
UserGroupInformation.getCurrentUser().reloginFromKeytab();
long now = System.currentTimeMillis();
long now = Time.now();
if (shouldCheckpointBasedOnCount() ||
now >= lastCheckpointTime + 1000 * checkpointConf.getPeriod()) {

View File

@ -33,6 +33,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@ -209,7 +210,7 @@ public class TransferFsImage {
//
// open connection to remote server
//
long startTime = Util.monotonicNow();
long startTime = Time.monotonicNow();
URL url = new URL(str);
HttpURLConnection connection = (HttpURLConnection)
@ -316,7 +317,7 @@ public class TransferFsImage {
}
}
double xferSec = Math.max(
((float)(Util.monotonicNow() - startTime)) / 1000.0, 0.001);
((float)(Time.monotonicNow() - startTime)) / 1000.0, 0.001);
long xferKb = received / 1024;
LOG.info(String.format("Transfer took %.2fs at %.2f KB/s",
xferSec, xferKb / xferSec));

View File

@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.SecurityUtil;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import static org.apache.hadoop.util.ExitUtil.terminate;
import com.google.common.annotations.VisibleForTesting;

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hdfs.util.Canceler;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import static org.apache.hadoop.util.Time.now;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hdfs.util;
import static org.apache.hadoop.hdfs.server.common.Util.monotonicNow;
import static org.apache.hadoop.util.Time.monotonicNow;
/**
* a class to throttle the data transfers.

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import static org.junit.Assert.*;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.junit.Test;
@ -171,7 +172,7 @@ public class TestLoadGenerator extends Configured implements Tool {
args = new String[] {"-readProbability", "0.3", "-writeProbability", "0.3",
"-root", TEST_SPACE_ROOT, "-maxDelayBetweenOps", "0",
"-numOfThreads", "1", "-startTime",
Long.toString(System.currentTimeMillis()), "-elapsedTime", "10"};
Long.toString(Time.now()), "-elapsedTime", "10"};
assertEquals(0, lg.run(args));
@ -227,7 +228,7 @@ public class TestLoadGenerator extends Configured implements Tool {
String[] scriptArgs = new String[] {
"-root", TEST_SPACE_ROOT, "-maxDelayBetweenOps", "0",
"-numOfThreads", "10", "-startTime",
Long.toString(System.currentTimeMillis()), "-scriptFile", script};
Long.toString(Time.now()), "-scriptFile", script};
assertEquals(0, lg.run(scriptArgs));

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.fs.ChecksumFileSystem;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@ -52,12 +53,12 @@ public class BenchmarkThroughput extends Configured implements Tool {
private int BUFFER_SIZE;
private void resetMeasurements() {
startTime = System.currentTimeMillis();
startTime = Time.now();
}
private void printMeasurements() {
System.out.println(" time: " +
((System.currentTimeMillis() - startTime)/1000));
((Time.now() - startTime)/1000));
}
private Path writeLocalFile(String name, Configuration conf,

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.namenode.CreateEditsLog;
import org.apache.hadoop.net.DNS;
import org.apache.hadoop.util.Time;
/**
@ -231,7 +232,6 @@ public class DataNodeCluster {
}
int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
return "/Rack-" + rand + "-"+ ip + "-" +
System.currentTimeMillis();
return "/Rack-" + rand + "-"+ ip + "-" + Time.now();
}
}

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Time;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
@ -105,16 +106,16 @@ public class TestAppendDifferentChecksum {
FileSystem fsWithCrc32C = createFsWithChecksum("CRC32C", 512);
Path p = new Path("/testAlgoSwitchRandomized");
long seed = System.currentTimeMillis();
long seed = Time.now();
System.out.println("seed: " + seed);
Random r = new Random(seed);
// Create empty to start
IOUtils.closeStream(fsWithCrc32.create(p));
long st = System.currentTimeMillis();
long st = Time.now();
int len = 0;
while (System.currentTimeMillis() - st < RANDOM_TEST_RUNTIME) {
while (Time.now() - st < RANDOM_TEST_RUNTIME) {
int thisLen = r.nextInt(500);
FileSystem fs = (r.nextBoolean() ? fsWithCrc32 : fsWithCrc32C);
FSDataOutputStream stm = fs.append(p);

View File

@ -73,6 +73,7 @@ import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Level;
import org.mockito.Mockito;
import org.mockito.internal.stubbing.answers.ThrowsException;
@ -483,9 +484,9 @@ public class TestDFSClientRetries extends TestCase {
//
// Test 1: might fail
//
long timestamp = System.currentTimeMillis();
long timestamp = Time.now();
boolean pass = busyTest(xcievers, threads, fileLen, timeWin, retries);
long timestamp2 = System.currentTimeMillis();
long timestamp2 = Time.now();
if ( pass ) {
LOG.info("Test 1 succeeded! Time spent: " + (timestamp2-timestamp)/1000.0 + " sec.");
} else {
@ -496,9 +497,9 @@ public class TestDFSClientRetries extends TestCase {
// Test 2: should never fail
//
retries = 50;
timestamp = System.currentTimeMillis();
timestamp = Time.now();
pass = busyTest(xcievers, threads, fileLen, timeWin, retries);
timestamp2 = System.currentTimeMillis();
timestamp2 = Time.now();
assertTrue("Something wrong! Test 2 got Exception with maxmum retries!", pass);
LOG.info("Test 2 succeeded! Time spent: " + (timestamp2-timestamp)/1000.0 + " sec.");
@ -507,9 +508,9 @@ public class TestDFSClientRetries extends TestCase {
//
retries = 3;
timeWin = 1000;
timestamp = System.currentTimeMillis();
timestamp = Time.now();
pass = busyTest(xcievers, threads, fileLen, timeWin, retries);
timestamp2 = System.currentTimeMillis();
timestamp2 = Time.now();
if ( pass ) {
LOG.info("Test 3 succeeded! Time spent: " + (timestamp2-timestamp)/1000.0 + " sec.");
} else {
@ -521,9 +522,9 @@ public class TestDFSClientRetries extends TestCase {
//
retries = 50;
timeWin = 1000;
timestamp = System.currentTimeMillis();
timestamp = Time.now();
pass = busyTest(xcievers, threads, fileLen, timeWin, retries);
timestamp2 = System.currentTimeMillis();
timestamp2 = Time.now();
assertTrue("Something wrong! Test 4 got Exception with maxmum retries!", pass);
LOG.info("Test 4 succeeded! Time spent: " + (timestamp2-timestamp)/1000.0 + " sec.");
}
@ -564,7 +565,7 @@ public class TestDFSClientRetries extends TestCase {
LOG.info("Path : \"" + file1 + "\"");
// write 1 block to file
byte[] buffer = AppendTestUtil.randomBytes(System.currentTimeMillis(), fileLen);
byte[] buffer = AppendTestUtil.randomBytes(Time.now(), fileLen);
stm.write(buffer, 0, fileLen);
stm.close();

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.Time;
/**
@ -85,11 +86,11 @@ public class TestDFSMkdirs extends TestCase {
DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
try {
// Create a dir in root dir, should succeed
assertTrue(dfs.mkdir(new Path("/mkdir-" + System.currentTimeMillis()),
assertTrue(dfs.mkdir(new Path("/mkdir-" + Time.now()),
FsPermission.getDefault()));
// Create a dir when parent dir exists as a file, should fail
IOException expectedException = null;
String filePath = "/mkdir-file-" + System.currentTimeMillis();
String filePath = "/mkdir-file-" + Time.now();
writeFile(dfs, new Path(filePath));
try {
dfs.mkdir(new Path(filePath + "/mkdir"), FsPermission.getDefault());
@ -103,7 +104,7 @@ public class TestDFSMkdirs extends TestCase {
// Create a dir in a non-exist directory, should fail
expectedException = null;
try {
dfs.mkdir(new Path("/non-exist/mkdir-" + System.currentTimeMillis()),
dfs.mkdir(new Path("/non-exist/mkdir-" + Time.now()),
FsPermission.getDefault());
} catch (IOException e) {
expectedException = e;

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Time;
/** Unit tests for permission */
public class TestDFSPermission extends TestCase {
@ -77,7 +78,7 @@ public class TestDFSPermission extends TestCase {
static {
try {
// Initiate the random number generator and logging the seed
long seed = Util.now();
long seed = Time.now();
r = new Random(seed);
LOG.info("Random number generator uses seed " + seed);
LOG.info("NUM_TEST_PERMISSIONS=" + NUM_TEST_PERMISSIONS);

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Time;
/**
* This test verifies that block verification occurs on the datanode
@ -72,15 +73,15 @@ public class TestDatanodeBlockScanner extends TestCase {
throws IOException, TimeoutException {
URL url = new URL("http://localhost:" + infoPort +
"/blockScannerReport?listblocks");
long lastWarnTime = System.currentTimeMillis();
long lastWarnTime = Time.now();
if (newTime <= 0) newTime = 1L;
long verificationTime = 0;
String block = DFSTestUtil.getFirstBlock(fs, file).getBlockName();
long failtime = (timeout <= 0) ? Long.MAX_VALUE
: System.currentTimeMillis() + timeout;
: Time.now() + timeout;
while (verificationTime < newTime) {
if (failtime < System.currentTimeMillis()) {
if (failtime < Time.now()) {
throw new TimeoutException("failed to achieve block verification after "
+ timeout + " msec. Current verification timestamp = "
+ verificationTime + ", requested verification time > "
@ -103,7 +104,7 @@ public class TestDatanodeBlockScanner extends TestCase {
}
if (verificationTime < newTime) {
long now = System.currentTimeMillis();
long now = Time.now();
if ((now - lastWarnTime) >= 5*1000) {
LOG.info("Waiting for verification of " + block);
lastWarnTime = now;
@ -118,7 +119,7 @@ public class TestDatanodeBlockScanner extends TestCase {
}
public void testDatanodeBlockScanner() throws IOException, TimeoutException {
long startTime = System.currentTimeMillis();
long startTime = Time.now();
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@ -311,7 +312,7 @@ public class TestDatanodeBlockScanner extends TestCase {
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 3L);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
long startTime = System.currentTimeMillis();
long startTime = Time.now();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(REPLICATION_FACTOR)
.build();
@ -395,10 +396,10 @@ public class TestDatanodeBlockScanner extends TestCase {
private static void waitForBlockDeleted(ExtendedBlock blk, int dnIndex,
long timeout) throws TimeoutException, InterruptedException {
File blockFile = MiniDFSCluster.getBlockFile(dnIndex, blk);
long failtime = System.currentTimeMillis()
long failtime = Time.now()
+ ((timeout > 0) ? timeout : Long.MAX_VALUE);
while (blockFile != null && blockFile.exists()) {
if (failtime < System.currentTimeMillis()) {
if (failtime < Time.now()) {
throw new TimeoutException("waited too long for blocks to be deleted: "
+ blockFile.getPath() + (blockFile.exists() ? " still exists; " : " is absent; "));
}

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Level;
import org.junit.Test;
@ -169,7 +170,7 @@ public class TestDistributedFileSystem {
for(int i = 0; i < filepaths.length; i++) {
filepaths[i] = new Path(filepathstring + i);
}
final long millis = System.currentTimeMillis();
final long millis = Time.now();
{
DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();

View File

@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Level;
/**
@ -668,13 +669,13 @@ public class TestFileCreation extends junit.framework.TestCase {
}
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
FileSystem fs = cluster.getFileSystem();
final Path path = new Path("/" + System.currentTimeMillis()
final Path path = new Path("/" + Time.now()
+ "-testFileCreationNonRecursive");
FSDataOutputStream out = null;
try {
IOException expectedException = null;
final String nonExistDir = "/non-exist-" + System.currentTimeMillis();
final String nonExistDir = "/non-exist-" + Time.now();
fs.delete(new Path(nonExistDir), true);
EnumSet<CreateFlag> createFlag = EnumSet.of(CreateFlag.CREATE);

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.util.Time;
/**
@ -72,7 +73,7 @@ public class TestInjectionForSimulatedStorage extends TestCase {
ClientProtocol namenode,
int expected, long maxWaitSec)
throws IOException {
long start = System.currentTimeMillis();
long start = Time.now();
//wait for all the blocks to be replicated;
LOG.info("Checking for block replication for " + filename);
@ -97,7 +98,7 @@ public class TestInjectionForSimulatedStorage extends TestCase {
actual + ".");
if (maxWaitSec > 0 &&
(System.currentTimeMillis() - start) > (maxWaitSec * 1000)) {
(Time.now() - start) > (maxWaitSec * 1000)) {
throw new IOException("Timedout while waiting for all blocks to " +
" be replicated for " + filename);
}

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Time;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
@ -82,7 +83,7 @@ public class TestLease {
// We don't need to wait the lease renewer thread to act.
// call renewLease() manually.
// make it look like lease has already expired.
dfs.lastLeaseRenewal = System.currentTimeMillis() - 300000;
dfs.lastLeaseRenewal = Time.now() - 300000;
dfs.renewLease();
// this should not work.

View File

@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -118,8 +119,8 @@ public class TestLeaseRenewer {
renewer.put(filePath, mockStream, MOCK_DFSCLIENT);
// Wait for lease to get renewed
long failTime = System.currentTimeMillis() + 5000;
while (System.currentTimeMillis() < failTime &&
long failTime = Time.now() + 5000;
while (Time.now() < failTime &&
leaseRenewalCount.get() == 0) {
Thread.sleep(50);
}
@ -206,8 +207,8 @@ public class TestLeaseRenewer {
renewer.closeFile(filePath, MOCK_DFSCLIENT);
// Should stop the renewer running within a few seconds
long failTime = System.currentTimeMillis() + 5000;
while (renewer.isRunning() && System.currentTimeMillis() < failTime) {
long failTime = Time.now() + 5000;
while (renewer.isRunning() && Time.now() < failTime) {
Thread.sleep(50);
}
Assert.assertFalse(renewer.isRunning());

Some files were not shown because too many files have changed in this diff Show More