You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/09/29 01:50:33 UTC

svn commit: r1391699 [2/2] - in /hbase/trunk: hbase-common/src/main/java/org/apache/hadoop/hbase/util/ hbase-server/src/main/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/ hbase-server/src/main/java/org/apache...

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java Fri Sep 28 23:50:31 2012
@@ -33,6 +33,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 import com.google.common.base.Preconditions;
 
@@ -83,7 +84,7 @@ public class CompactSplitThread implemen
           @Override
           public Thread newThread(Runnable r) {
             Thread t = new Thread(r);
-            t.setName(n + "-largeCompactions-" + System.currentTimeMillis());
+            t.setName(n + "-largeCompactions-" + EnvironmentEdgeManager.currentTimeMillis());
             return t;
           }
       });
@@ -95,7 +96,7 @@ public class CompactSplitThread implemen
           @Override
           public Thread newThread(Runnable r) {
             Thread t = new Thread(r);
-            t.setName(n + "-smallCompactions-" + System.currentTimeMillis());
+            t.setName(n + "-smallCompactions-" + EnvironmentEdgeManager.currentTimeMillis());
             return t;
           }
       });
@@ -107,7 +108,7 @@ public class CompactSplitThread implemen
           @Override
           public Thread newThread(Runnable r) {
             Thread t = new Thread(r);
-            t.setName(n + "-splits-" + System.currentTimeMillis());
+            t.setName(n + "-splits-" + EnvironmentEdgeManager.currentTimeMillis());
             return t;
           }
       });

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java Fri Sep 28 23:50:31 2012
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
  * State and utility processing {@link HRegion#getClosestRowBefore(byte[], byte[])}.
@@ -72,7 +73,7 @@ class GetClosestRowBeforeTracker {
         HConstants.DELIMITER) - this.rowoffset;
     }
     this.tablenamePlusDelimiterLength = metaregion? l + 1: -1;
-    this.oldestts = System.currentTimeMillis() - ttl;
+    this.oldestts = EnvironmentEdgeManager.currentTimeMillis() - ttl;
     this.kvcomparator = c;
     KeyValue.RowComparator rc = new KeyValue.RowComparator(this.kvcomparator);
     this.deletes = new TreeMap<KeyValue, NavigableSet<KeyValue>>(rc);

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

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java Fri Sep 28 23:50:31 2012
@@ -1178,7 +1178,7 @@ public class HStore extends SchemaConfig
     }
     // TODO: Use better method for determining stamp of last major (HBASE-2990)
     long lowTimestamp = getLowestTimestamp(filesToCompact);
