HBASE-6476 Replace all occurrances of System.currentTimeMillis() with EnvironmentEdge equivalent; REVERT

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1391852 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-09-29 17:57:49 +00:00
parent 98f566edf2
commit c7781e9a37
72 changed files with 213 additions and 269 deletions

View File

@ -142,7 +142,7 @@ public class Threads {
* @param msToWait the amount of time to sleep in milliseconds
*/
public static void sleepWithoutInterrupt(final long msToWait) {
long timeMillis = EnvironmentEdgeManager.currentTimeMillis();
long timeMillis = System.currentTimeMillis();
long endTime = timeMillis + msToWait;
boolean interrupted = false;
while (timeMillis < endTime) {
@ -151,7 +151,7 @@ public class Threads {
} catch (InterruptedException ex) {
interrupted = true;
}
timeMillis = EnvironmentEdgeManager.currentTimeMillis();
timeMillis = System.currentTimeMillis();
}
if (interrupted) {

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.Sleeper;
@ -61,7 +60,7 @@ public abstract class Chore extends HasThread {
try {
boolean initialChoreComplete = false;
while (!this.stopper.isStopped()) {
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
try {
if (!initialChoreComplete) {
initialChoreComplete = initialChore();

View File

@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.JenkinsHash;
import org.apache.hadoop.hbase.util.MD5Hash;
@ -262,7 +261,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
public HRegionInfo(final byte[] tableName, final byte[] startKey, final byte[] endKey,
final boolean split)
throws IllegalArgumentException {
this(tableName, startKey, endKey, split, EnvironmentEdgeManager.currentTimeMillis());
this(tableName, startKey, endKey, split, System.currentTimeMillis());
}

View File

@ -24,7 +24,6 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.io.VersionedWritable;
import org.apache.hadoop.io.WritableComparable;
@ -57,7 +56,7 @@ implements WritableComparable<HServerInfo> {
* @param webuiport Port the webui runs on.
*/
public HServerInfo(final HServerAddress serverAddress, final int webuiport) {
this(serverAddress, EnvironmentEdgeManager.currentTimeMillis(), webuiport);
this(serverAddress, System.currentTimeMillis(), webuiport);
}
public HServerInfo(HServerAddress serverAddress, long startCode,

View File

@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
@ -109,7 +108,7 @@ public class RegionTransition {
ZooKeeperProtos.RegionTransition.Builder builder = ZooKeeperProtos.RegionTransition.newBuilder().
setEventTypeCode(type.getCode()).setRegionName(ByteString.copyFrom(regionName)).
setOriginServerName(pbsn);
builder.setCreateTime(EnvironmentEdgeManager.currentTimeMillis());
builder.setCreateTime(System.currentTimeMillis());
if (payload != null) builder.setPayload(ByteString.copyFrom(payload));
return new RegionTransition(builder.build());
}

View File

@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.MetaNodeTracker;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -471,10 +470,10 @@ public class CatalogTracker {
*/
public ServerName waitForMeta(long timeout)
throws InterruptedException, IOException, NotAllMetaRegionsOnlineException {
long stop = EnvironmentEdgeManager.currentTimeMillis() + timeout;
long stop = System.currentTimeMillis() + timeout;
long waitTime = Math.min(50, timeout);
synchronized (metaAvailable) {
while(!stopped && (timeout == 0 || EnvironmentEdgeManager.currentTimeMillis() < stop)) {
while(!stopped && (timeout == 0 || System.currentTimeMillis() < stop)) {
if (getMetaServerConnection() != null) {
return metaLocation;
}

View File

@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.io.DataOutputBuffer;
/**
@ -98,7 +97,7 @@ public class ClientScanner extends AbstractClientScanner {
}
this.scan = scan;
this.tableName = tableName;
this.lastNext = EnvironmentEdgeManager.currentTimeMillis();
this.lastNext = System.currentTimeMillis();
this.connection = connection;
if (scan.getMaxResultSize() > 0) {
this.maxScannerResultSize = scan.getMaxResultSize();
@ -286,8 +285,8 @@ public class ClientScanner extends AbstractClientScanner {
// If we are over the timeout, throw this exception to the client
// Else, it's because the region moved and we used the old id
// against the new region server; reset the scanner.
if (timeout < EnvironmentEdgeManager.currentTimeMillis()) {
long elapsed = EnvironmentEdgeManager.currentTimeMillis() - lastNext;
if (timeout < System.currentTimeMillis()) {
long elapsed = System.currentTimeMillis() - lastNext;
ScannerTimeoutException ex = new ScannerTimeoutException(
elapsed + "ms passed since the last invocation, " +
"timeout is currently set to " + scannerTimeout);
@ -314,7 +313,7 @@ public class ClientScanner extends AbstractClientScanner {
callable = null;
continue;
}
long currentTime = EnvironmentEdgeManager.currentTimeMillis();
long currentTime = System.currentTimeMillis();
if (this.scanMetrics != null ) {
this.scanMetrics.sumOfMillisSecBetweenNexts.inc(currentTime-lastNext);
}

View File

@ -54,7 +54,7 @@ import java.util.Map;
* Specifying timestamps, deleteFamily and deleteColumns will delete all
* versions with a timestamp less than or equal to that passed. If no
* timestamp is specified, an entry is added with a timestamp of 'now'
* where 'now' is the servers's EnvironmentEdgeManager.currentTimeMillis().
* where 'now' is the servers's System.currentTimeMillis().
* Specifying a timestamp to the deleteColumn method will
* delete versions only with a timestamp equal to that specified.
* If no timestamp is passed to deleteColumn, internally, it figures the

View File

@ -86,7 +86,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDe
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.SoftValueSortedMap;
import org.apache.hadoop.hbase.util.Triple;
@ -1447,7 +1446,7 @@ public class HConnectionManager {
--keepAliveZookeeperUserCount;
if (keepAliveZookeeperUserCount <=0 ){
keepZooKeeperWatcherAliveUntil =
EnvironmentEdgeManager.currentTimeMillis() + keepAlive;
System.currentTimeMillis() + keepAlive;
}
}
}
@ -1485,7 +1484,7 @@ public class HConnectionManager {
}
protected void closeMasterProtocol(MasterProtocolState protocolState) {
if (EnvironmentEdgeManager.currentTimeMillis() > protocolState.keepAliveUntil) {
if (System.currentTimeMillis() > protocolState.keepAliveUntil) {
hci.closeMasterProtocol(protocolState);
protocolState.keepAliveUntil = Long.MAX_VALUE;
}
@ -1495,7 +1494,7 @@ public class HConnectionManager {
protected void chore() {
synchronized (hci.masterAndZKLock) {
if (hci.canCloseZKW) {
if (EnvironmentEdgeManager.currentTimeMillis() >
if (System.currentTimeMillis() >
hci.keepZooKeeperWatcherAliveUntil) {
hci.closeZooKeeperWatcher();
@ -1660,7 +1659,7 @@ public class HConnectionManager {
--protocolState.userCount;
if (protocolState.userCount <= 0) {
protocolState.keepAliveUntil =
EnvironmentEdgeManager.currentTimeMillis() + keepAlive;
System.currentTimeMillis() + keepAlive;
}
}
}
@ -2097,12 +2096,12 @@ public class HConnectionManager {
final Callable<MultiResponse> delegate = hci.createCallable(loc, multi, tableName);
return new Callable<MultiResponse>() {
private final long creationTime = EnvironmentEdgeManager.currentTimeMillis();
private final long creationTime = System.currentTimeMillis();
@Override
public MultiResponse call() throws Exception {
try {
final long waitingTime = delay + creationTime - EnvironmentEdgeManager.currentTimeMillis();
final long waitingTime = delay + creationTime - System.currentTimeMillis();
if (waitingTime > 0) {
Thread.sleep(waitingTime);
}

View File

@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.PairOfSameType;
/**
@ -410,7 +409,7 @@ public class MetaScanner {
HRegionInfo splitB = daughters.getSecond();
HTable metaTable = getMetaTable();
long start = EnvironmentEdgeManager.currentTimeMillis();
long start = System.currentTimeMillis();
Result resultA = getRegionResultBlocking(metaTable, blockingTimeout,
splitA.getRegionName());
if (resultA != null) {
@ -420,7 +419,7 @@ public class MetaScanner {
throw new RegionOfflineException("Split daughter region " +
splitA.getRegionNameAsString() + " cannot be found in META.");
}
long rem = blockingTimeout - (EnvironmentEdgeManager.currentTimeMillis() - start);
long rem = blockingTimeout - (System.currentTimeMillis() - start);
Result resultB = getRegionResultBlocking(metaTable, rem,
splitB.getRegionName());
@ -441,8 +440,8 @@ public class MetaScanner {
if (LOG.isDebugEnabled()) {
LOG.debug("blocking until region is in META: " + Bytes.toStringBinary(regionName));
}
long start = EnvironmentEdgeManager.currentTimeMillis();
while (EnvironmentEdgeManager.currentTimeMillis() - start < timeout) {
long start = System.currentTimeMillis();
while (System.currentTimeMillis() - start < timeout) {
Get get = new Get(regionName);
Result result = metaTable.get(get);
HRegionInfo info = getHRegionInfo(result);

View File

@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.DNS;
@ -142,10 +141,10 @@ public class ScannerCallable extends ServerCallable<Result[]> {
RequestConverter.buildScanRequest(scannerId, caching, false);
try {
ScanResponse response = server.scan(null, request);
long timestamp = EnvironmentEdgeManager.currentTimeMillis();
long timestamp = System.currentTimeMillis();
rrs = ResponseConverter.getResults(response);
if (logScannerActivity) {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
if (now - timestamp > logCutOffLatency) {
int rows = rrs == null ? 0 : rrs.length;
LOG.info("Took " + (now-timestamp) + "ms to fetch "

View File

@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.ipc.RemoteException;
import com.google.protobuf.ServiceException;
@ -115,12 +114,12 @@ public abstract class ServerCallable<T> implements Callable<T> {
public void beforeCall() {
HBaseRPC.setRpcTimeout(this.callTimeout);
this.startTime = EnvironmentEdgeManager.currentTimeMillis();
this.startTime = System.currentTimeMillis();
}
public void afterCall() {
HBaseRPC.resetRpcTimeout();
this.endTime = EnvironmentEdgeManager.currentTimeMillis();
this.endTime = System.currentTimeMillis();
}
public void shouldRetry(Throwable throwable) throws IOException {
@ -183,7 +182,7 @@ public abstract class ServerCallable<T> implements Callable<T> {
}
RetriesExhaustedException.ThrowableWithExtraContext qt =
new RetriesExhaustedException.ThrowableWithExtraContext(t,
EnvironmentEdgeManager.currentTimeMillis(), toString());
System.currentTimeMillis(), toString());
exceptions.add(qt);
if (tries == numRetries - 1) {
throw new RetriesExhaustedException(tries, exceptions);

View File

@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.Server;
@ -188,7 +187,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
FileSystem fs = path.getFileSystem(HBaseConfiguration.create());
Path dst = new Path(System.getProperty("java.io.tmpdir") +
java.io.File.separator +"." + pathPrefix +
"." + className + "." + EnvironmentEdgeManager.currentTimeMillis() + ".jar");
"." + className + "." + System.currentTimeMillis() + ".jar");
fs.copyToLocalFile(path, dst);
File tmpLocal = new File(dst.toString());
tmpLocal.deleteOnExit();
@ -214,7 +213,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
if (entry.getName().matches("/lib/[^/]+\\.jar")) {
File file = new File(System.getProperty("java.io.tmpdir") +
java.io.File.separator +"." + pathPrefix +
"." + className + "." + EnvironmentEdgeManager.currentTimeMillis() + "." + entry.getName().substring(5));
"." + className + "." + System.currentTimeMillis() + "." + entry.getName().substring(5));
IOUtils.copyBytes(jarFile.getInputStream(entry), new FileOutputStream(file), conf, true);
file.deleteOnExit();
paths.add(file.toURL());

View File

@ -266,7 +266,7 @@ public class HBaseClient {
protected Call(RpcRequestBody param) {
this.param = param;
this.startTime = EnvironmentEdgeManager.currentTimeMillis();
this.startTime = System.currentTimeMillis();
synchronized (HBaseClient.this) {
this.id = counter++;
}
@ -432,7 +432,7 @@ public class HBaseClient {
/** Update lastActivity with the current time. */
protected void touch() {
lastActivity.set(EnvironmentEdgeManager.currentTimeMillis());
lastActivity.set(System.currentTimeMillis());
}
/**
@ -604,7 +604,7 @@ public class HBaseClient {
protected synchronized boolean waitForWork() {
if (calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
long timeout = maxIdleTime-
(EnvironmentEdgeManager.currentTimeMillis()-lastActivity.get());
(System.currentTimeMillis()-lastActivity.get());
if (timeout>0) {
try {
wait(timeout);
@ -634,7 +634,7 @@ public class HBaseClient {
* since last I/O activity is equal to or greater than the ping interval
*/
protected synchronized void sendPing() throws IOException {
long curTime = EnvironmentEdgeManager.currentTimeMillis();
long curTime = System.currentTimeMillis();
if ( curTime - lastActivity.get() >= pingInterval) {
lastActivity.set(curTime);
//noinspection SynchronizeOnNonFinalField
@ -1056,7 +1056,7 @@ public class HBaseClient {
Iterator<Entry<Integer, Call>> itor = calls.entrySet().iterator();
while (itor.hasNext()) {
Call c = itor.next().getValue();
long waitTime = EnvironmentEdgeManager.currentTimeMillis() - c.getStartTime();
long waitTime = System.currentTimeMillis() - c.getStartTime();
if (waitTime >= rpcTimeout) {
if (this.closeException == null) {
// There may be no exception in the case that there are many calls
@ -1080,7 +1080,7 @@ public class HBaseClient {
try {
if (!calls.isEmpty()) {
Call firstCall = calls.get(calls.firstKey());
long maxWaitTime = EnvironmentEdgeManager.currentTimeMillis() - firstCall.getStartTime();
long maxWaitTime = System.currentTimeMillis() - firstCall.getStartTime();
if (maxWaitTime < rpcTimeout) {
rpcTimeout -= maxWaitTime;
}

View File

@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.util.ReflectionUtils;
@ -225,7 +224,7 @@ public class HBaseRPC {
long timeout
) throws IOException {
// HBase does limited number of reconnects which is different from hadoop.
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
IOException ioe;
int reconnectAttempts = 0;
while (true) {
@ -258,7 +257,7 @@ public class HBaseRPC {
}
}
// check if timed out
if (EnvironmentEdgeManager.currentTimeMillis() - timeout >= startTime) {
if (System.currentTimeMillis() - timeout >= startTime) {
throw ioe;
}

View File

@ -84,7 +84,6 @@ import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslDigestCallbackHan
import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslGssCallbackHandler;
import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslStatus;
import org.apache.hadoop.hbase.util.ByteBufferOutputStream;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Writable;
@ -334,7 +333,7 @@ public abstract class HBaseServer implements RpcServer {
this.id = id;
this.rpcRequestBody = rpcRequestBody;
this.connection = connection;
this.timestamp = EnvironmentEdgeManager.currentTimeMillis();
this.timestamp = System.currentTimeMillis();
this.response = null;
this.delayResponse = false;
this.responder = responder;
@ -465,7 +464,7 @@ public abstract class HBaseServer implements RpcServer {
@Override
public void throwExceptionIfCallerDisconnected() throws CallerDisconnectedException {
if (!connection.channel.isOpen()) {
long afterTime = EnvironmentEdgeManager.currentTimeMillis() - timestamp;
long afterTime = System.currentTimeMillis() - timestamp;
throw new CallerDisconnectedException(
"Aborting call " + this + " after " + afterTime + " ms, since " +
"caller disconnected");
@ -617,7 +616,7 @@ public abstract class HBaseServer implements RpcServer {
*/
private void cleanupConnections(boolean force) {
if (force || numConnections > thresholdIdleConnections) {
long currentTime = EnvironmentEdgeManager.currentTimeMillis();
long currentTime = System.currentTimeMillis();
if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
return;
}
@ -654,7 +653,7 @@ public abstract class HBaseServer implements RpcServer {
}
else i++;
}
lastCleanupRunTime = EnvironmentEdgeManager.currentTimeMillis();
lastCleanupRunTime = System.currentTimeMillis();
}
}
@ -752,7 +751,7 @@ public abstract class HBaseServer implements RpcServer {
try {
reader.startAdd();
SelectionKey readKey = reader.registerChannel(channel);
c = getConnection(channel, EnvironmentEdgeManager.currentTimeMillis());
c = getConnection(channel, System.currentTimeMillis());
readKey.attach(c);
synchronized (connectionList) {
connectionList.add(numConnections, c);
@ -775,7 +774,7 @@ public abstract class HBaseServer implements RpcServer {
if (c == null) {
return;
}
c.setLastContact(EnvironmentEdgeManager.currentTimeMillis());
c.setLastContact(System.currentTimeMillis());
try {
count = c.readAndProcess();
@ -794,7 +793,7 @@ public abstract class HBaseServer implements RpcServer {
// c = null;
}
else {
c.setLastContact(EnvironmentEdgeManager.currentTimeMillis());
c.setLastContact(System.currentTimeMillis());
}
}
@ -868,7 +867,7 @@ public abstract class HBaseServer implements RpcServer {
LOG.info(getName() + ": doAsyncWrite threw exception " + e);
}
}
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
if (now < lastPurgeTime + purgeTimeout) {
continue;
}
@ -1023,7 +1022,7 @@ public abstract class HBaseServer implements RpcServer {
if (inHandler) {
// set the serve time when the response has to be sent later
call.timestamp = EnvironmentEdgeManager.currentTimeMillis();
call.timestamp = System.currentTimeMillis();
if (enqueueInSelector(call))
done = true;
}
@ -1071,7 +1070,7 @@ public abstract class HBaseServer implements RpcServer {
//
void doRespond(Call call) throws IOException {
// set the serve time when the response has to be sent later
call.timestamp = EnvironmentEdgeManager.currentTimeMillis();
call.timestamp = System.currentTimeMillis();
responseQueueLen++;
boolean doRegister = false;

View File

@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Objects;
import org.codehaus.jackson.map.ObjectMapper;
@ -170,7 +169,7 @@ class ProtobufRpcEngine implements RpcEngine {
throws ServiceException {
long startTime = 0;
if (LOG.isDebugEnabled()) {
startTime = EnvironmentEdgeManager.currentTimeMillis();
startTime = System.currentTimeMillis();
}
RpcRequestBody rpcRequest = constructRpcRequest(method, args);
@ -179,7 +178,7 @@ class ProtobufRpcEngine implements RpcEngine {
val = client.call(rpcRequest, address, protocol, ticket, rpcTimeout);
if (LOG.isDebugEnabled()) {
long callTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
long callTime = System.currentTimeMillis() - startTime;
if (LOG.isTraceEnabled()) LOG.trace("Call: " + method.getName() + " " + callTime);
}
return val;
@ -351,7 +350,7 @@ class ProtobufRpcEngine implements RpcEngine {
throw new HBaseRPC.UnknownProtocolException(protocol);
}
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
if (method.getParameterTypes().length == 2) {
// RpcController + Message in the method args
// (generated code from RPC bits in .proto files have RpcController)
@ -364,7 +363,7 @@ class ProtobufRpcEngine implements RpcEngine {
+ method.getName() + "]" + ", allowed (at most): 2, Actual: "
+ method.getParameterTypes().length);
}
int processingTime = (int) (EnvironmentEdgeManager.currentTimeMillis() - startTime);
int processingTime = (int) (System.currentTimeMillis() - startTime);
int qTime = (int) (startTime-receiveTime);
if (TRACELOG.isDebugEnabled()) {
TRACELOG.debug("Call #" + CurCall.get().id +

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.util.StringUtils;
@ -93,7 +92,7 @@ public class TableRecordReaderImpl {
}
if (logScannerActivity) {
LOG.info("Current scan=" + currentScan.toString());
timestamp = EnvironmentEdgeManager.currentTimeMillis();
timestamp = System.currentTimeMillis();
rowcount = 0;
}
}
@ -198,7 +197,7 @@ public class TableRecordReaderImpl {
if (logScannerActivity) {
rowcount ++;
if (rowcount >= logPerRowCount) {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
LOG.info("Mapper took " + (now-timestamp)
+ "ms to process " + rowcount + " rows");
timestamp = now;
@ -233,7 +232,7 @@ public class TableRecordReaderImpl {
return false;
} catch (IOException ioe) {
if (logScannerActivity) {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
LOG.info("Mapper took " + (now-timestamp)
+ "ms to process " + rowcount + " rows");
LOG.info(ioe);

View File

@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.WritableUtils;
@ -127,7 +126,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
private final Map<byte [], WriterLength> writers =
new TreeMap<byte [], WriterLength>(Bytes.BYTES_COMPARATOR);
private byte [] previousRow = HConstants.EMPTY_BYTE_ARRAY;
private final byte [] now = Bytes.toBytes(EnvironmentEdgeManager.currentTimeMillis());
private final byte [] now = Bytes.toBytes(System.currentTimeMillis());
private boolean rollRequested = false;
public void write(ImmutableBytesWritable row, KeyValue kv)
@ -214,7 +213,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
private void close(final HFile.Writer w) throws IOException {
if (w != null) {
w.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY,
Bytes.toBytes(EnvironmentEdgeManager.currentTimeMillis()));
Bytes.toBytes(System.currentTimeMillis()));
w.appendFileInfo(StoreFile.BULKLOAD_TASK_KEY,
Bytes.toBytes(context.getTaskAttemptID().toString()));
w.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY,

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.mapreduce;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import java.io.IOException;
import java.util.ArrayList;
@ -418,7 +417,7 @@ public class ImportTsv {
// If timestamp option is not specified, use current system time.
long timstamp = conf
.getLong(TIMESTAMP_CONF_KEY, EnvironmentEdgeManager.currentTimeMillis());
.getLong(TIMESTAMP_CONF_KEY, System.currentTimeMillis());
// Set it back to replace invalid timestamp (non-numeric) with current
// system time

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.mapreduce.InputSplit;
@ -85,7 +84,7 @@ public class TableRecordReaderImpl {
this.scanner = this.htable.getScanner(currentScan);
if (logScannerActivity) {
LOG.info("Current scan=" + currentScan.toString());
timestamp = EnvironmentEdgeManager.currentTimeMillis();
timestamp = System.currentTimeMillis();
rowcount = 0;
}
}
@ -196,7 +195,7 @@ public class TableRecordReaderImpl {
if (logScannerActivity) {
rowcount ++;
if (rowcount >= logPerRowCount) {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
LOG.info("Mapper took " + (now-timestamp)
+ "ms to process " + rowcount + " rows");
timestamp = now;
@ -232,7 +231,7 @@ public class TableRecordReaderImpl {
return false;
} catch (IOException ioe) {
if (logScannerActivity) {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
LOG.info("Mapper took " + (now-timestamp)
+ "ms to process " + rowcount + " rows");
LOG.info(ioe);

View File

@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.KeyLocker;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
@ -666,7 +665,7 @@ public class AssignmentManager extends ZooKeeperListener {
Lock lock = locker.acquireLock(encodedName);
try {
// Printing if the event was created a long time ago helps debugging
boolean lateEvent = createTime < (EnvironmentEdgeManager.currentTimeMillis() - 15000);
boolean lateEvent = createTime < (System.currentTimeMillis() - 15000);
RegionState regionState = regionStates.getRegionTransitionState(encodedName);
LOG.debug("Handling transition=" + rt.getEventType() +
", server=" + sn + ", region=" +
@ -1216,7 +1215,7 @@ public class AssignmentManager extends ZooKeeperListener {
try {
// Send OPEN RPC. If it fails on a IOE or RemoteException, the
// TimeoutMonitor will pick up the pieces.
long maxWaitTime = EnvironmentEdgeManager.currentTimeMillis() +
long maxWaitTime = System.currentTimeMillis() +
this.server.getConfiguration().
getLong("hbase.regionserver.rpc.startup.waittime", 60000);
while (!this.server.isStopped()) {
@ -1245,7 +1244,7 @@ public class AssignmentManager extends ZooKeeperListener {
} else if (decodedException instanceof ServerNotRunningYetException) {
// This is the one exception to retry. For all else we should just fail
// the startup.
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
if (now > maxWaitTime) throw e;
LOG.debug("Server is not yet up; waiting up to " +
(maxWaitTime - now) + "ms", e);
@ -1339,7 +1338,7 @@ public class AssignmentManager extends ZooKeeperListener {
// call to open risks our writing PENDING_OPEN after state has been moved
// to OPENING by the regionserver.
regionStates.updateRegionState(state.getRegion(),
RegionState.State.PENDING_OPEN, EnvironmentEdgeManager.currentTimeMillis(),
RegionState.State.PENDING_OPEN, System.currentTimeMillis(),
destination);
this.counter.addAndGet(1);
}
@ -1442,7 +1441,7 @@ public class AssignmentManager extends ZooKeeperListener {
" to " + plan.getDestination().toString());
// Transition RegionState to PENDING_OPEN
regionStates.updateRegionState(state.getRegion(),
RegionState.State.PENDING_OPEN, EnvironmentEdgeManager.currentTimeMillis(),
RegionState.State.PENDING_OPEN, System.currentTimeMillis(),
plan.getDestination());
// Send OPEN RPC. This can fail if the server on other end is is not up.
// Pass the version that was obtained while setting the node to OFFLINE.
@ -2100,10 +2099,10 @@ public class AssignmentManager extends ZooKeeperListener {
// that if it returns without an exception that there was a period of time
// with no regions in transition from the point-of-view of the in-memory
// state of the Master.
final long endTime = EnvironmentEdgeManager.currentTimeMillis() + timeout;
final long endTime = System.currentTimeMillis() + timeout;
while (!this.server.isStopped() && regionStates.isRegionsInTransition()
&& endTime > EnvironmentEdgeManager.currentTimeMillis()) {
&& endTime > System.currentTimeMillis()) {
regionStates.waitForUpdate(100);
}
@ -2300,7 +2299,7 @@ public class AssignmentManager extends ZooKeeperListener {
* on a frequent interval.
*/
public void updateRegionsInTransitionMetrics() {
long currentTime = EnvironmentEdgeManager.currentTimeMillis();
long currentTime = System.currentTimeMillis();
int totalRITs = 0;
int totalRITsOverThreshold = 0;
long oldestRITTime = 0;
@ -2431,7 +2430,7 @@ public class AssignmentManager extends ZooKeeperListener {
boolean noRSAvailable = this.serverManager.createDestinationServersList().isEmpty();
// Iterate all regions in transition checking for time outs
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
// no lock concurrent access ok: we will be working on a copy, and it's java-valid to do
// a copy while another thread is adding/removing items
for (RegionState regionState : regionStates.getRegionsInTransition().values()) {

View File

@ -36,7 +36,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* Run bulk assign. Does one RCP per regionserver passing a
@ -104,10 +103,10 @@ public class GeneralBulkAssigner extends BulkAssigner {
pool.shutdown(); // no more task allowed
int serverCount = bulkPlan.size();
int regionCount = regionSet.size();
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
long rpcWaitTime = startTime + timeout;
while (!server.isStopped() && !pool.isTerminated()
&& rpcWaitTime > EnvironmentEdgeManager.currentTimeMillis()) {
&& rpcWaitTime > System.currentTimeMillis()) {
if (failedPlans.isEmpty()) {
pool.awaitTermination(100, TimeUnit.MILLISECONDS);
} else {
@ -116,7 +115,7 @@ public class GeneralBulkAssigner extends BulkAssigner {
}
if (!pool.isTerminated()) {
LOG.warn("bulk assigner is still running after "
+ (EnvironmentEdgeManager.currentTimeMillis() - startTime) + "ms, shut it down now");
+ (System.currentTimeMillis() - startTime) + "ms, shut it down now");
// some assigner hangs, can't wait any more, shutdown the pool now
List<Runnable> notStarted = pool.shutdownNow();
if (notStarted != null && !notStarted.isEmpty()) {
@ -134,11 +133,11 @@ public class GeneralBulkAssigner extends BulkAssigner {
Configuration conf = server.getConfiguration();
long perRegionOpenTimeGuesstimate =
conf.getLong("hbase.bulk.assignment.perregion.open.time", 1000);
long endTime = Math.max(EnvironmentEdgeManager.currentTimeMillis(), rpcWaitTime)
long endTime = Math.max(System.currentTimeMillis(), rpcWaitTime)
+ perRegionOpenTimeGuesstimate * (reassigningRegions + 1);
RegionStates regionStates = assignmentManager.getRegionStates();
// We're not synchronizing on regionsInTransition now because we don't use any iterator.
while (!regionSet.isEmpty() && !server.isStopped() && endTime > EnvironmentEdgeManager.currentTimeMillis()) {
while (!regionSet.isEmpty() && !server.isStopped() && endTime > System.currentTimeMillis()) {
Iterator<HRegionInfo> regionInfoIterator = regionSet.iterator();
while (regionInfoIterator.hasNext()) {
HRegionInfo hri = regionInfoIterator.next();
@ -154,7 +153,7 @@ public class GeneralBulkAssigner extends BulkAssigner {
}
if (LOG.isDebugEnabled()) {
long elapsedTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
long elapsedTime = System.currentTimeMillis() - startTime;
String status = "successfully";
if (!regionSet.isEmpty()) {
status = "with " + regionSet.size() + " regions still not assigned yet";

View File

@ -164,7 +164,6 @@ import org.apache.hadoop.hbase.replication.regionserver.Replication;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.HasThread;
@ -356,7 +355,7 @@ Server {
// Set our address.
this.isa = this.rpcServer.getListenerAddress();
this.serverName = new ServerName(this.isa.getHostName(),
this.isa.getPort(), EnvironmentEdgeManager.currentTimeMillis());
this.isa.getPort(), System.currentTimeMillis());
this.rsFatals = new MemoryBoundedLogMessageBuffer(
conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
@ -432,7 +431,7 @@ Server {
MonitoredTask startupStatus =
TaskMonitor.get().createStatus("Master startup");
startupStatus.setDescription("Master startup");
masterStartTime = EnvironmentEdgeManager.currentTimeMillis();
masterStartTime = System.currentTimeMillis();
try {
/*
* Block on becoming the active master.
@ -572,10 +571,10 @@ Server {
long lastMsgTs = 0l;
long now = 0l;
while (!this.stopped) {
now = EnvironmentEdgeManager.currentTimeMillis();
now = System.currentTimeMillis();
if ((now - lastMsgTs) >= this.msgInterval) {
doMetrics();
lastMsgTs = EnvironmentEdgeManager.currentTimeMillis();
lastMsgTs = System.currentTimeMillis();
}
stopSleeper.sleep();
}
@ -626,7 +625,7 @@ Server {
*/
status.setStatus("Initializing Master file system");
this.masterActiveTime = EnvironmentEdgeManager.currentTimeMillis();
this.masterActiveTime = System.currentTimeMillis();
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
this.fileSystemManager = new MasterFileSystem(this, this, metrics, masterRecovery);
@ -1266,7 +1265,7 @@ Server {
if (!this.loadBalancerTracker.isBalancerOn()) return false;
// Do this call outside of synchronized block.
int maximumBalanceTime = getBalancerCutoffTime();
long cutoffTime = EnvironmentEdgeManager.currentTimeMillis() + maximumBalanceTime;
long cutoffTime = System.currentTimeMillis() + maximumBalanceTime;
boolean balancerRan;
synchronized (this.balancer) {
// Only allow one balance run at at time.
@ -1312,13 +1311,13 @@ Server {
if (plans != null && !plans.isEmpty()) {
for (RegionPlan plan: plans) {
LOG.info("balance " + plan);
long balStartTime = EnvironmentEdgeManager.currentTimeMillis();
long balStartTime = System.currentTimeMillis();
this.assignmentManager.balance(plan);
totalRegPlanExecTime += EnvironmentEdgeManager.currentTimeMillis()-balStartTime;
totalRegPlanExecTime += System.currentTimeMillis()-balStartTime;
rpCount++;
if (rpCount < plans.size() &&
// if performing next balance exceeds cutoff time, exit the loop
(EnvironmentEdgeManager.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) {
(System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) {
LOG.debug("No more balancing till next balance run; maximumBalanceTime=" +
maximumBalanceTime);
break;

View File

@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* State of a Region while undergoing transitions.
@ -56,11 +55,11 @@ public class RegionState implements org.apache.hadoop.io.Writable {
private volatile State state;
public RegionState() {
this.stamp = new AtomicLong(EnvironmentEdgeManager.currentTimeMillis());
this.stamp = new AtomicLong(System.currentTimeMillis());
}
public RegionState(HRegionInfo region, State state) {
this(region, state, EnvironmentEdgeManager.currentTimeMillis(), null);
this(region, state, System.currentTimeMillis(), null);
}
public RegionState(HRegionInfo region,
@ -72,7 +71,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
}
public void updateTimestampToNow() {
this.stamp.set(EnvironmentEdgeManager.currentTimeMillis());
this.stamp.set(System.currentTimeMillis());
}
public State getState() {
@ -140,7 +139,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
*/
public String toDescriptiveString() {
long lstamp = stamp.get();
long relTime = EnvironmentEdgeManager.currentTimeMillis() - lstamp;
long relTime = System.currentTimeMillis() - lstamp;
return region.getRegionNameAsString()
+ " state=" + state

View File

@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
/**
@ -212,7 +211,7 @@ public class RegionStates {
*/
public synchronized RegionState updateRegionState(
final HRegionInfo hri, final State state, final ServerName serverName) {
return updateRegionState(hri, state, EnvironmentEdgeManager.currentTimeMillis(), serverName);
return updateRegionState(hri, state, System.currentTimeMillis(), serverName);
}
/**

View File

@ -60,7 +60,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.protobuf.ServiceException;
@ -288,7 +287,7 @@ public class ServerManager {
*/
private void checkClockSkew(final ServerName serverName, final long serverCurrentTime)
throws ClockOutOfSyncException {
long skew = EnvironmentEdgeManager.currentTimeMillis() - serverCurrentTime;
long skew = System.currentTimeMillis() - serverCurrentTime;
if (skew > maxSkew) {
String message = "Server " + serverName + " has been " +
"rejected; Reported time is too far out of sync with master. " +
@ -409,7 +408,7 @@ public class ServerManager {
long previousLogTime = 0;
while (!onlineServers.isEmpty()) {
if (EnvironmentEdgeManager.currentTimeMillis() > (previousLogTime + 1000)) {
if (System.currentTimeMillis() > (previousLogTime + 1000)) {
StringBuilder sb = new StringBuilder();
for (ServerName key : this.onlineServers.keySet()) {
if (sb.length() > 0) {
@ -418,7 +417,7 @@ public class ServerManager {
sb.append(key);
}
LOG.info("Waiting on regionserver(s) to go down " + sb.toString());
previousLogTime = EnvironmentEdgeManager.currentTimeMillis();
previousLogTime = System.currentTimeMillis();
}
synchronized (onlineServers) {
@ -695,7 +694,7 @@ public class ServerManager {
final int maxToStart = this.master.getConfiguration().
getInt("hbase.master.wait.on.regionservers.maxtostart", Integer.MAX_VALUE);
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
final long startTime = now;
long slept = 0;
long lastLogTime = 0;
@ -724,7 +723,7 @@ public class ServerManager {
// We sleep for some time
final long sleepTime = 50;
Thread.sleep(sleepTime);
now = EnvironmentEdgeManager.currentTimeMillis();
now = System.currentTimeMillis();
slept = now - startTime;
oldCount = count;

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.base.Joiner;
import com.google.common.collect.ArrayListMultimap;
@ -50,7 +49,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
// slop for regions
private float slop;
private Configuration config;
private static final Random RANDOM = new Random(EnvironmentEdgeManager.currentTimeMillis());
private static final Random RANDOM = new Random(System.currentTimeMillis());
private static final Log LOG = LogFactory.getLog(BaseLoadBalancer.class);
protected MasterServices services;

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.collect.MinMaxPriorityQueue;
@ -57,7 +56,7 @@ import com.google.common.collect.MinMaxPriorityQueue;
@InterfaceAudience.Private
public class DefaultLoadBalancer extends BaseLoadBalancer {
private static final Log LOG = LogFactory.getLog(DefaultLoadBalancer.class);
private static final Random RANDOM = new Random(EnvironmentEdgeManager.currentTimeMillis());
private static final Random RANDOM = new Random(System.currentTimeMillis());
private RegionInfoComparator riComparator = new RegionInfoComparator();
private RegionPlan.RegionPlanComparator rpComparator = new RegionPlan.RegionPlanComparator();
@ -182,7 +181,7 @@ public class DefaultLoadBalancer extends BaseLoadBalancer {
public List<RegionPlan> balanceCluster(
Map<ServerName, List<HRegionInfo>> clusterMap) {
boolean emptyRegionServerPresent = false;
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
ClusterLoadState cs = new ClusterLoadState(clusterMap);
@ -319,7 +318,7 @@ public class DefaultLoadBalancer extends BaseLoadBalancer {
// If none needed to fill all to min and none left to drain all to max,
// we are done
if (neededRegions == 0 && regionsToMove.isEmpty()) {
long endTime = EnvironmentEdgeManager.currentTimeMillis();
long endTime = System.currentTimeMillis();
LOG.info("Calculated a load balance in " + (endTime-startTime) + "ms. " +
"Moving " + totalNumMoved + " regions off of " +
serversOverloaded + " overloaded servers onto " +
@ -397,7 +396,7 @@ public class DefaultLoadBalancer extends BaseLoadBalancer {
}
}
long endTime = EnvironmentEdgeManager.currentTimeMillis();
long endTime = System.currentTimeMillis();
if (!regionsToMove.isEmpty() || neededRegions != 0) {
// Emit data so can diagnose how balancer went astray.

View File

@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import java.util.ArrayList;
import java.util.HashMap;
@ -107,7 +106,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
private static final String MAX_MOVES_KEY = "hbase.master.balancer.stochastic.maxMoveRegions";
private static final String KEEP_REGION_LOADS = "hbase.master.balancer.stochastic.numRegionLoadsToRemember";
private static final Random RANDOM = new Random(EnvironmentEdgeManager.currentTimeMillis());
private static final Random RANDOM = new Random(System.currentTimeMillis());
private static final Log LOG = LogFactory.getLog(StochasticLoadBalancer.class);
private final RegionLocationFinder regionFinder = new RegionLocationFinder();
private ClusterStatus clusterStatus = null;
@ -184,7 +183,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
return null;
}
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
// Keep track of servers to iterate through them.
List<ServerName> servers = new ArrayList<ServerName>(clusterState.keySet());
@ -249,7 +248,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
}
long endTime = EnvironmentEdgeManager.currentTimeMillis();
long endTime = System.currentTimeMillis();
if (initCost > currentCost) {
List<RegionPlan> plans = createRegionPlans(initialRegionMapping, clusterState);

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.commons.logging.Log;
@ -41,7 +40,7 @@ public class TimeToLiveLogCleaner extends BaseLogCleanerDelegate {
@Override
public boolean isLogDeletable(Path filePath) {
long time = 0;
long currentTime = EnvironmentEdgeManager.currentTimeMillis();
long currentTime = System.currentTimeMillis();
try {
FileStatus fStat = filePath.getFileSystem(this.getConf()).getFileStatus(filePath);
time = fStat.getModificationTime();

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.zookeeper.KeeperException;
@ -60,8 +59,8 @@ public class DeleteTableHandler extends TableEventHandler {
long waitTime = server.getConfiguration().
getLong("hbase.master.wait.on.region", 5 * 60 * 1000);
for (HRegionInfo region : regions) {
long done = EnvironmentEdgeManager.currentTimeMillis() + waitTime;
while (EnvironmentEdgeManager.currentTimeMillis() < done) {
long done = System.currentTimeMillis() + waitTime;
while (System.currentTimeMillis() < done) {
if (!am.getRegionStates().isRegionInTransition(region)) break;
Threads.sleep(waitingTimeForEvents);
LOG.debug("Waiting on region to clear regions in transition; "

View File

@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.zookeeper.KeeperException;
import org.cloudera.htrace.Trace;
@ -179,14 +178,14 @@ public class DisableTableHandler extends EventHandler {
@Override
protected boolean waitUntilDone(long timeout)
throws InterruptedException {
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
long remaining = timeout;
List<HRegionInfo> regions = null;
while (!server.isStopped() && remaining > 0) {
Thread.sleep(waitingTimeForEvents);
regions = assignmentManager.getRegionStates().getRegionsOfTable(tableName);
if (regions.isEmpty()) break;
remaining = timeout - (EnvironmentEdgeManager.currentTimeMillis() - startTime);
remaining = timeout - (System.currentTimeMillis() - startTime);
}
return regions != null && regions.isEmpty();
}

View File

@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.master.BulkAssigner;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.zookeeper.KeeperException;
import org.cloudera.htrace.Trace;
@ -220,7 +219,7 @@ public class EnableTableHandler extends EventHandler {
@Override
protected boolean waitUntilDone(long timeout)
throws InterruptedException {
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
long remaining = timeout;
List<HRegionInfo> regions = null;
int lastNumberOfRegions = 0;
@ -235,7 +234,7 @@ public class EnableTableHandler extends EventHandler {
lastNumberOfRegions = regions.size();
timeout += waitingTimeForEvents;
}
remaining = timeout - (EnvironmentEdgeManager.currentTimeMillis() - startTime);
remaining = timeout - (System.currentTimeMillis() - startTime);
}
return isDone(regions);
}

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.metrics;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.metrics.MetricsRecord;
import org.apache.hadoop.metrics.util.MetricsBase;
import org.apache.hadoop.metrics.util.MetricsRegistry;
@ -44,7 +43,7 @@ public class MetricsRate extends MetricsBase {
super(name, description);
this.value = 0;
this.prevRate = 0;
this.ts = EnvironmentEdgeManager.currentTimeMillis();
this.ts = System.currentTimeMillis();
registry.add(name, this);
}
@ -61,7 +60,7 @@ public class MetricsRate extends MetricsBase {
}
public synchronized void intervalHeartBeat() {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
long diff = (now-ts) / 1000;
if (diff < 1){
// To make sure our averages aren't skewed by fast repeated calls,

View File

@ -25,7 +25,6 @@ import java.util.LinkedList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
@ -55,7 +54,7 @@ public class MemoryBoundedLogMessageBuffer {
* older messages until the desired memory limit is achieved.
*/
public synchronized void add(String messageText) {
LogMessage message = new LogMessage(messageText, EnvironmentEdgeManager.currentTimeMillis());
LogMessage message = new LogMessage(messageText, System.currentTimeMillis());
usage += message.estimateHeapUsage();
messages.add(message);

View File

@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.client.Operation;
import org.apache.hadoop.hbase.io.WritableWithSize;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.io.Writable;
import org.codehaus.jackson.map.ObjectMapper;
@ -192,7 +191,7 @@ public class MonitoredRPCHandlerImpl extends MonitoredTaskImpl
long queueTime) {
this.methodName = methodName;
this.params = params;
this.rpcStartTime = EnvironmentEdgeManager.currentTimeMillis();
this.rpcStartTime = System.currentTimeMillis();
this.rpcQueueTime = queueTime;
this.state = State.RUNNING;
}

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.monitoring;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.codehaus.jackson.map.ObjectMapper;
import java.io.IOException;
@ -38,7 +37,7 @@ class MonitoredTaskImpl implements MonitoredTask {
protected volatile State state = State.RUNNING;
public MonitoredTaskImpl() {
startTime = EnvironmentEdgeManager.currentTimeMillis();
startTime = System.currentTimeMillis();
statusTime = startTime;
stateTime = startTime;
}
@ -117,12 +116,12 @@ class MonitoredTaskImpl implements MonitoredTask {
@Override
public void setStatus(String status) {
this.status = status;
statusTime = EnvironmentEdgeManager.currentTimeMillis();
statusTime = System.currentTimeMillis();
}
protected void setState(State state) {
this.state = state;
stateTime = EnvironmentEdgeManager.currentTimeMillis();
stateTime = System.currentTimeMillis();
}
@Override

View File

@ -30,7 +30,6 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
@ -141,12 +140,12 @@ public class TaskMonitor {
private boolean canPurge(MonitoredTask stat) {
long cts = stat.getCompletionTimestamp();
return (cts > 0 && EnvironmentEdgeManager.currentTimeMillis() - cts > EXPIRATION_TIME);
return (cts > 0 && System.currentTimeMillis() - cts > EXPIRATION_TIME);
}
public void dumpAsText(PrintWriter out) {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
List<MonitoredTask> tasks = getTasks();
for (MonitoredTask task : tasks) {

View File

@ -33,7 +33,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.base.Preconditions;
@ -84,7 +83,7 @@ public class CompactSplitThread implements CompactionRequestor {
@Override
public Thread newThread(Runnable r) {
Thread t = new Thread(r);
t.setName(n + "-largeCompactions-" + EnvironmentEdgeManager.currentTimeMillis());
t.setName(n + "-largeCompactions-" + System.currentTimeMillis());
return t;
}
});
@ -96,7 +95,7 @@ public class CompactSplitThread implements CompactionRequestor {
@Override
public Thread newThread(Runnable r) {
Thread t = new Thread(r);
t.setName(n + "-smallCompactions-" + EnvironmentEdgeManager.currentTimeMillis());
t.setName(n + "-smallCompactions-" + System.currentTimeMillis());
return t;
}
});
@ -108,7 +107,7 @@ public class CompactSplitThread implements CompactionRequestor {
@Override
public Thread newThread(Runnable r) {
Thread t = new Thread(r);
t.setName(n + "-splits-" + EnvironmentEdgeManager.currentTimeMillis());
t.setName(n + "-splits-" + System.currentTimeMillis());
return t;
}
});

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* State and utility processing {@link HRegion#getClosestRowBefore(byte[], byte[])}.
@ -73,7 +72,7 @@ class GetClosestRowBeforeTracker {
HConstants.DELIMITER) - this.rowoffset;
}
this.tablenamePlusDelimiterLength = metaregion? l + 1: -1;
this.oldestts = EnvironmentEdgeManager.currentTimeMillis() - ttl;
this.oldestts = System.currentTimeMillis() - ttl;
this.kvcomparator = c;
KeyValue.RowComparator rc = new KeyValue.RowComparator(this.kvcomparator);
this.deletes = new TreeMap<KeyValue, NavigableSet<KeyValue>>(rc);

View File

@ -522,7 +522,7 @@ public class HRegionServer implements ClientProtocol,
this.rpcServer.setErrorHandler(this);
this.rpcServer.setQosFunction((qosFunction = new QosFunction()));
this.startcode = EnvironmentEdgeManager.currentTimeMillis();
this.startcode = System.currentTimeMillis();
// login the server principal (if using secure Hadoop)
User.login(this.conf, "hbase.regionserver.keytab.file",
@ -885,11 +885,11 @@ public class HRegionServer implements ClientProtocol,
LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
}
}
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
if ((now - lastMsg) >= msgInterval) {
doMetrics();
tryRegionServerReport(lastMsg, now);
lastMsg = EnvironmentEdgeManager.currentTimeMillis();
lastMsg = System.currentTimeMillis();
}
if (!this.stopped) this.sleeper.sleep();
} // for
@ -1083,8 +1083,8 @@ public class HRegionServer implements ClientProtocol,
// Only print a message if the count of regions has changed.
if (count != lastCount) {
// Log every second at most
if (EnvironmentEdgeManager.currentTimeMillis() > (previousLogTime + 1000)) {
previousLogTime = EnvironmentEdgeManager.currentTimeMillis();
if (System.currentTimeMillis() > (previousLogTime + 1000)) {
previousLogTime = System.currentTimeMillis();
lastCount = count;
LOG.info("Waiting on " + count + " regions to close");
// Only print out regions still closing if a small number else will
@ -1928,7 +1928,7 @@ public class HRegionServer implements ClientProtocol,
return null;
}
LOG.debug("No master found; retry");
previousLogTime = EnvironmentEdgeManager.currentTimeMillis();
previousLogTime = System.currentTimeMillis();
refresh = true; // let's try pull it from ZK directly
sleeper.sleep();
@ -1952,14 +1952,14 @@ public class HRegionServer implements ClientProtocol,
e = e instanceof RemoteException ?
((RemoteException)e).unwrapRemoteException() : e;
if (e instanceof ServerNotRunningYetException) {
if (EnvironmentEdgeManager.currentTimeMillis() > (previousLogTime+1000)){
if (System.currentTimeMillis() > (previousLogTime+1000)){
LOG.info("Master isn't available yet, retrying");
previousLogTime = EnvironmentEdgeManager.currentTimeMillis();
previousLogTime = System.currentTimeMillis();
}
} else {
if (EnvironmentEdgeManager.currentTimeMillis() > (previousLogTime + 1000)) {
if (System.currentTimeMillis() > (previousLogTime + 1000)) {
LOG.warn("Unable to connect to master. Retrying. Error was:", e);
previousLogTime = EnvironmentEdgeManager.currentTimeMillis();
previousLogTime = System.currentTimeMillis();
}
}
try {
@ -4043,7 +4043,7 @@ public class HRegionServer implements ClientProtocol,
}
protected void addToMovedRegions(String encodedName, ServerName destination){
final Long time = EnvironmentEdgeManager.currentTimeMillis();
final Long time = System.currentTimeMillis();
movedRegions.put(
encodedName,
@ -4054,7 +4054,7 @@ public class HRegionServer implements ClientProtocol,
Pair<Long, ServerName> dest = movedRegions.get(encodedRegionName);
if (dest != null) {
if (dest.getFirst() > (EnvironmentEdgeManager.currentTimeMillis() - TIMEOUT_REGION_MOVED)) {
if (dest.getFirst() > (System.currentTimeMillis() - TIMEOUT_REGION_MOVED)) {
return dest.getSecond();
} else {
movedRegions.remove(encodedRegionName);
@ -4068,7 +4068,7 @@ public class HRegionServer implements ClientProtocol,
* Remove the expired entries from the moved regions list.
*/
protected void cleanMovedRegions(){
final long cutOff = EnvironmentEdgeManager.currentTimeMillis() - TIMEOUT_REGION_MOVED;
final long cutOff = System.currentTimeMillis() - TIMEOUT_REGION_MOVED;
Iterator<Entry<String, Pair<Long, ServerName>>> it = movedRegions.entrySet().iterator();
while (it.hasNext()){

View File

@ -1178,7 +1178,7 @@ public class HStore extends SchemaConfigured implements Store {
}
// TODO: Use better method for determining stamp of last major (HBASE-2990)
long lowTimestamp = getLowestTimestamp(filesToCompact);
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) {
// Major compaction time has elapsed.
if (filesToCompact.size() == 1) {

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.regionserver;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import java.util.ConcurrentModificationException;
@ -279,7 +278,7 @@ public class Leases extends HasThread {
}
public long getDelay(TimeUnit unit) {
return unit.convert(this.expirationTime - EnvironmentEdgeManager.currentTimeMillis(),
return unit.convert(this.expirationTime - System.currentTimeMillis(),
TimeUnit.MILLISECONDS);
}
@ -294,7 +293,7 @@ public class Leases extends HasThread {
* Resets the expiration time of the lease.
*/
public void resetExpirationTime() {
this.expirationTime = EnvironmentEdgeManager.currentTimeMillis() + this.leaseTimeoutPeriod;
this.expirationTime = System.currentTimeMillis() + this.leaseTimeoutPeriod;
}
}
}

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import java.io.IOException;
@ -49,7 +48,7 @@ class LogRoller extends HasThread implements WALActionsListener {
private final AtomicBoolean rollLog = new AtomicBoolean(false);
private final Server server;
private final RegionServerServices services;
private volatile long lastrolltime = EnvironmentEdgeManager.currentTimeMillis();
private volatile long lastrolltime = System.currentTimeMillis();
// Period to roll log.
private final long rollperiod;
private final int threadWakeFrequency;
@ -68,7 +67,7 @@ class LogRoller extends HasThread implements WALActionsListener {
@Override
public void run() {
while (!server.isStopped()) {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
boolean periodic = false;
if (!rollLog.get()) {
periodic = (now - this.lastrolltime) > this.rollperiod;

View File

@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.util.StringUtils;
@ -347,7 +346,7 @@ class MemStoreFlusher extends HasThread implements FlushRequester {
if (!fqe.region.getRegionInfo().isMetaRegion() &&
isTooManyStoreFiles(region)) {
if (fqe.isMaximumWait(this.blockingWaitTime)) {
LOG.info("Waited " + (EnvironmentEdgeManager.currentTimeMillis() - fqe.createTime) +
LOG.info("Waited " + (System.currentTimeMillis() - fqe.createTime) +
"ms on a compaction to clean up 'too many store files'; waited " +
"long enough... proceeding with flush of " +
region.getRegionNameAsString());
@ -526,7 +525,7 @@ class MemStoreFlusher extends HasThread implements FlushRequester {
FlushRegionEntry(final HRegion r) {
this.region = r;
this.createTime = EnvironmentEdgeManager.currentTimeMillis();
this.createTime = System.currentTimeMillis();
this.whenToExpire = this.createTime;
}
@ -535,7 +534,7 @@ class MemStoreFlusher extends HasThread implements FlushRequester {
* @return True if we have been delayed > <code>maximumWait</code> milliseconds.
*/
public boolean isMaximumWait(final long maximumWait) {
return (EnvironmentEdgeManager.currentTimeMillis() - this.createTime) > maximumWait;
return (System.currentTimeMillis() - this.createTime) > maximumWait;
}
/**
@ -548,19 +547,19 @@ class MemStoreFlusher extends HasThread implements FlushRequester {
/**
* @param when When to expire, when to come up out of the queue.
* Specify in milliseconds. This method adds EnvironmentEdgeManager.currentTimeMillis()
* Specify in milliseconds. This method adds System.currentTimeMillis()
* to whatever you pass.
* @return This.
*/
public FlushRegionEntry requeue(final long when) {
this.whenToExpire = EnvironmentEdgeManager.currentTimeMillis() + when;
this.whenToExpire = System.currentTimeMillis() + when;
this.requeueCount++;
return this;
}
@Override
public long getDelay(TimeUnit unit) {
return unit.convert(this.whenToExpire - EnvironmentEdgeManager.currentTimeMillis(),
return unit.convert(this.whenToExpire - System.currentTimeMillis(),
TimeUnit.MILLISECONDS);
}

View File

@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.SplitLogTask;
import org.apache.hadoop.hbase.master.SplitLogManager;
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -270,7 +269,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
SplitLogCounters.tot_wkr_task_acquired.incrementAndGet();
getDataSetWatchAsync();
t = EnvironmentEdgeManager.currentTimeMillis();
t = System.currentTimeMillis();
TaskExecutor.Status status;
status = splitTaskExecutor.exec(ZKSplitLog.getFileName(currentTask),
@ -315,7 +314,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
} finally {
if (t > 0) {
LOG.info("worker " + serverName + " done with task " + path +
" in " + (EnvironmentEdgeManager.currentTimeMillis() - t) + "ms");
" in " + (System.currentTimeMillis() - t) + "ms");
}
synchronized (grabTaskLock) {
workerInGrabTask = false;

View File

@ -25,7 +25,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.util.StringUtils;
import com.google.common.base.Preconditions;
@ -60,14 +59,14 @@ class SplitRequest implements Runnable {
return;
}
try {
final long startTime = EnvironmentEdgeManager.currentTimeMillis();
final long startTime = System.currentTimeMillis();
SplitTransaction st = new SplitTransaction(parent, midKey);
// If prepare does not return true, for some reason -- logged inside in
// the prepare call -- we are not ready to split just now. Just return.
if (!st.prepare()) return;
try {
st.execute(this.server, this.server);
this.server.getMetrics().incrementSplitSuccessCount(EnvironmentEdgeManager.currentTimeMillis() - startTime);
this.server.getMetrics().incrementSplitSuccessCount(System.currentTimeMillis() - startTime);
} catch (Exception e) {
if (this.server.isStopping() || this.server.isStopped()) {
LOG.info(
@ -99,7 +98,7 @@ class SplitRequest implements Runnable {
+ parent.getRegionInfo().getRegionNameAsString() + ", new regions: "
+ st.getFirstDaughter().getRegionNameAsString() + ", "
+ st.getSecondDaughter().getRegionNameAsString() + ". Split took "
+ StringUtils.formatTimeDiff(EnvironmentEdgeManager.currentTimeMillis(), startTime));
+ StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
} catch (IOException ex) {
LOG.error("Split failed " + this, RemoteExceptionHandler
.checkIOException(ex));

View File

@ -529,7 +529,7 @@ public class SplitTransaction {
@Override
public boolean progress() {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
if (now - lastLog > this.interval) {
LOG.info("Opening " + this.hri.getRegionNameAsString());
this.lastLog = now;

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.zookeeper.KeeperException;
@ -169,7 +168,7 @@ public class OpenRegionHandler extends EventHandler {
// Total timeout for meta edit. If we fail adding the edit then close out
// the region and let it be assigned elsewhere.
long timeout = assignmentTimeout * 10;
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
long endTime = now + timeout;
// Let our period at which we update OPENING state to be be 1/3rd of the
// regions-in-transition timeout period.
@ -191,7 +190,7 @@ public class OpenRegionHandler extends EventHandler {
// Go to the loop check.
}
}
now = EnvironmentEdgeManager.currentTimeMillis();
now = System.currentTimeMillis();
}
// Is thread still alive? We may have left above loop because server is
// stopping or we timed out the edit. Is so, interrupt it.

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.metrics.histogram.MetricsHistogram;
import org.apache.hadoop.hbase.metrics.PersistentMetricsTimeVaryingRate;
import com.yammer.metrics.stats.Snapshot;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.hadoop.metrics.ContextFactory;
@ -62,8 +61,8 @@ public class RegionServerMetrics implements Updater {
@SuppressWarnings({"FieldCanBeLocal"})
private final Log LOG = LogFactory.getLog(this.getClass());
private final MetricsRecord metricsRecord;
private long lastUpdate = EnvironmentEdgeManager.currentTimeMillis();
private long lastExtUpdate = EnvironmentEdgeManager.currentTimeMillis();
private long lastUpdate = System.currentTimeMillis();
private long lastExtUpdate = System.currentTimeMillis();
private long extendedPeriod = 0;
private static final int MB = 1024*1024;
private MetricsRegistry registry = new MetricsRegistry();
@ -350,7 +349,7 @@ public class RegionServerMetrics implements Updater {
*/
public void doUpdates(MetricsContext caller) {
synchronized (this) {
this.lastUpdate = EnvironmentEdgeManager.currentTimeMillis();
this.lastUpdate = System.currentTimeMillis();
// has the extended period for long-living stats elapsed?
if (this.extendedPeriod > 0 &&

View File

@ -67,7 +67,6 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.Threads;
@ -621,7 +620,7 @@ public class HLog implements Syncable {
if (currentFilenum > 0) {
oldPath = computeFilename(currentFilenum);
}
this.filenum = EnvironmentEdgeManager.currentTimeMillis();
this.filenum = System.currentTimeMillis();
Path newPath = computeFilename();
// Tell our listeners that a new log is about to be created
@ -1299,7 +1298,7 @@ public class HLog implements Syncable {
}
try {
long doneUpto;
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
// First flush all the pending writes to HDFS. Then
// issue the sync to HDFS. If sync is successful, then update
// syncedTillHere to indicate that transactions till this
@ -1335,7 +1334,7 @@ public class HLog implements Syncable {
}
this.syncedTillHere = Math.max(this.syncedTillHere, doneUpto);
syncTime.inc(EnvironmentEdgeManager.currentTimeMillis() - now);
syncTime.inc(System.currentTimeMillis() - now);
if (!this.logRollRunning) {
checkLowReplication();
try {
@ -1462,13 +1461,13 @@ public class HLog implements Syncable {
}
}
try {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
// coprocessor hook:
if (!coprocessorHost.preWALWrite(info, logKey, logEdit)) {
// write to our buffer for the Hlog file.
logSyncerThread.append(new HLog.Entry(logKey, logEdit));
}
long took = EnvironmentEdgeManager.currentTimeMillis() - now;
long took = System.currentTimeMillis() - now;
coprocessorHost.postWALWrite(info, logKey, logEdit);
writeTime.inc(took);
long len = 0;
@ -1591,13 +1590,13 @@ public class HLog implements Syncable {
}
long txid = 0;
synchronized (updateLock) {
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
WALEdit edit = completeCacheFlushLogEdit();
HLogKey key = makeKey(encodedRegionName, tableName, logSeqId,
EnvironmentEdgeManager.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID);
System.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID);
logSyncerThread.append(new Entry(key, edit));
txid = this.unflushedEntries.incrementAndGet();
writeTime.inc(EnvironmentEdgeManager.currentTimeMillis() - now);
writeTime.inc(System.currentTimeMillis() - now);
long len = 0;
for (KeyValue kv : edit.getKeyValues()) {
len += kv.getLength();
@ -1619,7 +1618,7 @@ public class HLog implements Syncable {
private WALEdit completeCacheFlushLogEdit() {
KeyValue kv = new KeyValue(METAROW, METAFAMILY, null,
EnvironmentEdgeManager.currentTimeMillis(), COMPLETE_CACHE_FLUSH);
System.currentTimeMillis(), COMPLETE_CACHE_FLUSH);
WALEdit e = new WALEdit();
e.add(kv);
return e;
@ -1902,7 +1901,7 @@ public class HLog implements Syncable {
final Path edits)
throws IOException {
Path moveAsideName = new Path(edits.getParent(), edits.getName() + "." +
EnvironmentEdgeManager.currentTimeMillis());
System.currentTimeMillis());
if (!fs.rename(edits, moveAsideName)) {
LOG.warn("Rename failed from " + edits + " to " + moveAsideName);
}

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.replication.regionserver.metrics;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* This class is for maintaining the various replication statistics for a sink and publishing them
@ -45,7 +44,7 @@ public class ReplicationSinkMetrics {
* @param timestamp The timestamp of the last operation applied.
*/
public void setAgeOfLastAppliedOp(long timestamp) {
long age = EnvironmentEdgeManager.currentTimeMillis() - timestamp;
long age = System.currentTimeMillis() - timestamp;
rms.setGauge(SINK_AGE_OF_LAST_APPLIED_OP, age);
}

View File

@ -22,7 +22,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* This class is for maintaining the various replication statistics for a source and publishing them
@ -76,7 +75,7 @@ public class ReplicationSourceMetrics {
* @param timestamp write time of the edit
*/
public void setAgeOfLastShippedOp(long timestamp) {
long age = EnvironmentEdgeManager.currentTimeMillis() - timestamp;
long age = System.currentTimeMillis() - timestamp;
rms.setGauge(ageOfLastShippedOpKey, age);
rms.setGauge(SOURCE_AGE_OF_LAST_SHIPPED_OP, age);
this.lastTimestamp = timestamp;

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.rest.model.ScannerModel;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.util.StringUtils;
@InterfaceAudience.Private
@ -89,7 +88,7 @@ public class ScannerResultGenerator extends ResultGenerator {
scan.setCacheBlocks(false);
scanner = table.getScanner(scan);
cached = null;
id = Long.toString(EnvironmentEdgeManager.currentTimeMillis()) +
id = Long.toString(System.currentTimeMillis()) +
Integer.toHexString(scanner.hashCode());
} finally {
table.close();

View File

@ -42,7 +42,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* A wrapper around HttpClient which provides some useful function and
@ -187,9 +186,9 @@ public class Client {
method.addRequestHeader(header);
}
}
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
int code = httpClient.executeMethod(method);
long endTime = EnvironmentEdgeManager.currentTimeMillis();
long endTime = System.currentTimeMillis();
if (LOG.isDebugEnabled()) {
LOG.debug(method.getName() + " " + uri + " " + code + " " +
method.getStatusText() + " in " + (endTime - startTime) + " ms");

View File

@ -30,7 +30,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.thrift.CallQueue.Call;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.thrift.TException;
import org.apache.thrift.TProcessor;
@ -220,7 +219,7 @@ public class TBoundedThreadPoolServer extends TServer {
long msLeftToWait =
serverOptions.stopTimeoutUnit.toMillis(serverOptions.stopTimeoutVal);
long timeMillis = EnvironmentEdgeManager.currentTimeMillis();
long timeMillis = System.currentTimeMillis();
LOG.info("Waiting for up to " + msLeftToWait + " ms to finish processing" +
" pending requests");
@ -230,7 +229,7 @@ public class TBoundedThreadPoolServer extends TServer {
executorService.awaitTermination(msLeftToWait, TimeUnit.MILLISECONDS);
break;
} catch (InterruptedException ix) {
long timePassed = EnvironmentEdgeManager.currentTimeMillis() - timeMillis;
long timePassed = System.currentTimeMillis() - timeMillis;
msLeftToWait -= timePassed;
timeMillis += timePassed;
interrupted = true;

View File

@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* HBase Canary Tool, that that can be used to do
@ -235,9 +234,9 @@ public final class Canary implements Tool {
get.addFamily(column.getName());
try {
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
table.get(get);
long time = EnvironmentEdgeManager.currentTimeMillis() - startTime;
long time = System.currentTimeMillis() - startTime;
sink.publishReadTiming(region, column, time);
} catch (Exception e) {

View File

@ -68,7 +68,7 @@ public class FSHDFSUtils extends FSUtils{
return;
}
LOG.info("Recovering file " + p);
long startWaiting = EnvironmentEdgeManager.currentTimeMillis();
long startWaiting = System.currentTimeMillis();
// Trying recovery
boolean recovered = false;
@ -95,7 +95,7 @@ public class FSHDFSUtils extends FSUtils{
// within its soft limit, but if we get it past that, it means
// that the RS is holding onto the file even though it lost its
// znode. We could potentially abort after some time here.
long waitedFor = EnvironmentEdgeManager.currentTimeMillis() - startWaiting;
long waitedFor = System.currentTimeMillis() - startWaiting;
if (waitedFor > LEASE_SOFTLIMIT_PERIOD) {
LOG.warn("Waited " + waitedFor + "ms for lease recovery on " + p +
":" + e.getMessage());

View File

@ -538,7 +538,7 @@ public abstract class FSUtils {
throws IOException {
// Rewrite the file as pb. Move aside the old one first, write new
// then delete the moved-aside file.
Path movedAsideName = new Path(p + "." + EnvironmentEdgeManager.currentTimeMillis());
Path movedAsideName = new Path(p + "." + System.currentTimeMillis());
if (!fs.rename(p, movedAsideName)) throw new IOException("Failed rename of " + p);
setClusterId(fs, rootdir, cid, 100);
if (!fs.delete(movedAsideName, false)) {

View File

@ -170,7 +170,7 @@ public class HBaseFsck {
private HBaseAdmin admin;
private HTable meta;
protected ExecutorService executor; // threads to retrieve data from regionservers
private long startMillis = EnvironmentEdgeManager.currentTimeMillis();
private long startMillis = System.currentTimeMillis();
private HFileCorruptionChecker hfcc;
private int retcode = 0;
@ -1255,7 +1255,7 @@ public class HBaseFsck {
throw new IOException(e);
}
MetaEntry m =
new MetaEntry(rootLocation.getRegionInfo(), sn, EnvironmentEdgeManager.currentTimeMillis());
new MetaEntry(rootLocation.getRegionInfo(), sn, System.currentTimeMillis());
HbckInfo hbInfo = new HbckInfo(m);
regionInfoMap.put(rootLocation.getRegionInfo().getEncodedName(), hbInfo);
return true;
@ -1508,7 +1508,7 @@ public class HBaseFsck {
(hbi.metaEntry == null)? false: hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline();
boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry);
boolean recentlyModified = hbi.getHdfsRegionDir() != null &&
hbi.getModTime() + timelag > EnvironmentEdgeManager.currentTimeMillis();
hbi.getModTime() + timelag > System.currentTimeMillis();
// ========== First the healthy cases =============
if (hbi.containsOnlyHdfsEdits()) {
@ -2321,7 +2321,7 @@ public class HBaseFsck {
*/
HTableDescriptor[] getTables(AtomicInteger numSkipped) {
List<String> tableNames = new ArrayList<String>();
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
for (HbckInfo hbi : regionInfoMap.values()) {
MetaEntry info = hbi.metaEntry;

View File

@ -119,8 +119,8 @@ public class HBaseFsckRepair {
public static void waitUntilAssigned(HBaseAdmin admin,
HRegionInfo region) throws IOException, InterruptedException {
long timeout = admin.getConfiguration().getLong("hbase.hbck.assign.timeout", 120000);
long expiration = timeout + EnvironmentEdgeManager.currentTimeMillis();
while (EnvironmentEdgeManager.currentTimeMillis() < expiration) {
long expiration = timeout + System.currentTimeMillis();
while (System.currentTimeMillis() < expiration) {
try {
Map<String, RegionState> rits=
admin.getClusterStatus().getRegionsInTransition();
@ -153,8 +153,8 @@ public class HBaseFsckRepair {
ProtobufUtil.closeRegion(rs, region.getRegionName(), false);
long timeout = admin.getConfiguration()
.getLong("hbase.hbck.close.timeout", 120000);
long expiration = timeout + EnvironmentEdgeManager.currentTimeMillis();
while (EnvironmentEdgeManager.currentTimeMillis() < expiration) {
long expiration = timeout + System.currentTimeMillis();
while (System.currentTimeMillis() < expiration) {
try {
HRegionInfo rsRegion =
ProtobufUtil.getRegionInfo(rs, region.getRegionName());

View File

@ -155,7 +155,7 @@ class HMerge {
Bytes.toString(tableName)
);
this.htd = FSTableDescriptors.getTableDescriptor(this.fs, this.tabledir);
Path logdir = new Path(tabledir, "merge_" + EnvironmentEdgeManager.currentTimeMillis() +
Path logdir = new Path(tabledir, "merge_" + System.currentTimeMillis() +
HConstants.HREGION_LOGDIR_NAME);
Path oldLogDir = new Path(tabledir, HConstants.HREGION_OLDLOGDIR_NAME);
this.hlog = new HLog(fs, logdir, oldLogDir, conf);

View File

@ -175,13 +175,13 @@ public class JVMClusterUtil {
// Wait for an active master
// having an active master before starting the region threads allows
// then to succeed on their connection to master
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
while (findActiveMaster(masters) == null) {
try {
Thread.sleep(100);
} catch (InterruptedException ignored) {
}
if (EnvironmentEdgeManager.currentTimeMillis() > startTime + 30000) {
if (System.currentTimeMillis() > startTime + 30000) {
throw new RuntimeException("Master not active after 30 seconds");
}
}
@ -197,13 +197,13 @@ public class JVMClusterUtil {
// Wait for an active master to be initialized (implies being master)
// with this, when we return the cluster is complete
startTime = EnvironmentEdgeManager.currentTimeMillis();
startTime = System.currentTimeMillis();
while (true) {
JVMClusterUtil.MasterThread t = findActiveMaster(masters);
if (t != null && t.master.isInitialized()) {
return t.master.getServerName().toString();
}
if (EnvironmentEdgeManager.currentTimeMillis() > startTime + 200000) {
if (System.currentTimeMillis() > startTime + 200000) {
throw new RuntimeException("Master not initialized after 200 seconds");
}
try {

View File

@ -335,7 +335,7 @@ public class Merge extends Configured implements Tool {
}
Delete delete = new Delete(regioninfo.getRegionName(),
EnvironmentEdgeManager.currentTimeMillis(), null);
System.currentTimeMillis(), null);
meta.delete(delete, null, true);
}

View File

@ -98,7 +98,7 @@ public class MetaUtils {
public synchronized HLog getLog() throws IOException {
if (this.log == null) {
Path logdir = new Path(this.fs.getHomeDirectory(),
HConstants.HREGION_LOGDIR_NAME + "_" + EnvironmentEdgeManager.currentTimeMillis());
HConstants.HREGION_LOGDIR_NAME + "_" + System.currentTimeMillis());
Path oldLogDir = new Path(this.fs.getHomeDirectory(),
HConstants.HREGION_OLDLOGDIR_NAME);
this.log = new HLog(this.fs, logdir, oldLogDir, this.conf);

View File

@ -426,7 +426,7 @@ public class RegionSplitter {
daughterRegions.get(rsLocation).add(dr);
}
LOG.debug("Done with bucketing. Split time!");
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
// open the split file and modify it as splits finish
FSDataInputStream tmpIn = fs.open(splitFile);
@ -544,7 +544,7 @@ public class RegionSplitter {
+ " " + splitAlgo.rowToStr(region.getSecond()) + "\n");
splitCount++;
if (splitCount % 10 == 0) {
long tDiff = (EnvironmentEdgeManager.currentTimeMillis() - startTime)
long tDiff = (System.currentTimeMillis() - startTime)
/ splitCount;
LOG.debug("STATUS UPDATE: " + splitCount + " / " + origCount
+ ". Avg Time / Split = "
@ -570,7 +570,7 @@ public class RegionSplitter {
}
LOG.debug("All regions have been successfully split!");
} finally {
long tDiff = EnvironmentEdgeManager.currentTimeMillis() - startTime;
long tDiff = System.currentTimeMillis() - startTime;
LOG.debug("TOTAL TIME = "
+ org.apache.hadoop.util.StringUtils.formatTime(tDiff));
LOG.debug("Splits = " + splitCount);

View File

@ -55,7 +55,7 @@ public class Sleeper {
* Sleep for period.
*/
public void sleep() {
sleep(EnvironmentEdgeManager.currentTimeMillis());
sleep(System.currentTimeMillis());
}
/**
@ -78,11 +78,11 @@ public class Sleeper {
if (this.stopper.isStopped()) {
return;
}
long now = EnvironmentEdgeManager.currentTimeMillis();
long now = System.currentTimeMillis();
long waitTime = this.period - (now - startTime);
if (waitTime > this.period) {
LOG.warn("Calculated wait time > " + this.period +
"; setting to this.period: " + EnvironmentEdgeManager.currentTimeMillis() + ", " +
"; setting to this.period: " + System.currentTimeMillis() + ", " +
startTime);
waitTime = this.period;
}
@ -93,7 +93,7 @@ public class Sleeper {
if (triggerWake) break;
sleepLock.wait(waitTime);
}
woke = EnvironmentEdgeManager.currentTimeMillis();
woke = System.currentTimeMillis();
long slept = woke - now;
if (slept - this.period > MINIMAL_DELTA_FOR_LOGGING) {
LOG.warn("We slept " + slept + "ms instead of " + this.period +
@ -109,7 +109,7 @@ public class Sleeper {
}
}
// Recalculate waitTime.
woke = (woke == -1)? EnvironmentEdgeManager.currentTimeMillis(): woke;
woke = (woke == -1)? System.currentTimeMillis(): woke;
waitTime = this.period - (woke - startTime);
}
triggerWake = false;

View File

@ -38,7 +38,6 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.zookeeper.server.NIOServerCnxnFactory;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.persistence.FileTxnLog;
@ -311,7 +310,7 @@ public class MiniZooKeeperCluster {
// XXX: From o.a.zk.t.ClientBase
private static boolean waitForServerDown(int port, long timeout) {
long start = EnvironmentEdgeManager.currentTimeMillis();
long start = System.currentTimeMillis();
while (true) {
try {
Socket sock = new Socket("localhost", port);
@ -326,7 +325,7 @@ public class MiniZooKeeperCluster {
return true;
}
if (EnvironmentEdgeManager.currentTimeMillis() > start + timeout) {
if (System.currentTimeMillis() > start + timeout) {
break;
}
try {
@ -340,7 +339,7 @@ public class MiniZooKeeperCluster {
// XXX: From o.a.zk.t.ClientBase
private static boolean waitForServerUp(int port, long timeout) {
long start = EnvironmentEdgeManager.currentTimeMillis();
long start = System.currentTimeMillis();
while (true) {
try {
Socket sock = new Socket("localhost", port);
@ -367,7 +366,7 @@ public class MiniZooKeeperCluster {
LOG.info("server localhost:" + port + " not up " + e);
}
if (EnvironmentEdgeManager.currentTimeMillis() > start + timeout) {
if (System.currentTimeMillis() > start + timeout) {
break;
}
try {

View File

@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.DeserializationException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
@ -1212,7 +1211,7 @@ public class ZKUtil {
byte[] data = null;
boolean finished = false;
final long endTime = EnvironmentEdgeManager.currentTimeMillis() + timeout;
final long endTime = System.currentTimeMillis() + timeout;
while (!finished) {
try {
data = ZKUtil.getData(zkw, znode);
@ -1220,7 +1219,7 @@ public class ZKUtil {
LOG.warn("Unexpected exception handling blockUntilAvailable", e);
}
if (data == null && (EnvironmentEdgeManager.currentTimeMillis() +
if (data == null && (System.currentTimeMillis() +
HConstants.SOCKET_RETRY_WAIT_MS < endTime)) {
Thread.sleep(HConstants.SOCKET_RETRY_WAIT_MS);
} else {

View File

@ -23,7 +23,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.zookeeper.KeeperException;
/**
@ -127,7 +126,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
throws InterruptedException {
if (timeout < 0) throw new IllegalArgumentException();
boolean notimeout = timeout == 0;
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long startTime = System.currentTimeMillis();
long remaining = timeout;
if (refresh) {
try {
@ -166,7 +165,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
// We expect a notification; but we wait with a
// a timeout to lower the impact of a race condition if any
wait(100);
remaining = timeout - (EnvironmentEdgeManager.currentTimeMillis() - startTime);
remaining = timeout - (System.currentTimeMillis() - startTime);
}
return this.data;
}

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
@ -325,9 +324,9 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
case SyncConnected:
// Now, this callback can be invoked before the this.zookeeper is set.
// Wait a little while.
long finished = EnvironmentEdgeManager.currentTimeMillis() +
long finished = System.currentTimeMillis() +
this.conf.getLong("hbase.zookeeper.watcher.sync.connected.wait", 2000);
while (EnvironmentEdgeManager.currentTimeMillis() < finished) {
while (System.currentTimeMillis() < finished) {
Threads.sleep(1);
if (this.recoverableZooKeeper != null) break;
}