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

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1391699 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-09-28 23:50:31 +00:00
parent db62a5eb0d
commit 040f76eb2c
72 changed files with 269 additions and 213 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator; import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/** /**
* State and utility processing {@link HRegion#getClosestRowBefore(byte[], byte[])}. * State and utility processing {@link HRegion#getClosestRowBefore(byte[], byte[])}.
@ -72,7 +73,7 @@ class GetClosestRowBeforeTracker {
HConstants.DELIMITER) - this.rowoffset; HConstants.DELIMITER) - this.rowoffset;
} }
this.tablenamePlusDelimiterLength = metaregion? l + 1: -1; this.tablenamePlusDelimiterLength = metaregion? l + 1: -1;
this.oldestts = System.currentTimeMillis() - ttl; this.oldestts = EnvironmentEdgeManager.currentTimeMillis() - ttl;
this.kvcomparator = c; this.kvcomparator = c;
KeyValue.RowComparator rc = new KeyValue.RowComparator(this.kvcomparator); KeyValue.RowComparator rc = new KeyValue.RowComparator(this.kvcomparator);
this.deletes = new TreeMap<KeyValue, NavigableSet<KeyValue>>(rc); 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.setErrorHandler(this);
this.rpcServer.setQosFunction((qosFunction = new QosFunction())); this.rpcServer.setQosFunction((qosFunction = new QosFunction()));
this.startcode = System.currentTimeMillis(); this.startcode = EnvironmentEdgeManager.currentTimeMillis();
// login the server principal (if using secure Hadoop) // login the server principal (if using secure Hadoop)
User.login(this.conf, "hbase.regionserver.keytab.file", User.login(this.conf, "hbase.regionserver.keytab.file",
@ -885,11 +885,11 @@ public class HRegionServer implements ClientProtocol,
LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString()); LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
} }
} }
long now = System.currentTimeMillis(); long now = EnvironmentEdgeManager.currentTimeMillis();
if ((now - lastMsg) >= msgInterval) { if ((now - lastMsg) >= msgInterval) {
doMetrics(); doMetrics();
tryRegionServerReport(lastMsg, now); tryRegionServerReport(lastMsg, now);
lastMsg = System.currentTimeMillis(); lastMsg = EnvironmentEdgeManager.currentTimeMillis();
} }
if (!this.stopped) this.sleeper.sleep(); if (!this.stopped) this.sleeper.sleep();
} // for } // for
@ -1083,8 +1083,8 @@ public class HRegionServer implements ClientProtocol,
// Only print a message if the count of regions has changed. // Only print a message if the count of regions has changed.
if (count != lastCount) { if (count != lastCount) {
// Log every second at most // Log every second at most
if (System.currentTimeMillis() > (previousLogTime + 1000)) { if (EnvironmentEdgeManager.currentTimeMillis() > (previousLogTime + 1000)) {
previousLogTime = System.currentTimeMillis(); previousLogTime = EnvironmentEdgeManager.currentTimeMillis();
lastCount = count; lastCount = count;
LOG.info("Waiting on " + count + " regions to close"); LOG.info("Waiting on " + count + " regions to close");
// Only print out regions still closing if a small number else will // Only print out regions still closing if a small number else will
@ -1928,7 +1928,7 @@ public class HRegionServer implements ClientProtocol,
return null; return null;
} }
LOG.debug("No master found; retry"); LOG.debug("No master found; retry");
previousLogTime = System.currentTimeMillis(); previousLogTime = EnvironmentEdgeManager.currentTimeMillis();
refresh = true; // let's try pull it from ZK directly refresh = true; // let's try pull it from ZK directly
sleeper.sleep(); sleeper.sleep();
@ -1952,14 +1952,14 @@ public class HRegionServer implements ClientProtocol,
e = e instanceof RemoteException ? e = e instanceof RemoteException ?
((RemoteException)e).unwrapRemoteException() : e; ((RemoteException)e).unwrapRemoteException() : e;
if (e instanceof ServerNotRunningYetException) { if (e instanceof ServerNotRunningYetException) {
if (System.currentTimeMillis() > (previousLogTime+1000)){ if (EnvironmentEdgeManager.currentTimeMillis() > (previousLogTime+1000)){
LOG.info("Master isn't available yet, retrying"); LOG.info("Master isn't available yet, retrying");
previousLogTime = System.currentTimeMillis(); previousLogTime = EnvironmentEdgeManager.currentTimeMillis();
} }
} else { } else {
if (System.currentTimeMillis() > (previousLogTime + 1000)) { if (EnvironmentEdgeManager.currentTimeMillis() > (previousLogTime + 1000)) {
LOG.warn("Unable to connect to master. Retrying. Error was:", e); LOG.warn("Unable to connect to master. Retrying. Error was:", e);
previousLogTime = System.currentTimeMillis(); previousLogTime = EnvironmentEdgeManager.currentTimeMillis();
} }
} }
try { try {
@ -4043,7 +4043,7 @@ public class HRegionServer implements ClientProtocol,
} }
protected void addToMovedRegions(String encodedName, ServerName destination){ protected void addToMovedRegions(String encodedName, ServerName destination){
final Long time = System.currentTimeMillis(); final Long time = EnvironmentEdgeManager.currentTimeMillis();
movedRegions.put( movedRegions.put(
encodedName, encodedName,
@ -4054,7 +4054,7 @@ public class HRegionServer implements ClientProtocol,
Pair<Long, ServerName> dest = movedRegions.get(encodedRegionName); Pair<Long, ServerName> dest = movedRegions.get(encodedRegionName);
if (dest != null) { if (dest != null) {
if (dest.getFirst() > (System.currentTimeMillis() - TIMEOUT_REGION_MOVED)) { if (dest.getFirst() > (EnvironmentEdgeManager.currentTimeMillis() - TIMEOUT_REGION_MOVED)) {
return dest.getSecond(); return dest.getSecond();
} else { } else {
movedRegions.remove(encodedRegionName); movedRegions.remove(encodedRegionName);
@ -4068,7 +4068,7 @@ public class HRegionServer implements ClientProtocol,
* Remove the expired entries from the moved regions list. * Remove the expired entries from the moved regions list.
*/ */
protected void cleanMovedRegions(){ protected void cleanMovedRegions(){
final long cutOff = System.currentTimeMillis() - TIMEOUT_REGION_MOVED; final long cutOff = EnvironmentEdgeManager.currentTimeMillis() - TIMEOUT_REGION_MOVED;
Iterator<Entry<String, Pair<Long, ServerName>>> it = movedRegions.entrySet().iterator(); Iterator<Entry<String, Pair<Long, ServerName>>> it = movedRegions.entrySet().iterator();
while (it.hasNext()){ 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) // TODO: Use better method for determining stamp of last major (HBASE-2990)
long lowTimestamp = getLowestTimestamp(filesToCompact); long lowTimestamp = getLowestTimestamp(filesToCompact);
long now = System.currentTimeMillis(); long now = EnvironmentEdgeManager.currentTimeMillis();
if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) { if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) {
// Major compaction time has elapsed. // Major compaction time has elapsed.
if (filesToCompact.size() == 1) { if (filesToCompact.size() == 1) {

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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