You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2018/02/27 11:16:13 UTC

[GitHub] sijie closed pull request #1208: use FutureUtils/FutureEventListener in bookkeeper-common for dlog

sijie closed pull request #1208: use FutureUtils/FutureEventListener in bookkeeper-common for dlog
URL: https://github.com/apache/bookkeeper/pull/1208
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/buildtools/src/main/resources/distributedlog/findbugsExclude.xml b/buildtools/src/main/resources/distributedlog/findbugsExclude.xml
index 7b2b80b99..f85686fcb 100644
--- a/buildtools/src/main/resources/distributedlog/findbugsExclude.xml
+++ b/buildtools/src/main/resources/distributedlog/findbugsExclude.xml
@@ -31,17 +31,8 @@
   </Match>
   <!-- distributedlog-common -->
   <Match>
-    <Class name="org.apache.distributedlog.common.concurrent.FutureUtils"/>
-    <Bug pattern="NP_NULL_PARAM_DEREF_NONVIRTUAL" />
-  </Match>
-  <Match>
-    <Class name="org.apache.distributedlog.common.concurrent.FutureUtils$2"/>
-    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
-  </Match>
-  <Match>
-    <Class name="org.apache.distributedlog.common.concurrent.FutureUtils"/>
-    <Method name="Void" />
-    <Bug pattern="NM_METHOD_NAMING_CONVENTION" />
+    <Class name="org.apache.distributedlog.util.OrderedScheduler"/>
+    <Bug pattern="NM_SAME_SIMPLE_NAME_AS_SUPERCLASS" />
   </Match>
   <!-- distributedlog-core -->
   <Match>
diff --git a/stream/distributedlog/common/pom.xml b/stream/distributedlog/common/pom.xml
index fe0628e3d..7735e550b 100644
--- a/stream/distributedlog/common/pom.xml
+++ b/stream/distributedlog/common/pom.xml
@@ -36,6 +36,11 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.bookkeeper</groupId>
+      <artifactId>bookkeeper-common</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
@@ -124,7 +129,7 @@
         </dependencies>
         <configuration>
           <configLocation>bookkeeper/checkstyle.xml</configLocation>
-          <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
+          <suppressionsLocation>bookkeeper/suppressions.xml</suppressionsLocation>
           <consoleOutput>true</consoleOutput>
           <failOnViolation>true</failOnViolation>
           <includeResources>false</includeResources>
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/AsyncSemaphore.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/AsyncSemaphore.java
index d6a9b34a7..095df411b 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/AsyncSemaphore.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/AsyncSemaphore.java
@@ -23,6 +23,7 @@
 import java.util.concurrent.RejectedExecutionException;
 import java.util.function.Supplier;
 import javax.annotation.concurrent.GuardedBy;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.util.Permit;
 
 /**
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/FutureEventListener.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/FutureEventListener.java
deleted file mode 100644
index ed5b9ac2e..000000000
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/FutureEventListener.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.distributedlog.common.concurrent;
-
-import java.util.concurrent.CompletionException;
-import java.util.function.BiConsumer;
-
-/**
- * Provide similar interface (as twitter future) over java future.
- */
-public interface FutureEventListener<T> extends BiConsumer<T, Throwable> {
-
-  void onSuccess(T value);
-
-  void onFailure(Throwable cause);
-
-  @Override
-  default void accept(T t, Throwable throwable) {
-    if (null != throwable) {
-      if (throwable instanceof CompletionException && null != throwable.getCause()) {
-        onFailure(throwable.getCause());
-      } else {
-        onFailure(throwable);
-      }
-      return;
-    }
-    onSuccess(t);
-  }
-}
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/FutureUtils.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/FutureUtils.java
deleted file mode 100644
index 15ecf1d14..000000000
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/FutureUtils.java
+++ /dev/null
@@ -1,376 +0,0 @@
-/*
- * 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.distributedlog.common.concurrent;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Lists;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.function.BiConsumer;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-import javax.annotation.Nullable;
-import lombok.SneakyThrows;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.distributedlog.common.stats.OpStatsListener;
-import org.apache.distributedlog.util.OrderedScheduler;
-
-/**
- * Future related utils.
- */
-@Slf4j
-public final class FutureUtils {
-
-    private FutureUtils() {}
-
-    private static final Function<Throwable, Exception> DEFAULT_EXCEPTION_HANDLER = cause -> {
-        if (cause instanceof Exception) {
-            return (Exception) cause;
-        } else {
-            return new Exception(cause);
-        }
-    };
-
-    public static CompletableFuture<Void> Void() {
-        return value(null);
-    }
-
-    public static <T> T result(CompletableFuture<T> future) throws Exception {
-        return FutureUtils.result(future, DEFAULT_EXCEPTION_HANDLER);
-    }
-
-    public static <T> T result(CompletableFuture<T> future, long timeout, TimeUnit timeUnit) throws Exception {
-        return FutureUtils.result(future, DEFAULT_EXCEPTION_HANDLER, timeout, timeUnit);
-    }
-
-    @SneakyThrows(InterruptedException.class)
-    public static <T, ExceptionT extends Throwable> T result(
-        CompletableFuture<T> future, Function<Throwable, ExceptionT> exceptionHandler) throws ExceptionT {
-        try {
-            return future.get();
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw e;
-        } catch (ExecutionException e) {
-            ExceptionT cause = exceptionHandler.apply(e.getCause());
-            if (null == cause) {
-                return null;
-            } else {
-                throw cause;
-            }
-        }
-    }
-
-    @SneakyThrows(InterruptedException.class)
-    public static <T, ExceptionT extends Throwable> T result(
-        CompletableFuture<T> future,
-        Function<Throwable, ExceptionT> exceptionHandler,
-        long timeout,
-        TimeUnit timeUnit) throws ExceptionT, TimeoutException {
-        try {
-            return future.get(timeout, timeUnit);
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw e;
-        } catch (ExecutionException e) {
-            ExceptionT cause = exceptionHandler.apply(e.getCause());
-            if (null == cause) {
-                return null;
-            } else {
-                throw cause;
-            }
-        }
-    }
-
-    public static <T> CompletableFuture<T> createFuture() {
-        return new CompletableFuture<T>();
-    }
-
-    public static <T> CompletableFuture<T> value(T value) {
-        return CompletableFuture.completedFuture(value);
-    }
-
-    public static <T> CompletableFuture<T> exception(Throwable cause) {
-        CompletableFuture<T> future = FutureUtils.createFuture();
-        future.completeExceptionally(cause);
-        return future;
-    }
-
-    public static <T> void complete(CompletableFuture<T> result,
-                                    T value) {
-        if (null == result) {
-            return;
-        }
-        result.complete(value);
-    }
-
-    public static <T> void completeExceptionally(CompletableFuture<T> result,
-                                                 Throwable cause) {
-        if (null == result) {
-            return;
-        }
-        result.completeExceptionally(cause);
-    }
-
-    /**
-     * Completing the {@code future} in the thread in the scheduler identified by
-     * the {@code scheduleKey}.
-     *
-     * @param future      future to complete
-     * @param action      action to execute when complete
-     * @param scheduler   scheduler to execute the action.
-     * @param scheduleKey key to choose the thread to execute the action
-     * @param <T>
-     * @return
-     */
-    public static <T> CompletableFuture<T> whenCompleteAsync(
-        CompletableFuture<T> future,
-        BiConsumer<? super T, ? super Throwable> action,
-        OrderedScheduler scheduler,
-        Object scheduleKey) {
-        return future.whenCompleteAsync(action, scheduler.chooseExecutor(scheduleKey));
-    }
-
-    public static <T> CompletableFuture<List<T>> collect(List<CompletableFuture<T>> futureList) {
-        CompletableFuture<Void> finalFuture =
-            CompletableFuture.allOf(futureList.toArray(new CompletableFuture[futureList.size()]));
-        return finalFuture.thenApply(result ->
-            futureList
-                .stream()
-                .map(CompletableFuture::join)
-                .collect(Collectors.toList()));
-    }
-
-    public static <T> void proxyTo(CompletableFuture<T> src,
-                                   CompletableFuture<T> target) {
-        src.whenComplete((value, cause) -> {
-            if (null == cause) {
-                target.complete(value);
-            } else {
-                target.completeExceptionally(cause);
-            }
-        });
-    }
-
-    //
-    // Process futures
-    //
-
-    private static class ListFutureProcessor<T, R>
-        implements FutureEventListener<R>, Runnable {
-
-        private volatile boolean done = false;
-        private final Iterator<T> itemsIter;
-        private final Function<T, CompletableFuture<R>> processFunc;
-        private final CompletableFuture<List<R>> promise;
-        private final List<R> results;
-        private final ExecutorService callbackExecutor;
-
-        ListFutureProcessor(List<T> items,
-                            Function<T, CompletableFuture<R>> processFunc,
-                            ExecutorService callbackExecutor) {
-            this.itemsIter = items.iterator();
-            this.processFunc = processFunc;
-            this.promise = new CompletableFuture<>();
-            this.results = Lists.newArrayListWithExpectedSize(items.size());
-            this.callbackExecutor = callbackExecutor;
-        }
-
-        @Override
-        public void onSuccess(R value) {
-            results.add(value);
-            if (null == callbackExecutor) {
-                run();
-            } else {
-                callbackExecutor.submit(this);
-            }
-        }
-
-        @Override
-        public void onFailure(final Throwable cause) {
-            done = true;
-
-            if (null == callbackExecutor) {
-                promise.completeExceptionally(cause);
-            } else {
-                callbackExecutor.submit(new Runnable() {
-                    @Override
-                    public void run() {
-                        promise.completeExceptionally(cause);
-                    }
-                });
-            }
-        }
-
-        @Override
-        public void run() {
-            if (done) {
-                log.debug("ListFutureProcessor is interrupted.");
-                return;
-            }
-            if (!itemsIter.hasNext()) {
-                promise.complete(results);
-                done = true;
-                return;
-            }
-            processFunc.apply(itemsIter.next()).whenComplete(this);
-        }
-    }
-
-    /**
-     * Process the list of items one by one using the process function <i>processFunc</i>.
-     * The process will be stopped immediately if it fails on processing any one.
-     *
-     * @param collection       list of items
-     * @param processFunc      process function
-     * @param callbackExecutor executor to process the item
-     * @return future presents the list of processed results
-     */
-    public static <T, R> CompletableFuture<List<R>> processList(List<T> collection,
-                                                                Function<T, CompletableFuture<R>> processFunc,
-                                                                @Nullable ExecutorService callbackExecutor) {
-        ListFutureProcessor<T, R> processor =
-            new ListFutureProcessor<T, R>(collection, processFunc, callbackExecutor);
-        if (null != callbackExecutor) {
-            callbackExecutor.submit(processor);
-        } else {
-            processor.run();
-        }
-        return processor.promise;
-    }
-
-    /**
-     * Raise an exception to the <i>promise</i> within a given <i>timeout</i> period.
-     * If the promise has been satisfied before raising, it won't change the state of the promise.
-     *
-     * @param promise   promise to raise exception
-     * @param timeout   timeout period
-     * @param unit      timeout period unit
-     * @param cause     cause to raise
-     * @param scheduler scheduler to execute raising exception
-     * @param key       the submit key used by the scheduler
-     * @return the promise applied with the raise logic
-     */
-    public static <T> CompletableFuture<T> within(final CompletableFuture<T> promise,
-                                                  final long timeout,
-                                                  final TimeUnit unit,
-                                                  final Throwable cause,
-                                                  final OrderedScheduler scheduler,
-                                                  final Object key) {
-        if (timeout < 0 || promise.isDone()) {
-            return promise;
-        }
-        // schedule a timeout to raise timeout exception
-        final java.util.concurrent.ScheduledFuture<?> task = scheduler.schedule(key, new Runnable() {
-            @Override
-            public void run() {
-                if (!promise.isDone() && promise.completeExceptionally(cause)) {
-                    log.info("Raise exception", cause);
-                }
-            }
-        }, timeout, unit);
-        // when the promise is satisfied, cancel the timeout task
-        promise.whenComplete((value, throwable) -> {
-                if (!task.cancel(true)) {
-                    log.debug("Failed to cancel the timeout task");
-                }
-            }
-        );
-        return promise;
-    }
-
-    /**
-     * Ignore exception from the <i>future</i>.
-     *
-     * @param future the original future
-     * @return a transformed future ignores exceptions
-     */
-    public static <T> CompletableFuture<Void> ignore(CompletableFuture<T> future) {
-        return ignore(future, null);
-    }
-
-    /**
-     * Ignore exception from the <i>future</i> and log <i>errorMsg</i> on exceptions.
-     *
-     * @param future   the original future
-     * @param errorMsg the error message to log on exceptions
-     * @return a transformed future ignores exceptions
-     */
-    public static <T> CompletableFuture<Void> ignore(CompletableFuture<T> future,
-                                                     final String errorMsg) {
-        final CompletableFuture<Void> promise = new CompletableFuture<Void>();
-        future.whenComplete(new FutureEventListener<T>() {
-            @Override
-            public void onSuccess(T value) {
-                promise.complete(null);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                if (null != errorMsg) {
-                    log.error(errorMsg, cause);
-                }
-                promise.complete(null);
-            }
-        });
-        return promise;
-    }
-
-    public static <T> CompletableFuture<T> ensure(CompletableFuture<T> future,
-                                                  Runnable ensureBlock) {
-        return future.whenComplete((value, cause) -> {
-            ensureBlock.run();
-        });
-    }
-
-    public static <T> CompletableFuture<T> rescue(CompletableFuture<T> future,
-                                                  Function<Throwable, CompletableFuture<T>> rescueFuc) {
-        CompletableFuture<T> result = FutureUtils.createFuture();
-        future.whenComplete((value, cause) -> {
-            if (null == cause) {
-                result.complete(value);
-                return;
-            }
-            proxyTo(rescueFuc.apply(cause), result);
-        });
-        return result;
-    }
-
-    /**
-      * Add a event listener over <i>result</i> for collecting the operation stats.
-      *
-      * @param result result to listen on
-      * @param opStatsLogger stats logger to record operations stats
-      * @param stopwatch stop watch to time operation
-      * @param <T>
-      * @return result after registered the event listener
-      */
-    public static <T> CompletableFuture<T> stats(CompletableFuture<T> result,
-                                                 OpStatsLogger opStatsLogger,
-                                                 Stopwatch stopwatch) {
-        return result.whenComplete(new OpStatsListener<T>(opStatsLogger, stopwatch));
-    }
-
-}
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/package-info.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/package-info.java
index dff0aced3..6125f8c07 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/package-info.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/concurrent/package-info.java
@@ -19,4 +19,4 @@
 /**
  * Utility classes commonly useful in concurrent programming.
  */
