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:
parent
790e970931
commit
dce88a6dbc
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 +
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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() +
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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]++;
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) <=
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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. */
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)");
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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));
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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; "));
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue