You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ha...@apache.org on 2017/05/18 21:31:09 UTC

zookeeper git commit: Ephemeral znode will not be removed when sesstion timeout, if the system time of ZooKeeper node changes unexpectedly.

Repository: zookeeper
Updated Branches:
  refs/heads/branch-3.4 593c99c3c -> 3c4f01f64


Ephemeral znode will not be removed when sesstion timeout, if the system time of ZooKeeper node changes unexpectedly.

Author: Jiang Jiafu <ji...@gmail.com>
Author: JiangJiafu <ji...@gmail.com>

Reviewers: Michael Han <ha...@apache.org>

Closes #253 from JiangJiafu/ZOOKEEPER-2774


Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/3c4f01f6
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/3c4f01f6
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/3c4f01f6

Branch: refs/heads/branch-3.4
Commit: 3c4f01f6450f55d2fcc605f66c6c6584ccff1584
Parents: 593c99c
Author: Jiang Jiafu <ji...@gmail.com>
Authored: Thu May 18 14:31:01 2017 -0700
Committer: Michael Han <ha...@apache.org>
Committed: Thu May 18 14:31:01 2017 -0700

----------------------------------------------------------------------
 .../main/org/apache/zookeeper/ClientCnxn.java   |   5 +-
 .../org/apache/zookeeper/ClientCnxnSocket.java  |   3 +-
 src/java/main/org/apache/zookeeper/Login.java   |  10 +-
 src/java/main/org/apache/zookeeper/Shell.java   |   7 +-
 src/java/main/org/apache/zookeeper/ZKUtil.java  |   2 +-
 .../main/org/apache/zookeeper/common/Time.java  |  52 +++++++++
 .../apache/zookeeper/server/ConnectionBean.java |   4 +-
 .../zookeeper/server/FinalRequestProcessor.java |   7 +-
 .../zookeeper/server/PrepRequestProcessor.java  |  12 +-
 .../org/apache/zookeeper/server/Request.java    |   3 +-
 .../apache/zookeeper/server/ServerStats.java    |   5 +-
 .../zookeeper/server/SessionTrackerImpl.java    |   9 +-
 .../zookeeper/server/ZooKeeperServer.java       |   4 -
 .../server/quorum/AuthFastLeaderElection.java   |   3 +-
 .../server/quorum/FastLeaderElection.java       |   3 +-
 .../zookeeper/server/quorum/Follower.java       |   3 +-
 .../apache/zookeeper/server/quorum/Leader.java  |  15 +--
 .../zookeeper/test/system/GenerateLoad.java     |  21 ++--
 .../org/apache/zookeeper/common/TimeTest.java   | 109 +++++++++++++++++++
 .../server/quorum/QuorumPeerMainTest.java       |   5 +-
 .../org/apache/zookeeper/test/ClientBase.java   |  21 ++--
 .../apache/zookeeper/test/ClientHammerTest.java |   7 +-
 .../apache/zookeeper/test/LoadFromLogTest.java  |  19 ++--
 .../apache/zookeeper/test/ReadOnlyModeTest.java |   8 +-
 .../zookeeper/test/StaticHostProviderTest.java  |   9 +-
 .../org/apache/zookeeper/test/TestHammer.java   |   5 +-
 .../zookeeper/test/ZooKeeperTestClient.java     |   3 +-
 27 files changed, 268 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/ClientCnxn.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java
index 08934b0..3cf2f75 100644
--- a/src/java/main/org/apache/zookeeper/ClientCnxn.java
+++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java
@@ -60,6 +60,7 @@ import org.apache.zookeeper.ZooKeeper.States;
 import org.apache.zookeeper.ZooKeeper.WatchRegistration;
 import org.apache.zookeeper.client.HostProvider;
 import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.proto.AuthPacket;
 import org.apache.zookeeper.proto.ConnectRequest;
 import org.apache.zookeeper.proto.CreateResponse;
@@ -1041,7 +1042,7 @@ public class ClientCnxn {
             clientCnxnSocket.updateNow();
             clientCnxnSocket.updateLastSendAndHeard();
             int to;
-            long lastPingRwServer = System.currentTimeMillis();
+            long lastPingRwServer = Time.currentElapsedTime();
             final int MAX_SEND_PING_INTERVAL = 10000; //10 seconds
             while (state.isAlive()) {
                 try {
@@ -1126,7 +1127,7 @@ public class ClientCnxn {
 
                     // If we are in read-only mode, seek for read/write server
                     if (state == States.CONNECTEDREADONLY) {
-                        long now = System.currentTimeMillis();
+                        long now = Time.currentElapsedTime();
                         int idlePingRwServer = (int) (now - lastPingRwServer);
                         if (idlePingRwServer >= pingRwTimeout) {
                             lastPingRwServer = now;

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java b/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java
index 5ca0ba7..b676531 100644
--- a/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java
+++ b/src/java/main/org/apache/zookeeper/ClientCnxnSocket.java
@@ -27,6 +27,7 @@ import java.util.List;
 
 import org.apache.jute.BinaryInputArchive;
 import org.apache.zookeeper.ClientCnxn.Packet;
+import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.proto.ConnectResponse;
 import org.apache.zookeeper.server.ByteBufferInputStream;
 import org.slf4j.Logger;
@@ -74,7 +75,7 @@ abstract class ClientCnxnSocket {
     }
 
     void updateNow() {
-        now = System.currentTimeMillis();
+        now = Time.currentElapsedTime();
     }
 
     int getIdleRecv() {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/Login.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/Login.java b/src/java/main/org/apache/zookeeper/Login.java
index 3e21aae..c4975be 100644
--- a/src/java/main/org/apache/zookeeper/Login.java
+++ b/src/java/main/org/apache/zookeeper/Login.java
@@ -33,6 +33,7 @@ import javax.security.auth.login.LoginException;
 import javax.security.auth.callback.CallbackHandler;
 
 import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.apache.zookeeper.common.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import javax.security.auth.kerberos.KerberosTicket;
@@ -74,7 +75,8 @@ public class Login {
     private String keytabFile = null;
     private String principal = null;
 
-    private long lastLogin = 0;
+    // Initialize 'lastLogin' to do a login at first time
+    private long lastLogin = Time.currentElapsedTime() - MIN_TIME_BEFORE_RELOGIN;
 
     /**
      * LoginThread constructor. The constructor starts the thread used
@@ -128,7 +130,7 @@ public class Login {
                 LOG.info("TGT refresh thread started.");
                 while (true) {  // renewal thread's main loop. if it exits from here, thread will exit.
                     KerberosTicket tgt = getTGT();
-                    long now = System.currentTimeMillis();
+                    long now = Time.currentWallTime();
                     long nextRefresh;
                     Date nextRefreshDate;
                     if (tgt == null) {
@@ -306,7 +308,7 @@ public class Login {
                 (TICKET_RENEW_WINDOW + (TICKET_RENEW_JITTER * rng.nextDouble())));
         if (proposedRefresh > expires) {
             // proposedRefresh is too far in the future: it's after ticket expires: simply return now.
-            return System.currentTimeMillis();
+            return Time.currentWallTime();
         }
         else {
             return proposedRefresh;
@@ -327,7 +329,7 @@ public class Login {
     }
 
     private boolean hasSufficientTimeElapsed() {
-        long now = System.currentTimeMillis();
+        long now = Time.currentElapsedTime();
         if (now - getLastLogin() < MIN_TIME_BEFORE_RELOGIN ) {
             LOG.warn("Not attempting to re-login since the last re-login was " +
                     "attempted less than " + (MIN_TIME_BEFORE_RELOGIN/1000) + " seconds"+

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/Shell.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/Shell.java b/src/java/main/org/apache/zookeeper/Shell.java
index 789c481..97efad3 100644
--- a/src/java/main/org/apache/zookeeper/Shell.java
+++ b/src/java/main/org/apache/zookeeper/Shell.java
@@ -39,8 +39,9 @@ import java.util.Timer;
 import java.util.TimerTask;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.log4j.Logger;
+import org.apache.zookeeper.common.Time;
 
-/** 
+/**
  * A base class for running a Unix command.
  * 
  * <code>Shell</code> can be used to run unix commands like <code>du</code> or
@@ -146,7 +147,7 @@ abstract public class Shell {
 
   /** check to see if a command needs to be executed and execute if needed */
   protected void run() throws IOException {
-    if (lastTime + interval > System.currentTimeMillis())
+    if (lastTime + interval > Time.currentElapsedTime())
       return;
     exitCode = 0; // reset for next run
     runCommand();
@@ -245,7 +246,7 @@ abstract public class Shell {
         LOG.warn("Error while closing the error stream", ioe);
       }
       process.destroy();
-      lastTime = System.currentTimeMillis();
+      lastTime = Time.currentElapsedTime();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/ZKUtil.java b/src/java/main/org/apache/zookeeper/ZKUtil.java
index 4713a08..e901832 100644
--- a/src/java/main/org/apache/zookeeper/ZKUtil.java
+++ b/src/java/main/org/apache/zookeeper/ZKUtil.java
@@ -121,4 +121,4 @@ public class ZKUtil {
         return tree;
     }
     
-}
\ No newline at end of file
+} 

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/common/Time.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/Time.java b/src/java/main/org/apache/zookeeper/common/Time.java
new file mode 100644
index 0000000..83e53f0
--- /dev/null
+++ b/src/java/main/org/apache/zookeeper/common/Time.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.common;
+
+import java.util.Date;
+
+public class Time {
+    /**
+     * Returns time in milliseconds as does System.currentTimeMillis(),
+     * but uses elapsed time from an arbitrary epoch more like System.nanoTime().
+     * The difference is that if somebody changes the system clock,
+     * Time.currentElapsedTime will change but nanoTime won't. On the other hand,
+     * all of ZK assumes that time is measured in milliseconds.
+     * @return  The time in milliseconds from some arbitrary point in time.
+     */
+    public static long currentElapsedTime() {
+        return System.nanoTime() / 1000000;
+    }
+
+    /**
+     * Explicitly returns system dependent current wall time.
+     * @return Current time in msec.
+     */
+    public static long currentWallTime() {
+        return System.currentTimeMillis();
+    }
+
+    /**
+     * This is to convert the elapsedTime to a Date.
+     * @return A date object indicated by the elapsedTime.
+     */
+    public static Date elapsedTimeToDate(long elapsedTime) {
+        long wallTime = currentWallTime() + elapsedTime - currentElapsedTime();
+        return new Date(wallTime);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/ConnectionBean.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/ConnectionBean.java b/src/java/main/org/apache/zookeeper/server/ConnectionBean.java
index 917aacf..58917e0 100644
--- a/src/java/main/org/apache/zookeeper/server/ConnectionBean.java
+++ b/src/java/main/org/apache/zookeeper/server/ConnectionBean.java
@@ -22,10 +22,10 @@ import java.net.Inet6Address;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.util.Arrays;
-import java.util.Date;
 
 import javax.management.ObjectName;
 
+import org.apache.zookeeper.common.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.jmx.MBeanRegistry;
@@ -164,7 +164,7 @@ public class ConnectionBean implements ConnectionMXBean, ZKMBeanInfo {
     }
 
     public String getLastResponseTime() {
-        return new Date(stats.getLastResponseTime()).toString();
+        return Time.elapsedTimeToDate(stats.getLastResponseTime()).toString();
     }
 
     public long getLastLatency() {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
index 7278064..65f7ac0 100644
--- a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
+++ b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.jute.Record;
+import org.apache.zookeeper.common.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.KeeperException;
@@ -165,7 +166,7 @@ public class FinalRequestProcessor implements RequestProcessor {
 
                 lastOp = "PING";
                 cnxn.updateStatsForResponse(request.cxid, request.zxid, lastOp,
-                        request.createTime, System.currentTimeMillis());
+                        request.createTime, Time.currentElapsedTime());
 
                 cnxn.sendResponse(new ReplyHeader(-2,
                         zks.getZKDatabase().getDataTreeLastProcessedZxid(), 0), null, "response");
@@ -176,7 +177,7 @@ public class FinalRequestProcessor implements RequestProcessor {
 
                 lastOp = "SESS";
                 cnxn.updateStatsForResponse(request.cxid, request.zxid, lastOp,
-                        request.createTime, System.currentTimeMillis());
+                        request.createTime, Time.currentElapsedTime());
 
                 zks.finishSessionInit(request.cnxn, true);
                 return;
@@ -385,7 +386,7 @@ public class FinalRequestProcessor implements RequestProcessor {
 
         zks.serverStats().updateLatency(request.createTime);
         cnxn.updateStatsForResponse(request.cxid, lastZxid, lastOp,
-                    request.createTime, System.currentTimeMillis());
+                    request.createTime, Time.currentElapsedTime());
 
         try {
             cnxn.sendResponse(hdr, rsp, "response");

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
index 1248b08..58497b7 100644
--- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
+++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
@@ -35,6 +35,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import org.apache.jute.Record;
 import org.apache.jute.BinaryOutputArchive;
 
+import org.apache.zookeeper.common.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.CreateMode;
@@ -318,7 +319,7 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements
         throws KeeperException, IOException, RequestProcessorException
     {
         request.hdr = new TxnHeader(request.sessionId, request.cxid, zxid,
-                                    zks.getTime(), type);
+                                    Time.currentWallTime(), type);
 
         switch (type) {
             case OpCode.create:                
@@ -557,9 +558,9 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements
                 try {
                     ByteBufferInputStream.byteBuffer2Record(request.request, multiRequest);
                 } catch(IOException e) {
-                   request.hdr =  new TxnHeader(request.sessionId, request.cxid, zks.getNextZxid(),
-                            zks.getTime(), OpCode.multi);
-                   throw e;
+                    request.hdr =  new TxnHeader(request.sessionId, request.cxid, zks.getNextZxid(),
+                            Time.currentWallTime(), OpCode.multi);
+                    throw e;
                 }
                 List<Txn> txns = new ArrayList<Txn>();
                 //Each op in a multi-op must have the same zxid!
@@ -616,7 +617,8 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements
                     index++;
                 }
 
-                request.hdr = new TxnHeader(request.sessionId, request.cxid, zxid, zks.getTime(), request.type);
+                request.hdr = new TxnHeader(request.sessionId, request.cxid, zxid,
+                        Time.currentWallTime(), request.type);
                 request.txn = new MultiTxn(txns);
                 
                 break;

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/Request.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/Request.java b/src/java/main/org/apache/zookeeper/server/Request.java
index 80d2b99..bb8b1ca 100644
--- a/src/java/main/org/apache/zookeeper/server/Request.java
+++ b/src/java/main/org/apache/zookeeper/server/Request.java
@@ -26,6 +26,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs.OpCode;
+import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.txn.TxnHeader;
 
@@ -75,7 +76,7 @@ public class Request {
 
     public final List<Id> authInfo;
 
-    public final long createTime = System.currentTimeMillis();
+    public final long createTime = Time.currentElapsedTime();
     
     private Object owner;
     

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/ServerStats.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/ServerStats.java b/src/java/main/org/apache/zookeeper/server/ServerStats.java
index dbee6d5..7885995 100644
--- a/src/java/main/org/apache/zookeeper/server/ServerStats.java
+++ b/src/java/main/org/apache/zookeeper/server/ServerStats.java
@@ -19,6 +19,9 @@
 package org.apache.zookeeper.server;
 
 
+
+import org.apache.zookeeper.common.Time;
+
 /**
  * Basic Server Statistics
  */
@@ -102,7 +105,7 @@ public class ServerStats {
     }
     // mutators
     synchronized void updateLatency(long requestCreateTime) {
-        long latency = System.currentTimeMillis() - requestCreateTime;
+        long latency = Time.currentElapsedTime() - requestCreateTime;
         totalLatency += latency;
         count++;
         if (latency < minLatency) {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java b/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java
index 938e9da..cdaaf2b 100644
--- a/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java
+++ b/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java
@@ -32,6 +32,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.SessionExpiredException;
+import org.apache.zookeeper.common.Time;
 
 /**
  * This is a full featured SessionTracker. It tracks session in grouped by tick
@@ -74,7 +75,7 @@ public class SessionTrackerImpl extends ZooKeeperCriticalThread implements Sessi
 
     public static long initializeNextSession(long id) {
         long nextSid = 0;
-        nextSid = (System.currentTimeMillis() << 24) >>> 8;
+        nextSid = (Time.currentElapsedTime() << 24) >>> 8;
         nextSid =  nextSid | (id <<56);
         return nextSid;
     }
@@ -98,7 +99,7 @@ public class SessionTrackerImpl extends ZooKeeperCriticalThread implements Sessi
         this.expirer = expirer;
         this.expirationInterval = tickTime;
         this.sessionsWithTimeout = sessionsWithTimeout;
-        nextExpirationTime = roundToInterval(System.currentTimeMillis());
+        nextExpirationTime = roundToInterval(Time.currentElapsedTime());
         this.nextSessionId = initializeNextSession(sid);
         for (Entry<Long, Integer> e : sessionsWithTimeout.entrySet()) {
             addSession(e.getKey(), e.getValue());
@@ -141,7 +142,7 @@ public class SessionTrackerImpl extends ZooKeeperCriticalThread implements Sessi
     synchronized public void run() {
         try {
             while (running) {
-                currentTime = System.currentTimeMillis();
+                currentTime = Time.currentElapsedTime();
                 if (nextExpirationTime > currentTime) {
                     this.wait(nextExpirationTime - currentTime);
                     continue;
@@ -174,7 +175,7 @@ public class SessionTrackerImpl extends ZooKeeperCriticalThread implements Sessi
         if (s == null || s.isClosing()) {
             return false;
         }
-        long expireTime = roundToInterval(System.currentTimeMillis() + timeout);
+        long expireTime = roundToInterval(Time.currentElapsedTime() + timeout);
         if (s.tickTime >= expireTime) {
             // Nothing needs to be done
             return true;

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
index 66e5afe..f29ee90 100644
--- a/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
+++ b/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
@@ -325,10 +325,6 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         hzxid.set(zxid);
     }
 
-    long getTime() {
-        return System.currentTimeMillis();
-    }
-
     private void close(long sessionId) {
         submitRequest(null, sessionId, OpCode.closeSession, 0, null, null);
     }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java b/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
index 5bf54f8..d32a725 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
@@ -36,6 +36,7 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import java.util.Random;
 
+import org.apache.zookeeper.common.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -412,7 +413,7 @@ public class AuthFastLeaderElection implements Election {
             WorkerSender(int attempts) {
                 maxAttempts = attempts;
                 rand = new Random(java.lang.Thread.currentThread().getId()
-                        + System.currentTimeMillis());
+                        + Time.currentElapsedTime());
             }
 
             long genChallenge() {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java b/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
index 2a3d4fd..dc5f099 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.server.ZooKeeperThread;
 import org.apache.zookeeper.server.quorum.QuorumCnxManager.Message;
@@ -801,7 +802,7 @@ public class FastLeaderElection implements Election {
             self.jmxLeaderElectionBean = null;
         }
         if (self.start_fle == 0) {
-           self.start_fle = System.currentTimeMillis();
+           self.start_fle = Time.currentElapsedTime();
         }
         try {
             HashMap<Long, Vote> recvset = new HashMap<Long, Vote>();

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/quorum/Follower.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Follower.java b/src/java/main/org/apache/zookeeper/server/quorum/Follower.java
index a17af49..e439aaa 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/Follower.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/Follower.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 
 import org.apache.jute.Record;
+import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.apache.zookeeper.server.util.SerializeUtils;
 import org.apache.zookeeper.server.util.ZxidUtils;
@@ -58,7 +59,7 @@ public class Follower extends Learner{
      * @throws InterruptedException
      */
     void followLeader() throws InterruptedException {
-        self.end_fle = System.currentTimeMillis();
+        self.end_fle = Time.currentElapsedTime();
         long electionTimeTaken = self.end_fle - self.start_fle;
         self.setElectionTimeTaken(electionTimeTaken);
         LOG.info("FOLLOWING - LEADER ELECTION TOOK - {}", electionTimeTaken);

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
index 44e6b4f..bd7bf35 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
@@ -41,6 +41,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import javax.security.sasl.SaslException;
 
 import org.apache.jute.BinaryOutputArchive;
+import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.server.FinalRequestProcessor;
 import org.apache.zookeeper.server.Request;
 import org.apache.zookeeper.server.RequestProcessor;
@@ -365,7 +366,7 @@ public class Leader {
      * @throws InterruptedException
      */
     void lead() throws IOException, InterruptedException {
-        self.end_fle = System.currentTimeMillis();
+        self.end_fle = Time.currentElapsedTime();
         long electionTimeTaken = self.end_fle - self.start_fle;
         self.setElectionTimeTaken(electionTimeTaken);
         LOG.info("LEADING - LEADER ELECTION TOOK - {}", electionTimeTaken);
@@ -885,12 +886,12 @@ public class Leader {
                 self.setAcceptedEpoch(epoch);
                 connectingFollowers.notifyAll();
             } else {
-                long start = System.currentTimeMillis();
+                long start = Time.currentElapsedTime();
                 long cur = start;
                 long end = start + self.getInitLimit()*self.getTickTime();
                 while(waitingForNewEpoch && cur < end) {
                     connectingFollowers.wait(end - cur);
-                    cur = System.currentTimeMillis();
+                    cur = Time.currentElapsedTime();
                 }
                 if (waitingForNewEpoch) {
                     throw new InterruptedException("Timeout while waiting for epoch from quorum");        
@@ -922,12 +923,12 @@ public class Leader {
                 electionFinished = true;
                 electingFollowers.notifyAll();
             } else {                
-                long start = System.currentTimeMillis();
+                long start = Time.currentElapsedTime();
                 long cur = start;
                 long end = start + self.getInitLimit()*self.getTickTime();
                 while(!electionFinished && cur < end) {
                     electingFollowers.wait(end - cur);
-                    cur = System.currentTimeMillis();
+                    cur = Time.currentElapsedTime();
                 }
                 if (!electionFinished) {
                     throw new InterruptedException("Timeout while waiting for epoch to be acked by quorum");
@@ -1010,12 +1011,12 @@ public class Leader {
                 quorumFormed = true;
                 newLeaderProposal.ackSet.notifyAll();
             } else {
-                long start = System.currentTimeMillis();
+                long start = Time.currentElapsedTime();
                 long cur = start;
                 long end = start + self.getInitLimit() * self.getTickTime();
                 while (!quorumFormed && cur < end) {
                     newLeaderProposal.ackSet.wait(end - cur);
-                    cur = System.currentTimeMillis();
+                    cur = Time.currentElapsedTime();
                 }
                 if (!quorumFormed) {
                     throw new InterruptedException(

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java
----------------------------------------------------------------------
diff --git a/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java b/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java
index cfd4e7b..b6ac04a 100644
--- a/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java
+++ b/src/java/systest/org/apache/zookeeper/test/system/GenerateLoad.java
@@ -53,6 +53,7 @@ import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.common.Time;
 
 public class GenerateLoad {
     protected static final Logger LOG = LoggerFactory.getLogger(GenerateLoad.class);
@@ -194,7 +195,7 @@ public class GenerateLoad {
 
         public void run() {
             try {
-                currentInterval = System.currentTimeMillis() / INTERVAL;
+                currentInterval = Time.currentElapsedTime() / INTERVAL;
                 // Give things time to report;
                 Thread.sleep(INTERVAL * 2);
                 long min = 99999;
@@ -202,7 +203,7 @@ public class GenerateLoad {
                 long total = 0;
                 int number = 0;
                 while (true) {
-                    long now = System.currentTimeMillis();
+                    long now = Time.currentElapsedTime();
                     long lastInterval = currentInterval;
                     currentInterval += 1;
                     long count = remove(lastInterval);
@@ -249,13 +250,13 @@ public class GenerateLoad {
     }
 
     synchronized static void sendChange(int percentage) {
-        long now = System.currentTimeMillis();
+        long now = Time.currentElapsedTime();
         long start = now;
         ReporterThread.percentage = percentage;
         for (SlaveThread st : slaves.toArray(new SlaveThread[0])) {
             st.send(percentage);
         }
-        now = System.currentTimeMillis();
+        now = Time.currentElapsedTime();
         long delay = now - start;
         if (delay > 1000) {
             System.out.println("Delay of " + delay + " to send new percentage");
@@ -387,7 +388,7 @@ public class GenerateLoad {
                         errors++;
                     } else {
                         finished++;
-                        rlatency += System.currentTimeMillis() - (Long) ctx;
+                        rlatency += Time.currentElapsedTime() - (Long) ctx;
                         reads++;
                     }
                 }
@@ -401,7 +402,7 @@ public class GenerateLoad {
                         errors++;
                     } else {
                         finished++;
-                        wlatency += System.currentTimeMillis() - (Long) ctx;
+                        wlatency += Time.currentElapsedTime() - (Long) ctx;
                         writes++;
                     }
                 }
@@ -427,7 +428,7 @@ public class GenerateLoad {
                         if (percentage == -1 || (finished == 0 && errors == 0)) {
                             continue;
                         }
-                        String report = System.currentTimeMillis() + " "
+                        String report = Time.currentElapsedTime() + " "
                                 + percentage + " " + finished + " " + errors + " "
                                 + outstanding + "\n";
                        /* String subreport = reads + " "
@@ -547,9 +548,9 @@ public class GenerateLoad {
 
         synchronized public boolean waitConnected(long timeout)
                 throws InterruptedException {
-            long endTime = System.currentTimeMillis() + timeout;
-            while (!connected && System.currentTimeMillis() < endTime) {
-                wait(endTime - System.currentTimeMillis());
+            long endTime = Time.currentElapsedTime() + timeout;
+            while (!connected && Time.currentElapsedTime() < endTime) {
+                wait(endTime - Time.currentElapsedTime());
             }
             return connected;
         }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/test/org/apache/zookeeper/common/TimeTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/common/TimeTest.java b/src/java/test/org/apache/zookeeper/common/TimeTest.java
new file mode 100644
index 0000000..d938556
--- /dev/null
+++ b/src/java/test/org/apache/zookeeper/common/TimeTest.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.common;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Calendar;
+import java.util.Date;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Command line program for demonstrating robustness to clock
+ * changes.
+ * <p/>
+ * How to run:
+ * ant clean compile-test
+ * echo build/test/lib/*.jar build/lib/*.jar build/classes build/test/classes | sed -e 's/ /:/g' > cp
+ * java -cp $(cat cp) org.apache.zookeeper.common.TimeTest | tee log-without-patch
+ * <p/>
+ * After test program starts, in another window, do commands:
+ * date -s '+1hour'
+ * date -s '-1hour'
+ * <p/>
+ * As long as there isn't any expired event, the experiment is successful.
+ */
+public class TimeTest extends ClientBase {
+    private static final long mt0 = System.currentTimeMillis();
+    private static final long nt0 = Time.currentElapsedTime();
+
+    private static AtomicInteger watchCount = new AtomicInteger(0);
+
+
+    public static void main(String[] args) throws Exception {
+        System.out.printf("Starting\n");
+        final TimeTest test = new TimeTest();
+        System.out.printf("After construct\n");
+        test.setUp();
+        ZooKeeper zk = test.createClient();
+        zk.create("/ephemeral", new byte[]{1, 2, 3},
+                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
+        while (Time.currentElapsedTime() - nt0 < 100000) {
+            System.out.printf("%d\t%s\n", discrepancy(),
+                    zk.exists("/ephemeral",
+                            watchCount.get() == 0 ? createWatcher() : null) != null);
+            waitByYielding(500);
+        }
+    }
+
+    private static Watcher createWatcher() {
+        watchCount.incrementAndGet();
+        return new Watcher() {
+            @Override
+            public void process(WatchedEvent event) {
+                watchCount.decrementAndGet();
+                System.out.printf("%d event = %s\n", discrepancy(), event);
+            }
+        };
+
+    }
+
+    private static void waitByYielding(long delay) {
+        long t0 = Time.currentElapsedTime();
+        while (Time.currentElapsedTime() < t0 + delay) {
+            Thread.yield();
+        }
+    }
+
+    private static long discrepancy() {
+        return (System.currentTimeMillis() - mt0) - (Time.currentElapsedTime() - nt0);
+    }
+
+    @Test
+    public void testElapsedTimeToDate() throws Exception {
+        long walltime = Time.currentWallTime();
+        long elapsedTime = Time.currentElapsedTime();
+        Thread.sleep(200);
+
+        Calendar cal = Calendar.getInstance();
+        cal.setTime(Time.elapsedTimeToDate(elapsedTime));
+        int calculatedDate = cal.get(Calendar.HOUR_OF_DAY);
+        cal.setTime(new Date(walltime));
+        int realDate = cal.get(Calendar.HOUR_OF_DAY);
+
+        Assert.assertEquals(calculatedDate, realDate);
+    }
+}

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
index 5ea92cb..e8ba8bb 100644
--- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
+++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
@@ -47,6 +47,7 @@ import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooKeeper.States;
+import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.common.AtomicFileOutputStream;
 import org.apache.zookeeper.server.quorum.Leader.Proposal;
 import org.apache.zookeeper.server.util.ZxidUtils;
@@ -814,9 +815,9 @@ public class QuorumPeerMainTest extends QuorumPeerTestBase {
         q1.start();
         // Let the notifications timeout
         Thread.sleep(30000);
-        long start = System.currentTimeMillis();
+        long start = Time.currentElapsedTime();
         q1.shutdown();
-        long end = System.currentTimeMillis();
+        long end = Time.currentElapsedTime();
         if ((end - start) > maxwait) {
            Assert.fail("QuorumPeer took " + (end -start) +
                     " to shutdown, expected " + maxwait);

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/test/org/apache/zookeeper/test/ClientBase.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/ClientBase.java b/src/java/test/org/apache/zookeeper/test/ClientBase.java
index 08cc13c..f2ad4d5 100644
--- a/src/java/test/org/apache/zookeeper/test/ClientBase.java
+++ b/src/java/test/org/apache/zookeeper/test/ClientBase.java
@@ -42,6 +42,7 @@ import javax.management.ObjectName;
 
 import junit.framework.TestCase;
 
+import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.TestableZooKeeper;
@@ -134,11 +135,11 @@ public abstract class ClientBase extends ZKTestCase {
         synchronized public void waitForConnected(long timeout)
             throws InterruptedException, TimeoutException
         {
-            long expire = System.currentTimeMillis() + timeout;
+            long expire = Time.currentElapsedTime() + timeout;
             long left = timeout;
             while(!connected && left > 0) {
                 wait(left);
-                left = expire - System.currentTimeMillis();
+                left = expire - Time.currentElapsedTime();
             }
             if (!connected) {
                 throw new TimeoutException("Did not connect");
@@ -161,11 +162,11 @@ public abstract class ClientBase extends ZKTestCase {
         synchronized public void waitForDisconnected(long timeout)
             throws InterruptedException, TimeoutException
         {
-            long expire = System.currentTimeMillis() + timeout;
+            long expire = Time.currentElapsedTime() + timeout;
             long left = timeout;
             while(connected && left > 0) {
                 wait(left);
-                left = expire - System.currentTimeMillis();
+                left = expire - Time.currentElapsedTime();
             }
             if (connected) {
                 throw new TimeoutException("Did not disconnect");
@@ -254,7 +255,7 @@ public abstract class ClientBase extends ZKTestCase {
     }
 
     public static boolean waitForServerUp(String hp, long timeout) {
-        long start = System.currentTimeMillis();
+        long start = Time.currentElapsedTime();
         while (true) {
             try {
                 // if there are multiple hostports, just take the first one
@@ -269,7 +270,7 @@ public abstract class ClientBase extends ZKTestCase {
                 LOG.info("server " + hp + " not up " + e);
             }
 
-            if (System.currentTimeMillis() > start + timeout) {
+            if (Time.currentElapsedTime() > start + timeout) {
                 break;
             }
             try {
@@ -281,7 +282,7 @@ public abstract class ClientBase extends ZKTestCase {
         return false;
     }
     public static boolean waitForServerDown(String hp, long timeout) {
-        long start = System.currentTimeMillis();
+        long start = Time.currentElapsedTime();
         while (true) {
             try {
                 HostPort hpobj = parseHostPortList(hp).get(0);
@@ -290,7 +291,7 @@ public abstract class ClientBase extends ZKTestCase {
                 return true;
             }
 
-            if (System.currentTimeMillis() > start + timeout) {
+            if (Time.currentElapsedTime() > start + timeout) {
                 break;
             }
             try {
@@ -304,7 +305,7 @@ public abstract class ClientBase extends ZKTestCase {
 
     public static boolean waitForServerState(QuorumPeer qp, int timeout,
             String serverState) {
-        long start = System.currentTimeMillis();
+        long start = Time.currentElapsedTime();
         while (true) {
             try {
                 Thread.sleep(250);
@@ -313,7 +314,7 @@ public abstract class ClientBase extends ZKTestCase {
             }
             if (qp.getServerState().equals(serverState))
                 return true;
-            if (System.currentTimeMillis() > start + timeout) {
+            if (Time.currentElapsedTime() > start + timeout) {
                 return false;
             }
         }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java b/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java
index 581402c..025ccbe 100644
--- a/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java
+++ b/src/java/test/org/apache/zookeeper/test/ClientHammerTest.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.Date;
 import java.util.List;
 
+import org.apache.zookeeper.common.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.CreateMode;
@@ -124,7 +125,7 @@ public class ClientHammerTest extends ClientBase {
     {
         try {
             HammerThread[] threads = new HammerThread[threadCount];
-            long start = System.currentTimeMillis();
+            long start = Time.currentElapsedTime();
             for (int i = 0; i < threads.length; i++) {
                 ZooKeeper zk = createClient();
                 String prefix = "/test-" + i;
@@ -157,7 +158,7 @@ public class ClientHammerTest extends ClientBase {
             final int childCount = 10;
 
             HammerThread[] threads = new HammerThread[threadCount];
-            long start = System.currentTimeMillis();
+            long start = Time.currentElapsedTime();
             for (int i = 0; i < threads.length; i++) {
                 String prefix = "/test-" + i;
                 {
@@ -218,7 +219,7 @@ public class ClientHammerTest extends ClientBase {
                     * HAMMERTHREAD_LATENCY * safetyFactor);
         }
         LOG.info(new Date() + " Total time "
-                + (System.currentTimeMillis() - start));
+                + (Time.currentElapsedTime() - start));
 
         ZooKeeper zk = createClient();
         try {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java b/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java
index 419683f..94b0f97 100644
--- a/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java
+++ b/src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java
@@ -26,6 +26,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.zookeeper.common.Time;
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.BinaryOutputArchive;
 import org.apache.jute.Record;
@@ -155,7 +156,7 @@ public class LoadFromLogTest extends ZKTestCase implements  Watcher {
         dt.createNode("/test", new byte[0], null, 0, -1, 1, 1);
         for (count = 1; count <= 3; count++) {
             dt.createNode("/test/" + count, new byte[0], null, 0, -1, count,
-                    System.currentTimeMillis());
+                    Time.currentElapsedTime());
         }
         DataNode zk = dt.getNode("/test");
 
@@ -204,15 +205,15 @@ public class LoadFromLogTest extends ZKTestCase implements  Watcher {
         if (type == OpCode.delete) {
             txn = new DeleteTxn(path);
             txnHeader = new TxnHeader(0xabcd, 0x123, prevPzxid + 1,
-                System.currentTimeMillis(), OpCode.delete);
+                Time.currentElapsedTime(), OpCode.delete);
         } else if (type == OpCode.create) {
             txnHeader = new TxnHeader(0xabcd, 0x123, prevPzxid + 1,
-                    System.currentTimeMillis(), OpCode.create);
+                    Time.currentElapsedTime(), OpCode.create);
             txn = new CreateTxn(path, new byte[0], null, false, cversion);
         }
         else if (type == OpCode.multi) {
             txnHeader = new TxnHeader(0xabcd, 0x123, prevPzxid + 1,
-                    System.currentTimeMillis(), OpCode.create);
+                    Time.currentElapsedTime(), OpCode.create);
             txn = new CreateTxn(path, new byte[0], null, false, cversion);                       
             ArrayList txnList = new ArrayList();
             ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -223,7 +224,7 @@ public class LoadFromLogTest extends ZKTestCase implements  Watcher {
             txnList.add(txact);
             txn = new MultiTxn(txnList);
             txnHeader = new TxnHeader(0xabcd, 0x123, prevPzxid + 1,
-                    System.currentTimeMillis(), OpCode.multi);
+                    Time.currentElapsedTime(), OpCode.multi);
         }
         logFile.processTransaction(txnHeader, dt, null, txn);
 
@@ -250,7 +251,7 @@ public class LoadFromLogTest extends ZKTestCase implements  Watcher {
         File tmpDir = ClientBase.createTmpDir();
         FileTxnLog txnLog = new FileTxnLog(tmpDir);
         TxnHeader txnHeader = new TxnHeader(0xabcd, 0x123, 0x123,
-              System.currentTimeMillis(), OpCode.create);
+              Time.currentElapsedTime(), OpCode.create);
         Record txn = new CreateTxn("/Test", new byte[0], null, false, 1);
         txnLog.append(txnHeader, txn);
         FileInputStream in = new FileInputStream(tmpDir.getPath() + "/log." +
@@ -444,9 +445,9 @@ public class LoadFromLogTest extends ZKTestCase implements  Watcher {
     private ZooKeeper getConnectedZkClient() throws IOException {
         ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
 
-        long start = System.currentTimeMillis();
+        long start = Time.currentElapsedTime();
         while (!connected) {
-            long end = System.currentTimeMillis();
+            long end = Time.currentElapsedTime();
             if (end - start > 5000) {
                 Assert.assertTrue("Could not connect with server in 5 seconds",
                         false);
@@ -459,4 +460,4 @@ public class LoadFromLogTest extends ZKTestCase implements  Watcher {
         }
         return zk;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java b/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java
index 1dbf647..cc266d1 100644
--- a/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java
+++ b/src/java/test/org/apache/zookeeper/test/ReadOnlyModeTest.java
@@ -42,6 +42,7 @@ import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper.States;
+import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.test.ClientBase.CountdownWatcher;
 import org.junit.After;
 import org.junit.Before;
@@ -178,13 +179,12 @@ public class ReadOnlyModeTest extends ZKTestCase {
         // kill peer and wait no more than 5 seconds for read-only server
         // to be started (which should take one tickTime (2 seconds))
         qu.shutdown(2);
-        long start = System.currentTimeMillis();
+        long start = Time.currentElapsedTime();
         while (!(zk.getState() == States.CONNECTEDREADONLY)) {
             Thread.sleep(200);
             // FIXME this was originally 5 seconds, but realistically, on random/slow/virt hosts, there is no way to guarantee this
-            Assert.assertTrue("Can't connect to the server", System
-                    .currentTimeMillis()
-                    - start < 30000);
+            Assert.assertTrue("Can't connect to the server",
+                              Time.currentElapsedTime() - start < 30000);
         }
 
         // At this point states list should contain, in the given order,

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java b/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java
index 75d3c59..aa78a4b 100644
--- a/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java
+++ b/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java
@@ -26,6 +26,7 @@ import static org.junit.Assert.fail;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.client.HostProvider;
 import org.apache.zookeeper.client.StaticHostProvider;
+import org.apache.zookeeper.common.Time;
 import org.junit.Test;
 
 import org.slf4j.Logger;
@@ -58,9 +59,9 @@ public class StaticHostProviderTest extends ZKTestCase {
             hostProvider.next(0);
             --size;
         }
-        long start = System.currentTimeMillis();
+        long start = Time.currentElapsedTime();
         hostProvider.next(1000);
-        long stop = System.currentTimeMillis();
+        long stop = Time.currentElapsedTime();
         assertTrue(900 <= stop - start);
     }
 
@@ -72,9 +73,9 @@ public class StaticHostProviderTest extends ZKTestCase {
             hostProvider.next(0);
             --size;
         }
-        long start = System.currentTimeMillis();
+        long start = Time.currentElapsedTime();
         hostProvider.next(0);
-        long stop = System.currentTimeMillis();
+        long stop = Time.currentElapsedTime();
         assertTrue(5 > stop - start);
     }
 

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/test/org/apache/zookeeper/test/TestHammer.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/TestHammer.java b/src/java/test/org/apache/zookeeper/test/TestHammer.java
index 09a678b..a73d6df 100644
--- a/src/java/test/org/apache/zookeeper/test/TestHammer.java
+++ b/src/java/test/org/apache/zookeeper/test/TestHammer.java
@@ -24,6 +24,7 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.AsyncCallback.VoidCallback;
 import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.common.Time;
 
 public class TestHammer implements VoidCallback {
 
@@ -32,7 +33,7 @@ public class TestHammer implements VoidCallback {
      */
     static int REPS = 50000;
     public static void main(String[] args) {
-            long startTime = System.currentTimeMillis();
+            long startTime = Time.currentElapsedTime();
             ZooKeeper zk = null;
             try {
                 zk = new ZooKeeper(args[0], 10000, null);
@@ -51,7 +52,7 @@ public class TestHammer implements VoidCallback {
                     e.printStackTrace();
                 }
             }
-            System.out.println("creates/sec=" + (REPS*1000/(System.currentTimeMillis()-startTime)));
+            System.out.println("creates/sec=" + (REPS*1000/(Time.currentElapsedTime()-startTime)));
     }
 
     public void processResult(int rc, String path, Object ctx) {

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3c4f01f6/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java b/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
index 67ca52f..0bbba61 100644
--- a/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
+++ b/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
@@ -32,6 +32,7 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.common.Time;
 import org.apache.zookeeper.data.Stat;
 import org.junit.Assert;
 
@@ -40,7 +41,7 @@ public class ZooKeeperTestClient extends ZKTestCase implements Watcher {
 
   protected static final String dirOnZK = "/test_dir";
 
-  protected String testDirOnZK = dirOnZK + "/" + System.currentTimeMillis();
+  protected String testDirOnZK = dirOnZK + "/" + Time.currentElapsedTime();
 
   LinkedBlockingQueue<WatchedEvent> events = new LinkedBlockingQueue<WatchedEvent>();