-package org.apache.distributedlog.common.concurrent;
\ No newline at end of file
+package org.apache.distributedlog.common.concurrent;
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/package-info.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/package-info.java
index 4c90bd23b..aa2ab9a49 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/package-info.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/package-info.java
@@ -19,4 +19,4 @@
 /**
  * Common functions and utils used across the project.
  */
-package org.apache.distributedlog.common;
\ No newline at end of file
+package org.apache.distributedlog.common;
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/rate/package-info.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/rate/package-info.java
index 3117c6401..03f68b302 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/rate/package-info.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/rate/package-info.java
@@ -19,4 +19,4 @@
 /**
  * Rate relate functions.
  */
-package org.apache.distributedlog.common.rate;
\ No newline at end of file
+package org.apache.distributedlog.common.rate;
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/stats/OpStatsListener.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/stats/OpStatsListener.java
index 4145b39ad..6a8fd743d 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/stats/OpStatsListener.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/stats/OpStatsListener.java
@@ -19,8 +19,8 @@
 
 import com.google.common.base.Stopwatch;
 import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
 
 /**
  * A {@link FutureEventListener} monitors the stats for a given operation.
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/stats/package-info.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/stats/package-info.java
index bf3859dfa..919c459be 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/stats/package-info.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/stats/package-info.java
@@ -18,4 +18,4 @@
 /**
  * Stats Related Utils.
  */
-package org.apache.distributedlog.common.stats;
\ No newline at end of file
+package org.apache.distributedlog.common.stats;
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/util/package-info.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/util/package-info.java
index e2bde3764..72b06b9a3 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/util/package-info.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/common/util/package-info.java
@@ -19,4 +19,4 @@
 /**
  * Common utility functions.
  */
-package org.apache.distributedlog.common.util;
\ No newline at end of file
+package org.apache.distributedlog.common.util;
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/Abortables.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/Abortables.java
index 73da1f1a3..9a34de411 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/Abortables.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/Abortables.java
@@ -24,7 +24,7 @@
 import java.util.concurrent.ExecutorService;
 import javax.annotation.Nullable;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.functions.VoidFunctions;
 
 /**
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java
index 7636c5706..b7b097b73 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java
@@ -19,7 +19,7 @@
 
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 
 /**
  * An {@code Abortable} is a source or destination of data that can be aborted.
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java
index 851f426e3..9e67e5369 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java
@@ -19,7 +19,7 @@
 
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 
 /**
  * A {@code AsyncCloseable} is a source or destination of data that can be closed asynchronously.
diff --git a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java
index 89b448e80..cfafca39e 100644
--- a/stream/distributedlog/common/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java
+++ b/stream/distributedlog/common/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java
@@ -33,7 +33,8 @@
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.distributedlog.common.util.MathUtil;
 
 /**
@@ -47,7 +48,9 @@
  * {@link OrderedScheduler#submit(Object, Runnable)} will be submitted to a dedicated executor based on
  * the hash value of submit <i>key</i>.
  */
-public class OrderedScheduler implements ScheduledExecutorService {
+public class OrderedScheduler
+        extends org.apache.bookkeeper.common.util.OrderedScheduler
+        implements ScheduledExecutorService {
 
     /**
      * Create a builder to build scheduler.
@@ -131,6 +134,14 @@ public OrderedScheduler build() {
     private OrderedScheduler(String name,
                              int corePoolSize,
                              ThreadFactory threadFactory) {
+        super(
+            name,
+            corePoolSize,
+            threadFactory,
+            NullStatsLogger.INSTANCE,
+            false,
+            Long.MAX_VALUE,
+            Integer.MAX_VALUE);
         this.name = name;
         this.corePoolSize = corePoolSize;
         this.executors = new ScheduledExecutorService[corePoolSize];
diff --git a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/concurrent/TestFutureUtils.java b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/concurrent/TestFutureUtils.java
deleted file mode 100644
index fecefbc3f..000000000
--- a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/concurrent/TestFutureUtils.java
+++ /dev/null
@@ -1,385 +0,0 @@
-/*
- * 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.distributedlog.common.concurrent;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyLong;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Function;
-import java.util.stream.LongStream;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.distributedlog.util.OrderedScheduler;
-import org.junit.Test;
-
-/**
- * Unit Test for {@link FutureUtils}.
- */
-public class TestFutureUtils {
-
-    /**
-     * Test Exception.
-     */
-    static class TestException extends IOException {
-        private static final long serialVersionUID = -6256482498453846308L;
-
-        public TestException() {
-            super("test-exception");
-        }
-    }
-
-    @Test
-    public void testComplete() throws Exception {
-        CompletableFuture<Long> future = FutureUtils.createFuture();
-        FutureUtils.complete(future, 1024L);
-        assertEquals(1024L, FutureUtils.result(future).longValue());
-    }
-
-    @Test(expected = TestException.class)
-    public void testCompleteExceptionally() throws Exception {
-        CompletableFuture<Long> future = FutureUtils.createFuture();
-        FutureUtils.completeExceptionally(future, new TestException());
-        FutureUtils.result(future);
-    }
-
-    @Test
-    public void testWhenCompleteAsync() throws Exception {
-        OrderedScheduler scheduler = OrderedScheduler.newBuilder()
-            .name("test-when-complete-async")
-            .corePoolSize(1)
-            .build();
-        AtomicLong resultHolder = new AtomicLong(0L);
-        CountDownLatch latch = new CountDownLatch(1);
-        CompletableFuture<Long> future = FutureUtils.createFuture();
-        FutureUtils.whenCompleteAsync(
-            future,
-            (result, cause) -> {
-                resultHolder.set(result);
-                latch.countDown();
-            },
-            scheduler,
-            new Object());
-        FutureUtils.complete(future, 1234L);
-        latch.await();
-        assertEquals(1234L, resultHolder.get());
-    }
-
-    @Test
-    public void testProxyToSuccess() throws Exception {
-        CompletableFuture<Long> src = FutureUtils.createFuture();
-        CompletableFuture<Long> target = FutureUtils.createFuture();
-        FutureUtils.proxyTo(src, target);
-        FutureUtils.complete(src, 10L);
-        assertEquals(10L, FutureUtils.result(target).longValue());
-    }
-
-    @Test(expected = TestException.class)
-    public void testProxyToFailure() throws Exception {
-        CompletableFuture<Long> src = FutureUtils.createFuture();
-        CompletableFuture<Long> target = FutureUtils.createFuture();
-        FutureUtils.proxyTo(src, target);
-        FutureUtils.completeExceptionally(src, new TestException());
-        FutureUtils.result(target);
-    }
-
-    @Test
-    public void testVoid() throws Exception {
-        CompletableFuture<Void> voidFuture = FutureUtils.Void();
-        assertTrue(voidFuture.isDone());
-        assertFalse(voidFuture.isCompletedExceptionally());
-        assertFalse(voidFuture.isCancelled());
-    }
-
-    @Test
-    public void testCollectEmptyList() throws Exception {
-        List<CompletableFuture<Integer>> futures = Lists.newArrayList();
-        List<Integer> result = FutureUtils.result(FutureUtils.collect(futures));
-        assertTrue(result.isEmpty());
-    }
-
-    @Test
-    public void testCollectTenItems() throws Exception {
-        List<CompletableFuture<Integer>> futures = Lists.newArrayList();
-        List<Integer> expectedResults = Lists.newArrayList();
-        for (int i = 0; i < 10; i++) {
-            futures.add(FutureUtils.value(i));
-            expectedResults.add(i);
-        }
-        List<Integer> results = FutureUtils.result(FutureUtils.collect(futures));
-        assertEquals(expectedResults, results);
-    }
-
-    @Test(expected = TestException.class)
-    public void testCollectFailures() throws Exception {
-        List<CompletableFuture<Integer>> futures = Lists.newArrayList();
-        List<Integer> expectedResults = Lists.newArrayList();
-        for (int i = 0; i < 10; i++) {
-            if (i == 9) {
-                futures.add(FutureUtils.value(i));
-            } else {
-                futures.add(FutureUtils.exception(new TestException()));
-            }
-            expectedResults.add(i);
-        }
-        FutureUtils.result(FutureUtils.collect(futures));
-    }
-
-    @Test
-    public void testWithinAlreadyDone() throws Exception {
-        OrderedScheduler scheduler = mock(OrderedScheduler.class);
-        CompletableFuture<Long> doneFuture = FutureUtils.value(1234L);
-        CompletableFuture<Long> withinFuture = FutureUtils.within(
-            doneFuture,
-            10,
-            TimeUnit.MILLISECONDS,
-            new TestException(),
-            scheduler,
-            1234L);
-        TimeUnit.MILLISECONDS.sleep(20);
-        assertTrue(withinFuture.isDone());
-        assertFalse(withinFuture.isCancelled());
-        assertFalse(withinFuture.isCompletedExceptionally());
-        verify(scheduler, times(0))
-            .schedule(eq(1234L), any(Runnable.class), eq(10), eq(TimeUnit.MILLISECONDS));
-    }
-
-    @Test
-    public void testWithinZeroTimeout() throws Exception {
-        OrderedScheduler scheduler = mock(OrderedScheduler.class);
-        CompletableFuture<Long> newFuture = FutureUtils.createFuture();
-        CompletableFuture<Long> withinFuture = FutureUtils.within(
-            newFuture,
-            0,
-            TimeUnit.MILLISECONDS,
-            new TestException(),
-            scheduler,
-            1234L);
-        TimeUnit.MILLISECONDS.sleep(20);
-        assertFalse(withinFuture.isDone());
-        assertFalse(withinFuture.isCancelled());
-        assertFalse(withinFuture.isCompletedExceptionally());
-        verify(scheduler, times(0))
-            .schedule(eq(1234L), any(Runnable.class), eq(10), eq(TimeUnit.MILLISECONDS));
-    }
-
-    @Test
-    public void testWithinCompleteBeforeTimeout() throws Exception {
-        OrderedScheduler scheduler = mock(OrderedScheduler.class);
-        ScheduledFuture<?> scheduledFuture = mock(ScheduledFuture.class);
-        when(scheduler.schedule(any(), any(Runnable.class), anyLong(), any(TimeUnit.class)))
-            .thenAnswer(invocationOnMock -> scheduledFuture);
-        CompletableFuture<Long> newFuture = FutureUtils.createFuture();
-        CompletableFuture<Long> withinFuture = FutureUtils.within(
-            newFuture,
-            Long.MAX_VALUE,
-            TimeUnit.MILLISECONDS,
-            new TestException(),
-            scheduler,
-            1234L);
-        assertFalse(withinFuture.isDone());
-        assertFalse(withinFuture.isCancelled());
-        assertFalse(withinFuture.isCompletedExceptionally());
-
-        newFuture.complete(5678L);
-
-        assertTrue(withinFuture.isDone());
-        assertFalse(withinFuture.isCancelled());
-        assertFalse(withinFuture.isCompletedExceptionally());
-        assertEquals((Long) 5678L, FutureUtils.result(withinFuture));
-
-        verify(scheduledFuture, times(1))
-            .cancel(eq(true));
-    }
-
-    @Test
-    public void testIgnoreSuccess() {
-        CompletableFuture<Long> underlyFuture = FutureUtils.createFuture();
-        CompletableFuture<Void> ignoredFuture = FutureUtils.ignore(underlyFuture);
-        underlyFuture.complete(1234L);
-        assertTrue(ignoredFuture.isDone());
-        assertFalse(ignoredFuture.isCompletedExceptionally());
-        assertFalse(ignoredFuture.isCancelled());
-    }
-
-    @Test
-    public void testIgnoreFailure() {
-        CompletableFuture<Long> underlyFuture = FutureUtils.createFuture();
-        CompletableFuture<Void> ignoredFuture = FutureUtils.ignore(underlyFuture);
-        underlyFuture.completeExceptionally(new TestException());
-        assertTrue(ignoredFuture.isDone());
-        assertFalse(ignoredFuture.isCompletedExceptionally());
-        assertFalse(ignoredFuture.isCancelled());
-    }
-
-    @Test
-    public void testEnsureSuccess() throws Exception {
-        CountDownLatch ensureLatch = new CountDownLatch(1);
-        CompletableFuture<Long> underlyFuture = FutureUtils.createFuture();
-        CompletableFuture<Long> ensuredFuture = FutureUtils.ensure(underlyFuture, () -> {
-            ensureLatch.countDown();
-        });
-        underlyFuture.complete(1234L);
-        FutureUtils.result(ensuredFuture);
-        assertTrue(ensuredFuture.isDone());
-        assertFalse(ensuredFuture.isCompletedExceptionally());
-        assertFalse(ensuredFuture.isCancelled());
-        ensureLatch.await();
-    }
-
-    @Test
-    public void testEnsureFailure() throws Exception {
-        CountDownLatch ensureLatch = new CountDownLatch(1);
-        CompletableFuture<Long> underlyFuture = FutureUtils.createFuture();
-        CompletableFuture<Long> ensuredFuture = FutureUtils.ensure(underlyFuture, () -> {
-            ensureLatch.countDown();
-        });
-        underlyFuture.completeExceptionally(new TestException());
-        FutureUtils.result(FutureUtils.ignore(ensuredFuture));
-        assertTrue(ensuredFuture.isDone());
-        assertTrue(ensuredFuture.isCompletedExceptionally());
-        assertFalse(ensuredFuture.isCancelled());
-        ensureLatch.await();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testRescueSuccess() throws Exception {
-        CompletableFuture<Long> underlyFuture = FutureUtils.createFuture();
-        Function<Throwable, CompletableFuture<Long>> rescueFuc = mock(Function.class);
-        CompletableFuture<Long> rescuedFuture = FutureUtils.rescue(underlyFuture, rescueFuc);
-        underlyFuture.complete(1234L);
-        FutureUtils.result(rescuedFuture);
-        assertTrue(rescuedFuture.isDone());
-        assertFalse(rescuedFuture.isCompletedExceptionally());
-        assertFalse(rescuedFuture.isCancelled());
-        verify(rescueFuc, times(0)).apply(any(Throwable.class));
-    }
-
-    @Test
-    public void testRescueFailure() throws Exception {
-        CompletableFuture<Long> futureCompletedAtRescue = FutureUtils.value(3456L);
-        CompletableFuture<Long> underlyFuture = FutureUtils.createFuture();
-        CompletableFuture<Long> rescuedFuture = FutureUtils.rescue(underlyFuture, (cause) -> futureCompletedAtRescue);
-        underlyFuture.completeExceptionally(new TestException());
-        FutureUtils.result(rescuedFuture);
-        assertTrue(rescuedFuture.isDone());
-        assertFalse(rescuedFuture.isCompletedExceptionally());
-        assertFalse(rescuedFuture.isCancelled());
-        assertEquals((Long) 3456L, FutureUtils.result(rescuedFuture));
-    }
-
-    @Test
-    public void testStatsSuccess() throws Exception {
-        OpStatsLogger statsLogger = mock(OpStatsLogger.class);
-        CompletableFuture<Long> underlyFuture = FutureUtils.createFuture();
-        CompletableFuture<Long> statsFuture = FutureUtils.stats(
-            underlyFuture,
-            statsLogger,
-            Stopwatch.createStarted());
-        underlyFuture.complete(1234L);
-        FutureUtils.result(statsFuture);
-        verify(statsLogger, times(1))
-            .registerSuccessfulEvent(anyLong(), eq(TimeUnit.MICROSECONDS));
-    }
-
-    @Test
-    public void testStatsFailure() throws Exception {
-        OpStatsLogger statsLogger = mock(OpStatsLogger.class);
-        CompletableFuture<Long> underlyFuture = FutureUtils.createFuture();
-        CompletableFuture<Long> statsFuture = FutureUtils.stats(
-            underlyFuture,
-            statsLogger,
-            Stopwatch.createStarted());
-        underlyFuture.completeExceptionally(new TestException());
-        FutureUtils.result(FutureUtils.ignore(statsFuture));
-        verify(statsLogger, times(1))
-            .registerFailedEvent(anyLong(), eq(TimeUnit.MICROSECONDS));
-    }
-
-    @Test
-    public void testProcessListSuccess() throws Exception {
-        List<Long> longList = Lists.newArrayList(LongStream.range(0L, 10L).iterator());
-        List<Long> expectedList = Lists.transform(
-            longList,
-            aLong -> 2 * aLong);
-        Function<Long, CompletableFuture<Long>> sumFunc = value -> FutureUtils.value(2 * value);
-        CompletableFuture<List<Long>> totalFuture = FutureUtils.processList(
-            longList,
-            sumFunc,
-            null);
-        assertEquals(expectedList, FutureUtils.result(totalFuture));
-    }
-
-    @Test
-    public void testProcessEmptyList() throws Exception {
-        List<Long> longList = Lists.newArrayList();
-        List<Long> expectedList = Lists.transform(
-            longList,
-            aLong -> 2 * aLong);
-        Function<Long, CompletableFuture<Long>> sumFunc = value -> FutureUtils.value(2 * value);
-        CompletableFuture<List<Long>> totalFuture = FutureUtils.processList(
-            longList,
-            sumFunc,
-            null);
-        assertEquals(expectedList, FutureUtils.result(totalFuture));
-    }
-
-    @Test
-    public void testProcessListFailures() throws Exception {
-        List<Long> longList = Lists.newArrayList(LongStream.range(0L, 10L).iterator());
-        AtomicLong total = new AtomicLong(0L);
-        Function<Long, CompletableFuture<Long>> sumFunc = value -> {
-            if (value < 5) {
-                total.addAndGet(value);
-                return FutureUtils.value(2 * value);
-            } else {
-                return FutureUtils.exception(new TestException());
-            }
-        };
-        CompletableFuture<List<Long>> totalFuture = FutureUtils.processList(
-            longList,
-            sumFunc,
-            null);
-        try {
-            FutureUtils.result(totalFuture);
-            fail("Should fail with TestException");
-        } catch (TestException te) {
-            // as expected
-        }
-        assertEquals(10L, total.get());
-    }
-
-}
diff --git a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/util/TestTimedOutTestsListener.java b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/util/TestTimedOutTestsListener.java
index 7a981d174..d38fcdf42 100644
--- a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/util/TestTimedOutTestsListener.java
+++ b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/util/TestTimedOutTestsListener.java
@@ -22,7 +22,6 @@
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
-
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.notification.Failure;
diff --git a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/util/TimedOutTestsListener.java b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/util/TimedOutTestsListener.java
index c86cf8f5e..1f61038aa 100644
--- a/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/util/TimedOutTestsListener.java
+++ b/stream/distributedlog/common/src/test/java/org/apache/distributedlog/common/util/TimedOutTestsListener.java
@@ -28,7 +28,6 @@
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Map;
-
 import org.apache.commons.lang.StringUtils;
 import org.junit.runner.notification.Failure;
 import org.junit.runner.notification.RunListener;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/package-info.java b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/package-info.java
index ca571079f..f1f73d960 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/package-info.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/bookkeeper/client/package-info.java
@@ -19,4 +19,4 @@
 /**
  * DistributedLog overrides on bookkeeper client.
  */
-package org.apache.bookkeeper.client;
\ No newline at end of file
+package org.apache.bookkeeper.client;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
index 8bc2ab6d3..357fc6c68 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
@@ -18,9 +18,9 @@
 package org.apache.distributedlog;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.io.IOException;
 import java.io.InputStream;
-
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.LogReader;
 import org.slf4j.Logger;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
index 93ad61040..9c22d2ca3 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
@@ -20,8 +20,8 @@
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.UnexpectedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
index fb40a6b8c..275c8d754 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
@@ -18,7 +18,6 @@
 package org.apache.distributedlog;
 
 import com.google.common.annotations.VisibleForTesting;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
@@ -26,8 +25,8 @@
 import java.util.concurrent.CompletionStage;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.util.PermitManager;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
index 0c6735a71..0612d3aa1 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
@@ -31,12 +31,12 @@
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.distributedlog.api.AsyncLogReader;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.exceptions.EndOfStreamException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
index a492f5152..18e6757ee 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
@@ -26,14 +26,14 @@
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.distributedlog.api.AsyncLogWriter;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.StreamNotReadyException;
 import org.apache.distributedlog.exceptions.WriteCancelledException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
index 5ca76d35d..1be0aec8d 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
@@ -23,25 +23,17 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
-
 import java.io.IOException;
 import java.net.URI;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
-
 import java.util.function.Function;
-
-
-
-
-
-
-
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
@@ -50,16 +42,11 @@
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.api.subscription.SubscriptionsStore;
-
 import org.apache.distributedlog.callback.LogSegmentListener;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.stats.BroadCastStatsLogger;
-
 import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.common.util.PermitManager;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.LogEmptyException;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
@@ -72,19 +59,14 @@
 import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
 import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
 import org.apache.distributedlog.logsegment.LogSegmentFilter;
-
 import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
-
 import org.apache.distributedlog.metadata.LogMetadataForReader;
 import org.apache.distributedlog.metadata.LogMetadataForWriter;
-
 import org.apache.distributedlog.metadata.LogStreamMetadataStore;
 import org.apache.distributedlog.namespace.NamespaceDriver;
 import org.apache.distributedlog.util.Allocator;
 import org.apache.distributedlog.util.DLUtils;
-
 import org.apache.distributedlog.util.OrderedScheduler;
-
 import org.apache.distributedlog.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
index d2e21695c..b4cf0f737 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
@@ -28,13 +28,13 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.distributedlog.acl.AccessControlManager;
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.callback.NamespaceListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.common.util.SchedulerUtils;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
@@ -47,7 +47,6 @@
 import org.apache.distributedlog.util.ConfUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java
index ddf07346f..f48baac1d 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogHandler.java
@@ -33,6 +33,8 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
@@ -40,8 +42,6 @@
 import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.distributedlog.callback.LogSegmentNamesListener;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.LogEmptyException;
 import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
@@ -52,11 +52,8 @@
 import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
 import org.apache.distributedlog.logsegment.PerStreamLogSegmentCache;
-
 import org.apache.distributedlog.metadata.LogMetadata;
-
 import org.apache.distributedlog.metadata.LogStreamMetadataStore;
-
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
index 5ebdd3b3b..4ce16121c 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
@@ -28,6 +28,8 @@
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
@@ -35,8 +37,6 @@
 import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.distributedlog.callback.LogSegmentListener;
 import org.apache.distributedlog.callback.LogSegmentNamesListener;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
 import org.apache.distributedlog.exceptions.LockingException;
@@ -49,8 +49,6 @@
 import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
 import org.apache.distributedlog.metadata.LogMetadataForReader;
 import org.apache.distributedlog.metadata.LogStreamMetadataStore;
-
-
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
 import org.slf4j.Logger;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
index 8584780c6..cecb4f917 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
@@ -40,6 +40,8 @@
 import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
@@ -49,8 +51,6 @@
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.util.MathUtils;
 import org.apache.distributedlog.Entry.Writer;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.stats.OpStatsListener;
 import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.common.util.Sizable;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
index 955fafed4..2c5aaf28b 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
@@ -30,14 +30,14 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
@@ -62,8 +62,6 @@
 import org.apache.distributedlog.util.Allocator;
 import org.apache.distributedlog.util.DLUtils;
 import org.apache.distributedlog.util.FailpointUtils;
-
-
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Transaction;
 import org.apache.distributedlog.util.Utils;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
index 35457b294..892ab063e 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
@@ -26,11 +26,11 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.distributedlog.api.AsyncLogReader;
 import org.apache.distributedlog.api.LogReader;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.EndOfStreamException;
 import org.apache.distributedlog.exceptions.IdleReaderException;
 import org.apache.distributedlog.util.Utils;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
index ef75a6dd0..82aaa22b9 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
@@ -19,8 +19,8 @@
 
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 
 class BKTransmitPacket {
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
index ee16e5d51..366178a9d 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog;
 
 import static com.google.common.base.Charsets.UTF_8;
+
 import com.google.common.base.Optional;
 import io.netty.channel.EventLoopGroup;
 import io.netty.util.HashedWheelTimer;
@@ -29,6 +30,7 @@
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.net.DNSToSwitchMapping;
@@ -38,7 +40,6 @@
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.distributedlog.ZooKeeperClient.Credentials;
 import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.net.NetUtils;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
index d348287ae..b16034cf2 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
@@ -19,6 +19,7 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
+
 import com.google.common.base.Optional;
 import io.netty.channel.EventLoopGroup;
 import io.netty.util.HashedWheelTimer;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
index 603859f71..5b16fe000 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
@@ -18,17 +18,16 @@
 package org.apache.distributedlog;
 
 import static com.google.common.base.Preconditions.checkArgument;
+
 import com.google.common.base.Optional;
 import com.google.common.collect.Sets;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Set;
-
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.net.DNSToSwitchMapping;
-
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.util.ReflectionUtils;
 import org.apache.commons.configuration.CompositeConfiguration;
@@ -38,13 +37,10 @@
 import org.apache.commons.configuration.SystemConfiguration;
 import org.apache.commons.lang.StringUtils;
 import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-
-
 import org.apache.distributedlog.bk.QuorumConfig;
 import org.apache.distributedlog.feature.DefaultFeatureProvider;
 import org.apache.distributedlog.net.DNSResolverForRacks;
 import org.apache.distributedlog.net.DNSResolverForRows;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
index d8bdc3da8..0ebb3f27c 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog;
 
 import static com.google.common.base.Charsets.UTF_8;
+
 import com.google.common.collect.ImmutableList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/Entry.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/Entry.java
index 2a44a5e28..5122b3b96 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/Entry.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/Entry.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+
 import io.netty.buffer.ByteBuf;
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
index a9b38841f..f13273fdd 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
@@ -18,13 +18,14 @@
 package org.apache.distributedlog;
 
 import static com.google.common.base.Charsets.UTF_8;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
 import java.io.File;
 import java.io.IOException;
 import java.util.Comparator;
 import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
 import org.apache.distributedlog.exceptions.UnsupportedMetadataVersionException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
index 33ff9915d..0469af118 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
@@ -33,11 +33,11 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.distributedlog.callback.LogSegmentListener;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java
index 29684a348..4b1991aa0 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ReadUtils.java
@@ -25,16 +25,14 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
 import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
 import org.apache.distributedlog.selector.FirstDLSNNotLessThanSelector;
 import org.apache.distributedlog.selector.FirstTxIdNotLessThanSelector;
 import org.apache.distributedlog.selector.LastRecordSelector;
 import org.apache.distributedlog.selector.LogRecordSelector;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
index 8f074ebed..9bb8813e0 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
@@ -18,8 +18,8 @@
 package org.apache.distributedlog;
 
 import static com.google.common.base.Charsets.UTF_8;
-import com.google.common.base.Stopwatch;
 
+import com.google.common.base.Stopwatch;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.List;
@@ -40,7 +40,6 @@
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.ACL;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
index 9e2ec96c4..3cc20e236 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
@@ -19,6 +19,7 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.net.URI;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
@@ -26,8 +27,6 @@
 import org.apache.distributedlog.ZooKeeperClient.Credentials;
 import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
-
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
index 1183869ac..52d19c3d5 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
@@ -18,8 +18,8 @@
 package org.apache.distributedlog.admin;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import com.google.common.collect.Lists;
 
+import com.google.common.collect.Lists;
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
@@ -32,16 +32,15 @@
 import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.CompletableFuture;
-
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.CountDownLatch;
-
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Function;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.util.IOUtils;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Options;
@@ -53,23 +52,16 @@
 import org.apache.distributedlog.ZooKeeperClientBuilder;
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.namespace.Namespace;
-
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.util.SchedulerUtils;
-
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
-
-
 import org.apache.distributedlog.impl.BKNamespaceDriver;
 import org.apache.distributedlog.impl.acl.ZKAccessControl;
 import org.apache.distributedlog.impl.federated.FederatedZKLogMetadataStore;
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
-
 import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
 import org.apache.distributedlog.metadata.DLMetadata;
 import org.apache.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
 import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-
 import org.apache.distributedlog.metadata.MetadataUpdater;
 import org.apache.distributedlog.namespace.NamespaceDriver;
 import org.apache.distributedlog.thrift.AccessControlEntry;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java
index db4b65e25..c5b9e03fc 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java
@@ -23,14 +23,12 @@
 import java.util.concurrent.CompletableFuture;
 import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public;
 import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving;
-
 import org.apache.distributedlog.AppendOnlyStreamReader;
 import org.apache.distributedlog.AppendOnlyStreamWriter;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.LogRecordWithDLSN;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.api.subscription.SubscriptionsStore;
-
 import org.apache.distributedlog.callback.LogSegmentListener;
 import org.apache.distributedlog.io.AsyncCloseable;
 import org.apache.distributedlog.namespace.NamespaceDriver;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java
index c55503128..8484be9cd 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java
@@ -18,9 +18,9 @@
 package org.apache.distributedlog.api.namespace;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.io.IOException;
 import java.net.URI;
-
 import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public;
 import org.apache.bookkeeper.common.annotation.InterfaceStability.Stable;
 import org.apache.bookkeeper.feature.Feature;
@@ -42,7 +42,6 @@
 import org.apache.distributedlog.util.DLUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.SimplePermitLimiter;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/package-info.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/package-info.java
index fa8f288a6..71ce026a7 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/package-info.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/api/namespace/package-info.java
@@ -19,4 +19,4 @@
 /**
  * Namespace API and the builder to build namespace instance.
  */
-package org.apache.distributedlog.api.namespace;
\ No newline at end of file
+package org.apache.distributedlog.api.namespace;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
index 695f6e834..655762c6d 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
@@ -44,32 +44,27 @@
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeperAccessor;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
 import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
 import org.apache.bookkeeper.zookeeper.RetryPolicy;
 import org.apache.commons.lang3.tuple.Pair;
-
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.BookKeeperClientBuilder;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
-
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
-
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
 import org.apache.distributedlog.namespace.NamespaceDriver;
 import org.apache.distributedlog.util.DLUtils;
-
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java
index 36d6cbb44..15db0bfe2 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 package org.apache.distributedlog.bk;
+
 import java.io.IOException;
 import org.apache.bookkeeper.client.LedgerHandle;
-
 import org.apache.distributedlog.util.Allocator;
 
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java
index f888f6f17..b86d39732 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java
@@ -16,12 +16,11 @@
  * limitations under the License.
  */
 package org.apache.distributedlog.bk;
+
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
-
 import org.apache.bookkeeper.client.LedgerHandle;
-
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.Transaction;
 import org.apache.distributedlog.util.Transaction.OpListener;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java
index 9f3668762..2f654bb6b 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java
@@ -32,14 +32,14 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.bookkeeper.versioning.LongVersion;
 import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.util.Transaction;
 import org.apache.distributedlog.util.Utils;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java
index aa0bc012d..367a97d30 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java
@@ -25,14 +25,14 @@
 import java.util.concurrent.CompletionStage;
 import java.util.function.Function;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.versioning.LongVersion;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.DLUtils;
 import org.apache.distributedlog.util.Transaction;
 import org.apache.distributedlog.util.Transaction.OpListener;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java
index 88a0535fd..3701d2abf 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java
@@ -18,7 +18,6 @@
 package org.apache.distributedlog.callback;
 
 import java.util.List;
-
 import org.apache.distributedlog.LogSegmentMetadata;
 
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java
index ca4f58179..260925df7 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java
@@ -18,7 +18,6 @@
 package org.apache.distributedlog.callback;
 
 import java.util.List;
-
 import org.apache.bookkeeper.versioning.Versioned;
 
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java
index 4b75fd41f..8a7a1bb87 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java
@@ -18,7 +18,6 @@
 package org.apache.distributedlog.callback;
 
 import com.google.common.annotations.Beta;
-
 import java.util.Iterator;
 
 /**
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java
index 3753014c8..1bfcd9cb3 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java
@@ -20,7 +20,6 @@
 import static com.google.common.base.Preconditions.checkNotNull;
 
 import com.google.common.base.Optional;
-
 import com.google.common.collect.Lists;
 import java.io.File;
 import java.net.MalformedURLException;
@@ -30,13 +29,8 @@
 import java.util.Map;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
-
-
-
-
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.distributedlog.DistributedLogConfiguration;
-
 import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
 import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
 import org.apache.distributedlog.common.config.ConfigurationSubscription;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
index 4e2755f12..a6f4d8dee 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
@@ -17,7 +17,37 @@
  */
 package org.apache.distributedlog.config;
 
-import static org.apache.distributedlog.DistributedLogConfiguration.*;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ACK_QUORUM_SIZE;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_DESERIALIZE_RECORDSET_ON_READS;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_DESERIALIZE_RECORDSET_ON_READS_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_ENABLE_LEDGER_ALLOCATOR_POOL;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_ENABLE_LEDGER_ALLOCATOR_POOL_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_IS_DURABLE_WRITE_ENABLED;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_MAX_CACHED_PARTITIONS_PER_PROXY;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_MAX_CACHED_PARTITIONS_PER_PROXY_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_OUTPUT_BUFFER_SIZE;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_OUTPUT_BUFFER_SIZE_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_OUTPUT_BUFFER_SIZE_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_BATCHSIZE;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_BATCHSIZE_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_BATCHSIZE_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_MAX_RECORDS;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_MAX_RECORDS_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_MAX_RECORDS_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS_OLD;
 
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.bk.QuorumConfig;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java
index 86b352f66..a6b5bf5de 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java
@@ -22,12 +22,10 @@
 import java.lang.reflect.InvocationTargetException;
 import org.apache.bookkeeper.feature.CacheableFeatureProvider;
 import org.apache.bookkeeper.feature.Feature;
-
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.distributedlog.DistributedLogConfiguration;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java
index 7475bf758..ae0c2dceb 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java
@@ -23,7 +23,6 @@
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.feature.SettableFeature;
 import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java
index bec6f582b..0e3544c65 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog.feature;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import java.io.File;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java
index 17f3e7ba4..28c699bd4 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java
@@ -27,7 +27,6 @@
 import io.netty.channel.epoll.EpollEventLoopGroup;
 import io.netty.channel.nio.NioEventLoopGroup;
 import io.netty.util.HashedWheelTimer;
-
 import java.io.IOException;
 import java.net.URI;
 import java.util.HashMap;
@@ -41,21 +40,17 @@
 import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
 import org.apache.bookkeeper.zookeeper.RetryPolicy;
 import org.apache.commons.lang.SystemUtils;
-
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.BookKeeperClientBuilder;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
-
 import org.apache.distributedlog.acl.AccessControlManager;
 import org.apache.distributedlog.acl.DefaultAccessControlManager;
 import org.apache.distributedlog.api.subscription.SubscriptionsStore;
 import org.apache.distributedlog.bk.LedgerAllocator;
 import org.apache.distributedlog.bk.LedgerAllocatorUtils;
-
-
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.InvalidStreamNameException;
@@ -74,7 +69,6 @@
 import org.apache.distributedlog.namespace.NamespaceDriverManager;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
-
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.common.PathUtils;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java
index fbbf2ab92..661688cd5 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.impl;
 
-import static org.apache.distributedlog.util.DLUtils.*;
+import static org.apache.distributedlog.util.DLUtils.isReservedStreamName;
 
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
@@ -25,11 +25,11 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.callback.NamespaceListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.metadata.LogMetadataStore;
 import org.apache.distributedlog.util.OrderedScheduler;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java
index 1b5e5d5bb..ebfe4c743 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java
@@ -31,6 +31,7 @@
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.bookkeeper.versioning.LongVersion;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
@@ -38,7 +39,6 @@
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.callback.LogSegmentNamesListener;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
 import org.apache.distributedlog.exceptions.ZKException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java
index 37a9eca92..87cfefa2b 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java
@@ -18,24 +18,22 @@
 package org.apache.distributedlog.impl;
 
 import static org.apache.distributedlog.impl.BKNamespaceDriver.getZKServersFromDLUri;
-import com.google.common.annotations.VisibleForTesting;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.io.IOException;
 import java.net.URI;
-
 import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
 import org.apache.bookkeeper.zookeeper.RetryPolicy;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
 import org.apache.distributedlog.util.Utils;
-
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java
index 6bd974507..ec0bc5b26 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.impl;
 
-import static org.apache.distributedlog.util.DLUtils.*;
+import static org.apache.distributedlog.util.DLUtils.isReservedStreamName;
 
 import java.net.URI;
 import java.util.ArrayList;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java
index 22efb7e61..af358ee11 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java
@@ -16,15 +16,14 @@
  * limitations under the License.
  */
 package org.apache.distributedlog.impl.acl;
+
 import static com.google.common.base.Charsets.UTF_8;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
-
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.util.concurrent.CompletableFuture;
-
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.thrift.AccessControlEntry;
 import org.apache.thrift.TException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java
index 925b14dc9..7b775c41d 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java
@@ -28,12 +28,12 @@
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.thrift.AccessControlEntry;
 import org.apache.zookeeper.AsyncCallback;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
index ef55f4d88..5dd6a88d7 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
@@ -33,27 +33,24 @@
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
-
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.callback.NamespaceListener;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.LogExistsException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
 import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.impl.ZKNamespaceWatcher;
 import org.apache.distributedlog.metadata.LogMetadataStore;
 import org.apache.distributedlog.namespace.NamespaceWatcher;
-
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.AsyncCallback;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
index 2ae27e2b4..1ab944303 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
@@ -21,7 +21,6 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
 import org.apache.bookkeeper.client.LedgerHandle;
-
 import org.apache.distributedlog.bk.LedgerAllocator;
 import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
 import org.apache.distributedlog.util.Allocator;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
index a3837ae03..be609a077 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
@@ -38,12 +38,12 @@
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerEntry;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.Entry;
 import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
@@ -52,7 +52,6 @@
 import org.apache.distributedlog.injector.AsyncFailureInjector;
 import org.apache.distributedlog.logsegment.LogSegmentEntryReader;
 import org.apache.distributedlog.util.OrderedScheduler;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
index d53fcd9cf..c4ffbe48d 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
@@ -18,12 +18,14 @@
 package org.apache.distributedlog.impl.logsegment;
 
 import static com.google.common.base.Charsets.UTF_8;
+
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 import org.apache.bookkeeper.client.AsyncCallback;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.DistributedLogConfiguration;
@@ -34,7 +36,6 @@
 import org.apache.distributedlog.bk.LedgerAllocatorDelegator;
 import org.apache.distributedlog.bk.QuorumConfigProvider;
 import org.apache.distributedlog.bk.SimpleLedgerAllocator;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.injector.AsyncFailureInjector;
@@ -45,7 +46,6 @@
 import org.apache.distributedlog.metadata.LogMetadataForWriter;
 import org.apache.distributedlog.util.Allocator;
 import org.apache.distributedlog.util.OrderedScheduler;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
index 62cfb98b7..8221ebfae 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
@@ -26,10 +26,10 @@
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.LedgerEntry;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.Entry;
 import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java
index 624d6197c..e507c32d8 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java
@@ -23,7 +23,7 @@
 import org.apache.bookkeeper.client.AsyncCallback;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.functions.VoidFunctions;
 
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java
index 4bcb47481..58cacae3e 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog.impl.metadata;
 
 import static com.google.common.base.Charsets.UTF_8;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
 import java.io.IOException;
@@ -25,8 +26,6 @@
 import java.net.URI;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-
-
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.ZooKeeperClient;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
index d8d7674e8..031b8a050 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
@@ -22,7 +22,13 @@
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.distributedlog.DistributedLogConstants.EMPTY_BYTES;
 import static org.apache.distributedlog.DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
-import static org.apache.distributedlog.metadata.LogMetadata.*;
+import static org.apache.distributedlog.metadata.LogMetadata.ALLOCATION_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.LAYOUT_VERSION;
+import static org.apache.distributedlog.metadata.LogMetadata.LOCK_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.LOGSEGMENTS_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.MAX_TXID_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.READ_LOCK_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.VERSION_PATH;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
@@ -36,6 +42,7 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.versioning.LongVersion;
 import org.apache.bookkeeper.versioning.Versioned;
@@ -44,7 +51,6 @@
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClient.ZooKeeperConnectionException;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.util.PermitManager;
 import org.apache.distributedlog.common.util.SchedulerUtils;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
index dd8754ddd..c68000006 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
@@ -21,11 +21,10 @@
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicReference;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.api.subscription.SubscriptionStateStore;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.AsyncCallback;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java
index c2c90fcfe..eda9af755 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java
@@ -25,13 +25,13 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.versioning.LongVersion;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.api.subscription.SubscriptionStateStore;
 import org.apache.distributedlog.api.subscription.SubscriptionsStore;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.AsyncCallback;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java
index 08f0135ba..e8e6c7d25 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog.limiter;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.distributedlog.exceptions.OverCapacityException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java
index 123c4375c..47f032fc5 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java
@@ -20,7 +20,7 @@
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java
index 79f5ae2fc..c8668e76f 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java
@@ -18,7 +18,7 @@
 package org.apache.distributedlog.lock;
 
 import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.LockingException;
 
 /**
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java
index fbe4eb2d6..1d18417a9 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java
@@ -25,12 +25,12 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.distributedlog.common.concurrent.AsyncSemaphore;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java
index 2d92152bb..3b42c2867 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java
@@ -33,6 +33,8 @@
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Function;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.OpStatsLogger;
@@ -41,9 +43,6 @@
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-
 import org.apache.distributedlog.common.stats.OpStatsListener;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.exceptions.LockingException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java
index 6c592f491..9de94d567 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java
@@ -23,9 +23,9 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.util.OrderedScheduler;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java
index ca4c36f9a..42c42b145 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog.metadata;
 
 import static com.google.common.base.Charsets.UTF_8;
+
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.StringReader;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
index db84983ad..9b57e11df 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
@@ -18,8 +18,8 @@
 package org.apache.distributedlog.metadata;
 
 import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
 import org.apache.distributedlog.util.Transaction;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java
index 6f5f86585..c07d5b4f2 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java
@@ -18,7 +18,6 @@
 package org.apache.distributedlog.metadata;
 
 import com.google.common.base.Optional;
-
 import java.net.URI;
 
 /**
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
index aef60ff2a..205de3afc 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
@@ -19,6 +19,7 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
+
 import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.DistributedLogConfiguration;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java
index e580c8bc8..b35de8fe7 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog.namespace;
 
-import static com.google.common.base.Preconditions.*;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import com.google.common.base.Objects;
 import com.google.common.collect.Sets;
 import java.net.URI;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/thrift/AccessControlEntry.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/thrift/AccessControlEntry.java
index 35056d9fa..cbfceb780 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/thrift/AccessControlEntry.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/thrift/AccessControlEntry.java
@@ -6,32 +6,21 @@
  */
 package org.apache.distributedlog.thrift;
 
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
 import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
+import java.util.BitSet;
+import java.util.Collections;
 import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
 import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import javax.annotation.Generated;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+import org.apache.thrift.scheme.TupleScheme;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 @Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2017-05-27")
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/thrift/BKDLConfigFormat.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/thrift/BKDLConfigFormat.java
index 6fb0dd672..ac9a03df3 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/thrift/BKDLConfigFormat.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/thrift/BKDLConfigFormat.java
@@ -6,32 +6,21 @@
  */
 package org.apache.distributedlog.thrift;
 
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
 import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
+import java.util.BitSet;
+import java.util.Collections;
 import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
 import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import javax.annotation.Generated;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+import org.apache.thrift.scheme.TupleScheme;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 @Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2017-05-27")
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
index 8ae05fe58..02edeba7e 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
@@ -19,6 +19,7 @@
 
 import static com.google.common.base.Charsets.UTF_8;
 import static com.google.common.base.Preconditions.checkArgument;
+
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import io.netty.buffer.ByteBuf;
@@ -61,6 +62,8 @@
 import org.apache.bookkeeper.client.LedgerEntry;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.client.LedgerReader;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
 import org.apache.bookkeeper.util.IOUtils;
@@ -92,8 +95,6 @@
 import org.apache.distributedlog.bk.LedgerAllocator;
 import org.apache.distributedlog.bk.LedgerAllocatorUtils;
 import org.apache.distributedlog.callback.NamespaceListener;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/DLUtils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/DLUtils.java
index a0083b244..c89db4745 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/DLUtils.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/DLUtils.java
@@ -20,6 +20,7 @@
 import static com.google.common.base.Charsets.UTF_8;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
+
 import com.google.common.base.Objects;
 import java.net.InetAddress;
 import java.net.URI;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java
index d37e86540..0d3be1b21 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 import java.util.concurrent.ConcurrentHashMap;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java
index c54de9363..b5bc49995 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java
@@ -18,14 +18,12 @@
 package org.apache.distributedlog.util;
 
 import com.google.common.annotations.VisibleForTesting;
-
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.Gauge;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
-
 import org.apache.distributedlog.common.util.PermitLimiter;
 
 /**
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java
index 155baaff8..8c4e18b76 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/util/Utils.java
@@ -33,11 +33,11 @@
 import javax.annotation.Nullable;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.versioning.LongVersion;
 import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.functions.VoidFunctions;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
diff --git a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java
index bd2a2d12d..37d1477ce 100644
--- a/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java
+++ b/stream/distributedlog/core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java
@@ -21,8 +21,8 @@
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.Transaction;
 import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.AsyncCallback;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
index 986ae9e2b..fa3b7430e 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
@@ -23,6 +23,7 @@
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.net.URI;
 import java.net.URL;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java
index df5284df5..10e8e907c 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java
@@ -17,7 +17,10 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 import org.apache.distributedlog.api.DistributedLogManager;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java
index 80ef2333a..69e57fb53 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java
@@ -17,7 +17,11 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.util.Arrays;
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.exceptions.EndOfStreamException;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
index 79f94f065..f9acd7c00 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
@@ -17,11 +17,13 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
 import java.net.URI;
-
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import org.apache.distributedlog.api.DistributedLogManager;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java
index 095333711..c6a4fdfab 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java
@@ -22,7 +22,9 @@
 import static org.apache.distributedlog.DLMTestUtil.validateFutureSucceededAndGetResult;
 import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
 import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java
index aec1b1feb..72a3ced44 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java
@@ -21,6 +21,7 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.concurrent.CancellationException;
@@ -30,13 +31,12 @@
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
-
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.api.AsyncLogReader;
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.api.subscription.SubscriptionsStore;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.exceptions.LockCancelledException;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
index 2efdca0d9..cd552d963 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
@@ -20,7 +20,13 @@
 import static com.google.common.base.Charsets.UTF_8;
 import static org.apache.distributedlog.DLMTestUtil.validateFutureFailed;
 import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
 import io.netty.buffer.ByteBuf;
@@ -36,14 +42,14 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import junit.framework.Assert;
-
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeperAccessor;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.feature.FixedValueFeature;
 import org.apache.bookkeeper.stats.NullStatsLogger;
-
 import org.apache.distributedlog.api.AsyncLogReader;
 import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.api.DistributedLogManager;
@@ -52,11 +58,8 @@
 import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
 import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
-
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
@@ -72,10 +75,7 @@
 import org.apache.distributedlog.lock.DistributedLock;
 import org.apache.distributedlog.util.FailpointUtils;
 import org.apache.distributedlog.util.SimplePermitLimiter;
-
-
 import org.apache.distributedlog.util.Utils;
-
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
index cf6413c96..5b9575bbe 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
@@ -17,7 +17,12 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import com.google.common.collect.Sets;
 import java.io.IOException;
 import java.net.URI;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
index 15711e7b3..a8b4f93f7 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
@@ -17,23 +17,25 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import com.google.common.base.Optional;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.LogWriter;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
 import org.apache.distributedlog.logsegment.LogSegmentFilter;
 import org.apache.distributedlog.util.Utils;
-
-
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
index 7b04b254e..6ad3c0f59 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
@@ -18,7 +18,11 @@
 package org.apache.distributedlog;
 
 import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
@@ -28,10 +32,10 @@
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.feature.SettableFeatureProvider;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.exceptions.EndOfStreamException;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
index 7b11ca527..3a3171cb9 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
@@ -17,7 +17,10 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.net.URI;
 import org.apache.distributedlog.api.AsyncLogWriter;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java
index cb98741f1..a5d4c1632 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java
@@ -17,7 +17,12 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java
index f92bce971..3f1602d16 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java
@@ -18,14 +18,12 @@
 package org.apache.distributedlog;
 
 import java.io.File;
-
 import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
 import org.apache.bookkeeper.util.IOUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
index f77ff5883..37f692dd2 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
@@ -29,6 +29,7 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.feature.SettableFeatureProvider;
 import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
 import org.apache.bookkeeper.stats.NullStatsLogger;
@@ -38,7 +39,6 @@
 import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.common.util.SchedulerUtils;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
index e2a970598..1b864cbe7 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
@@ -17,7 +17,10 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import com.google.common.base.Optional;
 import java.util.List;
 import org.apache.bookkeeper.net.DNSToSwitchMapping;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntry.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntry.java
index fef3dd4eb..59f9c8a76 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntry.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntry.java
@@ -30,10 +30,10 @@
 import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.Entry.Reader;
 import org.apache.distributedlog.Entry.Writer;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.LogRecordTooLongException;
 import org.apache.distributedlog.io.CompressionCodec;
 import org.apache.distributedlog.util.Utils;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntryPosition.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntryPosition.java
index 6c6b0f5b7..01a784dda 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntryPosition.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestEntryPosition.java
@@ -17,7 +17,10 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import org.junit.Test;
 
 
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
index e59df363b..7a80fd506 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
@@ -19,6 +19,7 @@
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.util.Utils;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
index 4f3212768..cd34e9585 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
@@ -17,7 +17,11 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.net.URI;
 import java.util.List;
 import org.apache.distributedlog.api.DistributedLogManager;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
index 3724b5a29..4912e88e9 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
@@ -22,6 +22,7 @@
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataBuilder;
 import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
index e15a40f70..4b17a1368 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
@@ -18,8 +18,13 @@
 package org.apache.distributedlog;
 
 
-import static org.apache.distributedlog.NonBlockingReadsTestUtil.*;
-import static org.junit.Assert.*;
+import static org.apache.distributedlog.NonBlockingReadsTestUtil.DEFAULT_SEGMENT_SIZE;
+import static org.apache.distributedlog.NonBlockingReadsTestUtil.readNonBlocking;
+import static org.apache.distributedlog.NonBlockingReadsTestUtil.writeRecordsForNonBlockingReads;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
index 854465ba5..faa31e51a 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
@@ -17,7 +17,8 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
 import com.google.common.util.concurrent.RateLimiter;
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
index ba374bd0b..18af307cd 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
@@ -17,18 +17,24 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import com.google.common.base.Optional;
 import com.google.common.base.Ticker;
 import com.google.common.collect.Lists;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
 import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java
index bc81087c5..033065bc3 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReadUtils.java
@@ -17,7 +17,10 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
 import java.util.List;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java
index 44ae00f56..be86b2cdb 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestReader.java
@@ -17,7 +17,10 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
@@ -26,9 +29,9 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.api.AsyncLogReader;
 import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
index 3eb82c92f..7a8d33c5a 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
@@ -18,7 +18,10 @@
 package org.apache.distributedlog;
 
 import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -26,11 +29,11 @@
 import java.util.concurrent.CountDownLatch;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.bookkeeper.feature.SettableFeature;
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.common.annotations.DistributedLogAnnotations.FlakyTest;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.feature.CoreFeatureKeys;
 import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryReader;
 import org.apache.distributedlog.util.FailpointUtils;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestSequenceID.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestSequenceID.java
index e45f14cec..19605203c 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestSequenceID.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestSequenceID.java
@@ -17,12 +17,15 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.util.List;
 import java.util.concurrent.LinkedBlockingQueue;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
 import org.apache.distributedlog.api.AsyncLogReader;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.util.Utils;
 import org.junit.Test;
 import org.slf4j.Logger;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTruncate.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTruncate.java
index b4c3f409d..5c779aca3 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTruncate.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestTruncate.java
@@ -35,7 +35,6 @@
 import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.util.Utils;
-
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
index 9a56edd7b..d009991bf 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
@@ -20,6 +20,7 @@
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
+
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.feature.SettableFeature;
 import org.apache.bookkeeper.stats.NullStatsLogger;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
index 5f3dae407..0c8daad3b 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
@@ -17,7 +17,11 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.util.ArrayList;
@@ -30,7 +34,6 @@
 import org.apache.distributedlog.ZooKeeperClient.Credentials;
 import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
 import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
-
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java
index cbf02b87b..a286ae50a 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java
@@ -17,7 +17,8 @@
  */
 package org.apache.distributedlog;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
+
 import com.google.common.base.Stopwatch;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java
index 3275b848e..b2190e95e 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java
@@ -18,12 +18,14 @@
 package org.apache.distributedlog.acl;
 
 import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.net.URI;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClusterTestCase;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.impl.acl.ZKAccessControl;
 import org.apache.distributedlog.thrift.AccessControlEntry;
 import org.apache.distributedlog.util.Utils;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java
index 56cfd7696..b4568c719 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java
@@ -17,7 +17,8 @@
  */
 package org.apache.distributedlog.acl;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
 import java.net.URI;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java
index e42b4ea55..42ff9aea6 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java
@@ -19,6 +19,7 @@
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+
 import java.net.URI;
 import java.util.HashMap;
 import java.util.List;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java
index 1174b561f..f5cd75e20 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog.admin;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
 
 import java.net.URI;
 import java.util.concurrent.CompletableFuture;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java
index f82129d95..2286d403e 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java
@@ -17,10 +17,10 @@
  */
 package org.apache.distributedlog.bk;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import com.google.common.collect.Lists;
-
 import java.io.IOException;
 import java.net.URI;
 import java.util.HashSet;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java
index caae38724..cba20c2ab 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java
@@ -17,7 +17,8 @@
  */
 package org.apache.distributedlog.config;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
 import com.google.common.base.Objects;
 import com.google.common.base.Optional;
 import java.io.File;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java
index 993182c30..eebf06e91 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java
@@ -18,8 +18,32 @@
 package org.apache.distributedlog.config;
 
 
-import static org.apache.distributedlog.DistributedLogConfiguration.*;
-import static org.junit.Assert.*;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ACK_QUORUM_SIZE;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_IS_DURABLE_WRITE_ENABLED;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_OUTPUT_BUFFER_SIZE;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_OUTPUT_BUFFER_SIZE_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_OUTPUT_BUFFER_SIZE_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_BATCHSIZE;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_BATCHSIZE_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_BATCHSIZE_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_MAX_RECORDS;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_MAX_RECORDS_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_READAHEAD_MAX_RECORDS_OLD;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT;
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS_OLD;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.bk.QuorumConfig;
 import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java
index 359484bc7..a94b5ba83 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog.feature;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.bookkeeper.feature.Feature;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
index d42d3ec7f..536a6c017 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
@@ -17,7 +17,10 @@
  */
 package org.apache.distributedlog.feature;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.distributedlog.DistributedLogConfiguration;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java
index 9eac19364..161222033 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java
@@ -17,7 +17,8 @@
  */
 package org.apache.distributedlog.impl;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import com.google.common.base.Optional;
 import com.google.common.collect.Sets;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java
index eac11a965..93ed56172 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java
@@ -17,8 +17,10 @@
  */
 package org.apache.distributedlog.impl;
 
-import static org.apache.distributedlog.impl.ZKLogSegmentFilters.*;
-import static org.junit.Assert.*;
+import static org.apache.distributedlog.impl.ZKLogSegmentFilters.WRITE_HANDLE_FILTER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import com.google.common.collect.Sets;
 import java.util.ArrayList;
 import java.util.Collection;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java
index 613787fd0..eda73a78c 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java
@@ -33,6 +33,7 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.versioning.LongVersion;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
@@ -44,7 +45,6 @@
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientUtils;
 import org.apache.distributedlog.callback.LogSegmentNamesListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.metadata.LogMetadata;
 import org.apache.distributedlog.metadata.LogMetadataForWriter;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java
index 1082d7e97..e2761b218 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java
@@ -17,9 +17,10 @@
  */
 package org.apache.distributedlog.impl;
 
-import static org.junit.Assert.*;
-import com.google.common.collect.Sets;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
+import com.google.common.collect.Sets;
 import java.net.URI;
 import java.util.Iterator;
 import java.util.Set;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
index 2faf13195..d2c67995e 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
@@ -21,6 +21,7 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import com.google.common.base.Optional;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
index f8848c2e7..5a2ffd134 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
@@ -17,7 +17,13 @@
  */
 package org.apache.distributedlog.impl.logsegment;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import com.google.common.collect.Lists;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
index baeaab921..fdd9f492f 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
@@ -19,8 +19,20 @@
 
 import static com.google.common.base.Charsets.UTF_8;
 import static org.apache.distributedlog.DistributedLogConstants.EMPTY_BYTES;
-import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.*;
-import static org.apache.distributedlog.metadata.LogMetadata.*;
+import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.checkLogMetadataPaths;
+import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.getLog;
+import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.getLogSegments;
+import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.getMissingPaths;
+import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.intToBytes;
+import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.pathExists;
+import static org.apache.distributedlog.metadata.LogMetadata.ALLOCATION_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.LAYOUT_VERSION;
+import static org.apache.distributedlog.metadata.LogMetadata.LOCK_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.LOGSEGMENTS_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.MAX_TXID_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.READ_LOCK_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.VERSION_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.getLogRootPath;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -36,6 +48,7 @@
 import java.net.URI;
 import java.util.Collections;
 import java.util.List;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.bookkeeper.versioning.LongVersion;
@@ -49,7 +62,6 @@
 import org.apache.distributedlog.ZooKeeperClusterTestCase;
 import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.exceptions.LogExistsException;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java
index 1c577f657..38f9c0b49 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java
@@ -21,6 +21,7 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.net.URI;
 import org.apache.distributedlog.DistributedLogConfiguration;
@@ -28,10 +29,8 @@
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClusterTestCase;
-
 import org.apache.distributedlog.metadata.DLMetadata;
 import org.apache.distributedlog.util.Utils;
-
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs;
 import org.junit.After;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java
index 5861718ad..c9ad2a153 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java
@@ -17,7 +17,8 @@
  */
 package org.apache.distributedlog.limiter;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
 import org.junit.Test;
 
 
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java
index 50fda95f4..bd3b71bd6 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java
@@ -22,6 +22,7 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -30,13 +31,13 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.distributedlog.TestDistributedLogBase;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClientUtils;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java
index 9e4e1e265..498f7f226 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java
@@ -17,11 +17,19 @@
  */
 package org.apache.distributedlog.lock;
 
-import static org.apache.distributedlog.lock.ZKSessionLock.*;
+import static org.apache.distributedlog.lock.ZKSessionLock.areLockWaitersInSameSession;
+import static org.apache.distributedlog.lock.ZKSessionLock.asyncParseClientID;
+import static org.apache.distributedlog.lock.ZKSessionLock.getLockIdFromPath;
+import static org.apache.distributedlog.lock.ZKSessionLock.getLockPathPrefixV1;
+import static org.apache.distributedlog.lock.ZKSessionLock.getLockPathPrefixV2;
+import static org.apache.distributedlog.lock.ZKSessionLock.getLockPathPrefixV3;
+import static org.apache.distributedlog.lock.ZKSessionLock.parseMemberID;
+import static org.apache.distributedlog.lock.ZKSessionLock.serializeClientId;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java
index d79250cd6..b9ae8f029 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java
@@ -17,7 +17,10 @@
  */
 package org.apache.distributedlog.logsegment;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java
index 4cfedbbfc..7a4deb6aa 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog.logsegment;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import org.apache.distributedlog.common.util.Sizable;
 import org.junit.Test;
 
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java
index 02097520b..093a56138 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java
@@ -20,6 +20,7 @@
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.net.URI;
 import org.apache.distributedlog.LocalDLMEmulator;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java
index d7fbb00ec..e3daed8e6 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java
@@ -18,8 +18,12 @@
 package org.apache.distributedlog.metadata;
 
 
-import static org.apache.distributedlog.metadata.LogMetadata.*;
-import static org.junit.Assert.*;
+import static org.apache.distributedlog.metadata.LogMetadata.ALLOCATION_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.LOCK_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.LOGSEGMENTS_PATH;
+import static org.apache.distributedlog.metadata.LogMetadata.MAX_TXID_PATH;
+import static org.junit.Assert.assertEquals;
+
 import java.net.URI;
 import org.apache.distributedlog.DLMTestUtil;
 import org.junit.Test;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
index 48ecde3fa..ba60e38a9 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
@@ -17,7 +17,11 @@
  */
 package org.apache.distributedlog.metadata;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java
index fe99362d6..7ee7cb3b3 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java
@@ -19,6 +19,7 @@
 
 import static org.apache.distributedlog.LocalDLMEmulator.DLOG_NAMESPACE;
 import static org.junit.Assert.assertTrue;
+
 import java.net.URI;
 import org.apache.distributedlog.BKDistributedLogNamespace;
 import org.apache.distributedlog.DistributedLogConfiguration;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java
index b32f55e5b..a804d9a0c 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java
@@ -17,7 +17,8 @@
  */
 package org.apache.distributedlog.net;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
 import java.util.ArrayList;
 import java.util.List;
 import org.junit.Test;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/net/TestNetUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/net/TestNetUtils.java
index fec785a7f..7cfc76f63 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/net/TestNetUtils.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/net/TestNetUtils.java
@@ -17,7 +17,8 @@
  */
 package org.apache.distributedlog.net;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
 import java.util.List;
 import org.apache.bookkeeper.net.DNSToSwitchMapping;
 import org.junit.Test;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/selector/TestLogRecordSelectors.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/selector/TestLogRecordSelectors.java
index d3ed32915..46b4c70f6 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/selector/TestLogRecordSelectors.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/selector/TestLogRecordSelectors.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog.selector;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
 import org.apache.distributedlog.DLMTestUtil;
 import org.apache.distributedlog.DLSN;
 import org.junit.Test;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java
index f982da6b2..753f23aa7 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java
@@ -17,9 +17,11 @@
  */
 package org.apache.distributedlog.tools;
 
-import static org.junit.Assert.*;
-import java.net.URI;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
+import java.net.URI;
 import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException;
 import org.apache.bookkeeper.util.ReflectionUtils;
 import org.apache.distributedlog.DLMTestUtil;
@@ -42,8 +44,6 @@
 import org.apache.distributedlog.tools.DistributedLogTool.ShowCommand;
 import org.apache.distributedlog.tools.DistributedLogTool.TruncateCommand;
 import org.apache.distributedlog.tools.DistributedLogTool.TruncateStreamCommand;
-
-
 import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
 import org.junit.Test;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java
index f323966d0..36c22c4fd 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java
@@ -17,7 +17,8 @@
  */
 package org.apache.distributedlog.util;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
 import org.apache.commons.configuration.CompositeConfiguration;
 import org.apache.commons.configuration.Configuration;
 import org.junit.Test;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java
index c712d6790..3c663c5df 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java
@@ -19,7 +19,10 @@
 
 import static com.google.common.base.Charsets.UTF_8;
 import static org.apache.distributedlog.util.DLUtils.validateAndNormalizeName;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
 import com.google.common.collect.Lists;
 import java.util.List;
 import org.apache.distributedlog.DLMTestUtil;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java
index ea6876f78..f47d0d9ec 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog.util;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Executors;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestTimeSequencer.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestTimeSequencer.java
index 59dfe867e..7230fa89f 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestTimeSequencer.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/util/TestTimeSequencer.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog.util;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import org.junit.Test;
 
 
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java
index 03d527917..e95aefc86 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java
@@ -17,8 +17,9 @@
  */
 package org.apache.distributedlog.zk;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+
 import java.util.concurrent.CountDownLatch;
 import javax.annotation.Nullable;
 import org.apache.distributedlog.ZooKeeperClient;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java
index d03563082..fab56ea4f 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java
@@ -17,8 +17,9 @@
  */
 package org.apache.distributedlog.zk;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.bookkeeper.versioning.Version;
diff --git a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java
index 75f193bd4..3b5908f1b 100644
--- a/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java
+++ b/stream/distributedlog/core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java
@@ -17,7 +17,8 @@
  */
 package org.apache.distributedlog.zk;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
 import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
diff --git a/stream/distributedlog/io/dlfs/pom.xml b/stream/distributedlog/io/dlfs/pom.xml
index b6df19bc2..0243bddc3 100644
--- a/stream/distributedlog/io/dlfs/pom.xml
+++ b/stream/distributedlog/io/dlfs/pom.xml
@@ -85,7 +85,7 @@
         </dependencies>
         <configuration>
           <configLocation>bookkeeper/checkstyle.xml</configLocation>
-          <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
+          <suppressionsLocation>bookkeeper/suppressions.xml</suppressionsLocation>
           <consoleOutput>true</consoleOutput>
           <failOnViolation>true</failOnViolation>
           <includeResources>false</includeResources>
diff --git a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java
index 3670bc52b..e6d076e8d 100644
--- a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java
+++ b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/DLOutputStream.java
@@ -27,12 +27,12 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicReference;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.concurrent.FutureEventListener;
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.exceptions.UnexpectedException;
 import org.apache.distributedlog.util.Utils;
 
diff --git a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/package-info.java b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/package-info.java
index 2af39b76c..335658fd1 100644
--- a/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/package-info.java
+++ b/stream/distributedlog/io/dlfs/src/main/java/org/apache/distributedlog/fs/package-info.java
@@ -19,4 +19,4 @@
 /**
  * A filesystem API built over distributedlog.
  */
-package org.apache.distributedlog.fs;
\ No newline at end of file
+package org.apache.distributedlog.fs;
diff --git a/stream/distributedlog/protocol/pom.xml b/stream/distributedlog/protocol/pom.xml
index 9d3f7702f..dfed60d26 100644
--- a/stream/distributedlog/protocol/pom.xml
+++ b/stream/distributedlog/protocol/pom.xml
@@ -72,7 +72,7 @@
         </dependencies>
         <configuration>
           <configLocation>bookkeeper/checkstyle.xml</configLocation>
-          <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
+          <suppressionsLocation>bookkeeper/suppressions.xml</suppressionsLocation>
           <consoleOutput>true</consoleOutput>
           <failOnViolation>true</failOnViolation>
           <includeResources>false</includeResources>
diff --git a/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java b/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java
index 0d04fd4e4..c80d3f4d3 100644
--- a/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java
+++ b/stream/distributedlog/protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java
@@ -30,9 +30,9 @@
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
 import org.apache.distributedlog.LogRecordSet.Reader;
 import org.apache.distributedlog.LogRecordSet.Writer;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.exceptions.LogRecordTooLongException;
 import org.apache.distributedlog.io.CompressionCodec.Type;
 import org.junit.Test;


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services