You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by si...@apache.org on 2014/03/10 02:58:20 UTC

svn commit: r1575792 - in /zookeeper/bookkeeper/branches/branch-4.2: ./ bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/ bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ bookkeeper-server/src/main/java/org/apache/bookkeepe...

Author: sijie
Date: Mon Mar 10 01:58:19 2014
New Revision: 1575792

URL: http://svn.apache.org/r1575792
Log:
BOOKKEEPER-602: we should have request timeouts rather than channel timeout in PerChannelBookieClient (Aniruddha via sijie)

Modified:
    zookeeper/bookkeeper/branches/branch-4.2/CHANGES.txt
    zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchReadThroughputLatency.java
    zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java
    zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java
    zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java
    zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
    zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/MathUtils.java
    zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java
    zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java
    zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java
    zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java
    zookeeper/bookkeeper/branches/branch-4.2/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookKeeperPersistenceManager.java
    zookeeper/bookkeeper/branches/branch-4.2/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestDeadlock.java

Modified: zookeeper/bookkeeper/branches/branch-4.2/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/CHANGES.txt?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/CHANGES.txt (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/CHANGES.txt Mon Mar 10 01:58:19 2014
@@ -12,6 +12,8 @@ Release 4.2.3 - 2013-12-04
 
         BOOKKEEPER-688: NPE exception in PerChannelBookieClient (ivank via sijie)
 
+        BOOKKEEPER-602: we should have request timeouts rather than channel timeout in PerChannelBookieClient (Aniruddha via sijie)
+
 Release 4.2.2 - 2013-10-02
 
   Backward compatible changes:

Modified: zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchReadThroughputLatency.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchReadThroughputLatency.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchReadThroughputLatency.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchReadThroughputLatency.java Mon Mar 10 01:58:19 2014
@@ -149,6 +149,7 @@ public class BenchReadThroughputLatency 
         formatter.printHelp("BenchReadThroughputLatency <options>", options);
     }
 
+    @SuppressWarnings("deprecation")
     public static void main(String[] args) throws Exception {
         Options options = new Options();
         options.addOption("ledger", true, "Ledger to read. If empty, read all ledgers which come available. " 

Modified: zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java Mon Mar 10 01:58:19 2014
@@ -228,6 +228,7 @@ public class BenchThroughputLatency impl
         }
     }
 
+    @SuppressWarnings("deprecation")
     public static void main(String[] args)
             throws KeeperException, IOException, InterruptedException, ParseException, BKException {
         Options options = new Options();

Modified: zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java Mon Mar 10 01:58:19 2014
@@ -17,7 +17,10 @@
  */
 package org.apache.bookkeeper.conf;
 
+import static com.google.common.base.Charsets.UTF_8;
+
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 
@@ -44,6 +47,10 @@ public class ClientConfiguration extends
     protected final static String CLIENT_TCP_NODELAY = "clientTcpNoDelay";
     protected final static String READ_TIMEOUT = "readTimeout";
     protected final static String SPECULATIVE_READ_TIMEOUT = "speculativeReadTimeout";
+    // Timeout Setting
+    protected final static String ADD_ENTRY_TIMEOUT_SEC = "addEntryTimeoutSec";
+    protected final static String READ_ENTRY_TIMEOUT_SEC = "readEntryTimeoutSec";
+    protected final static String TIMEOUT_TASK_INTERVAL_MILLIS = "timeoutTaskIntervalMillis";
 
     // Number Woker Threads
     protected final static String NUM_WORKER_THREADS = "numWorkerThreads";
@@ -234,7 +241,9 @@ public class ClientConfiguration extends
      * The default is 5 seconds.
      *
      * @return the current read timeout in seconds
+     * @deprecated use {@link getReadEntryTimeout()} or {@link getAddEntryTimeout()} instead
      */
+    @Deprecated
     public int getReadTimeout() {
         return getInt(READ_TIMEOUT, 5);
     }
@@ -244,13 +253,84 @@ public class ClientConfiguration extends
      * @see #getReadTimeout()
      * @param timeout The new read timeout in seconds
      * @return client configuration
+     * @deprecated use {@link setReadEntryTimeout(int)} or {@link setAddEntryTimeout(int)} instead
      */
+    @Deprecated
     public ClientConfiguration setReadTimeout(int timeout) {
         setProperty(READ_TIMEOUT, Integer.toString(timeout));
         return this;
     }
 
     /**
+     * Get the timeout for add request. This is the number of seconds we wait without hearing
+     * a response for add request from a bookie before we consider it failed.
+     *
+     * The default value is 5 second for backwards compatibility.
+     *
+     * @return add entry timeout.
+     */
+    @SuppressWarnings("deprecation")
+    public int getAddEntryTimeout() {
+        return getInt(ADD_ENTRY_TIMEOUT_SEC, getReadTimeout());
+    }
+
+    /**
+     * Set timeout for add entry request.
+     * @see #getAddEntryTimeout()
+     *
+     * @param timeout
+     *          The new add entry timeout in seconds.
+     * @return client configuration.
+     */
+    public ClientConfiguration setAddEntryTimeout(int timeout) {
+        setProperty(ADD_ENTRY_TIMEOUT_SEC, timeout);
+        return this;
+    }
+
+    /**
+     * Get the timeout for read entry. This is the number of seconds we wait without hearing
+     * a response for read entry request from a bookie before we consider it failed. By default,
+     * we use socket timeout specified at {@link #getReadTimeout()}.
+     *
+     * @return read entry timeout.
+     */
+    @SuppressWarnings("deprecation")
+    public int getReadEntryTimeout() {
+        return getInt(READ_ENTRY_TIMEOUT_SEC, getReadTimeout());
+    }
+
+    /**
+     * Set the timeout for read entry request.
+     * @see #getReadEntryTimeout()
+     *
+     * @param timeout
+     *          The new read entry timeout in seconds.
+     * @return client configuration.
+     */
+    public ClientConfiguration setReadEntryTimeout(int timeout) {
+        setProperty(READ_ENTRY_TIMEOUT_SEC, timeout);
+        return this;
+    }
+
+    /**
+     * Get the interval between successive executions of the PerChannelBookieClient's
+     * TimeoutTask. This value is in milliseconds. Every X milliseconds, the timeout task
+     * will be executed and it will error out entries that have timed out.
+     *
+     * We do it more aggressive to not accumulate pending requests due to slow responses.
+     * @return
+     */
+    public long getTimeoutTaskIntervalMillis() {
+        return getLong(TIMEOUT_TASK_INTERVAL_MILLIS,
+                TimeUnit.SECONDS.toMillis(Math.min(getAddEntryTimeout(), getReadEntryTimeout())));
+    }
+
+    public ClientConfiguration setTimeoutTaskIntervalMillis(long timeoutMillis) {
+        setProperty(TIMEOUT_TASK_INTERVAL_MILLIS, Long.toString(timeoutMillis));
+        return this;
+    }
+
+    /**
      * Get the number of worker threads. This is the number of
      * worker threads used by bookkeeper client to submit operations.
      *

Modified: zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java Mon Mar 10 01:58:19 2014
@@ -21,29 +21,34 @@ package org.apache.bookkeeper.proto;
  *
  */
 
-import java.util.Set;
-import java.util.HashSet;
+import static com.google.common.base.Charsets.UTF_8;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.atomic.AtomicLong;
-import org.apache.bookkeeper.conf.ClientConfiguration;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
+import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
 import org.apache.bookkeeper.util.OrderedSafeExecutor;
 import org.apache.bookkeeper.util.SafeRunnable;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBuffers;
 import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
 import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * Implements the client-side part of the BookKeeper protocol.
  *
@@ -52,15 +57,16 @@ public class BookieClient {
     static final Logger LOG = LoggerFactory.getLogger(BookieClient.class);
 
     // This is global state that should be across all BookieClients
-    AtomicLong totalBytesOutstanding = new AtomicLong();
-
-    OrderedSafeExecutor executor;
-    ClientSocketChannelFactory channelFactory;
-    ConcurrentHashMap<InetSocketAddress, PerChannelBookieClient> channels = new ConcurrentHashMap<InetSocketAddress, PerChannelBookieClient>();
+    final AtomicLong totalBytesOutstanding = new AtomicLong();
 
+    final OrderedSafeExecutor executor;
+    final ClientSocketChannelFactory channelFactory;
+    final ConcurrentHashMap<InetSocketAddress, PerChannelBookieClient> channels =
+        new ConcurrentHashMap<InetSocketAddress, PerChannelBookieClient>();
+    final ScheduledExecutorService timeoutExecutor = Executors.newSingleThreadScheduledExecutor();
     private final ClientConfiguration conf;
     private volatile boolean closed;
-    private ReentrantReadWriteLock closeLock;
+    private final ReentrantReadWriteLock closeLock;
 
     public BookieClient(ClientConfiguration conf, ClientSocketChannelFactory channelFactory, OrderedSafeExecutor executor) {
         this.conf = conf;
@@ -79,7 +85,8 @@ public class BookieClient {
                 if (closed) {
                     return null;
                 }
-                channel = new PerChannelBookieClient(conf, executor, channelFactory, addr, totalBytesOutstanding);
+                channel = new PerChannelBookieClient(conf, executor, channelFactory, addr, totalBytesOutstanding,
+                        timeoutExecutor);
                 PerChannelBookieClient prevChannel = channels.putIfAbsent(addr, channel);
                 if (prevChannel != null) {
                     channel = prevChannel;

Modified: zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java Mon Mar 10 01:58:19 2014
@@ -24,7 +24,8 @@ import java.util.Set;
 import java.util.Collections;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Semaphore;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.collect.ImmutableSet;
@@ -81,6 +82,7 @@ public class PerChannelBookieClient exte
     AtomicLong totalBytesOutstanding;
     ClientSocketChannelFactory channelFactory;
     OrderedSafeExecutor executor;
+    ScheduledExecutorService timeoutExecutor;
 
     ConcurrentHashMap<CompletionKey, AddCompletion> addCompletions = new ConcurrentHashMap<CompletionKey, AddCompletion>();
     ConcurrentHashMap<CompletionKey, ReadCompletion> readCompletions = new ConcurrentHashMap<CompletionKey, ReadCompletion>();
@@ -92,26 +94,77 @@ public class PerChannelBookieClient exte
     Queue<GenericCallback<Void>> pendingOps = new ArrayDeque<GenericCallback<Void>>();
     volatile Channel channel = null;
 
+    private class TimeoutTask implements Runnable {
+        @Override
+        public void run() {
+            errorOutTimedOutEntries();
+        }
+    }
+
     enum ConnectionState {
         DISCONNECTED, CONNECTING, CONNECTED, CLOSED
-            };
+    };
 
     volatile ConnectionState state;
     private final ClientConfiguration conf;
 
+    /**
+     * Error out any entries that have timed out.
+     */
+    private void errorOutTimedOutEntries() {
+        int numAdd = 0, numRead = 0;
+        int total = 0;
+        try {
+            for (CompletionKey key : addCompletions.keySet()) {
+                total++;
+                if (key.shouldTimeout(conf.getAddEntryTimeout() * 1000)) {
+                    errorOutAddKey(key);
+                    numAdd++;
+                }
+            }
+            for (CompletionKey key : readCompletions.keySet()) {
+                total++;
+                if (key.shouldTimeout(conf.getReadEntryTimeout() * 1000)) {
+                    errorOutReadKey(key);
+                    numRead++;
+                }
+            }
+        } catch (Throwable t) {
+            LOG.error("Caught RuntimeException while erroring out timed out entries : ", t);
+        }
+        if (numAdd + numRead > 0) {
+            LOG.info("Timeout task iterated through a total of {} keys.", total);
+            LOG.info("Timeout Task errored out {} add entry requests.", numAdd);
+            LOG.info("Timeout Task errored out {} read entry requests.", numRead);
+        }
+    }
+
     public PerChannelBookieClient(OrderedSafeExecutor executor, ClientSocketChannelFactory channelFactory,
-                                  InetSocketAddress addr, AtomicLong totalBytesOutstanding) {
-        this(new ClientConfiguration(), executor, channelFactory, addr, totalBytesOutstanding);
+                                  InetSocketAddress addr, AtomicLong totalBytesOutstanding,
+                                  ScheduledExecutorService timeoutExecutor) {
+        this(new ClientConfiguration(), executor, channelFactory, addr, totalBytesOutstanding, timeoutExecutor);
     }
-            
-    public PerChannelBookieClient(ClientConfiguration conf, OrderedSafeExecutor executor, ClientSocketChannelFactory channelFactory,
+
+    public PerChannelBookieClient(OrderedSafeExecutor executor, ClientSocketChannelFactory channelFactory,
                                   InetSocketAddress addr, AtomicLong totalBytesOutstanding) {
+        this(new ClientConfiguration(), executor, channelFactory, addr, totalBytesOutstanding, null);
+    }
+
+    public PerChannelBookieClient(ClientConfiguration conf, OrderedSafeExecutor executor,
+                                  ClientSocketChannelFactory channelFactory, InetSocketAddress addr,
+                                  AtomicLong totalBytesOutstanding, ScheduledExecutorService timeoutExecutor) {
         this.conf = conf;
         this.addr = addr;
         this.executor = executor;
         this.totalBytesOutstanding = totalBytesOutstanding;
         this.channelFactory = channelFactory;
         this.state = ConnectionState.DISCONNECTED;
+        this.timeoutExecutor = timeoutExecutor;
+        // scheudle the timeout task
+        if (null != this.timeoutExecutor) {
+            this.timeoutExecutor.scheduleWithFixedDelay(new TimeoutTask(), conf.getTimeoutTaskIntervalMillis(),
+                    conf.getTimeoutTaskIntervalMillis(), TimeUnit.MILLISECONDS);
+        }
     }
 
     private void connect() {
@@ -492,8 +545,6 @@ public class PerChannelBookieClient exte
     public ChannelPipeline getPipeline() throws Exception {
         ChannelPipeline pipeline = Channels.pipeline();
 
-        pipeline.addLast("readTimeout", new ReadTimeoutHandler(new HashedWheelTimer(),
-                                                               conf.getReadTimeout()));
         pipeline.addLast("lengthbasedframedecoder", new LengthFieldBasedFrameDecoder(MAX_FRAME_LENGTH, 0, 4, 0, 4));
         pipeline.addLast("mainhandler", this);
         return pipeline;
@@ -535,19 +586,6 @@ public class PerChannelBookieClient exte
                       e.getChannel().getRemoteAddress());
             return;
         }
-        if (t instanceof ReadTimeoutException) {
-            for (CompletionKey key : addCompletions.keySet()) {
-                if (key.shouldTimeout()) {
-                    errorOutAddKey(key);
-                }
-            }
-            for (CompletionKey key : readCompletions.keySet()) {
-                if (key.shouldTimeout()) {
-                    errorOutReadKey(key);
-                }
-            }
-            return;
-        }
 
         if (t instanceof IOException) {
             // these are thrown when a bookie fails, logging them just pollutes
@@ -736,15 +774,15 @@ public class PerChannelBookieClient exte
     }
 
     // visable for testing
-    class CompletionKey {
+    static class CompletionKey {
         long ledgerId;
         long entryId;
-        final long timeoutAt;
+        final long requestAt;
 
         CompletionKey(long ledgerId, long entryId) {
             this.ledgerId = ledgerId;
             this.entryId = entryId;
-            this.timeoutAt = MathUtils.now() + (conf.getReadTimeout()*1000);
+            this.requestAt = MathUtils.nowInNano();
         }
 
         @Override
@@ -761,12 +799,17 @@ public class PerChannelBookieClient exte
             return ((int) ledgerId << 16) ^ ((int) entryId);
         }
 
+        @Override
         public String toString() {
             return String.format("LedgerEntry(%d, %d)", ledgerId, entryId);
         }
 
-        public boolean shouldTimeout() {
-            return this.timeoutAt <= MathUtils.now();
+        public boolean shouldTimeout(long timeout) {
+            return elapsedTime() >= timeout;
+        }
+
+        public long elapsedTime() {
+            return MathUtils.elapsedMSec(requestAt);
         }
     }
 

Modified: zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/MathUtils.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/MathUtils.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/MathUtils.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/MathUtils.java Mon Mar 10 01:58:19 2014
@@ -48,4 +48,29 @@ public class MathUtils {
         return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
     }
 
+    /**
+     * Current time from some arbitrary time base in the past, counting in
+     * nanoseconds, and not affected by settimeofday or similar system clock
+     * changes. This is appropriate to use when computing how much longer to
+     * wait for an interval to expire.
+     *
+     * NOTE: only use it for measuring.
+     * http://docs.oracle.com/javase/1.5.0/docs/api/java/lang/System.html#nanoTime%28%29
+     *
+     * @return current time in nanoseconds.
+     */
+    public static long nowInNano() {
+        return System.nanoTime();
+    }
+
+    /**
+     * Milliseconds elapsed since the time specified, the input is nanoTime
+     * the only conversion happens when computing the elapsed time
+     *
+     * @param startNanoTime the start of the interval that we are measuring
+     * @return elapsed time in milliseconds.
+     */
+    public static long elapsedMSec (long startNanoTime) {
+       return (System.nanoTime() - startNanoTime)/ NANOSECONDS_PER_MILLISECOND;
+    }
 }

Modified: zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java Mon Mar 10 01:58:19 2014
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 /**
  * This class tests the ledger close logic.
  */
+@SuppressWarnings("deprecation")
 public class LedgerCloseTest extends BookKeeperClusterTestCase {
 
     static Logger LOG = LoggerFactory.getLogger(LedgerCloseTest.class);

Modified: zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java Mon Mar 10 01:58:19 2014
@@ -39,6 +39,7 @@ import org.apache.bookkeeper.proto.Bookk
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 
+@SuppressWarnings("deprecation")
 public class SlowBookieTest extends BookKeeperClusterTestCase {
     static Logger LOG = LoggerFactory.getLogger(SlowBookieTest.class);
 

Modified: zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java Mon Mar 10 01:58:19 2014
@@ -57,6 +57,7 @@ public class TestReadTimeout extends Boo
         this.digestType = DigestType.CRC32;
     }
 
+    @SuppressWarnings("deprecation")
     @Test(timeout=60000)
     public void testReadTimeout() throws Exception {
         final AtomicBoolean completed = new AtomicBoolean(false);

Modified: zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java Mon Mar 10 01:58:19 2014
@@ -67,6 +67,7 @@ public class TestSpeculativeRead extends
         return l.getId();
     }
 
+    @SuppressWarnings("deprecation")
     BookKeeper createClient(int specTimeout) throws Exception {
         ClientConfiguration conf = new ClientConfiguration()
             .setSpeculativeReadTimeout(specTimeout)

Modified: zookeeper/bookkeeper/branches/branch-4.2/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookKeeperPersistenceManager.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookKeeperPersistenceManager.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookKeeperPersistenceManager.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestBookKeeperPersistenceManager.java Mon Mar 10 01:58:19 2014
@@ -255,6 +255,7 @@ public class TestBookKeeperPersistenceMa
         });
     }
 
+    @SuppressWarnings("deprecation")
     private void startCluster(long delay) throws Exception {
         bktb = new BookKeeperTestBase(numBookies, 0L);
         bktb.setUp();

Modified: zookeeper/bookkeeper/branches/branch-4.2/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestDeadlock.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/branches/branch-4.2/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestDeadlock.java?rev=1575792&r1=1575791&r2=1575792&view=diff
==============================================================================
--- zookeeper/bookkeeper/branches/branch-4.2/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestDeadlock.java (original)
+++ zookeeper/bookkeeper/branches/branch-4.2/hedwig-server/src/test/java/org/apache/hedwig/server/persistence/TestDeadlock.java Mon Mar 10 01:58:19 2014
@@ -195,6 +195,7 @@ public class TestDeadlock extends Hedwig
         }
     }
 
+    @SuppressWarnings("deprecation")
     @Override
     protected ServerConfiguration getServerConfiguration(int serverPort, int sslServerPort) {
         ServerConfiguration serverConf = new TestServerConfiguration(serverPort, sslServerPort);