-    long now = System.currentTimeMillis();
+    long now = EnvironmentEdgeManager.currentTimeMillis();
     if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) {
       // Major compaction time has elapsed.
       if (filesToCompact.size() == 1) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java Fri Sep 28 23:50:31 2012
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionse
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.HasThread;
 
 import java.util.ConcurrentModificationException;
@@ -278,7 +279,7 @@ public class Leases extends HasThread {
     }
 
     public long getDelay(TimeUnit unit) {
-      return unit.convert(this.expirationTime - System.currentTimeMillis(),
+      return unit.convert(this.expirationTime - EnvironmentEdgeManager.currentTimeMillis(),
           TimeUnit.MILLISECONDS);
     }
 
@@ -293,7 +294,7 @@ public class Leases extends HasThread {
      * Resets the expiration time of the lease.
      */
     public void resetExpirationTime() {
-      this.expirationTime = System.currentTimeMillis() + this.leaseTimeoutPeriod;
+      this.expirationTime = EnvironmentEdgeManager.currentTimeMillis() + this.leaseTimeoutPeriod;
     }
   }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java Fri Sep 28 23:50:31 2012
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.HasThread;
 
 import java.io.IOException;
@@ -48,7 +49,7 @@ class LogRoller extends HasThread implem
   private final AtomicBoolean rollLog = new AtomicBoolean(false);
   private final Server server;
   private final RegionServerServices services;
-  private volatile long lastrolltime = System.currentTimeMillis();
+  private volatile long lastrolltime = EnvironmentEdgeManager.currentTimeMillis();
   // Period to roll log.
   private final long rollperiod;
   private final int threadWakeFrequency;
@@ -67,7 +68,7 @@ class LogRoller extends HasThread implem
   @Override
   public void run() {
     while (!server.isStopped()) {
-      long now = System.currentTimeMillis();
+      long now = EnvironmentEdgeManager.currentTimeMillis();
       boolean periodic = false;
       if (!rollLog.get()) {
         periodic = (now - this.lastrolltime) > this.rollperiod;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java Fri Sep 28 23:50:31 2012
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.DroppedSn
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.util.StringUtils;
 
@@ -346,7 +347,7 @@ class MemStoreFlusher extends HasThread 
     if (!fqe.region.getRegionInfo().isMetaRegion() &&
         isTooManyStoreFiles(region)) {
       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 " +
           "long enough... proceeding with flush of " +
           region.getRegionNameAsString());
@@ -525,7 +526,7 @@ class MemStoreFlusher extends HasThread 
 
     FlushRegionEntry(final HRegion r) {
       this.region = r;
-      this.createTime = System.currentTimeMillis();
+      this.createTime = EnvironmentEdgeManager.currentTimeMillis();
       this.whenToExpire = this.createTime;
     }
 
@@ -534,7 +535,7 @@ class MemStoreFlusher extends HasThread 
      * @return True if we have been delayed > <code>maximumWait</code> milliseconds.
      */
     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 
 
     /**
      * @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.
      * @return This.
      */
     public FlushRegionEntry requeue(final long when) {
-      this.whenToExpire = System.currentTimeMillis() + when;
+      this.whenToExpire = EnvironmentEdgeManager.currentTimeMillis() + when;
       this.requeueCount++;
       return this;
     }
 
     @Override
     public long getDelay(TimeUnit unit) {
-      return unit.convert(this.whenToExpire - System.currentTimeMillis(),
+      return unit.convert(this.whenToExpire - EnvironmentEdgeManager.currentTimeMillis(),
           TimeUnit.MILLISECONDS);
     }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java Fri Sep 28 23:50:31 2012
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.SplitLogT
 import org.apache.hadoop.hbase.master.SplitLogManager;
 import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -269,7 +270,7 @@ public class SplitLogWorker extends ZooK
       SplitLogCounters.tot_wkr_task_acquired.incrementAndGet();
       getDataSetWatchAsync();
 
-      t = System.currentTimeMillis();
+      t = EnvironmentEdgeManager.currentTimeMillis();
       TaskExecutor.Status status;
 
       status = splitTaskExecutor.exec(ZKSplitLog.getFileName(currentTask),
@@ -314,7 +315,7 @@ public class SplitLogWorker extends ZooK
     } finally {
       if (t > 0) {
         LOG.info("worker " + serverName + " done with task " + path +
-            " in " + (System.currentTimeMillis() - t) + "ms");
+            " in " + (EnvironmentEdgeManager.currentTimeMillis() - t) + "ms");
       }
       synchronized (grabTaskLock) {
         workerInGrabTask = false;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java Fri Sep 28 23:50:31 2012
@@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.base.Preconditions;
@@ -59,14 +60,14 @@ class SplitRequest implements Runnable {
       return;
     }
     try {
-      final long startTime = System.currentTimeMillis();
+      final long startTime = EnvironmentEdgeManager.currentTimeMillis();
       SplitTransaction st = new SplitTransaction(parent, midKey);
       // If prepare does not return true, for some reason -- logged inside in
       // the prepare call -- we are not ready to split just now. Just return.
       if (!st.prepare()) return;
       try {
         st.execute(this.server, this.server);
-        this.server.getMetrics().incrementSplitSuccessCount(System.currentTimeMillis() - startTime);
+        this.server.getMetrics().incrementSplitSuccessCount(EnvironmentEdgeManager.currentTimeMillis() - startTime);
       } catch (Exception e) {
         if (this.server.isStopping() || this.server.isStopped()) {
           LOG.info(
@@ -98,7 +99,7 @@ class SplitRequest implements Runnable {
           + parent.getRegionInfo().getRegionNameAsString() + ", new regions: "
           + st.getFirstDaughter().getRegionNameAsString() + ", "
           + st.getSecondDaughter().getRegionNameAsString() + ". Split took "
-          + StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
+          + StringUtils.formatTimeDiff(EnvironmentEdgeManager.currentTimeMillis(), startTime));
     } catch (IOException ex) {
       LOG.error("Split failed " + this, RemoteExceptionHandler
           .checkIOException(ex));

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java Fri Sep 28 23:50:31 2012
@@ -523,7 +523,7 @@ public class SplitTransaction {
 
     @Override
     public boolean progress() {
-      long now = System.currentTimeMillis();
+      long now = EnvironmentEdgeManager.currentTimeMillis();
       if (now - lastLog > this.interval) {
         LOG.info("Opening " + this.hri.getRegionNameAsString());
         this.lastLog = now;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java Fri Sep 28 23:50:31 2012
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.zookeeper.KeeperException;
 
@@ -168,7 +169,7 @@ public class OpenRegionHandler extends E
     // Total timeout for meta edit.  If we fail adding the edit then close out
     // the region and let it be assigned elsewhere.
     long timeout = assignmentTimeout * 10;
-    long now = System.currentTimeMillis();
+    long now = EnvironmentEdgeManager.currentTimeMillis();
     long endTime = now + timeout;
     // Let our period at which we update OPENING state to be be 1/3rd of the
     // regions-in-transition timeout period.
@@ -190,7 +191,7 @@ public class OpenRegionHandler extends E
           // Go to the loop check.
         }
       }
-      now = System.currentTimeMillis();
+      now = EnvironmentEdgeManager.currentTimeMillis();
     }
     // Is thread still alive?  We may have left above loop because server is
     // stopping or we timed out the edit.  Is so, interrupt it.

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java Fri Sep 28 23:50:31 2012
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.metrics.h
 import org.apache.hadoop.hbase.metrics.PersistentMetricsTimeVaryingRate;
 import com.yammer.metrics.stats.Snapshot;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Strings;
 import org.apache.hadoop.metrics.ContextFactory;
@@ -61,8 +62,8 @@ public class RegionServerMetrics impleme
   @SuppressWarnings({"FieldCanBeLocal"})
   private final Log LOG = LogFactory.getLog(this.getClass());
   private final MetricsRecord metricsRecord;
-  private long lastUpdate = System.currentTimeMillis();
-  private long lastExtUpdate = System.currentTimeMillis();
+  private long lastUpdate = EnvironmentEdgeManager.currentTimeMillis();
+  private long lastExtUpdate = EnvironmentEdgeManager.currentTimeMillis();
   private long extendedPeriod = 0;
   private static final int MB = 1024*1024;
   private MetricsRegistry registry = new MetricsRegistry();
@@ -349,7 +350,7 @@ public class RegionServerMetrics impleme
    */
   public void doUpdates(MetricsContext caller) {
     synchronized (this) {
-      this.lastUpdate = System.currentTimeMillis();
+      this.lastUpdate = EnvironmentEdgeManager.currentTimeMillis();
 
       // has the extended period for long-living stats elapsed?
       if (this.extendedPeriod > 0 &&

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

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationSinkMetrics.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationSinkMetrics.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationSinkMetrics.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationSinkMetrics.java Fri Sep 28 23:50:31 2012
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replicat
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
  * This class is for maintaining the various replication statistics for a sink and publishing them
@@ -44,7 +45,7 @@ public class ReplicationSinkMetrics {
    * @param timestamp The timestamp of the last operation applied.
    */
   public void setAgeOfLastAppliedOp(long timestamp) {
-    long age = System.currentTimeMillis() - timestamp;
+    long age = EnvironmentEdgeManager.currentTimeMillis() - timestamp;
     rms.setGauge(SINK_AGE_OF_LAST_APPLIED_OP, age);
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationSourceMetrics.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationSourceMetrics.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationSourceMetrics.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/metrics/ReplicationSourceMetrics.java Fri Sep 28 23:50:31 2012
@@ -22,6 +22,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
  * This class is for maintaining the various replication statistics for a source and publishing them
@@ -75,7 +76,7 @@ public class ReplicationSourceMetrics {
    * @param timestamp write time of the edit
    */
   public void setAgeOfLastShippedOp(long timestamp) {
-    long age = System.currentTimeMillis() - timestamp;
+    long age = EnvironmentEdgeManager.currentTimeMillis() - timestamp;
     rms.setGauge(ageOfLastShippedOpKey, age);
     rms.setGauge(SOURCE_AGE_OF_LAST_SHIPPED_OP, age);
     this.lastTimestamp = timestamp;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResultGenerator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResultGenerator.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResultGenerator.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResultGenerator.java Fri Sep 28 23:50:31 2012
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.Re
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.rest.model.ScannerModel;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.StringUtils;
 
 @InterfaceAudience.Private
@@ -88,7 +89,7 @@ public class ScannerResultGenerator exte
       scan.setCacheBlocks(false);
       scanner = table.getScanner(scan);
       cached = null;
-      id = Long.toString(System.currentTimeMillis()) +
+      id = Long.toString(EnvironmentEdgeManager.currentTimeMillis()) +
              Integer.toHexString(scanner.hashCode());
     } finally {
       table.close();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java Fri Sep 28 23:50:31 2012
@@ -42,6 +42,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
  * A wrapper around HttpClient which provides some useful function and
@@ -186,9 +187,9 @@ public class Client {
         method.addRequestHeader(header);
       }
     }
-    long startTime = System.currentTimeMillis();
+    long startTime = EnvironmentEdgeManager.currentTimeMillis();
     int code = httpClient.executeMethod(method);
-    long endTime = System.currentTimeMillis();
+    long endTime = EnvironmentEdgeManager.currentTimeMillis();
     if (LOG.isDebugEnabled()) {
       LOG.debug(method.getName() + " " + uri + " " + code + " " +
         method.getStatusText() + " in " + (endTime - startTime) + " ms");

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/TBoundedThreadPoolServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/TBoundedThreadPoolServer.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/TBoundedThreadPoolServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/TBoundedThreadPoolServer.java Fri Sep 28 23:50:31 2012
@@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.thrift.CallQueue.Call;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.thrift.TException;
 import org.apache.thrift.TProcessor;
@@ -219,7 +220,7 @@ public class TBoundedThreadPoolServer ex
 
     long msLeftToWait =
         serverOptions.stopTimeoutUnit.toMillis(serverOptions.stopTimeoutVal);
-    long timeMillis = System.currentTimeMillis();
+    long timeMillis = EnvironmentEdgeManager.currentTimeMillis();
 
     LOG.info("Waiting for up to " + msLeftToWait + " ms to finish processing" +
         " pending requests");
@@ -229,7 +230,7 @@ public class TBoundedThreadPoolServer ex
         executorService.awaitTermination(msLeftToWait, TimeUnit.MILLISECONDS);
         break;
       } catch (InterruptedException ix) {
-        long timePassed = System.currentTimeMillis() - timeMillis;
+        long timePassed = EnvironmentEdgeManager.currentTimeMillis() - timeMillis;
         msLeftToWait -= timePassed;
         timeMillis += timePassed;
         interrupted = true;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java Fri Sep 28 23:50:31 2012
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.TableNotF
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
  * HBase Canary Tool, that that can be used to do
@@ -234,9 +235,9 @@ public final class Canary implements Too
       get.addFamily(column.getName());
 
       try {
-        long startTime = System.currentTimeMillis();
+        long startTime = EnvironmentEdgeManager.currentTimeMillis();
         table.get(get);
-        long time = System.currentTimeMillis() - startTime;
+        long time = EnvironmentEdgeManager.currentTimeMillis() - startTime;
 
         sink.publishReadTiming(region, column, time);
       } catch (Exception e) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java Fri Sep 28 23:50:31 2012
@@ -68,7 +68,7 @@ public class FSHDFSUtils extends FSUtils
       return;
     }
     LOG.info("Recovering file " + p);
-    long startWaiting = System.currentTimeMillis();
+    long startWaiting = EnvironmentEdgeManager.currentTimeMillis();
 
     // Trying recovery
     boolean recovered = false;
@@ -95,7 +95,7 @@ public class FSHDFSUtils extends FSUtils
           // within its soft limit, but if we get it past that, it means
           // that the RS is holding onto the file even though it lost its
           // znode. We could potentially abort after some time here.
-          long waitedFor = System.currentTimeMillis() - startWaiting;
+          long waitedFor = EnvironmentEdgeManager.currentTimeMillis() - startWaiting;
           if (waitedFor > LEASE_SOFTLIMIT_PERIOD) {
             LOG.warn("Waited " + waitedFor + "ms for lease recovery on " + p +
               ":" + e.getMessage());

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java Fri Sep 28 23:50:31 2012
@@ -538,7 +538,7 @@ public abstract class FSUtils {
   throws IOException {
     // Rewrite the file as pb.  Move aside the old one first, write new
     // then delete the moved-aside file.
-    Path movedAsideName = new Path(p + "." + System.currentTimeMillis());
+    Path movedAsideName = new Path(p + "." + EnvironmentEdgeManager.currentTimeMillis());
     if (!fs.rename(p, movedAsideName)) throw new IOException("Failed rename of " + p);
     setClusterId(fs, rootdir, cid, 100);
     if (!fs.delete(movedAsideName, false)) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java Fri Sep 28 23:50:31 2012
@@ -170,7 +170,7 @@ public class HBaseFsck {
   private HBaseAdmin admin;
   private HTable meta;
   protected ExecutorService executor; // threads to retrieve data from regionservers
-  private long startMillis = System.currentTimeMillis();
+  private long startMillis = EnvironmentEdgeManager.currentTimeMillis();
   private HFileCorruptionChecker hfcc;
   private int retcode = 0;
 
@@ -1255,7 +1255,7 @@ public class HBaseFsck {
       throw new IOException(e);
     }
     MetaEntry m =
-      new MetaEntry(rootLocation.getRegionInfo(), sn, System.currentTimeMillis());
+      new MetaEntry(rootLocation.getRegionInfo(), sn, EnvironmentEdgeManager.currentTimeMillis());
     HbckInfo hbInfo = new HbckInfo(m);
     regionInfoMap.put(rootLocation.getRegionInfo().getEncodedName(), hbInfo);
     return true;
@@ -1508,7 +1508,7 @@ public class HBaseFsck {
       (hbi.metaEntry == null)? false: hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline();
     boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry);
     boolean recentlyModified = hbi.getHdfsRegionDir() != null &&
-      hbi.getModTime() + timelag > System.currentTimeMillis();
+      hbi.getModTime() + timelag > EnvironmentEdgeManager.currentTimeMillis();
 
     // ========== First the healthy cases =============
     if (hbi.containsOnlyHdfsEdits()) {
@@ -2321,7 +2321,7 @@ public class HBaseFsck {
    */
    HTableDescriptor[] getTables(AtomicInteger numSkipped) {
     List<String> tableNames = new ArrayList<String>();
-    long now = System.currentTimeMillis();
+    long now = EnvironmentEdgeManager.currentTimeMillis();
 
     for (HbckInfo hbi : regionInfoMap.values()) {
       MetaEntry info = hbi.metaEntry;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java Fri Sep 28 23:50:31 2012
@@ -119,8 +119,8 @@ public class HBaseFsckRepair {
   public static void waitUntilAssigned(HBaseAdmin admin,
       HRegionInfo region) throws IOException, InterruptedException {
     long timeout = admin.getConfiguration().getLong("hbase.hbck.assign.timeout", 120000);
-    long expiration = timeout + System.currentTimeMillis();
-    while (System.currentTimeMillis() < expiration) {
+    long expiration = timeout + EnvironmentEdgeManager.currentTimeMillis();
+    while (EnvironmentEdgeManager.currentTimeMillis() < expiration) {
       try {
         Map<String, RegionState> rits=
             admin.getClusterStatus().getRegionsInTransition();
@@ -153,8 +153,8 @@ public class HBaseFsckRepair {
     ProtobufUtil.closeRegion(rs, region.getRegionName(), false);
     long timeout = admin.getConfiguration()
       .getLong("hbase.hbck.close.timeout", 120000);
-    long expiration = timeout + System.currentTimeMillis();
-    while (System.currentTimeMillis() < expiration) {
+    long expiration = timeout + EnvironmentEdgeManager.currentTimeMillis();
+    while (EnvironmentEdgeManager.currentTimeMillis() < expiration) {
       try {
         HRegionInfo rsRegion =
           ProtobufUtil.getRegionInfo(rs, region.getRegionName());

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java Fri Sep 28 23:50:31 2012
@@ -155,7 +155,7 @@ class HMerge {
           Bytes.toString(tableName)
       );
       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);
       Path oldLogDir = new Path(tabledir, HConstants.HREGION_OLDLOGDIR_NAME);
       this.hlog = new HLog(fs, logdir, oldLogDir, conf);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java Fri Sep 28 23:50:31 2012
@@ -175,13 +175,13 @@ public class JVMClusterUtil {
     // Wait for an active master
     //  having an active master before starting the region threads allows
     //  then to succeed on their connection to master
-    long startTime = System.currentTimeMillis();
+    long startTime = EnvironmentEdgeManager.currentTimeMillis();
     while (findActiveMaster(masters) == null) {
       try {
         Thread.sleep(100);
       } catch (InterruptedException ignored) {
       }
-      if (System.currentTimeMillis() > startTime + 30000) {
+      if (EnvironmentEdgeManager.currentTimeMillis() > startTime + 30000) {
         throw new RuntimeException("Master not active after 30 seconds");
       }
     }
@@ -197,13 +197,13 @@ public class JVMClusterUtil {
 
     // Wait for an active master to be initialized (implies being master)
     //  with this, when we return the cluster is complete
-    startTime = System.currentTimeMillis();
+    startTime = EnvironmentEdgeManager.currentTimeMillis();
     while (true) {
       JVMClusterUtil.MasterThread t = findActiveMaster(masters);
       if (t != null && t.master.isInitialized()) {
         return t.master.getServerName().toString();
       }
-      if (System.currentTimeMillis() > startTime + 200000) {
+      if (EnvironmentEdgeManager.currentTimeMillis() > startTime + 200000) {
         throw new RuntimeException("Master not initialized after 200 seconds");
       }
       try {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java Fri Sep 28 23:50:31 2012
@@ -335,7 +335,7 @@ public class Merge extends Configured im
     }
 
     Delete delete  = new Delete(regioninfo.getRegionName(),
-        System.currentTimeMillis(), null);
+        EnvironmentEdgeManager.currentTimeMillis(), null);
     meta.delete(delete, null, true);
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java Fri Sep 28 23:50:31 2012
@@ -98,7 +98,7 @@ public class MetaUtils {
   public synchronized HLog getLog() throws IOException {
     if (this.log == null) {
       Path logdir = new Path(this.fs.getHomeDirectory(),
-          HConstants.HREGION_LOGDIR_NAME + "_" + System.currentTimeMillis());
+          HConstants.HREGION_LOGDIR_NAME + "_" + EnvironmentEdgeManager.currentTimeMillis());
       Path oldLogDir = new Path(this.fs.getHomeDirectory(),
           HConstants.HREGION_OLDLOGDIR_NAME);
       this.log = new HLog(this.fs, logdir, oldLogDir, this.conf);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java Fri Sep 28 23:50:31 2012
@@ -426,7 +426,7 @@ public class RegionSplitter {
       daughterRegions.get(rsLocation).add(dr);
     }
     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
     FSDataInputStream tmpIn = fs.open(splitFile);
@@ -544,7 +544,7 @@ public class RegionSplitter {
                 + " " + splitAlgo.rowToStr(region.getSecond()) + "\n");
             splitCount++;
             if (splitCount % 10 == 0) {
-              long tDiff = (System.currentTimeMillis() - startTime)
+              long tDiff = (EnvironmentEdgeManager.currentTimeMillis() - startTime)
                   / splitCount;
               LOG.debug("STATUS UPDATE: " + splitCount + " / " + origCount
                   + ". Avg Time / Split = "
@@ -570,7 +570,7 @@ public class RegionSplitter {
       }
       LOG.debug("All regions have been successfully split!");
     } finally {
-      long tDiff = System.currentTimeMillis() - startTime;
+      long tDiff = EnvironmentEdgeManager.currentTimeMillis() - startTime;
       LOG.debug("TOTAL TIME = "
           + org.apache.hadoop.util.StringUtils.formatTime(tDiff));
       LOG.debug("Splits = " + splitCount);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Sleeper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Sleeper.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Sleeper.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Sleeper.java Fri Sep 28 23:50:31 2012
@@ -55,7 +55,7 @@ public class Sleeper {
    * Sleep for period.
    */
   public void sleep() {
-    sleep(System.currentTimeMillis());
+    sleep(EnvironmentEdgeManager.currentTimeMillis());
   }
 
   /**
@@ -78,11 +78,11 @@ public class Sleeper {
     if (this.stopper.isStopped()) {
       return;
     }
-    long now = System.currentTimeMillis();
+    long now = EnvironmentEdgeManager.currentTimeMillis();
     long waitTime = this.period - (now - startTime);
     if (waitTime > this.period) {
       LOG.warn("Calculated wait time > " + this.period +
-        "; setting to this.period: " + System.currentTimeMillis() + ", " +
+        "; setting to this.period: " + EnvironmentEdgeManager.currentTimeMillis() + ", " +
         startTime);
       waitTime = this.period;
     }
@@ -93,7 +93,7 @@ public class Sleeper {
           if (triggerWake) break;
           sleepLock.wait(waitTime);
         }
-        woke = System.currentTimeMillis();
+        woke = EnvironmentEdgeManager.currentTimeMillis();
         long slept = woke - now;
         if (slept - this.period > MINIMAL_DELTA_FOR_LOGGING) {
           LOG.warn("We slept " + slept + "ms instead of " + this.period +
@@ -109,7 +109,7 @@ public class Sleeper {
         }
       }
       // Recalculate waitTime.
-      woke = (woke == -1)? System.currentTimeMillis(): woke;
+      woke = (woke == -1)? EnvironmentEdgeManager.currentTimeMillis(): woke;
       waitTime = this.period - (woke - startTime);
     }
     triggerWake = false;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java Fri Sep 28 23:50:31 2012
@@ -38,6 +38,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.zookeeper.server.NIOServerCnxnFactory;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.persistence.FileTxnLog;
@@ -310,7 +311,7 @@ public class MiniZooKeeperCluster {
 
   // XXX: From o.a.zk.t.ClientBase
   private static boolean waitForServerDown(int port, long timeout) {
-    long start = System.currentTimeMillis();
+    long start = EnvironmentEdgeManager.currentTimeMillis();
     while (true) {
       try {
         Socket sock = new Socket("localhost", port);
@@ -325,7 +326,7 @@ public class MiniZooKeeperCluster {
         return true;
       }
 
-      if (System.currentTimeMillis() > start + timeout) {
+      if (EnvironmentEdgeManager.currentTimeMillis() > start + timeout) {
         break;
       }
       try {
@@ -339,7 +340,7 @@ public class MiniZooKeeperCluster {
 
   // XXX: From o.a.zk.t.ClientBase
   private static boolean waitForServerUp(int port, long timeout) {
-    long start = System.currentTimeMillis();
+    long start = EnvironmentEdgeManager.currentTimeMillis();
     while (true) {
       try {
         Socket sock = new Socket("localhost", port);
@@ -366,7 +367,7 @@ public class MiniZooKeeperCluster {
         LOG.info("server localhost:" + port + " not up " + e);
       }
 
-      if (System.currentTimeMillis() > start + timeout) {
+      if (EnvironmentEdgeManager.currentTimeMillis() > start + timeout) {
         break;
       }
       try {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Fri Sep 28 23:50:31 2012
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.Deseriali
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
@@ -1211,7 +1212,7 @@ public class ZKUtil {
 
     byte[] data = null;
     boolean finished = false;
-    final long endTime = System.currentTimeMillis() + timeout;
+    final long endTime = EnvironmentEdgeManager.currentTimeMillis() + timeout;
     while (!finished) {
       try {
         data = ZKUtil.getData(zkw, znode);
@@ -1219,7 +1220,7 @@ public class ZKUtil {
         LOG.warn("Unexpected exception handling blockUntilAvailable", e);
       }
 
-      if (data == null && (System.currentTimeMillis() +
+      if (data == null && (EnvironmentEdgeManager.currentTimeMillis() +
         HConstants.SOCKET_RETRY_WAIT_MS < endTime)) {
         Thread.sleep(HConstants.SOCKET_RETRY_WAIT_MS);
       } else {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java Fri Sep 28 23:50:31 2012
@@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -126,7 +127,7 @@ public abstract class ZooKeeperNodeTrack
   throws InterruptedException {
     if (timeout < 0) throw new IllegalArgumentException();
     boolean notimeout = timeout == 0;
-    long startTime = System.currentTimeMillis();
+    long startTime = EnvironmentEdgeManager.currentTimeMillis();
     long remaining = timeout;
     if (refresh) {
       try {
@@ -165,7 +166,7 @@ public abstract class ZooKeeperNodeTrack
       // We expect a notification; but we wait with a
       //  a timeout to lower the impact of a race condition if any
       wait(100);
-      remaining = timeout - (System.currentTimeMillis() - startTime);
+      remaining = timeout - (EnvironmentEdgeManager.currentTimeMillis() - startTime);
     }
     return this.data;
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java?rev=1391699&r1=1391698&r2=1391699&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java Fri Sep 28 23:50:31 2012
@@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -324,9 +325,9 @@ public class ZooKeeperWatcher implements
       case SyncConnected:
         // Now, this callback can be invoked before the this.zookeeper is set.
         // Wait a little while.
-        long finished = System.currentTimeMillis() +
+        long finished = EnvironmentEdgeManager.currentTimeMillis() +
           this.conf.getLong("hbase.zookeeper.watcher.sync.connected.wait", 2000);
-        while (System.currentTimeMillis() < finished) {
+        while (EnvironmentEdgeManager.currentTimeMillis() < finished) {
           Threads.sleep(1);
           if (this.recoverableZooKeeper != null) break;
         }