DL-124: Use Java8 Future rather than twitter Future

Switch to use Java8 CompletableFuture, to reduce dependencies introduced by twitter future and make it more friendly to users (users don't think of using which version of scala).

This change is based on #132 . Gitsha ce0686e is the change to review.

The changes:

- Change Future to CompletableFuture
- Map to thenApply
- flatMap to thenCompose
- Added a FutureEventListener, and switch addEvenListener to whenComplete (or whenCompleteAsync)
- setValue to complete
- setException to completeExceptionally
- add rescue, ignore, ensure to FutureUtils as util functions.

Author: Sijie Guo <sijie@apache.org>

Reviewers: Jia Zhai <None>, Leigh Stewart <lstewart@apache.org>

Closes #133 from sijie/change_twitter_future_to_java_future
diff --git a/distributedlog-benchmark/conf/log4j.properties b/distributedlog-benchmark/conf/log4j.properties
index 930db8d..af1cf5f 100644
--- a/distributedlog-benchmark/conf/log4j.properties
+++ b/distributedlog-benchmark/conf/log4j.properties
@@ -30,11 +30,7 @@
 log4j.logger.org.apache.bookkeeper=INFO
 
 # redirect executor output to executors.log since slow op warnings can be quite verbose
-log4j.logger.org.apache.distributedlog.util.MonitoredFuturePool=INFO, Executors
-log4j.logger.org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
 log4j.logger.org.apache.bookkeeper.util.SafeRunnable=INFO, Executors
-log4j.additivity.org.apache.distributedlog.util.MonitoredFuturePool=false
-log4j.additivity.org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
 log4j.additivity.org.apache.bookkeeper.util.SafeRunnable=false
 
 log4j.appender.Executors=org.apache.log4j.RollingFileAppender
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/DLWriterWorker.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/DLWriterWorker.java
index a5e7a0a..5c9b2a9 100644
--- a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/DLWriterWorker.java
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/DLWriterWorker.java
@@ -19,17 +19,17 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.benchmark.utils.ShiftableRateLimiter;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import org.apache.distributedlog.util.FutureUtils;
-import org.apache.distributedlog.util.SchedulerUtils;
-import com.twitter.util.FutureEventListener;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.util.SchedulerUtils;
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
@@ -66,7 +66,7 @@
     final ScheduledExecutorService rescueService;
     final ShiftableRateLimiter rateLimiter;
     final Random random;
-    final DistributedLogNamespace namespace;
+    final Namespace namespace;
     final List<DistributedLogManager> dlms;
     final List<AsyncLogWriter> streamWriters;
     final int numStreams;
@@ -98,7 +98,7 @@
         this.rescueService = Executors.newSingleThreadScheduledExecutor();
         this.random = new Random(System.currentTimeMillis());
 
-        this.namespace = DistributedLogNamespaceBuilder.newBuilder()
+        this.namespace = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .statsLogger(statsLogger.scope("dl"))
@@ -120,7 +120,7 @@
                             FutureUtils.result(writer.asyncClose());
                         }
                         latch.countDown();
-                    } catch (IOException e) {
+                    } catch (Exception e) {
                         LOG.error("Failed to intialize writer for stream : {}", streamName, e);
                     }
 
@@ -148,7 +148,7 @@
         if (streamWriters.get(idx) == writer) {
             try {
                 FutureUtils.result(writer.asyncClose());
-            } catch (IOException e) {
+            } catch (Exception e) {
                 LOG.error("Failed to close writer for stream {}.", idx);
             }
             AsyncLogWriter newWriter = null;
@@ -185,7 +185,7 @@
         SchedulerUtils.shutdownScheduler(this.executorService, 2, TimeUnit.MINUTES);
         SchedulerUtils.shutdownScheduler(this.rescueService, 2, TimeUnit.MINUTES);
         for (AsyncLogWriter writer : streamWriters) {
-            FutureUtils.result(writer.asyncClose());
+            org.apache.distributedlog.util.Utils.ioResult(writer.asyncClose());
         }
         for (DistributedLogManager dlm : dlms) {
             dlm.close();
@@ -225,7 +225,7 @@
                     LOG.error("Error on generating message : ", e);
                     break;
                 }
-                writer.write(new LogRecord(requestMillis, data)).addEventListener(new FutureEventListener<DLSN>() {
+                writer.write(new LogRecord(requestMillis, data)).whenComplete(new FutureEventListener<DLSN>() {
                     @Override
                     public void onSuccess(DLSN value) {
                         requestStat.registerSuccessfulEvent(System.currentTimeMillis() - requestMillis);
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/ReaderWorker.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/ReaderWorker.java
index 11cba6f..ad95a59 100644
--- a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/ReaderWorker.java
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/ReaderWorker.java
@@ -22,21 +22,21 @@
 import com.google.common.base.Stopwatch;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.twitter.common.zookeeper.ServerSet;
-import org.apache.distributedlog.AsyncLogReader;
+import org.apache.distributedlog.api.AsyncLogReader;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.LogRecordSet;
 import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.benchmark.thrift.Message;
 import org.apache.distributedlog.client.serverset.DLZkServerSet;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.service.DistributedLogClient;
 import org.apache.distributedlog.service.DistributedLogClientBuilder;
-import org.apache.distributedlog.util.FutureUtils;
-import org.apache.distributedlog.util.SchedulerUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.util.SchedulerUtils;
 import com.twitter.finagle.builder.ClientBuilder;
 import com.twitter.finagle.stats.StatsReceiver;
 import com.twitter.finagle.thrift.ClientId$;
@@ -75,7 +75,7 @@
     final int endStreamId;
     final ScheduledExecutorService executorService;
     final ExecutorService callbackExecutor;
-    final DistributedLogNamespace namespace;
+    final Namespace namespace;
     final DistributedLogManager[] dlms;
     final AsyncLogReader[] logReaders;
     final StreamReader[] streamReaders;
@@ -100,7 +100,9 @@
     final Counter invalidRecordsCounter;
     final Counter outOfOrderSequenceIdCounter;
 
-    class StreamReader implements FutureEventListener<List<LogRecordWithDLSN>>, Runnable, Gauge<Number> {
+    class StreamReader implements
+        org.apache.distributedlog.common.concurrent.FutureEventListener<List<LogRecordWithDLSN>>,
+        Runnable, Gauge<Number> {
 
         final int streamIdx;
         final String streamName;
@@ -184,7 +186,7 @@
             if (!running) {
                 return;
             }
-            logReaders[streamIdx].readBulk(10).addEventListener(this);
+            logReaders[streamIdx].readBulk(10).whenComplete(this);
         }
 
         @Override
@@ -305,7 +307,7 @@
         }
 
         // construct the factory
-        this.namespace = DistributedLogNamespaceBuilder.newBuilder()
+        this.namespace = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .statsLogger(statsLogger.scope("dl"))
@@ -369,7 +371,7 @@
         if (logReaders[idx] != null) {
             try {
                 FutureUtils.result(logReaders[idx].asyncClose());
-            } catch (IOException e) {
+            } catch (Exception e) {
                 LOG.warn("Failed on closing stream reader {} : ", streamName, e);
             }
             logReaders[idx] = null;
@@ -434,7 +436,7 @@
         this.running = false;
         for (AsyncLogReader reader : logReaders) {
             if (null != reader) {
-                FutureUtils.result(reader.asyncClose());
+                org.apache.distributedlog.util.Utils.ioResult(reader.asyncClose());
             }
         }
         for (DistributedLogManager dlm : dlms) {
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java
index fa96dfb..46f9dfc 100644
--- a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java
@@ -29,7 +29,7 @@
 import org.apache.distributedlog.service.DistributedLogClient;
 import org.apache.distributedlog.service.DistributedLogClientBuilder;
 import org.apache.distributedlog.thrift.service.StatusCode;
-import org.apache.distributedlog.util.SchedulerUtils;
+import org.apache.distributedlog.common.util.SchedulerUtils;
 import com.twitter.finagle.builder.ClientBuilder;
 import com.twitter.finagle.stats.StatsReceiver;
 import com.twitter.finagle.thrift.ClientId;
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AsyncReaderBenchmark.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
index 4930b8a..4c8e372 100644
--- a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
@@ -18,12 +18,12 @@
 package org.apache.distributedlog.benchmark.stream;
 
 import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.AsyncLogReader;
+import org.apache.distributedlog.api.AsyncLogReader;
 import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.LogRecordWithDLSN;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
@@ -34,14 +34,14 @@
 import org.slf4j.LoggerFactory;
 
 /**
- * Benchmark on {@link org.apache.distributedlog.AsyncLogReader} reading from a stream.
+ * Benchmark on {@link AsyncLogReader} reading from a stream.
  */
 public class AsyncReaderBenchmark extends AbstractReaderBenchmark {
 
     private static final Logger logger = LoggerFactory.getLogger(AsyncReaderBenchmark.class);
 
     @Override
-    protected void benchmark(DistributedLogNamespace namespace, String logName, StatsLogger statsLogger) {
+    protected void benchmark(Namespace namespace, String logName, StatsLogger statsLogger) {
         DistributedLogManager dlm = null;
         while (null == dlm) {
             try {
@@ -112,7 +112,7 @@
                 openReaderStats.registerSuccessfulEvent(elapsedMs);
                 logger.info("It took {} ms to position the reader to transaction id = {}, dlsn = {}",
                         lastTxId, lastDLSN);
-            } catch (IOException ioe) {
+            } catch (Exception ioe) {
                 openReaderStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
                 logger.warn("Failed to create reader for stream {} reading from tx id = {}, dlsn = {}.",
                         new Object[] { streamName, lastTxId, lastDLSN });
@@ -141,7 +141,7 @@
                         lastDLSN = lastRecord.getDlsn();
                     }
                     stopwatch.reset();
-                } catch (IOException e) {
+                } catch (Exception e) {
                     logger.warn("Encountered reading record from stream {} : ", streamName, e);
                     reader = null;
                     break;
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerReadBenchmark.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerReadBenchmark.java
index 489e5af..9fb46ad 100644
--- a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerReadBenchmark.java
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerReadBenchmark.java
@@ -21,12 +21,12 @@
 
 import com.google.common.base.Stopwatch;
 import org.apache.distributedlog.BookKeeperClientBuilder;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
@@ -47,7 +47,7 @@
     private static final Logger logger = LoggerFactory.getLogger(AsyncReaderBenchmark.class);
 
     @Override
-    protected void benchmark(DistributedLogNamespace namespace, String logName, StatsLogger statsLogger) {
+    protected void benchmark(Namespace namespace, String logName, StatsLogger statsLogger) {
         DistributedLogManager dlm = null;
         while (null == dlm) {
             try {
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/StreamBenchmark.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/StreamBenchmark.java
index d3083ca..427b9f3 100644
--- a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/StreamBenchmark.java
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/StreamBenchmark.java
@@ -18,8 +18,8 @@
 package org.apache.distributedlog.benchmark.stream;
 
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import java.io.File;
 import java.net.URI;
 import org.apache.bookkeeper.stats.NullStatsProvider;
@@ -107,8 +107,8 @@
         statsProvider.start(conf);
         // run the benchmark
         StatsLogger statsLogger = statsProvider.getStatsLogger("dl");
-        DistributedLogNamespace namespace =
-                DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace =
+                NamespaceBuilder.newBuilder()
                         .conf(conf)
                         .uri(uri)
                         .statsLogger(statsLogger)
@@ -121,7 +121,7 @@
         }
     }
 
-    protected abstract void benchmark(DistributedLogNamespace namespace,
+    protected abstract void benchmark(Namespace namespace,
                                       String logName,
                                       StatsLogger statsLogger);
 
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/SyncReaderBenchmark.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/SyncReaderBenchmark.java
index 4abb317..cbd7f67 100644
--- a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/SyncReaderBenchmark.java
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/SyncReaderBenchmark.java
@@ -18,10 +18,10 @@
 package org.apache.distributedlog.benchmark.stream;
 
 import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.DistributedLogManager;
-import org.apache.distributedlog.LogReader;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.LogRecord;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.api.namespace.Namespace;
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 import org.apache.bookkeeper.stats.Counter;
@@ -31,7 +31,7 @@
 import org.slf4j.LoggerFactory;
 
 /**
- * Benchmark on {@link org.apache.distributedlog.LogReader} reading from a stream.
+ * Benchmark on {@link LogReader} reading from a stream.
  */
 public class SyncReaderBenchmark extends AbstractReaderBenchmark {
 
@@ -40,7 +40,7 @@
     public SyncReaderBenchmark() {}
 
     @Override
-    protected void benchmark(DistributedLogNamespace namespace, String streamName, StatsLogger statsLogger) {
+    protected void benchmark(Namespace namespace, String streamName, StatsLogger statsLogger) {
         DistributedLogManager dlm = null;
         while (null == dlm) {
             try {
diff --git a/distributedlog-build-tools/src/main/resources/distributedlog/checkstyle.xml b/distributedlog-build-tools/src/main/resources/distributedlog/checkstyle.xml
index e520b2c..28cd4ec 100644
--- a/distributedlog-build-tools/src/main/resources/distributedlog/checkstyle.xml
+++ b/distributedlog-build-tools/src/main/resources/distributedlog/checkstyle.xml
@@ -240,7 +240,7 @@
     <module name="MethodNameCheck">
       <!-- Validates identifiers for method names. -->
       <metadata name="altname" value="MethodName"/>
-      <property name="format" value="^[a-z][a-zA-Z0-9]*(_[a-zA-Z0-9]+)*$"/>
+      <property name="format" value="(^[a-z][a-zA-Z0-9]*(_[a-zA-Z0-9]+)*$|Void)"/>
       <property name="severity" value="error"/>
     </module>
 
@@ -271,12 +271,12 @@
     </module>
 
     <module name="MethodTypeParameterName">
-      <property name="format" value="^(((T|K|V|W|X)[0-9]*)|([A-Z][a-z][a-zA-Z]*T))$"/>
+      <property name="format" value="^(((T|K|V|W|X|R)[0-9]*)|([A-Z][a-z][a-zA-Z]*T))$"/>
       <property name="severity" value="error"/>
     </module>
 
     <module name="InterfaceTypeParameterName">
-      <property name="format" value="^(((T|K|V|W|X)[0-9]*)|([A-Z][a-z][a-zA-Z]*T))$"/>
+      <property name="format" value="^(((T|K|V|W|X|R)[0-9]*)|([A-Z][a-z][a-zA-Z]*T))$"/>
       <property name="severity" value="error"/>
     </module>
 
diff --git a/distributedlog-common/pom.xml b/distributedlog-common/pom.xml
new file mode 100644
index 0000000..cad2bc8
--- /dev/null
+++ b/distributedlog-common/pom.xml
@@ -0,0 +1,167 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.distributedlog</groupId>
+    <artifactId>distributedlog</artifactId>
+    <version>0.5.0-incubating-SNAPSHOT</version>
+  </parent>
+  <artifactId>distributedlog-common</artifactId>
+  <name>Apache DistributedLog :: Common</name>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.bookkeeper.stats</groupId>
+      <artifactId>bookkeeper-stats-api</artifactId>
+      <version>${bookkeeper.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <version>${slf4j.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.projectlombok</groupId>
+      <artifactId>lombok</artifactId>
+      <version>${lombok.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.inferred</groupId>
+      <artifactId>freebuilder</artifactId>
+      <version>${freebuilder.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <version>${guava.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+      <version>${commons-lang.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>commons-codec</groupId>
+      <artifactId>commons-codec</artifactId>
+      <version>${commons-codec.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <version>${slf4j.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <version>${mockito.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.jmock</groupId>
+      <artifactId>jmock</artifactId>
+      <version>${jmock.version}</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/src/main/resources/findbugsExclude.xml</excludeFilterFile>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>${maven-compiler-plugin.version}</version>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>${maven-surefire-plugin.version}</version>
+        <configuration>
+          <redirectTestOutputToFile>true</redirectTestOutputToFile>
+          <argLine>-Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G</argLine>
+          <forkMode>always</forkMode>
+          <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <version>${maven-checkstyle-plugin.version}</version>
+        <dependencies>
+          <dependency>
+            <groupId>com.puppycrawl.tools</groupId>
+            <artifactId>checkstyle</artifactId>
+            <version>${puppycrawl.checkstyle.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.distributedlog</groupId>
+            <artifactId>distributedlog-build-tools</artifactId>
+            <version>${project.version}</version>
+          </dependency>
+        </dependencies>
+        <configuration>
+          <configLocation>distributedlog/checkstyle.xml</configLocation>
+          <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
+          <consoleOutput>true</consoleOutput>
+          <failOnViolation>true</failOnViolation>
+          <includeResources>false</includeResources>
+          <includeTestSourceDirectory>true</includeTestSourceDirectory>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>test-compile</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/DistributedLogAnnotations.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/annotations/DistributedLogAnnotations.java
similarity index 95%
rename from distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/DistributedLogAnnotations.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/annotations/DistributedLogAnnotations.java
index f8cdea4..a5144b8 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/DistributedLogAnnotations.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/annotations/DistributedLogAnnotations.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.annotations;
+package org.apache.distributedlog.common.annotations;
 
 /**
  * Common annotation types.
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/package-info.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/annotations/package-info.java
similarity index 93%
rename from distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/package-info.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/annotations/package-info.java
index 0922f14..a390700 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/package-info.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/annotations/package-info.java
@@ -18,4 +18,4 @@
 /**
  * Defines annotations used across distributedlog project.
  */
-package org.apache.distributedlog.annotations;
+package org.apache.distributedlog.common.annotations;
diff --git a/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/AsyncSemaphore.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/AsyncSemaphore.java
new file mode 100644
index 0000000..d6a9b34
--- /dev/null
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/AsyncSemaphore.java
@@ -0,0 +1,160 @@
+/**
+ * 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.LinkedList;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.GuardedBy;
+import org.apache.distributedlog.common.util.Permit;
+
+/**
+ * An AsyncSemaphore is a traditional semaphore but with asynchronous
+ * execution.
+ *
+ * <p>Grabbing a permit returns a `Future[Permit]`.
+ *
+ * <p>Basic usage:
+ * {{{
+ *   val semaphore = new AsyncSemaphore(n)
+ *   ...
+ *   semaphore.acquireAndRun() {
+ *     somethingThatReturnsFutureT()
+ *   }
+ * }}}
+ *
+ * <p>Calls to acquire() and acquireAndRun are serialized, and tickets are
+ * given out fairly (in order of arrival).
+ */
+public class AsyncSemaphore {
+
+    private final Optional<Integer> maxWaiters;
+
+    private final Permit semaphorePermit = new Permit() {
+        @Override
+        public void release() {
+            releasePermit(this);
+        }
+    };
+
+    @GuardedBy("this")
+    private Optional<Throwable> closed = Optional.empty();
+    @GuardedBy("this")
+    private final LinkedList<CompletableFuture<Permit>> waitq;
+    @GuardedBy("this")
+    private int availablePermits;
+
+    public AsyncSemaphore(int initialPermits,
+                          Optional<Integer> maxWaiters) {
+        this.availablePermits = initialPermits;
+        this.waitq = new LinkedList<>();
+        this.maxWaiters = maxWaiters;
+    }
+
+    private synchronized void releasePermit(Permit permit) {
+        CompletableFuture<Permit> next = waitq.pollFirst();
+        if (null != next) {
+            next.complete(permit);
+        } else {
+            availablePermits += 1;
+        }
+    }
+
+    private CompletableFuture<Permit> newFuturePermit() {
+        return FutureUtils.value(semaphorePermit);
+    }
+
+    /**
+     * Acquire a [[Permit]], asynchronously.
+     *
+     * <p>Be sure to `permit.release()` in a
+     * - `finally` block of your `onSuccess` callback
+     * - `ensure` block of your future chain
+     *
+     * <p>Interrupting this future is only advisory, and will not release the permit
+     * if the future has already been satisfied.
+     *
+     * @note This method always return the same instance of [[Permit]].
+     * @return a `Future[Permit]` when the `Future` is satisfied, computation can proceed,
+     *         or a Future.Exception[RejectedExecutionException]` if the configured maximum
+     *         number of waiters would be exceeded.
+     */
+    public synchronized CompletableFuture<Permit> acquire() {
+        if (closed.isPresent()) {
+            return FutureUtils.exception(closed.get());
+        }
+
+        if (availablePermits > 0) {
+            availablePermits -= 1;
+            return newFuturePermit();
+        } else {
+            if (maxWaiters.isPresent() && waitq.size() >= maxWaiters.get()) {
+                return FutureUtils.exception(new RejectedExecutionException("Max waiters exceeded"));
+            } else {
+                CompletableFuture<Permit> future = FutureUtils.createFuture();
+                future.whenComplete((value, cause) -> {
+                    synchronized (AsyncSemaphore.this) {
+                        waitq.remove(future);
+                    }
+                });
+                waitq.addLast(future);
+                return future;
+            }
+        }
+    }
+
+    /**
+     * Fail the semaphore and stop it from distributing further permits. Subsequent
+     * attempts to acquire a permit fail with `exc`. This semaphore's queued waiters
+     * are also failed with `exc`.
+     */
+    public synchronized void fail(Throwable exc) {
+        closed = Optional.of(exc);
+        for (CompletableFuture<Permit> future : waitq) {
+            future.cancel(true);
+        }
+        waitq.clear();
+    }
+
+    /**
+     * Execute the function asynchronously when a permit becomes available.
+     *
+     * <p>If the function throws a non-fatal exception, the exception is returned as part of the Future.
+     * For all exceptions, the permit would be released before returning.
+     *
+     * @return a Future[T] equivalent to the return value of the input function. If the configured
+     *         maximum value of waitq is reached, Future.Exception[RejectedExecutionException] is
+     *         returned.
+     */
+    public <T> CompletableFuture<T> acquireAndRun(Supplier<CompletableFuture<T>> func) {
+        return acquire().thenCompose(permit -> {
+            CompletableFuture<T> future;
+            try {
+                future = func.get();
+                future.whenComplete((value, cause) -> permit.release());
+                return future;
+            } catch (Throwable cause) {
+                permit.release();
+                throw cause;
+            }
+        });
+    }
+
+}
diff --git a/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/FutureEventListener.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/FutureEventListener.java
new file mode 100644
index 0000000..ed5b9ac
--- /dev/null
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/FutureEventListener.java
@@ -0,0 +1,44 @@
+/*
+ * 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/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/FutureUtils.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/FutureUtils.java
new file mode 100644
index 0000000..15ecf1d
--- /dev/null
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/FutureUtils.java
@@ -0,0 +1,376 @@
+/*
+ * 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/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/package-info.java
similarity index 81%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
copy to distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/package-info.java
index d81f8a4..dff0ace 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/concurrent/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -7,7 +7,7 @@
  * "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
+ *       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,
@@ -15,7 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- * ReadAhead Mechanism for distributedlog streaming reads
+ * Utility classes commonly useful in concurrent programming.
  */
-package org.apache.distributedlog.readahead;
+package org.apache.distributedlog.common.concurrent;
\ No newline at end of file
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentBaseConfiguration.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConcurrentBaseConfiguration.java
similarity index 87%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentBaseConfiguration.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConcurrentBaseConfiguration.java
index f189ad3..83e8e0e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentBaseConfiguration.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConcurrentBaseConfiguration.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,17 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.config;
+package org.apache.distributedlog.common.config;
 
-import com.google.common.base.Preconditions;
-
-import org.apache.commons.configuration.AbstractConfiguration;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import static com.google.common.base.Preconditions.checkNotNull;
 
 import java.util.Iterator;
 import java.util.concurrent.ConcurrentHashMap;
+import org.apache.commons.configuration.AbstractConfiguration;
 
 /**
  * Configuration view built on concurrent hash map for fast thread-safe access.
@@ -35,7 +31,6 @@
  * This class simply overwrites any existing mapping.
  */
 public class ConcurrentBaseConfiguration extends AbstractConfiguration {
-    static final Logger LOG = LoggerFactory.getLogger(ConcurrentBaseConfiguration.class);
 
     private final ConcurrentHashMap<String, Object> map;
 
@@ -45,7 +40,7 @@
 
     @Override
     protected void addPropertyDirect(String key, Object value) {
-        Preconditions.checkNotNull(value);
+        checkNotNull(value);
         map.put(key, value);
     }
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentConstConfiguration.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConcurrentConstConfiguration.java
similarity index 87%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentConstConfiguration.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConcurrentConstConfiguration.java
index 4e7f886..1131409 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentConstConfiguration.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConcurrentConstConfiguration.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.config;
+package org.apache.distributedlog.common.config;
 
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.commons.configuration.Configuration;
 
 /**
@@ -25,7 +26,7 @@
  */
 public class ConcurrentConstConfiguration extends ConcurrentBaseConfiguration {
     public ConcurrentConstConfiguration(Configuration conf) {
-        Preconditions.checkNotNull(conf);
+        checkNotNull(conf);
         copy(conf);
     }
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationListener.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConfigurationListener.java
similarity index 95%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationListener.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConfigurationListener.java
index 70059d4..71bb12a 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationListener.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConfigurationListener.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.config;
+package org.apache.distributedlog.common.config;
 
 /**
  * Configuration listener triggered when reloading configuration settings.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationSubscription.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java
similarity index 93%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationSubscription.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java
index 0e5c897..72a5657 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationSubscription.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/ConfigurationSubscription.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,8 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.config;
+package org.apache.distributedlog.common.config;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 import java.io.FileNotFoundException;
 import java.util.List;
 import java.util.Set;
@@ -24,11 +30,6 @@
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.Iterator;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.FileConfiguration;
 import org.apache.commons.configuration.reloading.FileChangedReloadingStrategy;
@@ -63,10 +64,10 @@
                                      int reloadPeriod,
                                      TimeUnit reloadUnit)
             throws ConfigurationException {
-        Preconditions.checkNotNull(fileConfigBuilders);
-        Preconditions.checkArgument(!fileConfigBuilders.isEmpty());
-        Preconditions.checkNotNull(executorService);
-        Preconditions.checkNotNull(viewConfig);
+        checkNotNull(fileConfigBuilders);
+        checkArgument(!fileConfigBuilders.isEmpty());
+        checkNotNull(executorService);
+        checkNotNull(viewConfig);
         this.viewConfig = viewConfig;
         this.executorService = executorService;
         this.reloadPeriod = reloadPeriod;
@@ -158,8 +159,8 @@
     }
 
     private boolean fileNotFound(Exception ex) {
-        return ex instanceof FileNotFoundException ||
-                ex.getCause() != null && ex.getCause() instanceof FileNotFoundException;
+        return ex instanceof FileNotFoundException
+            || ex.getCause() != null && ex.getCause() instanceof FileNotFoundException;
     }
 
     private void loadView(FileConfiguration fileConfig) {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/FileConfigurationBuilder.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/FileConfigurationBuilder.java
similarity index 95%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/config/FileConfigurationBuilder.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/config/FileConfigurationBuilder.java
index dbf8fe7..0ff967d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/FileConfigurationBuilder.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/FileConfigurationBuilder.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.config;
+package org.apache.distributedlog.common.config;
 
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.FileConfiguration;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/PropertiesConfigurationBuilder.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/PropertiesConfigurationBuilder.java
similarity index 96%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/config/PropertiesConfigurationBuilder.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/config/PropertiesConfigurationBuilder.java
index df1408c..2d07535 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/PropertiesConfigurationBuilder.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/PropertiesConfigurationBuilder.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,10 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.config;
+package org.apache.distributedlog.common.config;
 
 import java.net.URL;
-
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.FileConfiguration;
 import org.apache.commons.configuration.PropertiesConfiguration;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/package-info.java
similarity index 90%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
copy to distributedlog-common/src/main/java/org/apache/distributedlog/common/config/package-info.java
index eb81cfe..88e68f2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/config/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * IO Utils for distributedlog
+ * Configuration Related Utils.
  */
-package org.apache.distributedlog.io;
+package org.apache.distributedlog.common.config;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/functions/VoidFunctions.java
similarity index 72%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java
copy to distributedlog-common/src/main/java/org/apache/distributedlog/common/functions/VoidFunctions.java
index dccd2e8..8d5069e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/functions/VoidFunctions.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,11 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.callback;
+package org.apache.distributedlog.common.functions;
+
+import java.util.List;
+import java.util.function.Function;
 
 /**
- * ReadAhead Callback
+ * Functions for transforming structures related to {@link Void}.
  */
-public interface ReadAheadCallback {
-    void resumeReadAhead();
+public class VoidFunctions {
+
+    public static final Function<List<Void>, Void> LIST_TO_VOID_FUNC =
+      list -> null;
+
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/functions/package-info.java
similarity index 91%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
copy to distributedlog-common/src/main/java/org/apache/distributedlog/common/functions/package-info.java
index eb81cfe..9e88612 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/functions/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * IO Utils for distributedlog
+ * Common Functions.
  */
-package org.apache.distributedlog.io;
+package org.apache.distributedlog.common.functions;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/package-info.java
similarity index 83%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
copy to distributedlog-common/src/main/java/org/apache/distributedlog/common/package-info.java
index eb81cfe..4c90bd2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -7,7 +7,7 @@
  * "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
+ *      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,
@@ -15,7 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- * IO Utils for distributedlog
+ * Common functions and utils used across the project.
  */
-package org.apache.distributedlog.io;
+package org.apache.distributedlog.common;
\ No newline at end of file
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRate.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/MovingAverageRate.java
similarity index 91%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRate.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/MovingAverageRate.java
index 14db547..f3e8c33 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRate.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/MovingAverageRate.java
@@ -15,8 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.rate;
+package org.apache.distributedlog.common.rate;
 
+/**
+ * Moving Average Rate.
+ */
 public interface MovingAverageRate {
     double get();
     void add(long amount);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRateFactory.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/MovingAverageRateFactory.java
similarity index 60%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRateFactory.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/MovingAverageRateFactory.java
index cd33e24..790ba03 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRateFactory.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/MovingAverageRateFactory.java
@@ -15,35 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.rate;
+package org.apache.distributedlog.common.rate;
 
-import com.twitter.util.Duration;
-import com.twitter.util.Function0;
-import com.twitter.util.TimerTask;
-import com.twitter.util.Timer;
-import com.twitter.util.Time;
 import java.util.concurrent.CopyOnWriteArrayList;
-import scala.runtime.BoxedUnit;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 
-public class MovingAverageRateFactory {
+/**
+ * Factory to create {@link MovingAverageRate} instances.
+ */
+public class MovingAverageRateFactory implements Runnable {
 
     private static final int DEFAULT_INTERVAL_SECS = 1;
 
-    private final Timer timer;
-    private final TimerTask timerTask;
+    private final ScheduledExecutorService scheduler;
+    private final ScheduledFuture<?> scheduledFuture;
     private final CopyOnWriteArrayList<SampledMovingAverageRate> avgs;
 
-    public MovingAverageRateFactory(Timer timer) {
+    public MovingAverageRateFactory(ScheduledExecutorService scheduler) {
         this.avgs = new CopyOnWriteArrayList<SampledMovingAverageRate>();
-        this.timer = timer;
-        Function0<BoxedUnit> sampleTask = new Function0<BoxedUnit>() {
-            public BoxedUnit apply() {
-                sampleAll();
-                return null;
-            }
-        };
-        this.timerTask = timer.schedulePeriodically(
-            Time.now(), Duration.fromSeconds(DEFAULT_INTERVAL_SECS), sampleTask);
+        this.scheduler = scheduler;
+        this.scheduledFuture = this.scheduler.scheduleAtFixedRate(
+            this, DEFAULT_INTERVAL_SECS, DEFAULT_INTERVAL_SECS, TimeUnit.SECONDS);
     }
 
     public MovingAverageRate create(int intervalSecs) {
@@ -53,13 +47,16 @@
     }
 
     public void close() {
-        timerTask.cancel();
+        scheduledFuture.cancel(true);
         avgs.clear();
     }
 
+    @Override
+    public void run() {
+        sampleAll();
+    }
+
     private void sampleAll() {
-        for (SampledMovingAverageRate avg : avgs) {
-            avg.sample();
-        }
+        avgs.forEach(SampledMovingAverageRate::sample);
     }
 }
diff --git a/distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/SampledMovingAverageRate.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/SampledMovingAverageRate.java
new file mode 100644
index 0000000..2c89d64
--- /dev/null
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/SampledMovingAverageRate.java
@@ -0,0 +1,95 @@
+/**
+ * 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.rate;
+
+import com.google.common.base.Ticker;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.commons.lang3.tuple.Pair;
+
+/**
+ * Sampled {@link MovingAverageRate}.
+ */
+class SampledMovingAverageRate implements MovingAverageRate {
+
+    private static final long NANOS_PER_SEC = TimeUnit.SECONDS.toNanos(1);
+
+    private final AtomicLong total;
+    private final Ticker ticker;
+    private final double scaleFactor;
+    private final LinkedBlockingDeque<Pair<Long, Long>> samples;
+
+    private double value;
+
+    public SampledMovingAverageRate(int intervalSecs) {
+        this(intervalSecs, 1, Ticker.systemTicker());
+    }
+
+    SampledMovingAverageRate(int intervalSecs,
+                             double scaleFactor,
+                             Ticker ticker) {
+        this.value = 0;
+        this.total = new AtomicLong(0);
+        this.scaleFactor = scaleFactor;
+        this.ticker = ticker;
+        this.samples = new LinkedBlockingDeque<>(intervalSecs);
+    }
+
+    @Override
+    public double get() {
+        return value;
+    }
+
+    @Override
+    public void add(long amount) {
+        total.getAndAdd(amount);
+    }
+
+    @Override
+    public void inc() {
+        add(1);
+    }
+
+    void sample() {
+        value = doSample();
+    }
+
+    private double doSample() {
+        long newSample = total.get();
+        long newTimestamp = ticker.read();
+
+        double rate = 0;
+        if (!samples.isEmpty()) {
+            Pair<Long, Long> oldestSample = samples.peekLast();
+
+            double dy = newSample - oldestSample.getRight();
+            double dt = newTimestamp - oldestSample.getLeft();
+
+            rate = (dt == 0) ? 0 : (NANOS_PER_SEC * scaleFactor * dy) / dt;
+        }
+
+        if (samples.remainingCapacity() == 0) {
+            samples.removeLast();
+        } else {
+            samples.addFirst(Pair.of(newTimestamp, newSample));
+        }
+
+        return rate;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/package-info.java
similarity index 91%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
copy to distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/package-info.java
index eb81cfe..3117c64 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/rate/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- * IO Utils for distributedlog
+ * Rate relate functions.
  */
-package org.apache.distributedlog.io;
+package org.apache.distributedlog.common.rate;
\ No newline at end of file
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/stats/BroadCastStatsLogger.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/stats/BroadCastStatsLogger.java
similarity index 96%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/stats/BroadCastStatsLogger.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/stats/BroadCastStatsLogger.java
index b6ca733..61a20f1 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/stats/BroadCastStatsLogger.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/stats/BroadCastStatsLogger.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,9 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.stats;
+package org.apache.distributedlog.common.stats;
 
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkNotNull;
 
 import org.apache.bookkeeper.stats.CachingStatsLogger;
 import org.apache.bookkeeper.stats.Counter;
@@ -52,8 +52,8 @@
 
         private Two(StatsLogger first, StatsLogger second) {
             super();
-            Preconditions.checkNotNull(first);
-            Preconditions.checkNotNull(second);
+            checkNotNull(first);
+            checkNotNull(second);
             this.first = first;
             this.second = second;
         }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/stats/OpStatsListener.java
similarity index 89%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java
copy to distributedlog-common/src/main/java/org/apache/distributedlog/common/stats/OpStatsListener.java
index 43641f0..e71a799 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/stats/OpStatsListener.java
@@ -15,13 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.stats;
+package org.apache.distributedlog.common.stats;
 
 import com.google.common.base.Stopwatch;
-import com.twitter.util.FutureEventListener;
-import org.apache.bookkeeper.stats.OpStatsLogger;
 import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
 
+/**
+ * A {@link FutureEventListener} monitors the stats for a given operation.
+ */
 public class OpStatsListener<T> implements FutureEventListener<T> {
     OpStatsLogger opStatsLogger;
     Stopwatch stopwatch;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/stats/package-info.java
similarity index 91%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
copy to distributedlog-common/src/main/java/org/apache/distributedlog/common/stats/package-info.java
index eb81cfe..bf3859d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/stats/package-info.java
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * IO Utils for distributedlog
+ * Stats Related Utils.
  */
-package org.apache.distributedlog.io;
+package org.apache.distributedlog.common.stats;
\ No newline at end of file
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/BitMaskUtils.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/BitMaskUtils.java
similarity index 97%
rename from distributedlog-protocol/src/main/java/org/apache/distributedlog/util/BitMaskUtils.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/util/BitMaskUtils.java
index b5280c9..53f4ab2 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/BitMaskUtils.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/BitMaskUtils.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.util;
+package org.apache.distributedlog.common.util;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/MathUtil.java
similarity index 69%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java
copy to distributedlog-common/src/main/java/org/apache/distributedlog/common/util/MathUtil.java
index dccd2e8..38b3ed2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/MathUtil.java
@@ -7,7 +7,7 @@
  * "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
+ *      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,
@@ -15,11 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.callback;
+
+package org.apache.distributedlog.common.util;
 
 /**
- * ReadAhead Callback
+ * Helpers for math related utils.
  */
-public interface ReadAheadCallback {
-    void resumeReadAhead();
+public class MathUtil {
+
+  public static int signSafeMod(long dividend, int divisor) {
+    int mod = (int) (dividend % divisor);
+
+    if (mod < 0) {
+      mod += divisor;
+    }
+
+    return mod;
+  }
+
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/Permit.java
similarity index 85%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/util/Permit.java
index dccd2e8..24cb63d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/Permit.java
@@ -15,11 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.callback;
+package org.apache.distributedlog.common.util;
 
 /**
- * ReadAhead Callback
+ * Permit.
  */
-public interface ReadAheadCallback {
-    void resumeReadAhead();
+@FunctionalInterface
+public interface Permit {
+
+    void release();
+
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/PermitLimiter.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/PermitLimiter.java
similarity index 89%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/util/PermitLimiter.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/util/PermitLimiter.java
index 15394dc..8fcbf12 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/PermitLimiter.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/PermitLimiter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.util;
+package org.apache.distributedlog.common.util;
 
 /**
  * A simple limiter interface which tracks acquire/release of permits, for
@@ -23,7 +23,7 @@
  */
 public interface PermitLimiter {
 
-    public static PermitLimiter NULL_PERMIT_LIMITER = new PermitLimiter() {
+    PermitLimiter NULL_PERMIT_LIMITER = new PermitLimiter() {
         @Override
         public boolean acquire() {
             return true;
@@ -51,7 +51,7 @@
     void release(int permits);
 
     /**
-     * Close the resources created by the limiter
+     * Close the resources created by the limiter.
      */
     void close();
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/PermitManager.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/PermitManager.java
similarity index 84%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/util/PermitManager.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/util/PermitManager.java
index 24c7860..3b6e3a1 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/PermitManager.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/PermitManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,21 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.util;
+package org.apache.distributedlog.common.util;
 
+/**
+ * Permit manager for managing permits.
+ */
 public interface PermitManager {
 
-    public static interface Permit {
-        static final Permit ALLOWED = new Permit() {
-            @Override
-            public boolean isAllowed() {
-                return true;
-            }
-        };
+    /**
+     * A class present a permit managed by a permit manager.
+     */
+    interface Permit {
+        Permit ALLOWED = () -> true;
         boolean isAllowed();
     }
 
-    public static PermitManager UNLIMITED_PERMIT_MANAGER = new PermitManager() {
+    PermitManager UNLIMITED_PERMIT_MANAGER = new PermitManager() {
         @Override
         public Permit acquirePermit() {
             return Permit.ALLOWED;
@@ -87,7 +88,7 @@
     boolean disallowObtainPermits(Permit permit);
 
     /**
-     * Release the resources
+     * Release the resources.
      */
     void close();
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/SchedulerUtils.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/SchedulerUtils.java
similarity index 62%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/util/SchedulerUtils.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/util/SchedulerUtils.java
index 66e382c..f6d4f23 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/SchedulerUtils.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/SchedulerUtils.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,19 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.util;
-
-import org.apache.bookkeeper.util.OrderedSafeExecutor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+package org.apache.distributedlog.common.util;
 
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
 
+/**
+ * Scheduler related utils.
+ */
+@Slf4j
 public class SchedulerUtils {
 
-    static final Logger logger = LoggerFactory.getLogger(SchedulerUtils.class);
-
     public static void shutdownScheduler(ExecutorService service, long timeout, TimeUnit timeUnit) {
         if (null == service) {
             return;
@@ -36,21 +35,9 @@
         try {
             service.awaitTermination(timeout, timeUnit);
         } catch (InterruptedException e) {
-            logger.warn("Interrupted when shutting down scheduler : ", e);
+            log.warn("Interrupted when shutting down scheduler : ", e);
         }
         service.shutdownNow();
     }
 
-    public static void shutdownScheduler(OrderedSafeExecutor service, long timeout, TimeUnit timeUnit) {
-        if (null == service) {
-            return;
-        }
-        service.shutdown();
-        try {
-            service.awaitTermination(timeout, timeUnit);
-        } catch (InterruptedException e) {
-            logger.warn("Interrupted when shutting down scheduler : ", e);
-        }
-        service.forceShutdown(timeout, timeUnit);
-    }
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Sequencer.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/Sequencer.java
similarity index 95%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/util/Sequencer.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/util/Sequencer.java
index ab8de35..a40b8e2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Sequencer.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/Sequencer.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.util;
+package org.apache.distributedlog.common.util;
 
 /**
  * Sequencer generating transaction id.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Sizable.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/Sizable.java
similarity index 95%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/util/Sizable.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/common/util/Sizable.java
index 2f606e2..d418e0f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Sizable.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/Sizable.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.util;
+package org.apache.distributedlog.common.util;
 
 /**
  * The {@code Sizable} interface is to provide the capability of calculating size
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/package-info.java
similarity index 85%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
copy to distributedlog-common/src/main/java/org/apache/distributedlog/common/util/package-info.java
index eb81cfe..e2bde37 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/common/util/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -7,7 +7,7 @@
  * "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
+ *       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,
@@ -15,7 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- * IO Utils for distributedlog
+ * Common utility functions.
  */
-package org.apache.distributedlog.io;
+package org.apache.distributedlog.common.util;
\ No newline at end of file
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortable.java b/distributedlog-common/src/main/java/org/apache/distributedlog/io/Abortable.java
similarity index 96%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortable.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/io/Abortable.java
index b2b430d..4edc09d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortable.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/io/Abortable.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -37,5 +37,5 @@
      *
      * @throws IOException if an I/O error occurs.
      */
-    public void abort() throws IOException;
+    void abort() throws IOException;
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortables.java b/distributedlog-common/src/main/java/org/apache/distributedlog/io/Abortables.java
similarity index 78%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortables.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/io/Abortables.java
index a4838b1..b6101a8 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortables.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/io/Abortables.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,32 +18,29 @@
 package org.apache.distributedlog.io;
 
 import com.google.common.collect.Lists;
-import org.apache.distributedlog.function.VoidFunctions;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Future;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
+import javax.annotation.Nullable;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.distributedlog.common.functions.VoidFunctions;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 
 /**
  * Utility methods for working with {@link Abortable} objects.
  *
  * @since 0.3.32
  */
+@Slf4j
 public final class Abortables {
 
-    static final Logger logger = LoggerFactory.getLogger(Abortables.class);
-
     private Abortables() {}
 
-    public static Future<Void> asyncAbort(@Nullable AsyncAbortable abortable,
-                                          boolean swallowIOException) {
+    public static CompletableFuture<Void> asyncAbort(@Nullable AsyncAbortable abortable,
+                                                     boolean swallowIOException) {
         if (null == abortable) {
-            return Future.Void();
+            return FutureUtils.Void();
         } else if (swallowIOException) {
             return FutureUtils.ignore(abortable.asyncAbort());
         } else {
@@ -85,7 +82,7 @@
             abortable.abort();
         } catch (IOException ioe) {
             if (swallowIOException) {
-                logger.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe);
+                log.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe);
             } else {
                 throw ioe;
             }
@@ -93,7 +90,7 @@
     }
 
     /**
-     * Abort async <i>abortable</i>
+     * Abort async <i>abortable</i>.
      *
      * @param abortable the {@code AsyncAbortable} object to be aborted, or null, in which case this method
      *                  does nothing.
@@ -109,11 +106,15 @@
         }
         try {
             FutureUtils.result(abortable.asyncAbort());
-        } catch (IOException ioe) {
+        } catch (Exception e) {
             if (swallowIOException) {
-                logger.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe);
+                log.warn("IOException thrown while aborting Abortable {} : ", abortable, e);
             } else {
-                throw ioe;
+                if (e instanceof IOException) {
+                    throw (IOException) e;
+                } else {
+                    throw new IOException(e);
+                }
             }
         }
     }
@@ -122,7 +123,7 @@
      * Aborts the given {@code abortable}, logging any {@code IOException} that's thrown rather than
      * propagating it.
      *
-     * While it's not safe in the general case to ignore exceptions that are thrown when aborting an
+     * <p>While it's not safe in the general case to ignore exceptions that are thrown when aborting an
      * I/O resource, it should generally be safe in the case of a resource that's being used only for
      * reading.
      *
@@ -133,7 +134,7 @@
         try {
             abort(abortable, true);
         } catch (IOException e) {
-            logger.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e);
+            log.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e);
         }
     }
 
@@ -141,7 +142,7 @@
      * Aborts the given {@code abortable}, logging any {@code IOException} that's thrown rather than
      * propagating it.
      *
-     * While it's not safe in the general case to ignore exceptions that are thrown when aborting an
+     * <p>While it's not safe in the general case to ignore exceptions that are thrown when aborting an
      * I/O resource, it should generally be safe in the case of a resource that's being used only for
      * reading.
      *
@@ -152,7 +153,7 @@
         try {
             abort(abortable, true);
         } catch (IOException e) {
-            logger.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e);
+            log.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e);
         }
     }
 
@@ -165,7 +166,7 @@
      *          abortables to abort
      * @return future represents the abort future
      */
-    public static Future<Void> abortSequence(ExecutorService executorService,
+    public static CompletableFuture<Void> abortSequence(ExecutorService executorService,
                                              AsyncAbortable... abortables) {
         List<AsyncAbortable> abortableList = Lists.newArrayListWithExpectedSize(abortables.length);
         for (AsyncAbortable abortable : abortables) {
@@ -176,8 +177,9 @@
             }
         }
         return FutureUtils.processList(
-                abortableList,
-                AsyncAbortable.ABORT_FUNC,
-                executorService).map(VoidFunctions.LIST_TO_VOID_FUNC);
+            abortableList,
+            AsyncAbortable.ABORT_FUNC,
+            executorService
+        ).thenApply(VoidFunctions.LIST_TO_VOID_FUNC);
     }
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java b/distributedlog-common/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java
similarity index 74%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java
index 7ec26a1..7636c57 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,8 +17,9 @@
  */
 package org.apache.distributedlog.io;
 
-import com.twitter.util.Function;
-import com.twitter.util.Future;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 
 /**
  * An {@code Abortable} is a source or destination of data that can be aborted.
@@ -32,19 +33,9 @@
  */
 public interface AsyncAbortable {
 
-    Function<AsyncAbortable, Future<Void>> ABORT_FUNC = new Function<AsyncAbortable, Future<Void>>() {
-        @Override
-        public Future<Void> apply(AsyncAbortable abortable) {
-            return abortable.asyncAbort();
-        }
-    };
+    Function<AsyncAbortable, CompletableFuture<Void>> ABORT_FUNC = abortable -> abortable.asyncAbort();
 
-    AsyncAbortable NULL = new AsyncAbortable() {
-        @Override
-        public Future<Void> asyncAbort() {
-            return Future.Void();
-        }
-    };
+    AsyncAbortable NULL = () -> FutureUtils.Void();
 
     /**
      * Aborts the object and releases any resources associated with it.
@@ -53,5 +44,5 @@
      *
      * @return future represents the abort result
      */
-    Future<Void> asyncAbort();
+    CompletableFuture<Void> asyncAbort();
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java b/distributedlog-common/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java
similarity index 61%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java
index 2bf0119..851f426 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java
@@ -17,9 +17,9 @@
  */
 package org.apache.distributedlog.io;
 
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 
 /**
  * A {@code AsyncCloseable} is a source or destination of data that can be closed asynchronously.
@@ -28,26 +28,12 @@
  */
 public interface AsyncCloseable {
 
-    Function<AsyncCloseable, Future<Void>> CLOSE_FUNC = new Function<AsyncCloseable, Future<Void>>() {
-        @Override
-        public Future<Void> apply(AsyncCloseable closeable) {
-            return closeable.asyncClose();
-        }
-    };
+    Function<AsyncCloseable, CompletableFuture<Void>> CLOSE_FUNC = closeable -> closeable.asyncClose();
 
-    Function<AsyncCloseable, Future<Void>> CLOSE_FUNC_IGNORE_ERRORS = new Function<AsyncCloseable, Future<Void>>() {
-        @Override
-        public Future<Void> apply(AsyncCloseable closeable) {
-            return FutureUtils.ignore(closeable.asyncClose());
-        }
-    };
+    Function<AsyncCloseable, CompletableFuture<Void>> CLOSE_FUNC_IGNORE_ERRORS =
+        closeable -> FutureUtils.ignore(closeable.asyncClose());
 
-    AsyncCloseable NULL = new AsyncCloseable() {
-        @Override
-        public Future<Void> asyncClose() {
-            return Future.Void();
-        }
-    };
+    AsyncCloseable NULL = () -> FutureUtils.Void();
 
     /**
      * Closes this source and releases any system resources associated
@@ -56,5 +42,5 @@
      *
      * @return future representing the close result.
      */
-    Future<Void> asyncClose();
+    CompletableFuture<Void> asyncClose();
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncDeleteable.java b/distributedlog-common/src/main/java/org/apache/distributedlog/io/AsyncDeleteable.java
similarity index 93%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncDeleteable.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/io/AsyncDeleteable.java
index 046c731..f7c3e3b 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncDeleteable.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/io/AsyncDeleteable.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.io;
 
-import com.twitter.util.Future;
+import java.util.concurrent.CompletableFuture;
 
 /**
  * A {@code AsyncDeleteable} is a source or destination of data that can be deleted asynchronously.
@@ -30,5 +30,5 @@
      *
      * @return future representing the deletion result.
      */
-    Future<Void> delete();
+    CompletableFuture<Void> delete();
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-common/src/main/java/org/apache/distributedlog/io/package-info.java
similarity index 95%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/io/package-info.java
index eb81cfe..c8e957f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/io/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * IO Utils for distributedlog
+ * IO Utils for distributedlog.
  */
 package org.apache.distributedlog.io;
diff --git a/distributedlog-common/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java b/distributedlog-common/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java
new file mode 100644
index 0000000..89b448e
--- /dev/null
+++ b/distributedlog-common/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java
@@ -0,0 +1,353 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.util;
+
+import com.google.common.base.Objects;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+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.distributedlog.common.util.MathUtil;
+
+/**
+ * Ordered Scheduler. It is thread pool based {@link ScheduledExecutorService}, additionally providing
+ * the ability to execute/schedule tasks by <code>key</code>. Hence the tasks submitted by same <i>key</i>
+ * will be executed in order.
+ *
+ * <p>The scheduler is comprised of multiple {@link ScheduledExecutorService}s. Each
+ * {@link ScheduledExecutorService} is a single thread executor. Normal task submissions will
+ * be submitted to executors in a random manner to guarantee load balancing. Keyed task submissions (e.g
+ * {@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 {
+
+    /**
+     * Create a builder to build scheduler.
+     *
+     * @return scheduler builder
+     */
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Builder for {@link OrderedScheduler}.
+     */
+    public static class Builder {
+
+        private String name = "OrderedScheduler";
+        private int corePoolSize = -1;
+        private ThreadFactory threadFactory = null;
+
+        /**
+         * Set the name of this scheduler. It would be used as part of stats scope and thread name.
+         *
+         * @param name name of the scheduler.
+         * @return scheduler builder
+         */
+        public Builder name(String name) {
+            this.name = name;
+            return this;
+        }
+
+        /**
+         * Set the number of threads to be used in this scheduler.
+         *
+         * @param corePoolSize the number of threads to keep in the pool, even
+         *                     if they are idle
+         * @return scheduler builder
+         */
+        public Builder corePoolSize(int corePoolSize) {
+            this.corePoolSize = corePoolSize;
+            return this;
+        }
+
+        /**
+         * Set the thread factory that the scheduler uses to create a new thread.
+         *
+         * @param threadFactory the factory to use when the executor
+         *                      creates a new thread
+         * @return scheduler builder
+         */
+        public Builder threadFactory(ThreadFactory threadFactory) {
+            this.threadFactory = threadFactory;
+            return this;
+        }
+
+        /**
+         * Build the ordered scheduler.
+         *
+         * @return ordered scheduler
+         */
+        public OrderedScheduler build() {
+            if (corePoolSize <= 0) {
+                corePoolSize = Runtime.getRuntime().availableProcessors();
+            }
+            if (null == threadFactory) {
+                threadFactory = Executors.defaultThreadFactory();
+            }
+
+            return new OrderedScheduler(
+                name,
+                corePoolSize,
+                threadFactory);
+        }
+
+    }
+
+    protected final String name;
+    protected final int corePoolSize;
+    protected final ScheduledExecutorService[] executors;
+    protected final Random random;
+
+    private OrderedScheduler(String name,
+                             int corePoolSize,
+                             ThreadFactory threadFactory) {
+        this.name = name;
+        this.corePoolSize = corePoolSize;
+        this.executors = new ScheduledExecutorService[corePoolSize];
+        for (int i = 0; i < corePoolSize; i++) {
+            ThreadFactory tf = new ThreadFactoryBuilder()
+                .setNameFormat(name + "-scheduler-" + i + "-%d")
+                .setThreadFactory(threadFactory)
+                .build();
+            executors[i] = Executors.newSingleThreadScheduledExecutor(tf);
+        }
+        this.random = new Random(System.currentTimeMillis());
+    }
+
+    protected ScheduledExecutorService chooseExecutor() {
+        return corePoolSize == 1 ? executors[0] : executors[random.nextInt(corePoolSize)];
+    }
+
+    public ScheduledExecutorService chooseExecutor(Object key) {
+        if (null == key) {
+            return chooseExecutor();
+        }
+        return corePoolSize == 1 ? executors[0] :
+            executors[MathUtil.signSafeMod(Objects.hashCode(key), corePoolSize)];
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public ScheduledFuture<?> schedule(Runnable command, long delay, TimeUnit unit) {
+        return chooseExecutor().schedule(command, delay, unit);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <V> ScheduledFuture<V> schedule(Callable<V> callable, long delay, TimeUnit unit) {
+        return chooseExecutor().schedule(callable, delay, unit);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public ScheduledFuture<?> scheduleAtFixedRate(Runnable command,
+                                                  long initialDelay, long period, TimeUnit unit) {
+        return chooseExecutor().scheduleAtFixedRate(command, initialDelay, period, unit);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public ScheduledFuture<?> scheduleWithFixedDelay(Runnable command,
+                                                     long initialDelay, long delay, TimeUnit unit) {
+        return chooseExecutor().scheduleWithFixedDelay(command, initialDelay, delay, unit);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void shutdown() {
+        for (ScheduledExecutorService executor : executors) {
+            executor.shutdown();
+        }
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public List<Runnable> shutdownNow() {
+        List<Runnable> runnables = new ArrayList<Runnable>();
+        for (ScheduledExecutorService executor : executors) {
+            runnables.addAll(executor.shutdownNow());
+        }
+        return runnables;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean isShutdown() {
+        for (ScheduledExecutorService executor : executors) {
+            if (!executor.isShutdown()) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean isTerminated() {
+        for (ScheduledExecutorService executor : executors) {
+            if (!executor.isTerminated()) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean awaitTermination(long timeout, TimeUnit unit)
+        throws InterruptedException {
+        for (ScheduledExecutorService executor : executors) {
+            if (!executor.awaitTermination(timeout, unit)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> Future<T> submit(Callable<T> task) {
+        return chooseExecutor().submit(task);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> Future<T> submit(Runnable task, T result) {
+        return chooseExecutor().submit(task, result);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public Future<?> submit(Runnable task) {
+        return chooseExecutor().submit(task);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+        throws InterruptedException {
+        return chooseExecutor().invokeAll(tasks);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit)
+        throws InterruptedException {
+        return chooseExecutor().invokeAll(tasks, timeout, unit);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks)
+        throws InterruptedException, ExecutionException {
+        return chooseExecutor().invokeAny(tasks);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit)
+        throws InterruptedException, ExecutionException, TimeoutException {
+        return chooseExecutor().invokeAny(tasks, timeout, unit);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void execute(Runnable command) {
+        chooseExecutor().execute(command);
+    }
+
+    // Ordered Functions
+
+    public ScheduledFuture<?> schedule(Object key, Runnable command, long delay, TimeUnit unit) {
+        return chooseExecutor(key).schedule(command, delay, unit);
+    }
+
+    public ScheduledFuture<?> scheduleAtFixedRate(Object key,
+                                                  Runnable command,
+                                                  long initialDelay,
+                                                  long period,
+                                                  TimeUnit unit) {
+        return chooseExecutor(key).scheduleAtFixedRate(command, initialDelay, period, unit);
+    }
+
+    public Future<?> submit(Object key, Runnable command) {
+        return chooseExecutor(key).submit(command);
+    }
+
+    public <T> CompletableFuture<T> submit(Object key, Callable<T> callable) {
+        CompletableFuture<T> future = FutureUtils.createFuture();
+        chooseExecutor(key).submit(() -> {
+            try {
+                future.complete(callable.call());
+            } catch (Exception e) {
+                future.completeExceptionally(e);
+            }
+        });
+        return future;
+    }
+
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/package-info.java b/distributedlog-common/src/main/java/org/apache/distributedlog/util/package-info.java
similarity index 100%
rename from distributedlog-protocol/src/main/java/org/apache/distributedlog/util/package-info.java
rename to distributedlog-common/src/main/java/org/apache/distributedlog/util/package-info.java
diff --git a/distributedlog-common/src/main/resources/findbugsExclude.xml b/distributedlog-common/src/main/resources/findbugsExclude.xml
new file mode 100644
index 0000000..ce2c176
--- /dev/null
+++ b/distributedlog-common/src/main/resources/findbugsExclude.xml
@@ -0,0 +1,32 @@
+<!--
+    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.
+//-->
+<FindBugsFilter>
+  <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" />
+  </Match>
+</FindBugsFilter>
diff --git a/distributedlog-common/src/test/java/org/apache/distributedlog/common/concurrent/TestFutureUtils.java b/distributedlog-common/src/test/java/org/apache/distributedlog/common/concurrent/TestFutureUtils.java
new file mode 100644
index 0000000..ddfb7ae
--- /dev/null
+++ b/distributedlog-common/src/test/java/org/apache/distributedlog/common/concurrent/TestFutureUtils.java
@@ -0,0 +1,384 @@
+/*
+ * 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.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Matchers.isA;
+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), isA(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), isA(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(anyObject(), 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();
+    }
+
+    @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());
+    }
+
+    @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());
+    }
+
+    @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/distributedlog-common/src/test/java/org/apache/distributedlog/common/config/PropertiesWriter.java b/distributedlog-common/src/test/java/org/apache/distributedlog/common/config/PropertiesWriter.java
new file mode 100644
index 0000000..6b3ca58
--- /dev/null
+++ b/distributedlog-common/src/test/java/org/apache/distributedlog/common/config/PropertiesWriter.java
@@ -0,0 +1,69 @@
+/*
+ * 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.config;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.util.Properties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Writer to write properties to files.
+ */
+public class PropertiesWriter {
+    static final Logger LOG = LoggerFactory.getLogger(PropertiesWriter.class);
+
+    final FileOutputStream outputStream;
+    final File configFile;
+    final Properties properties;
+
+    public PropertiesWriter() throws Exception {
+        this(null);
+    }
+
+    public PropertiesWriter(File configFile) throws Exception {
+        if (null == configFile) {
+            this.configFile = File.createTempFile("temp", ".conf");
+        } else {
+            this.configFile = configFile;
+        }
+        this.configFile.deleteOnExit();
+        this.properties = new Properties();
+        this.outputStream = new FileOutputStream(this.configFile);
+    }
+
+    public void setProperty(String key, String value) {
+        properties.setProperty(key, value);
+    }
+
+    public void removeProperty(String key) {
+        properties.remove(key);
+    }
+
+    public void save() throws Exception {
+        FileOutputStream outputStream = new FileOutputStream(configFile);
+        properties.store(outputStream, null);
+        configFile.setLastModified(configFile.lastModified() + 1000);
+        LOG.debug("save modified={}", configFile.lastModified());
+    }
+
+    public File getFile() {
+        return configFile;
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestConcurrentBaseConfiguration.java b/distributedlog-common/src/test/java/org/apache/distributedlog/common/config/TestConcurrentBaseConfiguration.java
similarity index 87%
rename from distributedlog-core/src/test/java/org/apache/distributedlog/config/TestConcurrentBaseConfiguration.java
rename to distributedlog-common/src/test/java/org/apache/distributedlog/common/config/TestConcurrentBaseConfiguration.java
index 9563511..a54faa0 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestConcurrentBaseConfiguration.java
+++ b/distributedlog-common/src/test/java/org/apache/distributedlog/common/config/TestConcurrentBaseConfiguration.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.config;
+package org.apache.distributedlog.common.config;
+
+import static org.junit.Assert.assertEquals;
 
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import static org.junit.Assert.*;
-
+/**
+ * Unit test of {@link ConcurrentBaseConfiguration}.
+ */
 public class TestConcurrentBaseConfiguration {
-    static final Logger LOG = LoggerFactory.getLogger(TestConcurrentBaseConfiguration.class);
 
     @Test(timeout = 20000)
     public void testBasicOperations() throws Exception {
@@ -43,4 +43,5 @@
         assertEquals(null, conf.getInteger("prop1", null));
         assertEquals(2, conf.getInt("prop2"));
     }
+
 }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestConfigurationSubscription.java b/distributedlog-common/src/test/java/org/apache/distributedlog/common/config/TestConfigurationSubscription.java
similarity index 93%
rename from distributedlog-core/src/test/java/org/apache/distributedlog/config/TestConfigurationSubscription.java
rename to distributedlog-common/src/test/java/org/apache/distributedlog/common/config/TestConfigurationSubscription.java
index 8420a97..a474f89 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestConfigurationSubscription.java
+++ b/distributedlog-common/src/test/java/org/apache/distributedlog/common/config/TestConfigurationSubscription.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,10 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.config;
+package org.apache.distributedlog.common.config;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 import com.google.common.collect.Lists;
-import org.apache.distributedlog.DistributedLogConfiguration;
+import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.configuration.CompositeConfiguration;
 import org.apache.commons.configuration.event.ConfigurationEvent;
 import org.apache.commons.configuration.event.ConfigurationListener;
 import org.jmock.lib.concurrent.DeterministicScheduler;
@@ -26,14 +35,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.List;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.*;
-
 /**
  * Notes:
  * 1. lastModified granularity is platform dependent, generally 1 sec, so we can't wait 1ms for things to
@@ -43,8 +44,9 @@
     static final Logger LOG = LoggerFactory.getLogger(TestConfigurationSubscription.class);
 
     /**
-     * Give FileChangedReloadingStrategy some time to start reloading
-     * Make sure now!=lastChecked
+     * Give FileChangedReloadingStrategy some time to start reloading.
+     *
+     * <p>Make sure now!=lastChecked
      * {@link org.apache.commons.configuration.reloading.FileChangedReloadingStrategy#reloadingRequired()}
      */
     private void ensureConfigReloaded() throws InterruptedException {
@@ -57,13 +59,13 @@
     public void testReloadConfiguration() throws Exception {
         PropertiesWriter writer = new PropertiesWriter();
         FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
-        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new CompositeConfiguration());
         DeterministicScheduler executorService = new DeterministicScheduler();
         List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(builder);
         ConfigurationSubscription confSub =
                 new ConfigurationSubscription(conf, fileConfigBuilders, executorService, 100, TimeUnit.MILLISECONDS);
         final AtomicReference<ConcurrentBaseConfiguration> confHolder = new AtomicReference<>();
-        confSub.registerListener(new org.apache.distributedlog.config.ConfigurationListener() {
+        confSub.registerListener(new org.apache.distributedlog.common.config.ConfigurationListener() {
             @Override
             public void onReload(ConcurrentBaseConfiguration conf) {
                 confHolder.set(conf);
@@ -88,7 +90,7 @@
         PropertiesWriter writer = new PropertiesWriter();
         DeterministicScheduler mockScheduler = new DeterministicScheduler();
         FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
-        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new CompositeConfiguration());
         List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(builder);
         ConfigurationSubscription confSub =
                 new ConfigurationSubscription(conf, fileConfigBuilders, mockScheduler, 100, TimeUnit.MILLISECONDS);
@@ -116,7 +118,7 @@
 
         ScheduledExecutorService mockScheduler = new DeterministicScheduler();
         FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
-        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new CompositeConfiguration());
         List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(builder);
         ConfigurationSubscription confSub =
                 new ConfigurationSubscription(conf, fileConfigBuilders, mockScheduler, 100, TimeUnit.MILLISECONDS);
@@ -135,7 +137,7 @@
 
         DeterministicScheduler mockScheduler = new DeterministicScheduler();
         FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
-        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new CompositeConfiguration());
         List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(builder);
         ConfigurationSubscription confSub =
                 new ConfigurationSubscription(conf, fileConfigBuilders, mockScheduler, 100, TimeUnit.MILLISECONDS);
diff --git a/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestTimedOutTestsListener.java b/distributedlog-common/src/test/java/org/apache/distributedlog/common/util/TestTimedOutTestsListener.java
similarity index 98%
rename from distributedlog-protocol/src/test/java/org/apache/distributedlog/TestTimedOutTestsListener.java
rename to distributedlog-common/src/test/java/org/apache/distributedlog/common/util/TestTimedOutTestsListener.java
index 20cf53c..7a981d1 100644
--- a/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestTimedOutTestsListener.java
+++ b/distributedlog-common/src/test/java/org/apache/distributedlog/common/util/TestTimedOutTestsListener.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog;
+package org.apache.distributedlog.common.util;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
diff --git a/distributedlog-protocol/src/test/java/org/apache/distributedlog/TimedOutTestsListener.java b/distributedlog-common/src/test/java/org/apache/distributedlog/common/util/TimedOutTestsListener.java
similarity index 98%
rename from distributedlog-protocol/src/test/java/org/apache/distributedlog/TimedOutTestsListener.java
rename to distributedlog-common/src/test/java/org/apache/distributedlog/common/util/TimedOutTestsListener.java
index db0ee4e..c86cf8f 100644
--- a/distributedlog-protocol/src/test/java/org/apache/distributedlog/TimedOutTestsListener.java
+++ b/distributedlog-common/src/test/java/org/apache/distributedlog/common/util/TimedOutTestsListener.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog;
+package org.apache.distributedlog.common.util;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
diff --git a/distributedlog-core-twitter/pom.xml b/distributedlog-core-twitter/pom.xml
new file mode 100644
index 0000000..41116b9
--- /dev/null
+++ b/distributedlog-core-twitter/pom.xml
@@ -0,0 +1,141 @@
+<?xml version="1.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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.distributedlog</groupId>
+    <artifactId>distributedlog</artifactId>
+    <version>0.5.0-incubating-SNAPSHOT</version>
+  </parent>
+  <artifactId>distributedlog-core-twitter</artifactId>
+  <name>Apache DistributedLog :: Core Library (Twitter Future Interface)</name>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-core</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>util-core_2.11</artifactId>
+      <version>${finagle.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <version>${slf4j.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <version>${mockito.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-common</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>${maven-compiler-plugin.version}</version>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>${maven-surefire-plugin.version}</version>
+        <configuration>
+          <redirectTestOutputToFile>true</redirectTestOutputToFile>
+          <argLine>-Xmx3G -Djava.net.preferIPv4Stack=true -XX:MaxDirectMemorySize=2G</argLine>
+          <forkMode>always</forkMode>
+          <forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
+          <properties>
+            <property>
+              <name>listener</name>
+              <value>org.apache.distributedlog.common.util.TimedOutTestsListener</value>
+            </property>
+          </properties>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/src/main/resources/findbugsExclude.xml</excludeFilterFile>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <version>${maven-checkstyle-plugin.version}</version>
+        <dependencies>
+          <dependency>
+            <groupId>com.puppycrawl.tools</groupId>
+            <artifactId>checkstyle</artifactId>
+            <version>${puppycrawl.checkstyle.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.distributedlog</groupId>
+            <artifactId>distributedlog-build-tools</artifactId>
+            <version>${project.version}</version>
+          </dependency>
+        </dependencies>
+        <configuration>
+          <configLocation>distributedlog/checkstyle.xml</configLocation>
+          <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
+          <consoleOutput>true</consoleOutput>
+          <failOnViolation>true</failOnViolation>
+          <includeResources>false</includeResources>
+          <includeTestSourceDirectory>true</includeTestSourceDirectory>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>test-compile</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogReader.java
similarity index 73%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java
copy to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogReader.java
index e3ace05..4ec1dfa 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogReader.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,27 +17,28 @@
  */
 package org.apache.distributedlog;
 
-import org.apache.distributedlog.io.AsyncCloseable;
 import com.twitter.util.Future;
-
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
-public interface AsyncLogReader extends AsyncCloseable {
+/**
+ * A log reader to read records in asynchronous way.
+ */
+public interface AsyncLogReader {
 
     /**
      * Get stream name that the reader reads from.
      *
      * @return stream name.
      */
-    public String getStreamName();
+    String getStreamName();
 
     /**
-     * Read the next record from the log stream
+     * Read the next record from the log stream.
      *
      * @return A promise that when satisfied will contain the Log Record with its DLSN.
      */
-    public Future<LogRecordWithDLSN> readNext();
+    Future<LogRecordWithDLSN> readNext();
 
     /**
      * Read next <i>numEntries</i> entries. The future is only satisfied with non-empty list
@@ -48,12 +49,12 @@
      *          num entries
      * @return A promise that when satisfied will contain a non-empty list of records with their DLSN.
      */
-    public Future<List<LogRecordWithDLSN>> readBulk(int numEntries);
+    Future<List<LogRecordWithDLSN>> readBulk(int numEntries);
 
     /**
      * Read next <i>numEntries</i> entries in a given <i>waitTime</i>.
-     * <p>
-     * The future is satisfied when either reads <i>numEntries</i> entries or reaches <i>waitTime</i>.
+     *
+     * <p>The future is satisfied when either reads <i>numEntries</i> entries or reaches <i>waitTime</i>.
      * The only exception is if there isn't any new entries written within <i>waitTime</i>, it would
      * wait until new entries are available.
      *
@@ -65,5 +66,14 @@
      *          wait time unit
      * @return A promise that when satisfied will contain a non-empty list of records with their DLSN.
      */
-    public Future<List<LogRecordWithDLSN>> readBulk(int numEntries, long waitTime, TimeUnit timeUnit);
+    Future<List<LogRecordWithDLSN>> readBulk(int numEntries, long waitTime, TimeUnit timeUnit);
+
+    /**
+     * Closes this source and releases any system resources associated
+     * with it. If the source is already closed then invoking this
+     * method has no effect.
+     *
+     * @return future representing the close result.
+     */
+    Future<Void> asyncClose();
 }
diff --git a/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogReaderImpl.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogReaderImpl.java
new file mode 100644
index 0000000..4f4a90e
--- /dev/null
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogReaderImpl.java
@@ -0,0 +1,78 @@
+/*
+ * 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;
+
+import static org.apache.distributedlog.util.FutureUtils.newTFuture;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.twitter.util.Future;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import scala.Function1;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Implementation wrapper of {@link org.apache.distributedlog.api.AsyncLogReader}.
+ */
+class AsyncLogReaderImpl implements AsyncLogReader {
+
+    static final Function1<org.apache.distributedlog.api.AsyncLogReader, AsyncLogReader> MAP_FUNC =
+        new AbstractFunction1<org.apache.distributedlog.api.AsyncLogReader, AsyncLogReader>() {
+            @Override
+            public AsyncLogReader apply(org.apache.distributedlog.api.AsyncLogReader reader) {
+                return new AsyncLogReaderImpl(reader);
+            }
+        };
+
+    private final org.apache.distributedlog.api.AsyncLogReader impl;
+
+    AsyncLogReaderImpl(org.apache.distributedlog.api.AsyncLogReader impl) {
+        this.impl = impl;
+    }
+
+    @VisibleForTesting
+    org.apache.distributedlog.api.AsyncLogReader getImpl() {
+        return impl;
+    }
+
+    @Override
+    public String getStreamName() {
+        return impl.getStreamName();
+    }
+
+    @Override
+    public Future<LogRecordWithDLSN> readNext() {
+        return newTFuture(impl.readNext());
+    }
+
+    @Override
+    public Future<List<LogRecordWithDLSN>> readBulk(int numEntries) {
+        return newTFuture(impl.readBulk(numEntries));
+    }
+
+    @Override
+    public Future<List<LogRecordWithDLSN>> readBulk(int numEntries, long waitTime, TimeUnit timeUnit) {
+        return newTFuture(impl.readBulk(numEntries, waitTime, timeUnit));
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        return newTFuture(impl.asyncClose());
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
similarity index 69%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
copy to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
index 53b393b..915877c 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,21 +17,20 @@
  */
 package org.apache.distributedlog;
 
-import org.apache.distributedlog.io.AsyncAbortable;
-import org.apache.distributedlog.io.AsyncCloseable;
 import com.twitter.util.Future;
-
-import java.io.Closeable;
 import java.util.List;
 
-public interface AsyncLogWriter extends AsyncCloseable, AsyncAbortable {
+/**
+ * A writer that appends log records asynchronously.
+ */
+public interface AsyncLogWriter {
 
     /**
      * Get the last committed transaction id.
      *
      * @return last committed transaction id.
      */
-    public long getLastTxId();
+    long getLastTxId();
 
     /**
      * Write a log record to the stream.
@@ -40,7 +39,7 @@
      * @return A Future which contains a DLSN if the record was successfully written
      * or an exception if the write fails
      */
-    public Future<DLSN> write(LogRecord record);
+    Future<DLSN> write(LogRecord record);
 
     /**
      * Write log records to the stream in bulk. Each future in the list represents the result of
@@ -51,7 +50,7 @@
      * @return A Future which contains a list of Future DLSNs if the record was successfully written
      * or an exception if the operation fails.
      */
-    public Future<List<Future<DLSN>>> writeBulk(List<LogRecord> record);
+    Future<List<Future<DLSN>>> writeBulk(List<LogRecord> record);
 
     /**
      * Truncate the log until <i>dlsn</i>.
@@ -61,10 +60,28 @@
      * @return A Future indicates whether the operation succeeds or not, or an exception
      * if the truncation fails.
      */
-    public Future<Boolean> truncate(DLSN dlsn);
+    Future<Boolean> truncate(DLSN dlsn);
 
     /**
-     * Get the name of the stream this writer writes data to
+     * Get the name of the stream this writer writes data to.
      */
-    public String getStreamName();
+    String getStreamName();
+
+    /**
+     * Closes this source and releases any system resources associated
+     * with it. If the source is already closed then invoking this
+     * method has no effect.
+     *
+     * @return future representing the close result.
+     */
+    Future<Void> asyncClose();
+
+    /**
+     * Aborts the object and releases any resources associated with it.
+     * If the object is already aborted then invoking this method has no
+     * effect.
+     *
+     * @return future represents the abort result
+     */
+    Future<Void> asyncAbort();
 }
diff --git a/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogWriterImpl.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogWriterImpl.java
new file mode 100644
index 0000000..dc28bb1
--- /dev/null
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/AsyncLogWriterImpl.java
@@ -0,0 +1,88 @@
+/*
+ * 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;
+
+import static org.apache.distributedlog.util.FutureUtils.newTFuture;
+import static org.apache.distributedlog.util.FutureUtils.newTFutureList;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.twitter.util.Future;
+import java.util.List;
+import scala.Function1;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * The implementation of {@link AsyncLogWriter} built over {@link org.apache.distributedlog.api.AsyncLogWriter}.
+ */
+class AsyncLogWriterImpl implements AsyncLogWriter {
+
+    static final Function1<org.apache.distributedlog.api.AsyncLogWriter, AsyncLogWriter> MAP_FUNC =
+        new AbstractFunction1<org.apache.distributedlog.api.AsyncLogWriter, AsyncLogWriter>() {
+            @Override
+            public AsyncLogWriter apply(org.apache.distributedlog.api.AsyncLogWriter writer) {
+                return new AsyncLogWriterImpl(writer);
+            }
+        };
+
+    private final org.apache.distributedlog.api.AsyncLogWriter impl;
+
+    AsyncLogWriterImpl(org.apache.distributedlog.api.AsyncLogWriter impl) {
+        this.impl = impl;
+    }
+
+    @VisibleForTesting
+    org.apache.distributedlog.api.AsyncLogWriter getImpl() {
+        return impl;
+    }
+
+    @Override
+    public long getLastTxId() {
+        return impl.getLastTxId();
+    }
+
+    @Override
+    public Future<DLSN> write(LogRecord record) {
+        return newTFuture(impl.write(record));
+    }
+
+    @Override
+    public Future<List<Future<DLSN>>> writeBulk(List<LogRecord> record) {
+        return newTFutureList(impl.writeBulk(record));
+    }
+
+    @Override
+    public Future<Boolean> truncate(DLSN dlsn) {
+        return newTFuture(impl.truncate(dlsn));
+    }
+
+    @Override
+    public String getStreamName() {
+        return impl.getStreamName();
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        return newTFuture(impl.asyncClose());
+    }
+
+    @Override
+    public Future<Void> asyncAbort() {
+        return newTFuture(impl.asyncAbort());
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/DistributedLogManager.java
similarity index 66%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java
copy to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/DistributedLogManager.java
index 7d33e9c..14f05c3 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/DistributedLogManager.java
@@ -17,16 +17,13 @@
  */
 package org.apache.distributedlog;
 
-import org.apache.distributedlog.callback.LogSegmentListener;
-import org.apache.distributedlog.io.AsyncCloseable;
-import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.subscription.SubscriptionStateStore;
-import org.apache.distributedlog.subscription.SubscriptionsStore;
 import com.twitter.util.Future;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.subscription.SubscriptionsStore;
 
 /**
  * A DistributedLogManager is responsible for managing a single place of storing
@@ -35,20 +32,21 @@
  * each conceptual place of storage corresponds to exactly one instance of
  * this class, which is created when the EditLog is first opened.
  */
-public interface DistributedLogManager extends AsyncCloseable, Closeable {
+public interface DistributedLogManager extends Closeable {
 
     /**
-     * Get the name of the stream managed by this log manager
+     * Get the name of the stream managed by this log manager.
+     *
      * @return streamName
      */
-    public String getStreamName();
+    String getStreamName();
 
     /**
      * Get the namespace driver used by this manager.
      *
      * @return the namespace driver
      */
-    public NamespaceDriver getNamespaceDriver();
+    NamespaceDriver getNamespaceDriver();
 
     /**
      * Get log segments.
@@ -56,7 +54,7 @@
      * @return log segments
      * @throws IOException
      */
-    public List<LogSegmentMetadata> getLogSegments() throws IOException;
+    List<LogSegmentMetadata> getLogSegments() throws IOException;
 
     /**
      * Register <i>listener</i> on log segment updates of this stream.
@@ -64,7 +62,7 @@
      * @param listener
      *          listener to receive update log segment list.
      */
-    public void registerListener(LogSegmentListener listener) throws IOException ;
+    void registerListener(LogSegmentListener listener) throws IOException;
 
     /**
      * Unregister <i>listener</i> on log segment updates from this stream.
@@ -72,55 +70,55 @@
      * @param listener
      *          listener to receive update log segment list.
      */
-    public void unregisterListener(LogSegmentListener listener);
+    void unregisterListener(LogSegmentListener listener);
 
     /**
      * Open async log writer to write records to the log stream.
      *
      * @return result represents the open result
      */
-    public Future<AsyncLogWriter> openAsyncLogWriter();
+    Future<AsyncLogWriter> openAsyncLogWriter();
 
     /**
-     * Begin writing to the log stream identified by the name
+     * Begin writing to the log stream identified by the name.
      *
      * @return the writer interface to generate log records
      */
-    public LogWriter startLogSegmentNonPartitioned() throws IOException;
+    LogWriter startLogSegmentNonPartitioned() throws IOException;
 
     /**
-     * Begin writing to the log stream identified by the name
+     * Begin writing to the log stream identified by the name.
      *
      * @return the writer interface to generate log records
      */
     // @Deprecated
-    public AsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException;
+    AsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException;
 
     /**
-     * Begin appending to the end of the log stream which is being treated as a sequence of bytes
+     * Begin appending to the end of the log stream which is being treated as a sequence of bytes.
      *
      * @return the writer interface to generate log records
      */
-    public AppendOnlyStreamWriter getAppendOnlyStreamWriter() throws IOException;
+    AppendOnlyStreamWriter getAppendOnlyStreamWriter() throws IOException;
 
     /**
-     * Get a reader to read a log stream as a sequence of bytes
+     * Get a reader to read a log stream as a sequence of bytes.
      *
      * @return the writer interface to generate log records
      */
-    public AppendOnlyStreamReader getAppendOnlyStreamReader() throws IOException;
+    AppendOnlyStreamReader getAppendOnlyStreamReader() throws IOException;
 
     /**
-     * Get the input stream starting with fromTxnId for the specified log
+     * Get the input stream starting with fromTxnId for the specified log.
      *
      * @param fromTxnId - the first transaction id we want to read
      * @return the stream starting with transaction fromTxnId
      * @throws IOException if a stream cannot be found.
      */
-    public LogReader getInputStream(long fromTxnId)
+    LogReader getInputStream(long fromTxnId)
         throws IOException;
 
-    public LogReader getInputStream(DLSN fromDLSN) throws IOException;
+    LogReader getInputStream(DLSN fromDLSN) throws IOException;
 
     /**
      * Open an async log reader to read records from a log starting from <code>fromTxnId</code>.
@@ -129,24 +127,24 @@
      *          transaction id to start reading from
      * @return async log reader
      */
-    public Future<AsyncLogReader> openAsyncLogReader(long fromTxnId);
+    Future<AsyncLogReader> openAsyncLogReader(long fromTxnId);
 
     /**
-     * Open an async log reader to read records from a log starting from <code>fromDLSN</code>
+     * Open an async log reader to read records from a log starting from <code>fromDLSN</code>.
      *
      * @param fromDLSN
      *          dlsn to start reading from
      * @return async log reader
      */
-    public Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN);
+    Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN);
 
     // @Deprecated
-    public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException;
+    AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException;
 
     // @Deprecated
-    public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException;
+    AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException;
 
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN);
+    Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN);
 
     /**
      * Get a log reader with lock starting from <i>fromDLSN</i> and using <i>subscriberId</i>.
@@ -159,21 +157,21 @@
      *          subscriber id
      * @return async log reader
      */
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN, String subscriberId);
+    Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN, String subscriberId);
 
     /**
      * Get a log reader using <i>subscriberId</i> with lock. The reader will start reading from
      * its last commit position recorded in subscription store. If no last commit position found
      * in subscription store, it would start reading from head of the stream.
      *
-     * If the two readers tried to open using same subscriberId, one would succeed, while the other
+     * <p>If the two readers tried to open using same subscriberId, one would succeed, while the other
      * will be blocked until it gets the lock.
      *
      * @param subscriberId
      *          subscriber id
      * @return async log reader
      */
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId);
+    Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId);
 
     /**
      * Get the {@link DLSN} of first log record whose transaction id is not less than <code>transactionId</code>.
@@ -182,109 +180,111 @@
      *          transaction id
      * @return dlsn of first log record whose transaction id is not less than transactionId.
      */
-    public Future<DLSN> getDLSNNotLessThanTxId(long transactionId);
+    Future<DLSN> getDLSNNotLessThanTxId(long transactionId);
 
     /**
-     * Get the last log record in the stream
+     * Get the last log record in the stream.
      *
      * @return the last log record in the stream
      * @throws IOException if a stream cannot be found.
      */
-    public LogRecordWithDLSN getLastLogRecord()
+    LogRecordWithDLSN getLastLogRecord()
         throws IOException;
 
     /**
-     * Get the earliest Transaction Id available in the log
+     * Get the earliest Transaction Id available in the log.
      *
      * @return earliest transaction id
      * @throws IOException
      */
-    public long getFirstTxId() throws IOException;
+    long getFirstTxId() throws IOException;
 
     /**
-     * Get Latest Transaction Id in the log
+     * Get Latest Transaction Id in the log.
      *
      * @return latest transaction id
      * @throws IOException
      */
-    public long getLastTxId() throws IOException;
+    long getLastTxId() throws IOException;
 
     /**
-     * Get Latest DLSN in the log
+     * Get Latest DLSN in the log.
      *
      * @return last dlsn
      * @throws IOException
      */
-    public DLSN getLastDLSN() throws IOException;
+    DLSN getLastDLSN() throws IOException;
 
     /**
-     * Get Latest log record with DLSN in the log - async
+     * Get Latest log record with DLSN in the log - async.
      *
      * @return latest log record with DLSN
      */
-    public Future<LogRecordWithDLSN> getLastLogRecordAsync();
+    Future<LogRecordWithDLSN> getLastLogRecordAsync();
 
     /**
-     * Get Latest Transaction Id in the log - async
+     * Get Latest Transaction Id in the log - async.
      *
      * @return latest transaction id
      */
-    public Future<Long> getLastTxIdAsync();
+    Future<Long> getLastTxIdAsync();
 
     /**
      * Get first DLSN in the log.
      *
      * @return first dlsn in the stream
      */
-    public Future<DLSN> getFirstDLSNAsync();
+    Future<DLSN> getFirstDLSNAsync();
 
     /**
-     * Get Latest DLSN in the log - async
+     * Get Latest DLSN in the log - async.
      *
      * @return latest transaction id
      */
-    public Future<DLSN> getLastDLSNAsync();
+    Future<DLSN> getLastDLSNAsync();
 
     /**
-     * Get the number of log records in the active portion of the log
-     * Any log segments that have already been truncated will not be included
+     * Get the number of log records in the active portion of the log.
+     *
+     * <p>Any log segments that have already been truncated will not be included.
      *
      * @return number of log records
      * @throws IOException
      */
-    public long getLogRecordCount() throws IOException;
+    long getLogRecordCount() throws IOException;
 
     /**
      * Get the number of log records in the active portion of the log - async.
-     * Any log segments that have already been truncated will not be included
+     *
+     * <p>Any log segments that have already been truncated will not be included
      *
      * @return future number of log records
      * @throws IOException
      */
-    public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN);
+    Future<Long> getLogRecordCountAsync(final DLSN beginDLSN);
 
     /**
      * Run recovery on the log.
      *
      * @throws IOException
      */
-    public void recover() throws IOException;
+    void recover() throws IOException;
 
     /**
      * Check if an end of stream marker was added to the stream
-     * A stream with an end of stream marker cannot be appended to
+     * A stream with an end of stream marker cannot be appended to.
      *
      * @return true if the marker was added to the stream, false otherwise
      * @throws IOException
      */
-    public boolean isEndOfStreamMarked() throws IOException;
+    boolean isEndOfStreamMarked() throws IOException;
 
     /**
      * Delete the log.
      *
      * @throws IOException if the deletion fails
      */
-    public void delete() throws IOException;
+    void delete() throws IOException;
 
     /**
      * The DistributedLogManager may archive/purge any logs for transactionId
@@ -296,13 +296,22 @@
      * @param minTxIdToKeep the earliest txid that must be retained
      * @throws IOException if purging fails
      */
-    public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException;
+    void purgeLogsOlderThan(long minTxIdToKeep) throws IOException;
 
     /**
      * Get the subscriptions store provided by the distributedlog manager.
      *
      * @return subscriptions store manages subscriptions for current stream.
      */
-    public SubscriptionsStore getSubscriptionsStore();
+    SubscriptionsStore getSubscriptionsStore();
+
+    /**
+     * Closes this source and releases any system resources associated
+     * with it. If the source is already closed then invoking this
+     * method has no effect.
+     *
+     * @return future representing the close result.
+     */
+    Future<Void> asyncClose();
 
 }
diff --git a/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/DistributedLogManagerImpl.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/DistributedLogManagerImpl.java
new file mode 100644
index 0000000..aa3e94e
--- /dev/null
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/DistributedLogManagerImpl.java
@@ -0,0 +1,227 @@
+/*
+ * 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;
+
+import static org.apache.distributedlog.util.FutureUtils.newTFuture;
+
+import com.twitter.util.Future;
+import java.io.IOException;
+import java.util.List;
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.subscription.SubscriptionsStore;
+
+/**
+ * The wrapper of {@link org.apache.distributedlog.api.DistributedLogManager}.
+ */
+public class DistributedLogManagerImpl implements DistributedLogManager {
+
+    private final org.apache.distributedlog.api.DistributedLogManager impl;
+
+    public DistributedLogManagerImpl(org.apache.distributedlog.api.DistributedLogManager impl) {
+        this.impl = impl;
+    }
+
+    @Override
+    public String getStreamName() {
+        return impl.getStreamName();
+    }
+
+    @Override
+    public NamespaceDriver getNamespaceDriver() {
+        return impl.getNamespaceDriver();
+    }
+
+    @Override
+    public List<LogSegmentMetadata> getLogSegments() throws IOException {
+        return impl.getLogSegments();
+    }
+
+    @Override
+    public void registerListener(LogSegmentListener listener) throws IOException {
+        impl.registerListener(listener);
+    }
+
+    @Override
+    public void unregisterListener(LogSegmentListener listener) {
+        impl.unregisterListener(listener);
+    }
+
+    @Override
+    public Future<AsyncLogWriter> openAsyncLogWriter() {
+        return newTFuture(impl.openAsyncLogWriter()).map(AsyncLogWriterImpl.MAP_FUNC);
+    }
+
+    @Override
+    public LogWriter startLogSegmentNonPartitioned() throws IOException {
+        return new LogWriterImpl(impl.startLogSegmentNonPartitioned());
+    }
+
+    @Override
+    public AsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException {
+        return new AsyncLogWriterImpl(impl.startAsyncLogSegmentNonPartitioned());
+    }
+
+    @Override
+    public AppendOnlyStreamWriter getAppendOnlyStreamWriter() throws IOException {
+        return impl.getAppendOnlyStreamWriter();
+    }
+
+    @Override
+    public AppendOnlyStreamReader getAppendOnlyStreamReader() throws IOException {
+        return impl.getAppendOnlyStreamReader();
+    }
+
+    @Override
+    public LogReader getInputStream(long fromTxnId) throws IOException {
+        return new LogReaderImpl(impl.getInputStream(fromTxnId));
+    }
+
+    @Override
+    public LogReader getInputStream(DLSN fromDLSN) throws IOException {
+        return new LogReaderImpl(impl.getInputStream(fromDLSN));
+    }
+
+    @Override
+    public Future<AsyncLogReader> openAsyncLogReader(long fromTxnId) {
+        return newTFuture(impl.openAsyncLogReader(fromTxnId)).map(AsyncLogReaderImpl.MAP_FUNC);
+    }
+
+    @Override
+    public Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN) {
+        return newTFuture(impl.openAsyncLogReader(fromDLSN)).map(AsyncLogReaderImpl.MAP_FUNC);
+    }
+
+    @Override
+    public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException {
+        return new AsyncLogReaderImpl(impl.getAsyncLogReader(fromTxnId));
+    }
+
+    @Override
+    public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException {
+        return new AsyncLogReaderImpl(impl.getAsyncLogReader(fromDLSN));
+    }
+
+    @Override
+    public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN) {
+        return newTFuture(impl.getAsyncLogReaderWithLock(fromDLSN)).map(AsyncLogReaderImpl.MAP_FUNC);
+    }
+
+    @Override
+    public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN, String subscriberId) {
+        return newTFuture(impl.getAsyncLogReaderWithLock(fromDLSN, subscriberId))
+            .map(AsyncLogReaderImpl.MAP_FUNC);
+    }
+
+    @Override
+    public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId) {
+        return newTFuture(impl.getAsyncLogReaderWithLock(subscriberId))
+            .map(AsyncLogReaderImpl.MAP_FUNC);
+    }
+
+    @Override
+    public Future<DLSN> getDLSNNotLessThanTxId(long transactionId) {
+        return newTFuture(impl.getDLSNNotLessThanTxId(transactionId));
+    }
+
+    @Override
+    public LogRecordWithDLSN getLastLogRecord() throws IOException {
+        return impl.getLastLogRecord();
+    }
+
+    @Override
+    public long getFirstTxId() throws IOException {
+        return impl.getFirstTxId();
+    }
+
+    @Override
+    public long getLastTxId() throws IOException {
+        return impl.getLastTxId();
+    }
+
+    @Override
+    public DLSN getLastDLSN() throws IOException {
+        return impl.getLastDLSN();
+    }
+
+    @Override
+    public Future<LogRecordWithDLSN> getLastLogRecordAsync() {
+        return newTFuture(impl.getLastLogRecordAsync());
+    }
+
+    @Override
+    public Future<Long> getLastTxIdAsync() {
+        return newTFuture(impl.getLastTxIdAsync());
+    }
+
+    @Override
+    public Future<DLSN> getFirstDLSNAsync() {
+        return newTFuture(impl.getFirstDLSNAsync());
+    }
+
+    @Override
+    public Future<DLSN> getLastDLSNAsync() {
+        return newTFuture(impl.getLastDLSNAsync());
+    }
+
+    @Override
+    public long getLogRecordCount() throws IOException {
+        return impl.getLogRecordCount();
+    }
+
+    @Override
+    public Future<Long> getLogRecordCountAsync(DLSN beginDLSN) {
+        return newTFuture(impl.getLogRecordCountAsync(beginDLSN));
+    }
+
+    @Override
+    public void recover() throws IOException {
+        impl.recover();
+    }
+
+    @Override
+    public boolean isEndOfStreamMarked() throws IOException {
+        return impl.isEndOfStreamMarked();
+    }
+
+    @Override
+    public void delete() throws IOException {
+        impl.delete();
+    }
+
+    @Override
+    public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException {
+        impl.purgeLogsOlderThan(minTxIdToKeep);
+    }
+
+    @Override
+    public SubscriptionsStore getSubscriptionsStore() {
+        return new SubscriptionsStoreImpl(impl.getSubscriptionsStore());
+    }
+
+    @Override
+    public void close() throws IOException {
+        impl.close();
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        return newTFuture(impl.asyncClose());
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogReader.java
similarity index 79%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java
copy to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogReader.java
index baf3182..ce8b4f7 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogReader.java
@@ -17,6 +17,7 @@
  */
 package org.apache.distributedlog;
 
+import com.twitter.util.Future;
 import org.apache.distributedlog.io.AsyncCloseable;
 
 import java.io.Closeable;
@@ -28,21 +29,21 @@
  *
  * <h3>Lifecycle of a Reader</h3>
  *
- * A reader is a <i>sequential</i> reader that read records from a DL log starting
+ * <p>A reader is a <i>sequential</i> reader that read records from a DL log starting
  * from a given position. The position could be a <i>DLSN</i> (via {@link DistributedLogManager#getInputStream(DLSN)}
  * or a <i>Transaction ID</i> (via {@link DistributedLogManager#getInputStream(long)}.
- * <p>
- * After the reader is open, it could call {@link #readNext(boolean)} or {@link #readBulk(boolean, int)}
+ *
+ * <p>After the reader is open, it could call {@link #readNext(boolean)} or {@link #readBulk(boolean, int)}
  * to read records out the log from provided position.
- * <p>
- * Closing the reader (via {@link #close()} will release all the resources occupied
+ *
+ * <p>Closing the reader (via {@link #close()} will release all the resources occupied
  * by this reader instance.
- * <p>
- * Exceptions could be thrown during reading records. Once the exception is thrown,
+ *
+ * <p>Exceptions could be thrown during reading records. Once the exception is thrown,
  * the reader is set to an error state and it isn't usable anymore. It is the application's
  * responsibility to handle the exceptions and re-create readers if necessary.
- * <p>
- * Example:
+ *
+ * <p>Example:
  * <pre>
  * DistributedLogManager dlm = ...;
  * long nextTxId = ...;
@@ -66,7 +67,7 @@
  *
  * <h3>Read Records</h3>
  *
- * Reading records from an <i>endless</i> log in `synchronous` way isn't as
+ * <p>Reading records from an <i>endless</i> log in `synchronous` way isn't as
  * trivial as in `asynchronous` way (via {@link AsyncLogReader}. Because it
  * lacks of callback mechanism. LogReader introduces a flag `nonBlocking` on
  * controlling the <i>waiting</i> behavior on `synchronous` reads.
@@ -77,8 +78,8 @@
  * before returning read calls. While <i>NonBlocking</i> (nonBlocking = true)
  * means the reads will only check readahead cache and return whatever records
  * available in the readahead cache.
- * <p>
- * The <i>waiting</i> period varies in <i>blocking</i> mode. If the reader is
+ *
+ * <p>The <i>waiting</i> period varies in <i>blocking</i> mode. If the reader is
  * catching up with writer (there are records in the log), the read call will
  * wait until records are read and returned. If the reader is caught up with
  * writer (there are no more records in the log at read time), the read call
@@ -87,14 +88,14 @@
  * records available in the readahead cache. In other words, if a reader sees
  * no record on blocking reads, it means the reader is `caught-up` with the
  * writer.
- * <p>
- * <i>Blocking</i> and <i>NonBlocking</i> modes are useful for building replicated
+ *
+ * <p><i>Blocking</i> and <i>NonBlocking</i> modes are useful for building replicated
  * state machines. Applications could use <i>blocking</i> reads till caught up
  * with latest data. Once they are caught up with latest data, they could start
  * serving their service and turn to <i>non-blocking</i> read mode and tail read
  * data from the logs.
- * <p>
- * See examples below.
+ *
+ * <p>See examples below.
  *
  * <h4>Read Single Record</h4>
  *
@@ -132,7 +133,7 @@
  * int N = 10;
  *
  * // keep reading N records in blocking way until no records available in the log
- * List<LogRecord> records = reader.readBulk(false, N);
+ * List&lt;LogRecord&gt; records = reader.readBulk(false, N);
  * while (!records.isEmpty()) {
  *     // process the list of records
  *     ...
@@ -154,22 +155,21 @@
  *
  * </pre>
  *
- * <p>
- * NOTE: Extending {@link AsyncCloseable}: BKSyncLogReader is implemented based on BKAsyncLogReader, exposing
+ * <p>NOTE: Extending {@link AsyncCloseable}: BKSyncLogReader is implemented based on BKAsyncLogReader, exposing
  * the {@link AsyncCloseable} interface so the reader could be closed asynchronously
  *
  * @see AsyncLogReader
  */
-public interface LogReader extends Closeable, AsyncCloseable {
+public interface LogReader extends Closeable {
 
     /**
      * Read the next log record from the stream.
-     * <p>
-     * If <i>nonBlocking</i> is set to true, the call returns immediately by just polling
+     *
+     * <p>If <i>nonBlocking</i> is set to true, the call returns immediately by just polling
      * records from read ahead cache. It would return <i>null</i> if there isn't any records
      * available in the read ahead cache.
-     * <p>
-     * If <i>nonBlocking</i> is set to false, it would does blocking call. The call will
+     *
+     * <p>If <i>nonBlocking</i> is set to false, it would does blocking call. The call will
      * block until return a record if there are records in the stream (aka catching up).
      * Otherwise it would wait up to {@link DistributedLogConfiguration#getReadAheadWaitTime()}
      * milliseconds and return null if there isn't any more records in the stream.
@@ -179,10 +179,10 @@
      * @return an operation from the stream or null if at end of stream
      * @throws IOException if there is an error reading from the stream
      */
-    public LogRecordWithDLSN readNext(boolean nonBlocking) throws IOException;
+    LogRecordWithDLSN readNext(boolean nonBlocking) throws IOException;
 
     /**
-     * Read the next <i>numLogRecords</i> log records from the stream
+     * Read the next <i>numLogRecords</i> log records from the stream.
      *
      * @param nonBlocking should the read make blocking calls to the backend or rely on the
      * readAhead cache
@@ -191,5 +191,14 @@
      * @throws IOException if there is an error reading from the stream
      * @see #readNext(boolean)
      */
-    public List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords) throws IOException;
+    List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords) throws IOException;
+
+    /**
+     * Closes this source and releases any system resources associated
+     * with it. If the source is already closed then invoking this
+     * method has no effect.
+     *
+     * @return future representing the close result.
+     */
+    Future<Void> asyncClose();
 }
diff --git a/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogReaderImpl.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogReaderImpl.java
new file mode 100644
index 0000000..2a7031e
--- /dev/null
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogReaderImpl.java
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import static org.apache.distributedlog.util.FutureUtils.newTFuture;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.twitter.util.Future;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The wrapper over {@link org.apache.distributedlog.api.LogReader}.
+ */
+class LogReaderImpl implements LogReader {
+
+    private final org.apache.distributedlog.api.LogReader reader;
+
+    LogReaderImpl(org.apache.distributedlog.api.LogReader reader) {
+        this.reader = reader;
+    }
+
+    @VisibleForTesting
+    org.apache.distributedlog.api.LogReader getImpl() {
+        return reader;
+    }
+
+    @Override
+    public LogRecordWithDLSN readNext(boolean nonBlocking) throws IOException {
+        return reader.readNext(nonBlocking);
+    }
+
+    @Override
+    public List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords) throws IOException {
+        return reader.readBulk(nonBlocking, numLogRecords);
+    }
+
+    @Override
+    public void close() throws IOException {
+        reader.close();
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        return newTFuture(reader.asyncClose());
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogWriter.java
similarity index 82%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java
copy to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogWriter.java
index 8a4a30b..d965158 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogWriter.java
@@ -23,10 +23,10 @@
 import java.io.IOException;
 import java.util.List;
 
-/*
-* A generic interface class to support writing log records into
-* a persistent distributed log.
-*/
+/**
+ * A generic interface class to support writing log records into
+ * a persistent distributed log.
+ */
 public interface LogWriter extends Closeable, Abortable {
     /**
      * Write a log record to the stream.
@@ -34,8 +34,7 @@
      * @param record single log record
      * @throws IOException
      */
-    public void write(LogRecord record) throws IOException;
-
+    void write(LogRecord record) throws IOException;
 
     /**
      * Write a list of log records to the stream.
@@ -44,7 +43,7 @@
      * @throws IOException
      */
     @Deprecated
-    public int writeBulk(List<LogRecord> records) throws IOException;
+    int writeBulk(List<LogRecord> records) throws IOException;
 
     /**
      * All data that has been written to the stream so far will be sent to
@@ -52,9 +51,9 @@
      * The transmission is asynchronous and new data can be still written to the
      * stream while flushing is performed.
      *
-     * TODO: rename this to flush()
+     * <p>TODO: rename this to flush()
      */
-    public long setReadyToFlush() throws IOException;
+    long setReadyToFlush() throws IOException;
 
     /**
      * Flush and sync all data that is ready to be flush
@@ -63,7 +62,7 @@
      *
      * TODO: rename this to commit()
      */
-    public long flushAndSync() throws IOException;
+    long flushAndSync() throws IOException;
 
     /**
      * Flushes all the data up to this point,
@@ -73,6 +72,6 @@
      *
      * @throws IOException
      */
-    public void markEndOfStream() throws IOException;
+    void markEndOfStream() throws IOException;
 
 }
diff --git a/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogWriterImpl.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogWriterImpl.java
new file mode 100644
index 0000000..532b3e5
--- /dev/null
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/LogWriterImpl.java
@@ -0,0 +1,75 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The wrapper of {@link org.apache.distributedlog.api.LogWriter}.
+ */
+class LogWriterImpl implements LogWriter {
+
+    private final org.apache.distributedlog.api.LogWriter impl;
+
+    LogWriterImpl(org.apache.distributedlog.api.LogWriter impl) {
+        this.impl = impl;
+    }
+
+    @VisibleForTesting
+    org.apache.distributedlog.api.LogWriter getImpl() {
+        return impl;
+    }
+
+    @Override
+    public void write(LogRecord record) throws IOException {
+        impl.write(record);
+    }
+
+    @Override
+    public int writeBulk(List<LogRecord> records) throws IOException {
+        return impl.writeBulk(records);
+    }
+
+    @Override
+    public long setReadyToFlush() throws IOException {
+        return impl.setReadyToFlush();
+    }
+
+    @Override
+    public long flushAndSync() throws IOException {
+        return impl.flushAndSync();
+    }
+
+    @Override
+    public void markEndOfStream() throws IOException {
+        impl.markEndOfStream();
+    }
+
+    @Override
+    public void close() throws IOException {
+        impl.close();
+    }
+
+    @Override
+    public void abort() throws IOException {
+        impl.abort();
+    }
+}
diff --git a/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/SubscriptionsStoreImpl.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/SubscriptionsStoreImpl.java
new file mode 100644
index 0000000..5439976
--- /dev/null
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/SubscriptionsStoreImpl.java
@@ -0,0 +1,67 @@
+/*
+ * 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;
+
+import static org.apache.distributedlog.util.FutureUtils.newTFuture;
+
+import com.twitter.util.Future;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.distributedlog.subscription.SubscriptionsStore;
+
+/**
+ * A wrapper over {@link org.apache.distributedlog.api.subscription.SubscriptionsStore}.
+ */
+class SubscriptionsStoreImpl implements SubscriptionsStore {
+
+    private final org.apache.distributedlog.api.subscription.SubscriptionsStore impl;
+
+    SubscriptionsStoreImpl(org.apache.distributedlog.api.subscription.SubscriptionsStore impl) {
+        this.impl = impl;
+    }
+
+    org.apache.distributedlog.api.subscription.SubscriptionsStore getImpl() {
+        return impl;
+    }
+
+    @Override
+    public Future<DLSN> getLastCommitPosition(String subscriberId) {
+        return newTFuture(impl.getLastCommitPosition(subscriberId));
+    }
+
+    @Override
+    public Future<Map<String, DLSN>> getLastCommitPositions() {
+        return newTFuture(impl.getLastCommitPositions());
+    }
+
+    @Override
+    public Future<Void> advanceCommitPosition(String subscriberId, DLSN newPosition) {
+        return newTFuture(impl.advanceCommitPosition(subscriberId, newPosition));
+    }
+
+    @Override
+    public Future<Boolean> deleteSubscriber(String subscriberId) {
+        return newTFuture(impl.deleteSubscriber(subscriberId));
+    }
+
+    @Override
+    public void close() throws IOException {
+        impl.close();
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java
similarity index 94%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java
rename to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java
index 4cbee98..3bbfd95 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java
@@ -49,17 +49,17 @@
  *
  * <h4>Namespace Location</h4>
  *
- * At the highest level, a <code>DistributedLogNamespace</code> is located by a <code>URI</code>. The location
+ * <p>At the highest level, a <code>DistributedLogNamespace</code> is located by a <code>URI</code>. The location
  * URI is in string form has the syntax
  *
  * <blockquote>
  * distributedlog[<tt><b>-</b></tt><i>provider</i>]<tt><b>:</b></tt><i>provider-specific-path</i>
  * </blockquote>
  *
- * where square brackets [...] delineate optional components and the characters <tt><b>-</b></tt> and <tt><b>:</b></tt>
- * stand for themselves.
+ * <p>where square brackets [...] delineate optional components and the characters <tt><b>-</b></tt> and
+ * <tt><b>:</b></tt> stand for themselves.
  *
- * The <code>provider</code> part in the URI indicates what is the backend used for this namespace. For example:
+ * <p>The <code>provider</code> part in the URI indicates what is the backend used for this namespace. For example:
  * <i>distributedlog-bk</i> URI is storing logs in bookkeeper, while <i>distributedlog-mem</i> URI is storing logs in
  * memory. The <code>provider</code> part is optional. It would use bookkeeper backend if the <i>provider</i> part
  * is omitted.
diff --git a/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceBuilder.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceBuilder.java
new file mode 100644
index 0000000..e646b19
--- /dev/null
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceBuilder.java
@@ -0,0 +1,177 @@
+/**
+ * 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.namespace;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+import java.io.IOException;
+import java.net.URI;
+
+/**
+ * Builder to construct a <code>DistributedLogNamespace</code>.
+ * The builder takes the responsibility of loading backend according to the uri.
+ *
+ * @see DistributedLogNamespace
+ * @since 0.3.32
+ */
+public class DistributedLogNamespaceBuilder {
+
+    public static DistributedLogNamespaceBuilder newBuilder() {
+        return new DistributedLogNamespaceBuilder();
+    }
+
+    private final NamespaceBuilder builder;
+
+    // private constructor
+    private DistributedLogNamespaceBuilder() {
+        this(NamespaceBuilder.newBuilder());
+    }
+
+    @VisibleForTesting
+    DistributedLogNamespaceBuilder(NamespaceBuilder builder) {
+        this.builder = builder;
+    }
+
+    /**
+     * DistributedLog Configuration used for the namespace.
+     *
+     * @param conf
+     *          distributedlog configuration
+     * @return namespace builder.
+     */
+    public DistributedLogNamespaceBuilder conf(DistributedLogConfiguration conf) {
+        this.builder.conf(conf);
+        return this;
+    }
+
+    /**
+     * Dynamic DistributedLog Configuration used for the namespace.
+     *
+     * @param dynConf dynamic distributedlog configuration
+     * @return namespace builder
+     */
+    public DistributedLogNamespaceBuilder dynConf(DynamicDistributedLogConfiguration dynConf) {
+        this.builder.dynConf(dynConf);
+        return this;
+    }
+
+    /**
+     * Namespace Location.
+     *
+     * @param uri
+     *          namespace location uri.
+     * @see DistributedLogNamespace
+     * @return namespace builder.
+     */
+    public DistributedLogNamespaceBuilder uri(URI uri) {
+        this.builder.uri(uri);
+        return this;
+    }
+
+    /**
+     * Stats Logger used for stats collection.
+     *
+     * @param statsLogger
+     *          stats logger
+     * @return namespace builder.
+     */
+    public DistributedLogNamespaceBuilder statsLogger(StatsLogger statsLogger) {
+        this.builder.statsLogger(statsLogger);
+        return this;
+    }
+
+    /**
+     * Stats Logger used for collecting per log stats.
+     *
+     * @param statsLogger
+     *          stats logger for collecting per log stats
+     * @return namespace builder.
+     */
+    public DistributedLogNamespaceBuilder perLogStatsLogger(StatsLogger statsLogger) {
+        this.builder.perLogStatsLogger(statsLogger);
+        return this;
+    }
+
+    /**
+     * Feature provider used to control the availabilities of features in the namespace.
+     *
+     * @param featureProvider
+     *          feature provider to control availabilities of features.
+     * @return namespace builder.
+     */
+    public DistributedLogNamespaceBuilder featureProvider(FeatureProvider featureProvider) {
+        this.builder.featureProvider(featureProvider);
+        return this;
+    }
+
+    /**
+     * Client Id used for accessing the namespace.
+     *
+     * @param clientId
+     *          client id used for accessing the namespace
+     * @return namespace builder.
+     */
+    public DistributedLogNamespaceBuilder clientId(String clientId) {
+        this.builder.clientId(clientId);
+        return this;
+    }
+
+    /**
+     * Region Id used for encoding logs in the namespace. The region id
+     * is useful when the namespace is globally spanning over regions.
+     *
+     * @param regionId
+     *          region id.
+     * @return namespace builder.
+     */
+    public DistributedLogNamespaceBuilder regionId(int regionId) {
+        this.builder.regionId(regionId);
+        return this;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static StatsLogger normalizePerLogStatsLogger(StatsLogger statsLogger,
+                                                          StatsLogger perLogStatsLogger,
+                                                          DistributedLogConfiguration conf) {
+        StatsLogger normalizedPerLogStatsLogger = perLogStatsLogger;
+        if (perLogStatsLogger == NullStatsLogger.INSTANCE
+            && conf.getEnablePerStreamStat()) {
+            normalizedPerLogStatsLogger = statsLogger.scope("stream");
+        }
+        return normalizedPerLogStatsLogger;
+    }
+
+    /**
+     * Build the namespace.
+     *
+     * @return the namespace instance.
+     * @throws IllegalArgumentException when there is illegal argument provided in the builder
+     * @throws NullPointerException when there is null argument provided in the builder
+     * @throws IOException when fail to build the backend
+     */
+    public DistributedLogNamespace build()
+            throws IllegalArgumentException, NullPointerException, IOException {
+        return new DistributedLogNamespaceImpl(this.builder.build());
+    }
+}
diff --git a/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceImpl.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceImpl.java
new file mode 100644
index 0000000..a528d62
--- /dev/null
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceImpl.java
@@ -0,0 +1,101 @@
+/*
+ * 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.namespace;
+
+import com.google.common.base.Optional;
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.DistributedLogManagerImpl;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+
+/**
+ * A wapper over {@link org.apache.distributedlog.api.namespace.Namespace}.
+ */
+class DistributedLogNamespaceImpl implements DistributedLogNamespace {
+
+    private final Namespace impl;
+
+    DistributedLogNamespaceImpl(Namespace impl) {
+        this.impl = impl;
+    }
+
+
+    @Override
+    public NamespaceDriver getNamespaceDriver() {
+        return impl.getNamespaceDriver();
+    }
+
+    @Override
+    public void createLog(String logName) throws InvalidStreamNameException, IOException {
+        impl.createLog(logName);
+    }
+
+    @Override
+    public void deleteLog(String logName) throws InvalidStreamNameException, LogNotFoundException, IOException {
+        impl.deleteLog(logName);
+    }
+
+    @Override
+    public DistributedLogManager openLog(String logName) throws InvalidStreamNameException, IOException {
+        return new DistributedLogManagerImpl(impl.openLog(logName));
+    }
+
+    @Override
+    public DistributedLogManager openLog(String logName,
+                                         Optional<DistributedLogConfiguration> logConf,
+                                         Optional<DynamicDistributedLogConfiguration> dynamicLogConf,
+                                         Optional<StatsLogger> perStreamStatsLogger)
+            throws InvalidStreamNameException, IOException {
+        return new DistributedLogManagerImpl(impl.openLog(
+            logName, logConf, dynamicLogConf, perStreamStatsLogger));
+    }
+
+    @Override
+    public boolean logExists(String logName) throws IOException {
+        return impl.logExists(logName);
+    }
+
+    @Override
+    public Iterator<String> getLogs() throws IOException {
+        return impl.getLogs();
+    }
+
+    @Override
+    public void registerNamespaceListener(NamespaceListener listener) {
+        impl.registerNamespaceListener(listener);
+    }
+
+    @Override
+    public AccessControlManager createAccessControlManager() throws IOException {
+        return impl.createAccessControlManager();
+    }
+
+    @Override
+    public void close() {
+        impl.close();
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/package-info.java
similarity index 88%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
rename to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/package-info.java
index d81f8a4..2febeda 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/namespace/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- * ReadAhead Mechanism for distributedlog streaming reads
+ * {@link com.twitter.util.Future} based namespace API.
  */
-package org.apache.distributedlog.readahead;
+package org.apache.distributedlog.namespace;
\ No newline at end of file
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/package-info.java
similarity index 90%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
copy to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/package-info.java
index eb81cfe..ef26d51 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- * IO Utils for distributedlog
+ * {@link com.twitter.util.Future} based API.
  */
-package org.apache.distributedlog.io;
+package org.apache.distributedlog;
\ No newline at end of file
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java
similarity index 95%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java
rename to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java
index 43641f0..a9c7b21 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,13 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.distributedlog.stats;
 
 import com.google.common.base.Stopwatch;
 import com.twitter.util.FutureEventListener;
-import org.apache.bookkeeper.stats.OpStatsLogger;
 import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.OpStatsLogger;
 
+/**
+ * A {@link FutureEventListener} monitors the stats for a given operation.
+ */
 public class OpStatsListener<T> implements FutureEventListener<T> {
     OpStatsLogger opStatsLogger;
     Stopwatch stopwatch;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/stats/package-info.java
similarity index 88%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
copy to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/stats/package-info.java
index d81f8a4..e352591 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/stats/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- * ReadAhead Mechanism for distributedlog streaming reads
+ * {@link com.twitter.util.Future} based stats utils.
  */
-package org.apache.distributedlog.readahead;
+package org.apache.distributedlog.stats;
\ No newline at end of file
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionsStore.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/subscription/SubscriptionsStore.java
similarity index 85%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionsStore.java
rename to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/subscription/SubscriptionsStore.java
index 1974f1e..cb4ec55 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionsStore.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/subscription/SubscriptionsStore.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,15 +17,13 @@
  */
 package org.apache.distributedlog.subscription;
 
-import org.apache.distributedlog.DLSN;
 import com.twitter.util.Future;
-import scala.runtime.BoxedUnit;
-
 import java.io.Closeable;
 import java.util.Map;
+import org.apache.distributedlog.DLSN;
 
 /**
- * Store to manage subscriptions
+ * Store to manage subscriptions.
  */
 public interface SubscriptionsStore extends Closeable {
 
@@ -36,14 +34,14 @@
      *          subscriber id
      * @return future representing last committed position.
      */
-    public Future<DLSN> getLastCommitPosition(String subscriberId);
+    Future<DLSN> getLastCommitPosition(String subscriberId);
 
     /**
      * Get the last committed positions for all subscribers.
      *
      * @return future representing last committed positions for all subscribers.
      */
-    public Future<Map<String, DLSN>> getLastCommitPositions();
+    Future<Map<String, DLSN>> getLastCommitPositions();
 
     /**
      * Advance the last committed position for <i>subscriberId</i>.
@@ -54,7 +52,7 @@
      *          new committed position.
      * @return future representing advancing result.
      */
-    public Future<BoxedUnit> advanceCommitPosition(String subscriberId, DLSN newPosition);
+    Future<Void> advanceCommitPosition(String subscriberId, DLSN newPosition);
 
     /**
      * Delete the subscriber <i>subscriberId</i> permanently. Once the subscriber is deleted, all the
@@ -64,6 +62,6 @@
      * return true only if there's such subscriber and we removed it successfully.
      * return false if there's no such subscriber, or we failed to remove.
      */
-    public Future<Boolean> deleteSubscriber(String subscriberId);
+    Future<Boolean> deleteSubscriber(String subscriberId);
 
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/subscription/package-info.java
similarity index 87%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
copy to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/subscription/package-info.java
index d81f8a4..032a2ba 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/subscription/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- * ReadAhead Mechanism for distributedlog streaming reads
+ * {@link com.twitter.util.Future} based subscription API.
  */
-package org.apache.distributedlog.readahead;
+package org.apache.distributedlog.subscription;
\ No newline at end of file
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/FutureUtils.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/util/FutureUtils.java
similarity index 89%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/util/FutureUtils.java
rename to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/util/FutureUtils.java
index 8e4a8be..d121529 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/FutureUtils.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/util/FutureUtils.java
@@ -18,14 +18,7 @@
 package org.apache.distributedlog.util;
 
 import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.exceptions.BKTransmitException;
-import org.apache.distributedlog.exceptions.LockingException;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.stats.OpStatsListener;
+import com.google.common.collect.Lists;
 import com.twitter.util.Await;
 import com.twitter.util.Duration;
 import com.twitter.util.Function;
@@ -36,14 +29,8 @@
 import com.twitter.util.Return;
 import com.twitter.util.Throw;
 import com.twitter.util.Try;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
 import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -51,14 +38,32 @@
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.distributedlog.stats.OpStatsListener;
+import org.apache.zookeeper.KeeperException;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
 
 /**
- * Utilities to process future
+ * Utilities to process future.
  */
 public class FutureUtils {
 
     private static final Logger logger = LoggerFactory.getLogger(FutureUtils.class);
 
+    /**
+     * A future listener that is supposed to run in ordered scheduler.
+     */
     public static class OrderedFutureEventListener<R>
             implements FutureEventListener<R> {
 
@@ -102,6 +107,9 @@
         }
     }
 
+    /**
+     * A future listener is running a specific executor.
+     */
     public static class FutureEventListenerRunnable<R>
             implements FutureEventListener<R> {
 
@@ -275,7 +283,7 @@
      */
     public static int bkResultCode(Throwable throwable) {
         if (throwable instanceof BKException) {
-            return ((BKException)throwable).getCode();
+            return ((BKException) throwable).getCode();
         }
         return BKException.Code.UnexpectedConditionException;
     }
@@ -293,6 +301,7 @@
 
     /**
      * Wait for the result for a given <i>duration</i>.
+     *
      * <p>If the result is not ready within `duration`, an IOException will thrown wrapping with
      * corresponding {@link com.twitter.util.TimeoutException}.
      *
@@ -410,6 +419,7 @@
 
     /**
      * Satisfy the <i>promise</i> with provide value in an ordered scheduler.
+     *
      * <p>If the promise was already satisfied, nothing will be changed.
      *
      * @param promise promise to satisfy
@@ -431,6 +441,7 @@
 
     /**
      * Satisfy the <i>promise</i> with provide value.
+     *
      * <p>If the promise was already satisfied, nothing will be changed.
      *
      * @param promise promise to satisfy
@@ -450,7 +461,7 @@
      * Satisfy the <i>promise</i> with provided <i>cause</i> in an ordered scheduler.
      *
      * @param promise promise to satisfy
-     * @param throwable cause to satisfy
+     * @param cause cause to satisfy
      * @param scheduler the scheduler to satisfy the promise
      * @param key submit key of the ordered scheduler
      */
@@ -493,7 +504,7 @@
     }
 
     /**
-     * Ignore exception from the <i>future</i> and log <i>errorMsg</i> on exceptions
+     * 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
@@ -531,4 +542,55 @@
             + BKException.getMessage(transmitResult), transmitResult);
     }
 
+    public static <T> CompletableFuture<T> newJFuture(Promise<T> promise) {
+        CompletableFuture<T> jFuture = org.apache.distributedlog.common.concurrent.FutureUtils.createFuture();
+        jFuture.whenComplete((value, cause) -> {
+            if (null != cause) {
+                if (cause instanceof CompletionException) {
+                    promise.setException(cause.getCause());
+                } else {
+                    promise.setException(cause);
+                }
+            } else {
+                promise.setValue(value);
+            }
+        });
+        return jFuture;
+    }
+
+    public static <T> Future<T> newTFuture(CompletableFuture<T> jFuture) {
+        Promise<T> promise = new Promise<>();
+        jFuture.whenComplete((value, cause) -> {
+            if (null != cause) {
+                if (cause instanceof CompletionException) {
+                    promise.setException(cause.getCause());
+                } else {
+                    promise.setException(cause);
+                }
+            } else {
+                promise.setValue(value);
+            }
+        });
+        return promise;
+    }
+
+    public static <T> Future<List<Future<T>>> newTFutureList(
+            CompletableFuture<List<CompletableFuture<T>>> jFutureList) {
+        Promise<List<Future<T>>> promise = new Promise<>();
+        jFutureList.whenComplete((value, cause) -> {
+            if (null != cause) {
+                if (cause instanceof CompletionException) {
+                    promise.setException(cause.getCause());
+                } else {
+                    promise.setException(cause);
+                }
+            } else {
+                promise.setValue(Lists.transform(
+                    value,
+                    future -> newTFuture(future)));
+            }
+        });
+        return promise;
+    }
+
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/util/package-info.java
similarity index 89%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
copy to distributedlog-core-twitter/src/main/java/org/apache/distributedlog/util/package-info.java
index eb81cfe..ee00974 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
+++ b/distributedlog-core-twitter/src/main/java/org/apache/distributedlog/util/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- * IO Utils for distributedlog
+ * {@link com.twitter.util.Future} related utils.
  */
-package org.apache.distributedlog.io;
+package org.apache.distributedlog.util;
\ No newline at end of file
diff --git a/distributedlog-core-twitter/src/main/resources/findbugsExclude.xml b/distributedlog-core-twitter/src/main/resources/findbugsExclude.xml
new file mode 100644
index 0000000..1afb903
--- /dev/null
+++ b/distributedlog-core-twitter/src/main/resources/findbugsExclude.xml
@@ -0,0 +1,19 @@
+<!--
+    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.
+//-->
+<FindBugsFilter>
+</FindBugsFilter>
diff --git a/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestAsyncLogReaderImpl.java b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestAsyncLogReaderImpl.java
new file mode 100644
index 0000000..dfff56a
--- /dev/null
+++ b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestAsyncLogReaderImpl.java
@@ -0,0 +1,91 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.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 java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.distributedlog.util.FutureUtils;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link AsyncLogReaderImpl}.
+ */
+public class TestAsyncLogReaderImpl {
+
+    private final org.apache.distributedlog.api.AsyncLogReader underlying =
+        mock(org.apache.distributedlog.api.AsyncLogReader.class);
+    private final AsyncLogReaderImpl reader = new AsyncLogReaderImpl(underlying);
+
+    @Test
+    public void testRead() throws Exception {
+        LogRecordWithDLSN record = mock(LogRecordWithDLSN.class);
+        when(underlying.readNext())
+            .thenReturn(CompletableFuture.completedFuture(record));
+        assertEquals(record, FutureUtils.result(reader.readNext()));
+        verify(underlying, times(1)).readNext();
+    }
+
+    @Test
+    public void testReadBulk() throws Exception {
+        List<LogRecordWithDLSN> records = mock(List.class);
+        when(underlying.readBulk(anyInt()))
+            .thenReturn(CompletableFuture.completedFuture(records));
+        assertEquals(records, FutureUtils.result(reader.readBulk(100)));
+        verify(underlying, times(1)).readBulk(eq(100));
+    }
+
+    @Test
+    public void testReadBulkWithWaitTime() throws Exception {
+        List<LogRecordWithDLSN> records = mock(List.class);
+        when(underlying.readBulk(anyInt(), anyLong(), any(TimeUnit.class)))
+            .thenReturn(CompletableFuture.completedFuture(records));
+        assertEquals(records, FutureUtils.result(reader.readBulk(100, 10, TimeUnit.MICROSECONDS)));
+        verify(underlying, times(1))
+            .readBulk(eq(100), eq(10L), eq(TimeUnit.MICROSECONDS));
+    }
+
+    @Test
+    public void testGetStreamName() throws Exception {
+        String streamName = "test-stream-name";
+        when(underlying.getStreamName())
+            .thenReturn(streamName);
+        assertEquals(streamName, reader.getStreamName());
+        verify(underlying, times(1)).getStreamName();
+    }
+
+    @Test
+    public void testAsyncClose() throws Exception {
+        when(underlying.asyncClose())
+            .thenReturn(CompletableFuture.completedFuture(null));
+        FutureUtils.result(reader.asyncClose());
+        verify(underlying, times(1)).asyncClose();
+    }
+
+}
diff --git a/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestAsyncLogWriterImpl.java b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestAsyncLogWriterImpl.java
new file mode 100644
index 0000000..621e18d
--- /dev/null
+++ b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestAsyncLogWriterImpl.java
@@ -0,0 +1,117 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.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.collect.Lists;
+import com.twitter.util.Futures;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.util.FutureUtils;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link AsyncLogWriterImpl}.
+ */
+public class TestAsyncLogWriterImpl {
+
+    private final org.apache.distributedlog.api.AsyncLogWriter underlying =
+        mock(org.apache.distributedlog.api.AsyncLogWriter.class);
+    private final AsyncLogWriterImpl writer = new AsyncLogWriterImpl(underlying);
+
+    @Test
+    public void testWrite() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        LogRecord record = mock(LogRecord.class);
+        when(underlying.write(any(LogRecord.class)))
+            .thenReturn(CompletableFuture.completedFuture(dlsn));
+        assertEquals(dlsn, FutureUtils.result(writer.write(record)));
+        verify(underlying, times(1)).write(eq(record));
+    }
+
+    @Test
+    public void testWriteBulk() throws Exception {
+        List<LogRecord> records = mock(List.class);
+        List<CompletableFuture<DLSN>> futures = Lists.newArrayList();
+        List<DLSN> dlsns = Lists.newArrayList();
+        for (int i = 0; i < 10; i++) {
+            DLSN dlsn = mock(DLSN.class);
+            dlsns.add(dlsn);
+            futures.add(CompletableFuture.completedFuture(dlsn));
+        }
+        when(underlying.writeBulk(any(List.class)))
+            .thenReturn(CompletableFuture.completedFuture(futures));
+        assertEquals(
+            dlsns,
+            FutureUtils.result(Futures.collect(
+                FutureUtils.result(writer.writeBulk(records)))));
+        verify(underlying, times(1)).writeBulk(eq(records));
+    }
+
+    @Test
+    public void testGetLastTxId() throws Exception {
+        long txId = 123456L;
+        when(underlying.getLastTxId()).thenReturn(txId);
+        assertEquals(txId, writer.getLastTxId());
+        verify(underlying, times(1)).getLastTxId();
+    }
+
+    @Test
+    public void testTruncate() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        when(underlying.truncate(dlsn))
+            .thenReturn(CompletableFuture.completedFuture(true));
+        assertTrue(FutureUtils.result(writer.truncate(dlsn)));
+        verify(underlying, times(1)).truncate(eq(dlsn));
+    }
+
+    @Test
+    public void testGetStreamName() throws Exception {
+        String streamName = "test-stream-name";
+        when(underlying.getStreamName())
+            .thenReturn(streamName);
+        assertEquals(streamName, writer.getStreamName());
+        verify(underlying, times(1)).getStreamName();
+    }
+
+    @Test
+    public void testAsyncClose() throws Exception {
+        when(underlying.asyncClose())
+            .thenReturn(CompletableFuture.completedFuture(null));
+        FutureUtils.result(writer.asyncClose());
+        verify(underlying, times(1)).asyncClose();
+    }
+
+    @Test
+    public void testAsyncAbort() throws Exception {
+        when(underlying.asyncAbort())
+            .thenReturn(CompletableFuture.completedFuture(null));
+        FutureUtils.result(writer.asyncAbort());
+        verify(underlying, times(1)).asyncAbort();
+    }
+
+}
diff --git a/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestDistributedLogManagerImpl.java b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestDistributedLogManagerImpl.java
new file mode 100644
index 0000000..bed516d
--- /dev/null
+++ b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestDistributedLogManagerImpl.java
@@ -0,0 +1,351 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.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 java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.api.LogWriter;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.util.FutureUtils;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link DistributedLogManagerImpl}.
+ */
+public class TestDistributedLogManagerImpl {
+
+    private final org.apache.distributedlog.api.DistributedLogManager impl =
+        mock(org.apache.distributedlog.api.DistributedLogManager.class);
+    private final DistributedLogManagerImpl manager = new DistributedLogManagerImpl(impl);
+
+    @Test
+    public void testGetStreamName() throws Exception {
+        String name = "test-get-stream-name";
+        when(impl.getStreamName()).thenReturn(name);
+        assertEquals(name, manager.getStreamName());
+        verify(impl, times(1)).getStreamName();
+    }
+
+    @Test
+    public void testGetNamespaceDriver() throws Exception {
+        NamespaceDriver driver = mock(NamespaceDriver.class);
+        when(impl.getNamespaceDriver()).thenReturn(driver);
+        assertEquals(driver, manager.getNamespaceDriver());
+        verify(impl, times(1)).getNamespaceDriver();
+    }
+
+    @Test
+    public void testGetLogSegments() throws Exception {
+        List<LogSegmentMetadata> segments = mock(List.class);
+        when(impl.getLogSegments()).thenReturn(segments);
+        assertEquals(segments, manager.getLogSegments());
+        verify(impl, times(1)).getLogSegments();
+    }
+
+    @Test
+    public void testRegisterListener() throws Exception {
+        LogSegmentListener listener = mock(LogSegmentListener.class);
+        manager.registerListener(listener);
+        verify(impl, times(1)).registerListener(listener);
+    }
+
+    @Test
+    public void testUnregisterListener() throws Exception {
+        LogSegmentListener listener = mock(LogSegmentListener.class);
+        manager.unregisterListener(listener);
+        verify(impl, times(1)).unregisterListener(listener);
+    }
+
+    @Test
+    public void testOpenAsyncLogWriter() throws Exception {
+        AsyncLogWriter writer = mock(AsyncLogWriter.class);
+        when(impl.openAsyncLogWriter()).thenReturn(CompletableFuture.completedFuture(writer));
+        assertEquals(writer, ((AsyncLogWriterImpl) FutureUtils.result(manager.openAsyncLogWriter())).getImpl());
+        verify(impl, times(1)).openAsyncLogWriter();
+    }
+
+    @Test
+    public void testStartLogSegmentNonPartitioned() throws Exception {
+        LogWriter writer = mock(LogWriter.class);
+        when(impl.startLogSegmentNonPartitioned()).thenReturn(writer);
+        assertEquals(writer, ((LogWriterImpl) manager.startLogSegmentNonPartitioned()).getImpl());
+        verify(impl, times(1)).startLogSegmentNonPartitioned();
+    }
+
+    @Test
+    public void testStartAsyncLogSegmentNonPartitioned() throws Exception {
+        AsyncLogWriter writer = mock(AsyncLogWriter.class);
+        when(impl.startAsyncLogSegmentNonPartitioned()).thenReturn(writer);
+        assertEquals(writer, ((AsyncLogWriterImpl) manager.startAsyncLogSegmentNonPartitioned()).getImpl());
+        verify(impl, times(1)).startAsyncLogSegmentNonPartitioned();
+    }
+
+    @Test
+    public void testGetAppendOnlyStreamWriter() throws Exception {
+        AppendOnlyStreamWriter writer = mock(AppendOnlyStreamWriter.class);
+        when(impl.getAppendOnlyStreamWriter()).thenReturn(writer);
+        assertEquals(writer, manager.getAppendOnlyStreamWriter());
+        verify(impl, times(1)).getAppendOnlyStreamWriter();
+    }
+
+    @Test
+    public void testGetAppendOnlyStreamReader() throws Exception {
+        AppendOnlyStreamReader writer = mock(AppendOnlyStreamReader.class);
+        when(impl.getAppendOnlyStreamReader()).thenReturn(writer);
+        assertEquals(writer, manager.getAppendOnlyStreamReader());
+        verify(impl, times(1)).getAppendOnlyStreamReader();
+    }
+
+    @Test
+    public void testGetInputStream() throws Exception {
+        LogReader reader = mock(LogReader.class);
+        when(impl.getInputStream(anyLong())).thenReturn(reader);
+        assertEquals(reader, ((LogReaderImpl) manager.getInputStream(1234L)).getImpl());
+        verify(impl, times(1)).getInputStream(eq(1234L));
+    }
+
+    @Test
+    public void testGetInputStream2() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        LogReader reader = mock(LogReader.class);
+        when(impl.getInputStream(eq(dlsn))).thenReturn(reader);
+        assertEquals(reader, ((LogReaderImpl) manager.getInputStream(dlsn)).getImpl());
+        verify(impl, times(1)).getInputStream(eq(dlsn));
+    }
+
+    @Test
+    public void testOpenAsyncLogReader() throws Exception {
+        org.apache.distributedlog.api.AsyncLogReader reader = mock(org.apache.distributedlog.api.AsyncLogReader.class);
+        when(impl.openAsyncLogReader(eq(1234L))).thenReturn(CompletableFuture.completedFuture(reader));
+        assertEquals(reader,
+            ((AsyncLogReaderImpl) FutureUtils.result(manager.openAsyncLogReader(1234L))).getImpl());
+        verify(impl, times(1)).openAsyncLogReader(eq(1234L));
+    }
+
+    @Test
+    public void testOpenAsyncLogReader2() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        org.apache.distributedlog.api.AsyncLogReader reader = mock(org.apache.distributedlog.api.AsyncLogReader.class);
+        when(impl.openAsyncLogReader(eq(dlsn))).thenReturn(CompletableFuture.completedFuture(reader));
+        assertEquals(reader,
+            ((AsyncLogReaderImpl) FutureUtils.result(manager.openAsyncLogReader(dlsn))).getImpl());
+        verify(impl, times(1)).openAsyncLogReader(eq(dlsn));
+    }
+
+    @Test
+    public void testGetAsyncLogReader() throws Exception {
+        org.apache.distributedlog.api.AsyncLogReader reader = mock(org.apache.distributedlog.api.AsyncLogReader.class);
+        when(impl.getAsyncLogReader(eq(1234L))).thenReturn(reader);
+        assertEquals(reader,
+            ((AsyncLogReaderImpl) manager.getAsyncLogReader(1234L)).getImpl());
+        verify(impl, times(1)).getAsyncLogReader(eq(1234L));
+    }
+
+    @Test
+    public void testGetAsyncLogReader2() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        org.apache.distributedlog.api.AsyncLogReader reader = mock(org.apache.distributedlog.api.AsyncLogReader.class);
+        when(impl.getAsyncLogReader(eq(dlsn))).thenReturn(reader);
+        assertEquals(reader,
+            ((AsyncLogReaderImpl) manager.getAsyncLogReader(dlsn)).getImpl());
+        verify(impl, times(1)).getAsyncLogReader(eq(dlsn));
+    }
+
+    @Test
+    public void testOpenAsyncLogReaderWithLock() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        org.apache.distributedlog.api.AsyncLogReader reader = mock(org.apache.distributedlog.api.AsyncLogReader.class);
+        when(impl.getAsyncLogReaderWithLock(eq(dlsn))).thenReturn(CompletableFuture.completedFuture(reader));
+        assertEquals(reader,
+            ((AsyncLogReaderImpl) FutureUtils.result(manager.getAsyncLogReaderWithLock(dlsn))).getImpl());
+        verify(impl, times(1)).getAsyncLogReaderWithLock(eq(dlsn));
+    }
+
+    @Test
+    public void testOpenAsyncLogReaderWithLock2() throws Exception {
+        String subscriberId = "test-subscriber";
+        DLSN dlsn = mock(DLSN.class);
+        org.apache.distributedlog.api.AsyncLogReader reader = mock(org.apache.distributedlog.api.AsyncLogReader.class);
+        when(impl.getAsyncLogReaderWithLock(eq(dlsn), eq(subscriberId)))
+            .thenReturn(CompletableFuture.completedFuture(reader));
+        assertEquals(reader,
+            ((AsyncLogReaderImpl) FutureUtils.result(manager.getAsyncLogReaderWithLock(dlsn, subscriberId))).getImpl());
+        verify(impl, times(1)).getAsyncLogReaderWithLock(eq(dlsn), eq(subscriberId));
+    }
+
+    @Test
+    public void testOpenAsyncLogReaderWithLock3() throws Exception {
+        String subscriberId = "test-subscriber";
+        org.apache.distributedlog.api.AsyncLogReader reader = mock(org.apache.distributedlog.api.AsyncLogReader.class);
+        when(impl.getAsyncLogReaderWithLock(eq(subscriberId)))
+            .thenReturn(CompletableFuture.completedFuture(reader));
+        assertEquals(reader,
+            ((AsyncLogReaderImpl) FutureUtils.result(manager.getAsyncLogReaderWithLock(subscriberId))).getImpl());
+        verify(impl, times(1)).getAsyncLogReaderWithLock(eq(subscriberId));
+    }
+
+    @Test
+    public void testGetDLSNNotLessThanTxId() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        when(impl.getDLSNNotLessThanTxId(anyLong())).thenReturn(CompletableFuture.completedFuture(dlsn));
+        assertEquals(dlsn, FutureUtils.result(manager.getDLSNNotLessThanTxId(1234L)));
+        verify(impl, times(1)).getDLSNNotLessThanTxId(eq(1234L));
+    }
+
+    @Test
+    public void testGetLastLogRecord() throws Exception {
+        LogRecordWithDLSN record = mock(LogRecordWithDLSN.class);
+        when(impl.getLastLogRecord()).thenReturn(record);
+        assertEquals(record, manager.getLastLogRecord());
+        verify(impl, times(1)).getLastLogRecord();
+    }
+
+    @Test
+    public void testFirstTxId() throws Exception {
+        long txId = System.currentTimeMillis();
+        when(impl.getFirstTxId()).thenReturn(txId);
+        assertEquals(txId, manager.getFirstTxId());
+        verify(impl, times(1)).getFirstTxId();
+    }
+
+    @Test
+    public void testLastTxId() throws Exception {
+        long txId = System.currentTimeMillis();
+        when(impl.getLastTxId()).thenReturn(txId);
+        assertEquals(txId, manager.getLastTxId());
+        verify(impl, times(1)).getLastTxId();
+    }
+
+    @Test
+    public void testLastDLSN() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        when(impl.getLastDLSN()).thenReturn(dlsn);
+        assertEquals(dlsn, manager.getLastDLSN());
+        verify(impl, times(1)).getLastDLSN();
+    }
+
+    @Test
+    public void testGetLastLogRecordAsync() throws Exception {
+        LogRecordWithDLSN record = mock(LogRecordWithDLSN.class);
+        when(impl.getLastLogRecordAsync()).thenReturn(CompletableFuture.completedFuture(record));
+        assertEquals(record, FutureUtils.result(manager.getLastLogRecordAsync()));
+        verify(impl, times(1)).getLastLogRecordAsync();
+    }
+
+    @Test
+    public void testLastTxIdAsync() throws Exception {
+        long txId = System.currentTimeMillis();
+        when(impl.getLastTxIdAsync()).thenReturn(CompletableFuture.completedFuture(txId));
+        assertEquals(txId, FutureUtils.result(manager.getLastTxIdAsync()).longValue());
+        verify(impl, times(1)).getLastTxIdAsync();
+    }
+
+    @Test
+    public void testLastDLSNAsync() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        when(impl.getLastDLSNAsync()).thenReturn(CompletableFuture.completedFuture(dlsn));
+        assertEquals(dlsn, FutureUtils.result(manager.getLastDLSNAsync()));
+        verify(impl, times(1)).getLastDLSNAsync();
+    }
+
+    @Test
+    public void testFirstDLSNAsync() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        when(impl.getFirstDLSNAsync()).thenReturn(CompletableFuture.completedFuture(dlsn));
+        assertEquals(dlsn, FutureUtils.result(manager.getFirstDLSNAsync()));
+        verify(impl, times(1)).getFirstDLSNAsync();
+    }
+
+    @Test
+    public void testGetLogRecordCount() throws Exception {
+        long count = System.currentTimeMillis();
+        when(impl.getLogRecordCount()).thenReturn(count);
+        assertEquals(count, manager.getLogRecordCount());
+        verify(impl, times(1)).getLogRecordCount();
+    }
+
+    @Test
+    public void testGetLogRecordCountAsync() throws Exception {
+        DLSN dlsn = mock(DLSN.class);
+        long count = System.currentTimeMillis();
+        when(impl.getLogRecordCountAsync(eq(dlsn))).thenReturn(CompletableFuture.completedFuture(count));
+        assertEquals(count, FutureUtils.result(manager.getLogRecordCountAsync(dlsn)).longValue());
+        verify(impl, times(1)).getLogRecordCountAsync(eq(dlsn));
+    }
+
+    @Test
+    public void testRecover() throws Exception {
+        manager.recover();
+        verify(impl, times(1)).recover();
+    }
+
+    @Test
+    public void testIsEndOfStreamMarked() throws Exception {
+        when(impl.isEndOfStreamMarked()).thenReturn(true);
+        assertTrue(manager.isEndOfStreamMarked());
+        verify(impl, times(1)).isEndOfStreamMarked();
+    }
+
+    @Test
+    public void testDelete() throws Exception {
+        manager.delete();
+        verify(impl, times(1)).delete();
+    }
+
+    @Test
+    public void testPurgeLogsOlderThan() throws Exception {
+        long minTxIdToKeep = System.currentTimeMillis();
+        manager.purgeLogsOlderThan(minTxIdToKeep);
+        verify(impl, times(1)).purgeLogsOlderThan(eq(minTxIdToKeep));
+    }
+
+    @Test
+    public void testGetSubscriptionsStore() throws Exception {
+        SubscriptionsStore ss = mock(SubscriptionsStore.class);
+        when(impl.getSubscriptionsStore()).thenReturn(ss);
+        assertEquals(ss, ((SubscriptionsStoreImpl) manager.getSubscriptionsStore()).getImpl());
+        verify(impl, times(1)).getSubscriptionsStore();
+    }
+
+    @Test
+    public void testClose() throws Exception {
+        manager.close();
+        verify(impl, times(1)).close();
+    }
+
+    @Test
+    public void testAsyncClose() throws Exception {
+        when(impl.asyncClose()).thenReturn(CompletableFuture.completedFuture(null));
+        FutureUtils.result(manager.asyncClose());
+        verify(impl, times(1)).asyncClose();
+    }
+}
diff --git a/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestLogReaderImpl.java b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestLogReaderImpl.java
new file mode 100644
index 0000000..4adc386
--- /dev/null
+++ b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestLogReaderImpl.java
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import static org.mockito.Matchers.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 java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.util.FutureUtils;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link LogReaderImpl}.
+ */
+public class TestLogReaderImpl {
+
+    private final org.apache.distributedlog.api.LogReader underlying =
+        mock(org.apache.distributedlog.api.LogReader.class);
+    private final LogReaderImpl reader = new LogReaderImpl(underlying);
+
+    @Test
+    public void testReadNext() throws Exception {
+        reader.readNext(false);
+        verify(underlying, times(1)).readNext(eq(false));
+    }
+
+    @Test
+    public void testReadBulk() throws Exception {
+        reader.readBulk(false, 100);
+        verify(underlying, times(1)).readBulk(eq(false), eq(100));
+    }
+
+    @Test
+    public void testClose() throws Exception {
+        reader.close();
+        verify(underlying, times(1)).close();
+    }
+
+    @Test
+    public void testAsyncClose() throws Exception {
+        when(underlying.asyncClose())
+            .thenReturn(CompletableFuture.completedFuture(null));
+        FutureUtils.result(reader.asyncClose());
+        verify(underlying, times(1)).asyncClose();
+    }
+
+}
diff --git a/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestLogWriterImpl.java b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestLogWriterImpl.java
new file mode 100644
index 0000000..be69260
--- /dev/null
+++ b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestLogWriterImpl.java
@@ -0,0 +1,82 @@
+/*
+ * 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;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.util.List;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link LogWriterImpl}.
+ */
+public class TestLogWriterImpl {
+
+    private final org.apache.distributedlog.api.LogWriter underlying =
+        mock(org.apache.distributedlog.api.LogWriter.class);
+    private final LogWriterImpl writer = new LogWriterImpl(underlying);
+
+    @Test
+    public void testWrite() throws Exception {
+        LogRecord record = mock(LogRecord.class);
+        writer.write(record);
+        verify(underlying, times(1)).write(eq(record));
+    }
+
+    @Test
+    public void testWriteBulk() throws Exception {
+        List<LogRecord> records = mock(List.class);
+        writer.writeBulk(records);
+        verify(underlying, times(1)).writeBulk(eq(records));
+    }
+
+    @Test
+    public void testSetReadyToFlush() throws Exception {
+        writer.setReadyToFlush();
+        verify(underlying, times(1)).setReadyToFlush();
+    }
+
+    @Test
+    public void testFlushAndSync() throws Exception {
+        writer.flushAndSync();
+        verify(underlying, times(1)).flushAndSync();
+    }
+
+    @Test
+    public void testMarkEndOfStream() throws Exception {
+        writer.markEndOfStream();
+        verify(underlying, times(1)).markEndOfStream();
+    }
+
+    @Test
+    public void testClose() throws Exception {
+        writer.close();
+        verify(underlying, times(1)).close();
+    }
+
+    @Test
+    public void testAbort() throws Exception {
+        writer.abort();
+        verify(underlying, times(1)).abort();
+    }
+
+}
diff --git a/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestSubscriptionStoreImpl.java b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestSubscriptionStoreImpl.java
new file mode 100644
index 0000000..e6573aa
--- /dev/null
+++ b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/TestSubscriptionStoreImpl.java
@@ -0,0 +1,91 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.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 java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
+import org.apache.distributedlog.util.FutureUtils;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link SubscriptionsStoreImpl}.
+ */
+public class TestSubscriptionStoreImpl {
+
+    private final SubscriptionsStore underlying = mock(SubscriptionsStore.class);
+    private final SubscriptionsStoreImpl store = new SubscriptionsStoreImpl(underlying);
+
+    @Test
+    public void testGetLastCommitPosition() throws Exception {
+        String subscriber = "test-subscriber";
+        DLSN dlsn = mock(DLSN.class);
+        when(underlying.getLastCommitPosition(anyString()))
+            .thenReturn(CompletableFuture.completedFuture(dlsn));
+        assertEquals(dlsn,
+            FutureUtils.result(store.getLastCommitPosition(subscriber)));
+        verify(underlying, times(1)).getLastCommitPosition(eq(subscriber));
+    }
+
+    @Test
+    public void testGetLastCommitPositions() throws Exception {
+        Map<String, DLSN> positions = mock(Map.class);
+        when(underlying.getLastCommitPositions())
+            .thenReturn(CompletableFuture.completedFuture(positions));
+        assertEquals(positions, FutureUtils.result(store.getLastCommitPositions()));
+        verify(underlying, times(1)).getLastCommitPositions();
+    }
+
+    @Test
+    public void testAdvanceCommmitPosition() throws Exception {
+        String subscriber = "test-subscriber";
+        DLSN dlsn = mock(DLSN.class);
+        when(underlying.advanceCommitPosition(anyString(), any(DLSN.class)))
+            .thenReturn(CompletableFuture.completedFuture(null));
+        FutureUtils.result(store.advanceCommitPosition(subscriber, dlsn));
+        verify(underlying, times(1))
+            .advanceCommitPosition(eq(subscriber), eq(dlsn));
+    }
+
+    @Test
+    public void testDeleteSubscriber() throws Exception {
+        String subscriber = "test-subscriber";
+        when(underlying.deleteSubscriber(anyString()))
+            .thenReturn(CompletableFuture.completedFuture(true));
+        assertTrue(FutureUtils.result(store.deleteSubscriber(subscriber)));
+        verify(underlying, times(1)).deleteSubscriber(eq(subscriber));
+    }
+
+    @Test
+    public void testClose() throws Exception {
+        store.close();
+        verify(underlying, times(1)).close();
+    }
+
+}
diff --git a/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java
new file mode 100644
index 0000000..78dcb2a
--- /dev/null
+++ b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java
@@ -0,0 +1,104 @@
+/*
+ * 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.namespace;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.net.URI;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link DistributedLogNamespaceBuilder}.
+ */
+public class TestDistributedLogNamespaceBuilder {
+
+    private final NamespaceBuilder underlying = mock(NamespaceBuilder.class);
+    private final DistributedLogNamespaceBuilder builder = new DistributedLogNamespaceBuilder(underlying);
+
+    @Test
+    public void testConf() {
+        DistributedLogConfiguration conf = mock(DistributedLogConfiguration.class);
+        builder.conf(conf);
+        verify(underlying, times(1)).conf(eq(conf));
+    }
+
+    @Test
+    public void testDynConf() {
+        DynamicDistributedLogConfiguration conf = mock(DynamicDistributedLogConfiguration.class);
+        builder.dynConf(conf);
+        verify(underlying, times(1)).dynConf(eq(conf));
+    }
+
+    @Test
+    public void testUri() {
+        URI uri = URI.create("distributedlog://127.0.0.1/messaging/distributedlog");
+        builder.uri(uri);
+        verify(underlying, times(1)).uri(eq(uri));
+    }
+
+    @Test
+    public void testStatsLogger() {
+        StatsLogger statsLogger = mock(StatsLogger.class);
+        builder.statsLogger(statsLogger);
+        verify(underlying, times(1)).statsLogger(eq(statsLogger));
+    }
+
+    @Test
+    public void testPerLogStatsLogger() {
+        StatsLogger statsLogger = mock(StatsLogger.class);
+        builder.perLogStatsLogger(statsLogger);
+        verify(underlying, times(1)).perLogStatsLogger(eq(statsLogger));
+    }
+
+    @Test
+    public void testFeatureProvider() {
+        FeatureProvider provider = mock(FeatureProvider.class);
+        builder.featureProvider(provider);
+        verify(underlying, times(1)).featureProvider(eq(provider));
+    }
+
+    @Test
+    public void testClientId() {
+        String clientId = "test-client-id";
+        builder.clientId(clientId);
+        verify(underlying, times(1)).clientId(eq(clientId));
+    }
+
+    @Test
+    public void testRegionId() {
+        int regionId = 1234;
+        builder.regionId(regionId);
+        verify(underlying, times(1)).regionId(eq(regionId));
+    }
+
+    @Test
+    public void testBuild() throws Exception {
+        builder.build();
+        verify(underlying, times(1)).build();
+    }
+
+}
diff --git a/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceImpl.java b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceImpl.java
new file mode 100644
index 0000000..b562fe4
--- /dev/null
+++ b/distributedlog-core-twitter/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceImpl.java
@@ -0,0 +1,119 @@
+/*
+ * 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.namespace;
+
+import static junit.framework.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.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.Optional;
+import java.util.Iterator;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.callback.NamespaceListener;
+import org.junit.Test;
+
+/**
+ * Unit test of {@link DistributedLogNamespaceImpl}.
+ */
+public class TestDistributedLogNamespaceImpl {
+
+    private final Namespace impl = mock(Namespace.class);
+    private final DistributedLogNamespaceImpl namespace = new DistributedLogNamespaceImpl(impl);
+
+    @Test
+    public void testGetNamespaceDriver() {
+        NamespaceDriver driver = mock(NamespaceDriver.class);
+        when(impl.getNamespaceDriver()).thenReturn(driver);
+        assertEquals(driver, namespace.getNamespaceDriver());
+        verify(impl, times(1)).getNamespaceDriver();
+    }
+
+    @Test
+    public void testCreateLog() throws Exception {
+        String logName = "test-log-name";
+        namespace.createLog(logName);
+        verify(impl, times(1)).createLog(eq(logName));
+    }
+
+    @Test
+    public void testDeleteLog() throws Exception {
+        String logName = "test-log-name";
+        namespace.deleteLog(logName);
+        verify(impl, times(1)).deleteLog(eq(logName));
+    }
+
+    @Test
+    public void testOpenLog() throws Exception {
+        String logName = "test-open-log";
+        namespace.openLog(logName);
+        verify(impl, times(1)).openLog(eq(logName));
+    }
+
+    @Test
+    public void testOpenLog2() throws Exception {
+        String logName = "test-open-log";
+        namespace.openLog(logName, Optional.absent(), Optional.absent(), Optional.absent());
+        verify(impl, times(1))
+            .openLog(eq(logName), eq(Optional.absent()), eq(Optional.absent()), eq(Optional.absent()));
+    }
+
+    @Test
+    public void testLogExists() throws Exception {
+        String logName = "test-log-exists";
+        when(impl.logExists(anyString())).thenReturn(true);
+        assertTrue(namespace.logExists(logName));
+        verify(impl, times(1)).logExists(eq(logName));
+    }
+
+    @Test
+    public void testGetLogs() throws Exception {
+        Iterator<String> logs = mock(Iterator.class);
+        when(impl.getLogs()).thenReturn(logs);
+        assertEquals(logs, namespace.getLogs());
+        verify(impl, times(1)).getLogs();
+    }
+
+    @Test
+    public void testRegisterNamespaceListener() throws Exception {
+        NamespaceListener listener = mock(NamespaceListener.class);
+        namespace.registerNamespaceListener(listener);
+        verify(impl, times(1)).registerNamespaceListener(eq(listener));
+    }
+
+    @Test
+    public void testCreateAccessControlManager() throws Exception {
+        AccessControlManager manager = mock(AccessControlManager.class);
+        when(impl.createAccessControlManager()).thenReturn(manager);
+        assertEquals(manager, namespace.createAccessControlManager());
+        verify(impl, times(1)).createAccessControlManager();
+    }
+
+    @Test
+    public void testClose() {
+        namespace.close();
+        verify(impl, times(1)).close();
+    }
+
+}
diff --git a/distributedlog-core/conf/log4j.properties b/distributedlog-core/conf/log4j.properties
index 930db8d..af1cf5f 100644
--- a/distributedlog-core/conf/log4j.properties
+++ b/distributedlog-core/conf/log4j.properties
@@ -30,11 +30,7 @@
 log4j.logger.org.apache.bookkeeper=INFO
 
 # redirect executor output to executors.log since slow op warnings can be quite verbose
-log4j.logger.org.apache.distributedlog.util.MonitoredFuturePool=INFO, Executors
-log4j.logger.org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
 log4j.logger.org.apache.bookkeeper.util.SafeRunnable=INFO, Executors
-log4j.additivity.org.apache.distributedlog.util.MonitoredFuturePool=false
-log4j.additivity.org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
 log4j.additivity.org.apache.bookkeeper.util.SafeRunnable=false
 
 log4j.appender.Executors=org.apache.log4j.RollingFileAppender
diff --git a/distributedlog-core/pom.xml b/distributedlog-core/pom.xml
index a4f7568..1ad51c2 100644
--- a/distributedlog-core/pom.xml
+++ b/distributedlog-core/pom.xml
@@ -26,6 +26,16 @@
   <name>Apache DistributedLog :: Core Library</name>
   <dependencies>
     <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-common</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-protocol</artifactId>
+      <version>${project.parent.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.zookeeper</groupId>
       <artifactId>zookeeper</artifactId>
       <version>${zookeeper.version}</version>
@@ -41,50 +51,6 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>${junit.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.jmock</groupId>
-      <artifactId>jmock</artifactId>
-      <version>${jmock.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
-      <version>${slf4j.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-      <version>${slf4j.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter.common</groupId>
-      <artifactId>stats-util</artifactId>
-      <version>${stats-util.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-jdk14</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>util-core_2.11</artifactId>
-      <version>${finagle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.commons</groupId>
-      <artifactId>commons-lang3</artifactId>
-      <version>${commons-lang3.version}</version>
-    </dependency>
-    <dependency>
       <groupId>org.apache.thrift</groupId>
       <artifactId>libthrift</artifactId>
       <version>0.9.3</version>
@@ -114,19 +80,28 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>commons-cli</groupId>
-      <artifactId>commons-cli</artifactId>
-      <version>${commons-cli.version}</version>
+      <groupId>org.projectlombok</groupId>
+      <artifactId>lombok</artifactId>
+      <version>${lombok.version}</version>
+      <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-protocol</artifactId>
-      <version>${project.parent.version}</version>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>net.jpountz.lz4</groupId>
-      <artifactId>lz4</artifactId>
-      <version>${lz4.version}</version>
+      <groupId>org.jmock</groupId>
+      <artifactId>jmock</artifactId>
+      <version>${jmock.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <version>${slf4j.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
@@ -136,7 +111,7 @@
     </dependency> 
     <dependency>
       <groupId>org.apache.distributedlog</groupId>
-      <artifactId>distributedlog-protocol</artifactId>
+      <artifactId>distributedlog-common</artifactId>
       <version>${project.parent.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
@@ -172,7 +147,7 @@
           <properties>
             <property>
               <name>listener</name>
-              <value>org.apache.distributedlog.TimedOutTestsListener</value>
+              <value>org.apache.distributedlog.common.util.TimedOutTestsListener</value>
             </property>
           </properties>
         </configuration>
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
index 1d96f0e..3a31907 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
@@ -18,10 +18,10 @@
 package org.apache.distributedlog;
 
 import com.google.common.base.Preconditions;
-
 import java.io.IOException;
 import java.io.InputStream;
-
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
index 8278c68..dde78c2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
@@ -17,14 +17,12 @@
  */
 package org.apache.distributedlog;
 
-import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
 import java.io.Closeable;
 import java.io.IOException;
-
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,16 +42,16 @@
         this.requestPos = pos;
     }
 
-    public Future<DLSN> write(byte[] data) {
+    public CompletableFuture<DLSN> write(byte[] data) {
         requestPos += data.length;
-        Future<DLSN> writeResult = logWriter.write(new LogRecord(requestPos, data));
-        return writeResult.addEventListener(new WriteCompleteListener(requestPos));
+        CompletableFuture<DLSN> writeResult = logWriter.write(new LogRecord(requestPos, data));
+        return writeResult.whenComplete(new WriteCompleteListener(requestPos));
     }
 
     public void force(boolean metadata) throws IOException {
         long pos = 0;
         try {
-            pos = Await.result(logWriter.flushAndCommit());
+            pos = FutureUtils.result(logWriter.flushAndCommit());
         } catch (IOException ioe) {
             throw ioe;
         } catch (Exception ex) {
@@ -78,7 +76,7 @@
 
     public void markEndOfStream() throws IOException {
         try {
-            Await.result(logWriter.markEndOfStream());
+            FutureUtils.result(logWriter.markEndOfStream());
         } catch (IOException ioe) {
             throw ioe;
         } catch (Exception ex) {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
index c12bd10..367bb50 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog;
 
-public interface AsyncNotification {
+interface AsyncNotification {
     /**
      * Triggered when the background activity encounters an exception
      *
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
index 4a2ef30..8a0bffb 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
@@ -18,6 +18,9 @@
 package org.apache.distributedlog;
 
 import com.google.common.annotations.VisibleForTesting;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+import java.util.function.Function;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.LockingException;
@@ -27,18 +30,12 @@
 import org.apache.distributedlog.io.Abortables;
 import org.apache.distributedlog.io.AsyncAbortable;
 import org.apache.distributedlog.io.AsyncCloseable;
-import org.apache.distributedlog.util.FutureUtils;
-import org.apache.distributedlog.util.PermitManager;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.util.PermitManager;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -53,18 +50,18 @@
     protected final BKDistributedLogManager bkDistributedLogManager;
 
     // States
-    private Promise<Void> closePromise = null;
+    private CompletableFuture<Void> closePromise = null;
     private volatile boolean forceRolling = false;
     private boolean forceRecovery = false;
 
     // Truncation Related
-    private Future<List<LogSegmentMetadata>> lastTruncationAttempt = null;
+    private CompletableFuture<List<LogSegmentMetadata>> lastTruncationAttempt = null;
     @VisibleForTesting
     private Long minTimestampToKeepOverride = null;
 
     // Log Segment Writers
     protected BKLogSegmentWriter segmentWriter = null;
-    protected Future<BKLogSegmentWriter> segmentWriterFuture = null;
+    protected CompletableFuture<BKLogSegmentWriter> segmentWriterFuture = null;
     protected BKLogSegmentWriter allocatedSegmentWriter = null;
     protected BKLogWriteHandler writeHandler = null;
 
@@ -100,7 +97,7 @@
         // This code path will be executed when the handler is not set or has been closed
         // due to forceRecovery during testing
         BKLogWriteHandler newHandler =
-                FutureUtils.result(bkDistributedLogManager.asyncCreateWriteHandler(false));
+                Utils.ioResult(bkDistributedLogManager.asyncCreateWriteHandler(false));
         boolean success = false;
         try {
             synchronized (this) {
@@ -123,13 +120,13 @@
         return segmentWriter;
     }
 
-    protected synchronized Future<BKLogSegmentWriter> getCachedLogWriterFuture() {
+    protected synchronized CompletableFuture<BKLogSegmentWriter> getCachedLogWriterFuture() {
         return segmentWriterFuture;
     }
 
     protected synchronized void cacheLogWriter(BKLogSegmentWriter logWriter) {
         this.segmentWriter = logWriter;
-        this.segmentWriterFuture = Future.value(logWriter);
+        this.segmentWriterFuture = FutureUtils.value(logWriter);
     }
 
     protected synchronized BKLogSegmentWriter removeCachedLogWriter() {
@@ -157,12 +154,12 @@
         }
     }
 
-    private Future<Void> asyncCloseAndComplete(boolean shouldThrow) {
+    private CompletableFuture<Void> asyncCloseAndComplete(boolean shouldThrow) {
         BKLogSegmentWriter segmentWriter = getCachedLogWriter();
         BKLogWriteHandler writeHandler = getCachedWriteHandler();
         if (null != segmentWriter && null != writeHandler) {
             cancelTruncation();
-            Promise<Void> completePromise = new Promise<Void>();
+            CompletableFuture<Void> completePromise = new CompletableFuture<Void>();
             asyncCloseAndComplete(segmentWriter, writeHandler, completePromise, shouldThrow);
             return completePromise;
         } else {
@@ -172,10 +169,10 @@
 
     private void asyncCloseAndComplete(final BKLogSegmentWriter segmentWriter,
                                        final BKLogWriteHandler writeHandler,
-                                       final Promise<Void> completePromise,
+                                       final CompletableFuture<Void> completePromise,
                                        final boolean shouldThrow) {
         writeHandler.completeAndCloseLogSegment(segmentWriter)
-                .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+                .whenComplete(new FutureEventListener<LogSegmentMetadata>() {
                     @Override
                     public void onSuccess(LogSegmentMetadata segment) {
                         removeCachedLogWriter();
@@ -189,15 +186,11 @@
                     }
 
                     private void complete(final Throwable cause) {
-                        closeNoThrow().ensure(new AbstractFunction0<BoxedUnit>() {
-                            @Override
-                            public BoxedUnit apply() {
-                                if (null != cause && shouldThrow) {
-                                    FutureUtils.setException(completePromise, cause);
-                                } else {
-                                    FutureUtils.setValue(completePromise, null);
-                                }
-                                return BoxedUnit.UNIT;
+                        FutureUtils.ensure(closeNoThrow(), () -> {
+                            if (null != cause && shouldThrow) {
+                                FutureUtils.completeExceptionally(completePromise, cause);
+                            } else {
+                                FutureUtils.complete(completePromise, null);
                             }
                         });
                     }
@@ -206,63 +199,67 @@
 
     @VisibleForTesting
     void closeAndComplete() throws IOException {
-        FutureUtils.result(asyncCloseAndComplete(true));
+        Utils.ioResult(asyncCloseAndComplete(true));
     }
 
-    protected Future<Void> asyncCloseAndComplete() {
+    protected CompletableFuture<Void> asyncCloseAndComplete() {
         return asyncCloseAndComplete(true);
     }
 
     @Override
     public void close() throws IOException {
-        FutureUtils.result(asyncClose());
+        Utils.ioResult(asyncClose());
     }
 
     @Override
-    public Future<Void> asyncClose() {
+    public CompletableFuture<Void> asyncClose() {
         return asyncCloseAndComplete(false);
     }
 
     /**
      * Close the writer and release all the underlying resources
      */
-    protected Future<Void> closeNoThrow() {
-        Promise<Void> closeFuture;
+    protected CompletableFuture<Void> closeNoThrow() {
+        CompletableFuture<Void> closeFuture;
         synchronized (this) {
             if (null != closePromise) {
                 return closePromise;
             }
-            closeFuture = closePromise = new Promise<Void>();
+            closeFuture = closePromise = new CompletableFuture<Void>();
         }
         cancelTruncation();
-        Utils.closeSequence(bkDistributedLogManager.getScheduler(),
-                true, /** ignore close errors **/
-                getCachedLogWriter(),
-                getAllocatedLogWriter(),
-                getCachedWriteHandler()
-        ).proxyTo(closeFuture);
+        FutureUtils.proxyTo(
+            Utils.closeSequence(bkDistributedLogManager.getScheduler(),
+                    true, /** ignore close errors **/
+                    getCachedLogWriter(),
+                    getAllocatedLogWriter(),
+                    getCachedWriteHandler()
+            ),
+            closeFuture);
         return closeFuture;
     }
 
     @Override
     public void abort() throws IOException {
-        FutureUtils.result(asyncAbort());
+        Utils.ioResult(asyncAbort());
     }
 
     @Override
-    public Future<Void> asyncAbort() {
-        Promise<Void> closeFuture;
+    public CompletableFuture<Void> asyncAbort() {
+        CompletableFuture<Void> closeFuture;
         synchronized (this) {
             if (null != closePromise) {
                 return closePromise;
             }
-            closeFuture = closePromise = new Promise<Void>();
+            closeFuture = closePromise = new CompletableFuture<Void>();
         }
         cancelTruncation();
-        Abortables.abortSequence(bkDistributedLogManager.getScheduler(),
-                getCachedLogWriter(),
-                getAllocatedLogWriter(),
-                getCachedWriteHandler()).proxyTo(closeFuture);
+        FutureUtils.proxyTo(
+            Abortables.abortSequence(bkDistributedLogManager.getScheduler(),
+                    getCachedLogWriter(),
+                    getAllocatedLogWriter(),
+                    getCachedWriteHandler()),
+            closeFuture);
         return closeFuture;
     }
 
@@ -270,22 +267,22 @@
     protected BKLogSegmentWriter getLedgerWriter(final long startTxId,
                                                  final boolean allowMaxTxID)
             throws IOException {
-        Future<BKLogSegmentWriter> logSegmentWriterFuture = asyncGetLedgerWriter(true);
+        CompletableFuture<BKLogSegmentWriter> logSegmentWriterFuture = asyncGetLedgerWriter(true);
         BKLogSegmentWriter logSegmentWriter = null;
         if (null != logSegmentWriterFuture) {
-            logSegmentWriter = FutureUtils.result(logSegmentWriterFuture);
+            logSegmentWriter = Utils.ioResult(logSegmentWriterFuture);
         }
         if (null == logSegmentWriter || (shouldStartNewSegment(logSegmentWriter) || forceRolling)) {
-            logSegmentWriter = FutureUtils.result(rollLogSegmentIfNecessary(
+            logSegmentWriter = Utils.ioResult(rollLogSegmentIfNecessary(
                     logSegmentWriter, startTxId, true /* bestEffort */, allowMaxTxID));
         }
         return logSegmentWriter;
     }
 
     // used by async writer
-    synchronized protected Future<BKLogSegmentWriter> asyncGetLedgerWriter(boolean resetOnError) {
+    synchronized protected CompletableFuture<BKLogSegmentWriter> asyncGetLedgerWriter(boolean resetOnError) {
         final BKLogSegmentWriter ledgerWriter = getCachedLogWriter();
-        Future<BKLogSegmentWriter> ledgerWriterFuture = getCachedLogWriterFuture();
+        CompletableFuture<BKLogSegmentWriter> ledgerWriterFuture = getCachedLogWriterFuture();
         if (null == ledgerWriterFuture || null == ledgerWriter) {
             return null;
         }
@@ -293,38 +290,38 @@
         // Handle the case where the last call to write actually caused an error in the log
         if ((ledgerWriter.isLogSegmentInError() || forceRecovery) && resetOnError) {
             // Close the ledger writer so that we will recover and start a new log segment
-            Future<Void> closeFuture;
+            CompletableFuture<Void> closeFuture;
             if (ledgerWriter.isLogSegmentInError()) {
                 closeFuture = ledgerWriter.asyncAbort();
             } else {
                 closeFuture = ledgerWriter.asyncClose();
             }
-            return closeFuture.flatMap(
-                    new AbstractFunction1<Void, Future<BKLogSegmentWriter>>() {
+            return closeFuture.thenCompose(
+                    new Function<Void, CompletionStage<BKLogSegmentWriter>>() {
                 @Override
-                public Future<BKLogSegmentWriter> apply(Void result) {
+                public CompletableFuture<BKLogSegmentWriter> apply(Void result) {
                     removeCachedLogWriter();
 
                     if (ledgerWriter.isLogSegmentInError()) {
-                        return Future.value(null);
+                        return FutureUtils.value(null);
                     }
 
                     BKLogWriteHandler writeHandler;
                     try {
                         writeHandler = getWriteHandler();
                     } catch (IOException e) {
-                        return Future.exception(e);
+                        return FutureUtils.exception(e);
                     }
                     if (null != writeHandler && forceRecovery) {
                         return writeHandler.completeAndCloseLogSegment(ledgerWriter)
-                                .map(new AbstractFunction1<LogSegmentMetadata, BKLogSegmentWriter>() {
+                                .thenApply(new Function<LogSegmentMetadata, BKLogSegmentWriter>() {
                             @Override
                             public BKLogSegmentWriter apply(LogSegmentMetadata completedLogSegment) {
                                 return null;
                             }
                         });
                     } else {
-                        return Future.value(null);
+                        return FutureUtils.value(null);
                     }
                 }
             });
@@ -357,32 +354,25 @@
         // skip scheduling if there is task that's already running
         //
         synchronized (this) {
-            if (truncationEnabled && ((lastTruncationAttempt == null) || lastTruncationAttempt.isDefined())) {
+            if (truncationEnabled && ((lastTruncationAttempt == null) || lastTruncationAttempt.isDone())) {
                 lastTruncationAttempt = writeHandler.purgeLogSegmentsOlderThanTimestamp(minTimestampToKeep);
             }
         }
     }
 
-    private Future<BKLogSegmentWriter> asyncStartNewLogSegment(final BKLogWriteHandler writeHandler,
+    private CompletableFuture<BKLogSegmentWriter> asyncStartNewLogSegment(final BKLogWriteHandler writeHandler,
                                                                final long startTxId,
                                                                final boolean allowMaxTxID) {
         return writeHandler.recoverIncompleteLogSegments()
-                .flatMap(new AbstractFunction1<Long, Future<BKLogSegmentWriter>>() {
-            @Override
-            public Future<BKLogSegmentWriter> apply(Long lastTxId) {
-                return writeHandler.asyncStartLogSegment(startTxId, false, allowMaxTxID)
-                        .onSuccess(new AbstractFunction1<BKLogSegmentWriter, BoxedUnit>() {
-                    @Override
-                    public BoxedUnit apply(BKLogSegmentWriter newSegmentWriter) {
+            .thenCompose(
+                lastTxId -> writeHandler.asyncStartLogSegment(startTxId, false, allowMaxTxID)
+                    .thenApply(newSegmentWriter -> {
                         cacheLogWriter(newSegmentWriter);
-                        return BoxedUnit.UNIT;
-                    }
-                });
-            }
-        });
+                        return newSegmentWriter;
+                    }));
     }
 
-    private Future<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOneWithPermit(
+    private CompletableFuture<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOneWithPermit(
             final BKLogSegmentWriter oldSegmentWriter,
             final BKLogWriteHandler writeHandler,
             final long startTxId,
@@ -390,47 +380,46 @@
             final boolean allowMaxTxID) {
         final PermitManager.Permit switchPermit = bkDistributedLogManager.getLogSegmentRollingPermitManager().acquirePermit();
         if (switchPermit.isAllowed()) {
-            return closeOldLogSegmentAndStartNewOne(
-                    oldSegmentWriter,
-                    writeHandler,
-                    startTxId,
-                    bestEffort,
-                    allowMaxTxID
-            ).rescue(new Function<Throwable, Future<BKLogSegmentWriter>>() {
-                @Override
-                public Future<BKLogSegmentWriter> apply(Throwable cause) {
-                    if (cause instanceof LockingException) {
-                        LOG.warn("We lost lock during completeAndClose log segment for {}. Disable ledger rolling until it is recovered : ",
-                                writeHandler.getFullyQualifiedName(), cause);
-                        bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
-                        return Future.value(oldSegmentWriter);
-                    } else if (cause instanceof ZKException) {
-                        ZKException zke = (ZKException) cause;
-                        if (ZKException.isRetryableZKException(zke)) {
-                            LOG.warn("Encountered zookeeper connection issues during completeAndClose log segment for {}." +
-                                    " Disable ledger rolling until it is recovered : {}", writeHandler.getFullyQualifiedName(),
-                                    zke.getKeeperExceptionCode());
+            return FutureUtils.ensure(
+                FutureUtils.rescue(
+                     closeOldLogSegmentAndStartNewOne(
+                            oldSegmentWriter,
+                            writeHandler,
+                            startTxId,
+                            bestEffort,
+                            allowMaxTxID
+                    ),
+                    // rescue function
+                    cause -> {
+                        if (cause instanceof LockingException) {
+                            LOG.warn("We lost lock during completeAndClose log segment for {}. Disable ledger rolling until it is recovered : ",
+                                    writeHandler.getFullyQualifiedName(), cause);
                             bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
-                            return Future.value(oldSegmentWriter);
+                            return FutureUtils.value(oldSegmentWriter);
+                        } else if (cause instanceof ZKException) {
+                            ZKException zke = (ZKException) cause;
+                            if (ZKException.isRetryableZKException(zke)) {
+                                LOG.warn("Encountered zookeeper connection issues during completeAndClose log segment for {}." +
+                                        " Disable ledger rolling until it is recovered : {}", writeHandler.getFullyQualifiedName(),
+                                        zke.getKeeperExceptionCode());
+                                bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
+                                return FutureUtils.value(oldSegmentWriter);
+                            }
                         }
+                        return FutureUtils.exception(cause);
                     }
-                    return Future.exception(cause);
-                }
-            }).ensure(new AbstractFunction0<BoxedUnit>() {
-                @Override
-                public BoxedUnit apply() {
-                    bkDistributedLogManager.getLogSegmentRollingPermitManager()
-                            .releasePermit(switchPermit);
-                    return BoxedUnit.UNIT;
-                }
-            });
+                ),
+                // ensure function
+                () -> bkDistributedLogManager.getLogSegmentRollingPermitManager()
+                                .releasePermit(switchPermit)
+            );
         } else {
             bkDistributedLogManager.getLogSegmentRollingPermitManager().releasePermit(switchPermit);
-            return Future.value(oldSegmentWriter);
+            return FutureUtils.value(oldSegmentWriter);
         }
     }
 
-    private Future<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOne(
+    private CompletableFuture<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOne(
             final BKLogSegmentWriter oldSegmentWriter,
             final BKLogWriteHandler writeHandler,
             final long startTxId,
@@ -444,14 +433,14 @@
                         writeHandler.getFullyQualifiedName());
             }
             return writeHandler.asyncStartLogSegment(startTxId, bestEffort, allowMaxTxID)
-                    .flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<BKLogSegmentWriter>>() {
+                    .thenCompose(new Function<BKLogSegmentWriter, CompletableFuture<BKLogSegmentWriter>>() {
                         @Override
-                        public Future<BKLogSegmentWriter> apply(BKLogSegmentWriter newSegmentWriter) {
+                        public CompletableFuture<BKLogSegmentWriter> apply(BKLogSegmentWriter newSegmentWriter) {
                             if (null == newSegmentWriter) {
                                 if (bestEffort) {
-                                    return Future.value(oldSegmentWriter);
+                                    return FutureUtils.value(oldSegmentWriter);
                                 } else {
-                                    return Future.exception(
+                                    return FutureUtils.exception(
                                             new UnexpectedException("StartLogSegment returns null for bestEffort rolling"));
                                 }
                             }
@@ -468,30 +457,30 @@
         }
     }
 
-    private Future<BKLogSegmentWriter> completeOldSegmentAndCacheNewLogSegmentWriter(
+    private CompletableFuture<BKLogSegmentWriter> completeOldSegmentAndCacheNewLogSegmentWriter(
             BKLogSegmentWriter oldSegmentWriter,
             final BKLogSegmentWriter newSegmentWriter) {
-        final Promise<BKLogSegmentWriter> completePromise = new Promise<BKLogSegmentWriter>();
+        final CompletableFuture<BKLogSegmentWriter> completePromise = new CompletableFuture<BKLogSegmentWriter>();
         // complete the old log segment
         writeHandler.completeAndCloseLogSegment(oldSegmentWriter)
-                .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+                .whenComplete(new FutureEventListener<LogSegmentMetadata>() {
 
                     @Override
                     public void onSuccess(LogSegmentMetadata value) {
                         cacheLogWriter(newSegmentWriter);
                         removeAllocatedLogWriter();
-                        FutureUtils.setValue(completePromise, newSegmentWriter);
+                        FutureUtils.complete(completePromise, newSegmentWriter);
                     }
 
                     @Override
                     public void onFailure(Throwable cause) {
-                        FutureUtils.setException(completePromise, cause);
+                        FutureUtils.completeExceptionally(completePromise, cause);
                     }
                 });
         return completePromise;
     }
 
-    synchronized protected Future<BKLogSegmentWriter> rollLogSegmentIfNecessary(
+    synchronized protected CompletableFuture<BKLogSegmentWriter> rollLogSegmentIfNecessary(
             final BKLogSegmentWriter segmentWriter,
             long startTxId,
             boolean bestEffort,
@@ -500,18 +489,18 @@
         try {
             writeHandler = getWriteHandler();
         } catch (IOException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         }
-        Future<BKLogSegmentWriter> rollPromise;
+        CompletableFuture<BKLogSegmentWriter> rollPromise;
         if (null != segmentWriter && (writeHandler.shouldStartNewSegment(segmentWriter) || forceRolling)) {
             rollPromise = closeOldLogSegmentAndStartNewOneWithPermit(
                     segmentWriter, writeHandler, startTxId, bestEffort, allowMaxTxID);
         } else if (null == segmentWriter) {
             rollPromise = asyncStartNewLogSegment(writeHandler, startTxId, allowMaxTxID);
         } else {
-            rollPromise = Future.value(segmentWriter);
+            rollPromise = FutureUtils.value(segmentWriter);
         }
-        return rollPromise.map(new AbstractFunction1<BKLogSegmentWriter, BKLogSegmentWriter>() {
+        return rollPromise.thenApply(new Function<BKLogSegmentWriter, BKLogSegmentWriter>() {
             @Override
             public BKLogSegmentWriter apply(BKLogSegmentWriter newSegmentWriter) {
                 if (segmentWriter == newSegmentWriter) {
@@ -542,7 +531,7 @@
 
     protected synchronized void cancelTruncation() {
         if (null != lastTruncationAttempt) {
-            FutureUtils.cancel(lastTruncationAttempt);
+            lastTruncationAttempt.cancel(true);
             lastTruncationAttempt = null;
         }
     }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
index eedfbd6..26a4a76 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
@@ -21,6 +21,20 @@
 import com.google.common.base.Optional;
 import com.google.common.base.Stopwatch;
 import com.google.common.base.Ticker;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+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.exceptions.DLIllegalStateException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.exceptions.EndOfStreamException;
@@ -28,32 +42,12 @@
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.apache.distributedlog.exceptions.ReadCancelledException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Throw;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.versioning.Versioned;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Function1;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
 
 /**
  * BookKeeper based {@link AsyncLogReader} implementation.
@@ -76,13 +70,8 @@
 class BKAsyncLogReader implements AsyncLogReader, Runnable, AsyncNotification {
     static final Logger LOG = LoggerFactory.getLogger(BKAsyncLogReader.class);
 
-    private static final Function1<List<LogRecordWithDLSN>, LogRecordWithDLSN> READ_NEXT_MAP_FUNCTION =
-            new AbstractFunction1<List<LogRecordWithDLSN>, LogRecordWithDLSN>() {
-                @Override
-                public LogRecordWithDLSN apply(List<LogRecordWithDLSN> records) {
-                    return records.get(0);
-                }
-            };
+    private static final Function<List<LogRecordWithDLSN>, LogRecordWithDLSN> READ_NEXT_MAP_FUNCTION =
+        records -> records.get(0);
 
     private final String streamName;
     protected final BKDistributedLogManager bkDistributedLogManager;
@@ -104,7 +93,7 @@
     // last process time
     private final Stopwatch lastProcessTime;
 
-    protected Promise<Void> closeFuture = null;
+    protected CompletableFuture<Void> closeFuture = null;
 
     private boolean lockStream = false;
 
@@ -143,7 +132,7 @@
         private final Stopwatch enqueueTime;
         private final int numEntries;
         private final List<LogRecordWithDLSN> records;
-        private final Promise<List<LogRecordWithDLSN>> promise;
+        private final CompletableFuture<List<LogRecordWithDLSN>> promise;
         private final long deadlineTime;
         private final TimeUnit deadlineTimeUnit;
 
@@ -158,12 +147,12 @@
             } else {
                 this.records = new ArrayList<LogRecordWithDLSN>();
             }
-            this.promise = new Promise<List<LogRecordWithDLSN>>();
+            this.promise = new CompletableFuture<List<LogRecordWithDLSN>>();
             this.deadlineTime = deadlineTime;
             this.deadlineTimeUnit = deadlineTimeUnit;
         }
 
-        Promise<List<LogRecordWithDLSN>> getPromise() {
+        CompletableFuture<List<LogRecordWithDLSN>> getPromise() {
             return promise;
         }
 
@@ -171,9 +160,9 @@
             return enqueueTime.elapsed(timeUnit);
         }
 
-        void setException(Throwable throwable) {
+        void completeExceptionally(Throwable throwable) {
             Stopwatch stopwatch = Stopwatch.createStarted();
-            if (promise.updateIfEmpty(new Throw<List<LogRecordWithDLSN>>(throwable))) {
+            if (promise.completeExceptionally(throwable)) {
                 futureSetLatency.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
                 delayUntilPromiseSatisfied.registerFailedEvent(enqueueTime.elapsed(TimeUnit.MICROSECONDS));
             }
@@ -204,7 +193,7 @@
             }
             delayUntilPromiseSatisfied.registerSuccessfulEvent(enqueueTime.stop().elapsed(TimeUnit.MICROSECONDS));
             Stopwatch stopwatch = Stopwatch.createStarted();
-            promise.setValue(records);
+            promise.complete(records);
             futureSetLatency.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
         }
     }
@@ -333,7 +322,7 @@
         return startDLSN;
     }
 
-    public Future<Void> lockStream() {
+    public CompletableFuture<Void> lockStream() {
         this.lockStream = true;
         return readHandler.lockStream();
     }
@@ -381,16 +370,16 @@
      * @return A promise that when satisfied will contain the Log Record with its DLSN.
      */
     @Override
-    public synchronized Future<LogRecordWithDLSN> readNext() {
-        return readInternal(1, 0, TimeUnit.MILLISECONDS).map(READ_NEXT_MAP_FUNCTION);
+    public synchronized CompletableFuture<LogRecordWithDLSN> readNext() {
+        return readInternal(1, 0, TimeUnit.MILLISECONDS).thenApply(READ_NEXT_MAP_FUNCTION);
     }
 
-    public synchronized Future<List<LogRecordWithDLSN>> readBulk(int numEntries) {
+    public synchronized CompletableFuture<List<LogRecordWithDLSN>> readBulk(int numEntries) {
         return readInternal(numEntries, 0, TimeUnit.MILLISECONDS);
     }
 
     @Override
-    public synchronized Future<List<LogRecordWithDLSN>> readBulk(int numEntries,
+    public synchronized CompletableFuture<List<LogRecordWithDLSN>> readBulk(int numEntries,
                                                                  long waitTime,
                                                                  TimeUnit timeUnit) {
         return readInternal(numEntries, waitTime, timeUnit);
@@ -404,7 +393,7 @@
      *          num entries to read
      * @return A promise that satisfied with a non-empty list of log records with their DLSN.
      */
-    private synchronized Future<List<LogRecordWithDLSN>> readInternal(int numEntries,
+    private synchronized CompletableFuture<List<LogRecordWithDLSN>> readInternal(int numEntries,
                                                                       long deadlineTime,
                                                                       TimeUnit deadlineTimeUnit) {
         timeBetweenReadNexts.registerSuccessfulEvent(readNextDelayStopwatch.elapsed(TimeUnit.MICROSECONDS));
@@ -421,19 +410,15 @@
                     bkDistributedLogManager.getScheduler(),
                     Ticker.systemTicker(),
                     bkDistributedLogManager.alertStatsLogger);
-            readHandler.checkLogStreamExists().addEventListener(new FutureEventListener<Void>() {
+            readHandler.checkLogStreamExists().whenComplete(new FutureEventListener<Void>() {
                 @Override
                 public void onSuccess(Void value) {
                     try {
                         readHandler.registerListener(readAheadEntryReader);
                         readHandler.asyncStartFetchLogSegments()
-                                .map(new AbstractFunction1<Versioned<List<LogSegmentMetadata>>, BoxedUnit>() {
-                                    @Override
-                                    public BoxedUnit apply(Versioned<List<LogSegmentMetadata>> logSegments) {
-                                        readAheadEntryReader.addStateChangeNotification(BKAsyncLogReader.this);
-                                        readAheadEntryReader.start(logSegments.getValue());
-                                        return BoxedUnit.UNIT;
-                                    }
+                                .thenAccept(logSegments -> {
+                                    readAheadEntryReader.addStateChangeNotification(BKAsyncLogReader.this);
+                                    readAheadEntryReader.start(logSegments.getValue());
                                 });
                     } catch (Exception exc) {
                         notifyOnError(exc);
@@ -448,7 +433,7 @@
         }
 
         if (checkClosedOrInError("readNext")) {
-            readRequest.setException(lastException.get());
+            readRequest.completeExceptionally(lastException.get());
         } else {
             boolean queueEmpty = pendingRequests.isEmpty();
             pendingRequests.add(readRequest);
@@ -478,15 +463,15 @@
     }
 
     @Override
-    public Future<Void> asyncClose() {
+    public CompletableFuture<Void> asyncClose() {
         // Cancel the idle reader timeout task, interrupting if necessary
         ReadCancelledException exception;
-        Promise<Void> closePromise;
+        CompletableFuture<Void> closePromise;
         synchronized (this) {
             if (null != closeFuture) {
                 return closeFuture;
             }
-            closePromise = closeFuture = new Promise<Void>();
+            closePromise = closeFuture = new CompletableFuture<Void>();
             exception = new ReadCancelledException(readHandler.getFullyQualifiedName(), "Reader was closed");
             setLastException(exception);
         }
@@ -507,16 +492,18 @@
             readHandler.unregisterListener(readAheadReader);
             readAheadReader.removeStateChangeNotification(this);
         }
-        Utils.closeSequence(bkDistributedLogManager.getScheduler(), true,
-                readAheadReader,
-                readHandler
-        ).proxyTo(closePromise);
+        FutureUtils.proxyTo(
+            Utils.closeSequence(bkDistributedLogManager.getScheduler(), true,
+                    readAheadReader,
+                    readHandler
+            ),
+            closePromise);
         return closePromise;
     }
 
     private void cancelAllPendingReads(Throwable throwExc) {
         for (PendingReadRequest promise : pendingRequests) {
-            promise.setException(throwExc);
+            promise.completeExceptionally(throwExc);
         }
         pendingRequests.clear();
     }
@@ -591,7 +578,8 @@
                     }
 
                     if (disableProcessingReadRequests) {
-                        LOG.info("Reader of {} is forced to stop processing read requests", readHandler.getFullyQualifiedName());
+                        LOG.info("Reader of {} is forced to stop processing read requests",
+                            readHandler.getFullyQualifiedName());
                         return;
                     }
                 }
@@ -601,9 +589,9 @@
                 // the reader in error and abort all pending reads since we dont
                 // know the last consumed read
                 if (null == lastException.get()) {
-                    if (nextRequest.getPromise().isInterrupted().isDefined()) {
-                        setLastException(new DLInterruptedException("Interrupted on reading " + readHandler.getFullyQualifiedName() + " : ",
-                                nextRequest.getPromise().isInterrupted().get()));
+                    if (nextRequest.getPromise().isCancelled()) {
+                        setLastException(new DLInterruptedException("Interrupted on reading "
+                            + readHandler.getFullyQualifiedName()));
                     }
                 }
 
@@ -680,9 +668,9 @@
                     } else {
                         DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at dlsn = "
                                 + nextRequest.records.get(0).getDlsn());
-                        nextRequest.setException(ise);
+                        nextRequest.completeExceptionally(ise);
                         if (null != request) {
-                            request.setException(ise);
+                            request.completeExceptionally(ise);
                         }
                         // We should never get here as we should have exited the loop if
                         // pendingRequests were empty
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
index a1b1d5c..62b32f2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
@@ -19,33 +19,30 @@
 
 import com.google.common.base.Stopwatch;
 import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+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.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.StreamNotReadyException;
 import org.apache.distributedlog.exceptions.WriteCancelledException;
 import org.apache.distributedlog.exceptions.WriteException;
 import org.apache.distributedlog.feature.CoreFeatureKeys;
 import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Try;
-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.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Function1;
-import scala.Option;
-import scala.runtime.AbstractFunction1;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
 
 /**
  * BookKeeper based {@link AsyncLogWriter} implementation.
@@ -70,35 +67,30 @@
 
     static final Logger LOG = LoggerFactory.getLogger(BKAsyncLogWriter.class);
 
-    static Function1<List<LogSegmentMetadata>, Boolean> TruncationResultConverter =
-            new AbstractFunction1<List<LogSegmentMetadata>, Boolean>() {
-                @Override
-                public Boolean apply(List<LogSegmentMetadata> segments) {
-                    return true;
-                }
-            };
+    static Function<List<LogSegmentMetadata>, Boolean> TruncationResultConverter =
+        segments -> true;
 
     // Records pending for roll log segment.
     class PendingLogRecord implements FutureEventListener<DLSN> {
 
         final LogRecord record;
-        final Promise<DLSN> promise;
+        final CompletableFuture<DLSN> promise;
         final boolean flush;
 
         PendingLogRecord(LogRecord record, boolean flush) {
             this.record = record;
-            this.promise = new Promise<DLSN>();
+            this.promise = new CompletableFuture<DLSN>();
             this.flush = flush;
         }
 
         @Override
         public void onSuccess(DLSN value) {
-            promise.setValue(value);
+            promise.complete(value);
         }
 
         @Override
         public void onFailure(Throwable cause) {
-            promise.setException(cause);
+            promise.completeExceptionally(cause);
             encounteredError = true;
         }
     }
@@ -135,7 +127,7 @@
     private final boolean disableRollOnSegmentError;
     private LinkedList<PendingLogRecord> pendingRequests = null;
     private volatile boolean encounteredError = false;
-    private Promise<BKLogSegmentWriter> rollingFuture = null;
+    private CompletableFuture<BKLogSegmentWriter> rollingFuture = null;
     private long lastTxId = DistributedLogConstants.INVALID_TXID;
 
     private final StatsLogger statsLogger;
@@ -186,7 +178,7 @@
      *          log record
      * @return future of the write
      */
-    public Future<DLSN> writeControlRecord(final LogRecord record) {
+    public CompletableFuture<DLSN> writeControlRecord(final LogRecord record) {
         record.setControl();
         return write(record);
     }
@@ -206,7 +198,7 @@
         }
     }
 
-    private Future<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
+    private CompletableFuture<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
                                                            boolean bestEffort,
                                                            boolean rollLog,
                                                            boolean allowMaxTxID) {
@@ -217,24 +209,20 @@
                 stopwatch);
     }
 
-    private Future<BKLogSegmentWriter> doGetLogSegmentWriter(final long firstTxid,
+    private CompletableFuture<BKLogSegmentWriter> doGetLogSegmentWriter(final long firstTxid,
                                                              final boolean bestEffort,
                                                              final boolean rollLog,
                                                              final boolean allowMaxTxID) {
         if (encounteredError) {
-            return Future.exception(new WriteException(bkDistributedLogManager.getStreamName(),
+            return FutureUtils.exception(new WriteException(bkDistributedLogManager.getStreamName(),
                     "writer has been closed due to error."));
         }
-        Future<BKLogSegmentWriter> writerFuture = asyncGetLedgerWriter(!disableRollOnSegmentError);
+        CompletableFuture<BKLogSegmentWriter> writerFuture = asyncGetLedgerWriter(!disableRollOnSegmentError);
         if (null == writerFuture) {
             return rollLogSegmentIfNecessary(null, firstTxid, bestEffort, allowMaxTxID);
         } else if (rollLog) {
-            return writerFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<BKLogSegmentWriter>>() {
-                @Override
-                public Future<BKLogSegmentWriter> apply(BKLogSegmentWriter writer) {
-                    return rollLogSegmentIfNecessary(writer, firstTxid, bestEffort, allowMaxTxID);
-                }
-            });
+            return writerFuture.thenCompose(
+                writer -> rollLogSegmentIfNecessary(writer, firstTxid, bestEffort, allowMaxTxID));
         } else {
             return writerFuture;
         }
@@ -244,20 +232,20 @@
      * We write end of stream marker by writing a record with MAX_TXID, so we need to allow using
      * max txid when rolling for this case only.
      */
-    private Future<BKLogSegmentWriter> getLogSegmentWriterForEndOfStream() {
+    private CompletableFuture<BKLogSegmentWriter> getLogSegmentWriterForEndOfStream() {
         return getLogSegmentWriter(DistributedLogConstants.MAX_TXID,
                                      false /* bestEffort */,
                                      false /* roll log */,
                                      true /* allow max txid */);
     }
 
-    private Future<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
+    private CompletableFuture<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
                                                            boolean bestEffort,
                                                            boolean rollLog) {
         return getLogSegmentWriter(firstTxid, bestEffort, rollLog, false /* allow max txid */);
     }
 
-    Future<DLSN> queueRequest(LogRecord record, boolean flush) {
+    CompletableFuture<DLSN> queueRequest(LogRecord record, boolean flush) {
         PendingLogRecord pendingLogRecord = new PendingLogRecord(record, flush);
         pendingRequests.add(pendingLogRecord);
         return pendingLogRecord.promise;
@@ -276,25 +264,25 @@
     void startQueueingRequests() {
         assert(null == pendingRequests && null == rollingFuture);
         pendingRequests = new LinkedList<PendingLogRecord>();
-        rollingFuture = new Promise<BKLogSegmentWriter>();
+        rollingFuture = new CompletableFuture<BKLogSegmentWriter>();
     }
 
     // for ordering guarantee, we shouldn't send requests to next log segments until
     // previous log segment is done.
-    private synchronized Future<DLSN> asyncWrite(final LogRecord record,
+    private synchronized CompletableFuture<DLSN> asyncWrite(final LogRecord record,
                                                  boolean flush) {
         // The passed in writer may be stale since we acquire the writer outside of sync
         // lock. If we recently rolled and the new writer is cached, use that instead.
-        Future<DLSN> result = null;
+        CompletableFuture<DLSN> result = null;
         BKLogSegmentWriter w;
         try {
             w = getCachedLogSegmentWriter();
         } catch (WriteException we) {
-            return Future.exception(we);
+            return FutureUtils.exception(we);
         }
         if (null != rollingFuture) {
             if (streamFailFast) {
-                result = Future.exception(new StreamNotReadyException("Rolling log segment"));
+                result = FutureUtils.exception(new StreamNotReadyException("Rolling log segment"));
             } else {
                 result = queueRequest(record, flush);
             }
@@ -303,7 +291,7 @@
             startQueueingRequests();
             if (null != w) {
                 LastPendingLogRecord lastLogRecordInCurrentSegment = new LastPendingLogRecord(record, flush);
-                w.asyncWrite(record, true).addEventListener(lastLogRecordInCurrentSegment);
+                w.asyncWrite(record, true).whenComplete(lastLogRecordInCurrentSegment);
                 result = lastLogRecordInCurrentSegment.promise;
             } else { // no log segment yet. roll the log segment and issue pending requests.
                 result = queueRequest(record, flush);
@@ -314,26 +302,22 @@
         }
         // use map here rather than onSuccess because we want lastTxId to be updated before
         // satisfying the future
-        return result.map(new AbstractFunction1<DLSN, DLSN>() {
-            @Override
-            public DLSN apply(DLSN dlsn) {
-                setLastTxId(record.getTransactionId());
-                return dlsn;
-            }
+        return result.thenApply(dlsn -> {
+            setLastTxId(record.getTransactionId());
+            return dlsn;
         });
     }
 
-    private List<Future<DLSN>> asyncWriteBulk(List<LogRecord> records) {
-        final ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(records.size());
+    private List<CompletableFuture<DLSN>> asyncWriteBulk(List<LogRecord> records) {
+        final ArrayList<CompletableFuture<DLSN>> results = new ArrayList<CompletableFuture<DLSN>>(records.size());
         Iterator<LogRecord> iterator = records.iterator();
         while (iterator.hasNext()) {
             LogRecord record = iterator.next();
-            Future<DLSN> future = asyncWrite(record, !iterator.hasNext());
+            CompletableFuture<DLSN> future = asyncWrite(record, !iterator.hasNext());
             results.add(future);
 
             // Abort early if an individual write has already failed.
-            Option<Try<DLSN>> result = future.poll();
-            if (result.isDefined() && result.get().isThrow()) {
+            if (future.isDone() && future.isCompletedExceptionally()) {
                 break;
             }
         }
@@ -343,18 +327,18 @@
         return results;
     }
 
-    private void appendCancelledFutures(List<Future<DLSN>> futures, int numToAdd) {
+    private void appendCancelledFutures(List<CompletableFuture<DLSN>> futures, int numToAdd) {
         final WriteCancelledException cre =
             new WriteCancelledException(getStreamName());
         for (int i = 0; i < numToAdd; i++) {
-            Future<DLSN> cancelledFuture = Future.exception(cre);
+            CompletableFuture<DLSN> cancelledFuture = FutureUtils.exception(cre);
             futures.add(cancelledFuture);
         }
     }
 
     private void rollLogSegmentAndIssuePendingRequests(final long firstTxId) {
         getLogSegmentWriter(firstTxId, true, true)
-                .addEventListener(new FutureEventListener<BKLogSegmentWriter>() {
+                .whenComplete(new FutureEventListener<BKLogSegmentWriter>() {
             @Override
             public void onSuccess(BKLogSegmentWriter writer) {
                 try {
@@ -362,7 +346,7 @@
                         for (PendingLogRecord pendingLogRecord : pendingRequests) {
                             FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_LogWriterIssuePending);
                             writer.asyncWrite(pendingLogRecord.record, pendingLogRecord.flush)
-                                    .addEventListener(pendingLogRecord);
+                                    .whenComplete(pendingLogRecord);
                         }
                         // if there are no records in the pending queue, let's write a control record
                         // so that when a new log segment is rolled, a control record will be added and
@@ -373,10 +357,10 @@
                             controlRecord.setControl();
                             PendingLogRecord controlReq = new PendingLogRecord(controlRecord, false);
                             writer.asyncWrite(controlReq.record, controlReq.flush)
-                                    .addEventListener(controlReq);
+                                    .whenComplete(controlReq);
                         }
                         if (null != rollingFuture) {
-                            FutureUtils.setValue(rollingFuture, writer);
+                            FutureUtils.complete(rollingFuture, writer);
                         }
                         rollingFuture = null;
                         pendingRequestDispatch.add(pendingRequests.size());
@@ -401,7 +385,7 @@
             encounteredError = errorOutWriter;
             pendingRequests = null;
             if (null != rollingFuture) {
-                FutureUtils.setException(rollingFuture, cause);
+                FutureUtils.completeExceptionally(rollingFuture, cause);
             }
             rollingFuture = null;
         }
@@ -411,7 +395,7 @@
         // After erroring out the writer above, no more requests
         // will be enqueued to pendingRequests
         for (PendingLogRecord pendingLogRecord : pendingRequestsSnapshot) {
-            pendingLogRecord.promise.setException(cause);
+            pendingLogRecord.promise.completeExceptionally(cause);
         }
     }
 
@@ -425,7 +409,7 @@
      * @param record single log record
      */
     @Override
-    public Future<DLSN> write(final LogRecord record) {
+    public CompletableFuture<DLSN> write(final LogRecord record) {
         final Stopwatch stopwatch = Stopwatch.createStarted();
         return FutureUtils.stats(
                 asyncWrite(record, true),
@@ -442,30 +426,30 @@
      * @param records list of records
      */
     @Override
-    public Future<List<Future<DLSN>>> writeBulk(final List<LogRecord> records) {
+    public CompletableFuture<List<CompletableFuture<DLSN>>> writeBulk(final List<LogRecord> records) {
         final Stopwatch stopwatch = Stopwatch.createStarted();
         return FutureUtils.stats(
-                Future.value(asyncWriteBulk(records)),
+                FutureUtils.value(asyncWriteBulk(records)),
                 bulkWriteOpStatsLogger,
                 stopwatch);
     }
 
     @Override
-    public Future<Boolean> truncate(final DLSN dlsn) {
+    public CompletableFuture<Boolean> truncate(final DLSN dlsn) {
         if (DLSN.InvalidDLSN == dlsn) {
-            return Future.value(false);
+            return FutureUtils.value(false);
         }
         BKLogWriteHandler writeHandler;
         try {
             writeHandler = getWriteHandler();
         } catch (IOException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         }
-        return writeHandler.setLogSegmentsOlderThanDLSNTruncated(dlsn).map(TruncationResultConverter);
+        return writeHandler.setLogSegmentsOlderThanDLSNTruncated(dlsn).thenApply(TruncationResultConverter);
     }
 
-    Future<Long> flushAndCommit() {
-        Future<BKLogSegmentWriter> writerFuture;
+    CompletableFuture<Long> flushAndCommit() {
+        CompletableFuture<BKLogSegmentWriter> writerFuture;
         synchronized (this) {
             if (null != this.rollingFuture) {
                 writerFuture = this.rollingFuture;
@@ -474,19 +458,14 @@
             }
         }
         if (null == writerFuture) {
-            return Future.value(getLastTxId());
+            return FutureUtils.value(getLastTxId());
         }
-        return writerFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Long>>() {
-            @Override
-            public Future<Long> apply(BKLogSegmentWriter writer) {
-                return writer.flushAndCommit();
-            }
-        });
+        return writerFuture.thenCompose(writer -> writer.flushAndCommit());
     }
 
-    Future<Long> markEndOfStream() {
+    CompletableFuture<Long> markEndOfStream() {
         final Stopwatch stopwatch = Stopwatch.createStarted();
-        Future<BKLogSegmentWriter> logSegmentWriterFuture;
+        CompletableFuture<BKLogSegmentWriter> logSegmentWriterFuture;
         synchronized (this) {
             logSegmentWriterFuture = this.rollingFuture;
         }
@@ -495,19 +474,14 @@
         }
 
         return FutureUtils.stats(
-                logSegmentWriterFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Long>>() {
-                    @Override
-                    public Future<Long> apply(BKLogSegmentWriter w) {
-                        return w.markEndOfStream();
-                    }
-                }),
+                logSegmentWriterFuture.thenCompose(w -> w.markEndOfStream()),
                 markEndOfStreamOpStatsLogger,
                 stopwatch);
     }
 
     @Override
-    protected Future<Void> asyncCloseAndComplete() {
-        Future<BKLogSegmentWriter> logSegmentWriterFuture;
+    protected CompletableFuture<Void> asyncCloseAndComplete() {
+        CompletableFuture<BKLogSegmentWriter> logSegmentWriterFuture;
         synchronized (this) {
             logSegmentWriterFuture = this.rollingFuture;
         }
@@ -515,18 +489,13 @@
         if (null == logSegmentWriterFuture) {
             return super.asyncCloseAndComplete();
         } else {
-            return logSegmentWriterFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Void>>() {
-                @Override
-                public Future<Void> apply(BKLogSegmentWriter segmentWriter) {
-                    return BKAsyncLogWriter.super.asyncCloseAndComplete();
-                }
-            });
+            return logSegmentWriterFuture.thenCompose(segmentWriter1 -> super.asyncCloseAndComplete());
         }
     }
 
     @Override
     void closeAndComplete() throws IOException {
-        FutureUtils.result(asyncCloseAndComplete());
+        Utils.ioResult(asyncCloseAndComplete());
     }
 
     /**
@@ -539,12 +508,12 @@
     }
 
     @Override
-    public Future<Void> asyncAbort() {
-        Future<Void> result = super.asyncAbort();
+    public CompletableFuture<Void> asyncAbort() {
+        CompletableFuture<Void> result = super.asyncAbort();
         synchronized (this) {
             if (pendingRequests != null) {
                 for (PendingLogRecord pendingLogRecord : pendingRequests) {
-                    pendingLogRecord.promise.setException(new WriteException(bkDistributedLogManager.getStreamName(),
+                    pendingLogRecord.promise.completeExceptionally(new WriteException(bkDistributedLogManager.getStreamName(),
                             "abort wring: writer has been closed due to error."));
                 }
             }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
index 00e6b5c..cffe500 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
@@ -20,14 +20,22 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.AlertStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.callback.LogSegmentListener;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.LogEmptyException;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.function.CloseAsyncCloseableFunction;
-import org.apache.distributedlog.function.GetVersionedValueFunction;
 import org.apache.distributedlog.injector.AsyncFailureInjector;
 import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
 import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
@@ -41,40 +49,25 @@
 import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
 import org.apache.distributedlog.metadata.LogStreamMetadataStore;
 import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.stats.BroadCastStatsLogger;
-import org.apache.distributedlog.subscription.SubscriptionsStore;
+import org.apache.distributedlog.common.stats.BroadCastStatsLogger;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
 import org.apache.distributedlog.util.Allocator;
 import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.util.FutureUtils;
-import org.apache.distributedlog.util.MonitoredFuturePool;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.PermitLimiter;
-import org.apache.distributedlog.util.PermitManager;
-import org.apache.distributedlog.util.SchedulerUtils;
+import org.apache.distributedlog.common.util.PermitLimiter;
+import org.apache.distributedlog.common.util.PermitManager;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.ExceptionalFunction;
-import com.twitter.util.ExceptionalFunction0;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.net.URI;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
 
 import static org.apache.distributedlog.namespace.NamespaceDriver.Role.READER;
 import static org.apache.distributedlog.namespace.NamespaceDriver.Role.WRITER;
@@ -104,20 +97,10 @@
     static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogManager.class);
 
     static final Function<LogRecordWithDLSN, Long> RECORD_2_TXID_FUNCTION =
-            new Function<LogRecordWithDLSN, Long>() {
-                @Override
-                public Long apply(LogRecordWithDLSN record) {
-                    return record.getTransactionId();
-                }
-            };
+        record -> record.getTransactionId();
 
     static final Function<LogRecordWithDLSN, DLSN> RECORD_2_DLSN_FUNCTION =
-            new Function<LogRecordWithDLSN, DLSN>() {
-                @Override
-                public DLSN apply(LogRecordWithDLSN record) {
-                    return record.getDlsn();
-                }
-            };
+        record -> record.getDlsn();
 
     private final URI uri;
     private final String name;
@@ -127,7 +110,7 @@
     private final DistributedLogConfiguration conf;
     private final DynamicDistributedLogConfiguration dynConf;
     private final NamespaceDriver driver;
-    private Promise<Void> closePromise;
+    private CompletableFuture<Void> closePromise;
     private final OrderedScheduler scheduler;
     private final FeatureProvider featureProvider;
     private final AsyncFailureInjector failureInjector;
@@ -272,17 +255,18 @@
 
     @Override
     public List<LogSegmentMetadata> getLogSegments() throws IOException {
-        return FutureUtils.result(getLogSegmentsAsync());
+        return Utils.ioResult(getLogSegmentsAsync());
     }
 
-    protected Future<List<LogSegmentMetadata>> getLogSegmentsAsync() {
+    protected CompletableFuture<List<LogSegmentMetadata>> getLogSegmentsAsync() {
         final BKLogReadHandler readHandler = createReadHandler();
         return readHandler.readLogSegmentsFromStore(
                 LogSegmentMetadata.COMPARATOR,
                 LogSegmentFilter.DEFAULT_FILTER,
-                null)
-                .map(GetVersionedValueFunction.GET_LOGSEGMENT_LIST_FUNC)
-                .ensure(CloseAsyncCloseableFunction.of(readHandler));
+                null
+        )
+        .thenApply((versionedList) -> versionedList.getValue())
+        .whenComplete((value, cause) -> readHandler.asyncClose());
     }
 
     @Override
@@ -353,29 +337,26 @@
 
     public BKLogWriteHandler createWriteHandler(boolean lockHandler)
             throws IOException {
-        return FutureUtils.result(asyncCreateWriteHandler(lockHandler));
+        return Utils.ioResult(asyncCreateWriteHandler(lockHandler));
     }
 
-    Future<BKLogWriteHandler> asyncCreateWriteHandler(final boolean lockHandler) {
+    CompletableFuture<BKLogWriteHandler> asyncCreateWriteHandler(final boolean lockHandler) {
         // Fetching Log Metadata (create if not exists)
         return driver.getLogStreamMetadataStore(WRITER).getLog(
                 uri,
                 name,
                 true,
                 conf.getCreateStreamIfNotExists()
-        ).flatMap(new AbstractFunction1<LogMetadataForWriter, Future<BKLogWriteHandler>>() {
-            @Override
-            public Future<BKLogWriteHandler> apply(LogMetadataForWriter logMetadata) {
-                Promise<BKLogWriteHandler> createPromise = new Promise<BKLogWriteHandler>();
-                createWriteHandler(logMetadata, lockHandler, createPromise);
-                return createPromise;
-            }
+        ).thenCompose(logMetadata -> {
+            CompletableFuture<BKLogWriteHandler> createPromise = new CompletableFuture<BKLogWriteHandler>();
+            createWriteHandler(logMetadata, lockHandler, createPromise);
+            return createPromise;
         });
     }
 
     private void createWriteHandler(LogMetadataForWriter logMetadata,
                                     boolean lockHandler,
-                                    final Promise<BKLogWriteHandler> createPromise) {
+                                    final CompletableFuture<BKLogWriteHandler> createPromise) {
         // Build the locks
         DistributedLock lock;
         if (conf.isWriteLockEnabled()) {
@@ -389,7 +370,7 @@
             segmentAllocator = driver.getLogSegmentEntryStore(WRITER)
                     .newLogSegmentAllocator(logMetadata, dynConf);
         } catch (IOException ioe) {
-            FutureUtils.setException(createPromise, ioe);
+            FutureUtils.completeExceptionally(createPromise, ioe);
             return;
         }
 
@@ -412,25 +393,21 @@
                 dynConf,
                 lock);
         if (lockHandler) {
-            writeHandler.lockHandler().addEventListener(new FutureEventListener<DistributedLock>() {
+            writeHandler.lockHandler().whenComplete(new FutureEventListener<DistributedLock>() {
                 @Override
                 public void onSuccess(DistributedLock lock) {
-                    FutureUtils.setValue(createPromise, writeHandler);
+                    FutureUtils.complete(createPromise, writeHandler);
                 }
 
                 @Override
                 public void onFailure(final Throwable cause) {
-                    writeHandler.asyncClose().ensure(new AbstractFunction0<BoxedUnit>() {
-                        @Override
-                        public BoxedUnit apply() {
-                            FutureUtils.setException(createPromise, cause);
-                            return BoxedUnit.UNIT;
-                        }
-                    });
+                    FutureUtils.ensure(
+                        writeHandler.asyncClose(),
+                        () -> FutureUtils.completeExceptionally(createPromise, cause));
                 }
             });
         } else {
-            FutureUtils.setValue(createPromise, writeHandler);
+            FutureUtils.complete(createPromise, writeHandler);
         }
     }
 
@@ -438,18 +415,15 @@
         return driver.getLogStreamMetadataStore(WRITER).getPermitManager();
     }
 
-    <T> Future<T> processReaderOperation(final Function<BKLogReadHandler, Future<T>> func) {
-        return scheduler.apply(new ExceptionalFunction0<BKLogReadHandler>() {
-            @Override
-            public BKLogReadHandler applyE() throws Throwable {
-                return getReadHandlerAndRegisterListener(true, null);
-            }
-        }).flatMap(new ExceptionalFunction<BKLogReadHandler, Future<T>>() {
-            @Override
-            public Future<T> applyE(final BKLogReadHandler readHandler) throws Throwable {
-                return func.apply(readHandler);
-            }
+    <T> CompletableFuture<T> processReaderOperation(final Function<BKLogReadHandler, CompletableFuture<T>> func) {
+        CompletableFuture<T> future = FutureUtils.createFuture();
+        scheduler.submit(() -> {
+            BKLogReadHandler readHandler = getReadHandlerAndRegisterListener(true, null);
+            FutureUtils.proxyTo(
+                func.apply(readHandler),
+                future);
         });
+        return future;
     }
 
     /**
@@ -461,7 +435,7 @@
     @Override
     public boolean isEndOfStreamMarked() throws IOException {
         checkClosedOrInError("isEndOfStreamMarked");
-        long lastTxId = FutureUtils.result(getLastLogRecordAsyncInternal(false, true)).getTransactionId();
+        long lastTxId = Utils.ioResult(getLastLogRecordAsyncInternal(false, true)).getTransactionId();
         return lastTxId == DistributedLogConstants.MAX_TXID;
     }
 
@@ -473,7 +447,7 @@
     public AppendOnlyStreamWriter getAppendOnlyStreamWriter() throws IOException {
         long position;
         try {
-            position = FutureUtils.result(getLastLogRecordAsyncInternal(true, false)).getTransactionId();
+            position = Utils.ioResult(getLastLogRecordAsyncInternal(true, false)).getTransactionId();
             if (DistributedLogConstants.INVALID_TXID == position ||
                 DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID == position) {
                 position = 0;
@@ -508,7 +482,7 @@
         try {
             writer.createAndCacheWriteHandler();
             BKLogWriteHandler writeHandler = writer.getWriteHandler();
-            FutureUtils.result(writeHandler.lockHandler());
+            Utils.ioResult(writeHandler.lockHandler());
             success = true;
             return writer;
         } finally {
@@ -525,75 +499,63 @@
      */
     @Override
     public BKAsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException {
-        return (BKAsyncLogWriter) FutureUtils.result(openAsyncLogWriter());
+        return (BKAsyncLogWriter) Utils.ioResult(openAsyncLogWriter());
     }
 
     @Override
-    public Future<AsyncLogWriter> openAsyncLogWriter() {
+    public CompletableFuture<AsyncLogWriter> openAsyncLogWriter() {
         try {
             checkClosedOrInError("startLogSegmentNonPartitioned");
         } catch (AlreadyClosedException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         }
 
-        Future<BKLogWriteHandler> createWriteHandleFuture;
+        CompletableFuture<BKLogWriteHandler> createWriteHandleFuture;
         synchronized (this) {
             // 1. create the locked write handler
             createWriteHandleFuture = asyncCreateWriteHandler(true);
         }
-        return createWriteHandleFuture.flatMap(new AbstractFunction1<BKLogWriteHandler, Future<AsyncLogWriter>>() {
-            @Override
-            public Future<AsyncLogWriter> apply(final BKLogWriteHandler writeHandler) {
-                final BKAsyncLogWriter writer;
-                synchronized (BKDistributedLogManager.this) {
-                    // 2. create the writer with the handler
-                    writer = new BKAsyncLogWriter(
-                            conf,
-                            dynConf,
-                            BKDistributedLogManager.this,
-                            writeHandler,
-                            featureProvider,
-                            statsLogger);
-                }
-                // 3. recover the incomplete log segments
-                return writeHandler.recoverIncompleteLogSegments()
-                        .map(new AbstractFunction1<Long, AsyncLogWriter>() {
-                            @Override
-                            public AsyncLogWriter apply(Long lastTxId) {
-                                // 4. update last tx id if successfully recovered
-                                writer.setLastTxId(lastTxId);
-                                return writer;
-                            }
-                        }).onFailure(new AbstractFunction1<Throwable, BoxedUnit>() {
-                            @Override
-                            public BoxedUnit apply(Throwable cause) {
-                                // 5. close the writer if recovery failed
-                                writer.asyncAbort();
-                                return BoxedUnit.UNIT;
-                            }
-                        });
+        return createWriteHandleFuture.thenCompose(writeHandler -> {
+            final BKAsyncLogWriter writer;
+            synchronized (BKDistributedLogManager.this) {
+                // 2. create the writer with the handler
+                writer = new BKAsyncLogWriter(
+                        conf,
+                        dynConf,
+                        BKDistributedLogManager.this,
+                        writeHandler,
+                        featureProvider,
+                        statsLogger);
             }
+            // 3. recover the incomplete log segments
+            return writeHandler.recoverIncompleteLogSegments()
+                .thenApply(lastTxId -> {
+                    // 4. update last tx id if successfully recovered
+                    writer.setLastTxId(lastTxId);
+                    return (AsyncLogWriter) writer;
+                })
+                .whenComplete((lastTxId, cause) -> {
+                    if (null != cause) {
+                        // 5. close the writer if recovery failed
+                        writer.asyncAbort();
+                    }
+                });
         });
     }
 
     @Override
-    public Future<DLSN> getDLSNNotLessThanTxId(final long fromTxnId) {
-        return getLogSegmentsAsync().flatMap(new AbstractFunction1<List<LogSegmentMetadata>, Future<DLSN>>() {
-            @Override
-            public Future<DLSN> apply(List<LogSegmentMetadata> segments) {
-                return getDLSNNotLessThanTxId(fromTxnId, segments);
-            }
-        });
+    public CompletableFuture<DLSN> getDLSNNotLessThanTxId(final long fromTxnId) {
+        return getLogSegmentsAsync().thenCompose(segments -> getDLSNNotLessThanTxId(fromTxnId, segments));
     }
 
-    private Future<DLSN> getDLSNNotLessThanTxId(long fromTxnId,
+    private CompletableFuture<DLSN> getDLSNNotLessThanTxId(long fromTxnId,
                                                 final List<LogSegmentMetadata> segments) {
         if (segments.isEmpty()) {
             return getLastDLSNAsync();
         }
         final int segmentIdx = DLUtils.findLogSegmentNotLessThanTxnId(segments, fromTxnId);
         if (segmentIdx < 0) {
-            return Future.value(new DLSN(segments.get(0).getLogSegmentSequenceNumber(), 0L, 0L));
+            return FutureUtils.value(new DLSN(segments.get(0).getLogSegmentSequenceNumber(), 0L, 0L));
         }
         return getDLSNNotLessThanTxIdInSegment(
                 fromTxnId,
@@ -603,7 +565,7 @@
         );
     }
 
-    private Future<DLSN> getDLSNNotLessThanTxIdInSegment(final long fromTxnId,
+    private CompletableFuture<DLSN> getDLSNNotLessThanTxIdInSegment(final long fromTxnId,
                                                          final int segmentIdx,
                                                          final List<LogSegmentMetadata> segments,
                                                          final LogSegmentEntryStore entryStore) {
@@ -615,29 +577,23 @@
                 scheduler,
                 entryStore,
                 Math.max(2, dynConf.getReadAheadBatchSize())
-        ).flatMap(new AbstractFunction1<Optional<LogRecordWithDLSN>, Future<DLSN>>() {
-            @Override
-            public Future<DLSN> apply(Optional<LogRecordWithDLSN> foundRecord) {
-                if (foundRecord.isPresent()) {
-                    return Future.value(foundRecord.get().getDlsn());
-                }
-                if ((segments.size() - 1) == segmentIdx) {
-                    return getLastLogRecordAsync().map(new AbstractFunction1<LogRecordWithDLSN, DLSN>() {
-                        @Override
-                        public DLSN apply(LogRecordWithDLSN record) {
-                            if (record.getTransactionId() >= fromTxnId) {
-                                return record.getDlsn();
-                            }
-                            return record.getDlsn().getNextDLSN();
-                        }
-                    });
-                } else {
-                    return getDLSNNotLessThanTxIdInSegment(
-                            fromTxnId,
-                            segmentIdx + 1,
-                            segments,
-                            entryStore);
-                }
+        ).thenCompose(foundRecord -> {
+            if (foundRecord.isPresent()) {
+                return FutureUtils.value(foundRecord.get().getDlsn());
+            }
+            if ((segments.size() - 1) == segmentIdx) {
+                return getLastLogRecordAsync().thenApply(record -> {
+                    if (record.getTransactionId() >= fromTxnId) {
+                        return record.getDlsn();
+                    }
+                    return record.getDlsn().getNextDLSN();
+                });
+            } else {
+                return getDLSNNotLessThanTxIdInSegment(
+                        fromTxnId,
+                        segmentIdx + 1,
+                        segments,
+                        entryStore);
             }
         });
     }
@@ -662,7 +618,7 @@
 
     @Override
     public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException {
-        return FutureUtils.result(openAsyncLogReader(fromTxnId));
+        return Utils.ioResult(openAsyncLogReader(fromTxnId));
     }
 
     /**
@@ -687,39 +643,34 @@
      * @return future representing the open result.
      */
     @Override
-    public Future<AsyncLogReader> openAsyncLogReader(long fromTxnId) {
-        final Promise<DLSN> dlsnPromise = new Promise<DLSN>();
-        getDLSNNotLessThanTxId(fromTxnId).addEventListener(new FutureEventListener<DLSN>() {
+    public CompletableFuture<AsyncLogReader> openAsyncLogReader(long fromTxnId) {
+        final CompletableFuture<DLSN> dlsnPromise = new CompletableFuture<DLSN>();
+        getDLSNNotLessThanTxId(fromTxnId).whenComplete(new FutureEventListener<DLSN>() {
 
             @Override
             public void onSuccess(DLSN dlsn) {
-                dlsnPromise.setValue(dlsn);
+                dlsnPromise.complete(dlsn);
             }
 
             @Override
             public void onFailure(Throwable cause) {
                 if (cause instanceof LogEmptyException) {
-                    dlsnPromise.setValue(DLSN.InitialDLSN);
+                    dlsnPromise.complete(DLSN.InitialDLSN);
                 } else {
-                    dlsnPromise.setException(cause);
+                    dlsnPromise.completeExceptionally(cause);
                 }
             }
         });
-        return dlsnPromise.flatMap(new AbstractFunction1<DLSN, Future<AsyncLogReader>>() {
-            @Override
-            public Future<AsyncLogReader> apply(DLSN dlsn) {
-                return openAsyncLogReader(dlsn);
-            }
-        });
+        return dlsnPromise.thenCompose(dlsn -> openAsyncLogReader(dlsn));
     }
 
     @Override
     public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException {
-        return FutureUtils.result(openAsyncLogReader(fromDLSN));
+        return Utils.ioResult(openAsyncLogReader(fromDLSN));
     }
 
     @Override
-    public Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN) {
+    public CompletableFuture<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN) {
         Optional<String> subscriberId = Optional.absent();
         AsyncLogReader reader = new BKAsyncLogReader(
                 this,
@@ -729,7 +680,7 @@
                 false,
                 statsLogger);
         pendingReaders.add(reader);
-        return Future.value(reader);
+        return FutureUtils.value(reader);
     }
 
     /**
@@ -738,26 +689,26 @@
      * blocked.
      */
     @Override
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(final DLSN fromDLSN) {
+    public CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(final DLSN fromDLSN) {
         Optional<String> subscriberId = Optional.absent();
         return getAsyncLogReaderWithLock(Optional.of(fromDLSN), subscriberId);
     }
 
     @Override
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(final DLSN fromDLSN, final String subscriberId) {
+    public CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(final DLSN fromDLSN, final String subscriberId) {
         return getAsyncLogReaderWithLock(Optional.of(fromDLSN), Optional.of(subscriberId));
     }
 
     @Override
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId) {
+    public CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId) {
         Optional<DLSN> fromDLSN = Optional.absent();
         return getAsyncLogReaderWithLock(fromDLSN, Optional.of(subscriberId));
     }
 
-    protected Future<AsyncLogReader> getAsyncLogReaderWithLock(final Optional<DLSN> fromDLSN,
+    protected CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(final Optional<DLSN> fromDLSN,
                                                                final Optional<String> subscriberId) {
         if (!fromDLSN.isPresent() && !subscriberId.isPresent()) {
-            return Future.exception(new UnexpectedException("Neither from dlsn nor subscriber id is provided."));
+            return FutureUtils.exception(new UnexpectedException("Neither from dlsn nor subscriber id is provided."));
         }
         final BKAsyncLogReader reader = new BKAsyncLogReader(
                 BKDistributedLogManager.this,
@@ -767,55 +718,50 @@
                 false,
                 statsLogger);
         pendingReaders.add(reader);
-        final Future<Void> lockFuture = reader.lockStream();
-        final Promise<AsyncLogReader> createPromise = new Promise<AsyncLogReader>(
-                new Function<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable cause) {
+        final CompletableFuture<Void> lockFuture = reader.lockStream();
+        final CompletableFuture<AsyncLogReader> createPromise = FutureUtils.createFuture();
+        createPromise.whenComplete((value, cause) -> {
+            if (cause instanceof CancellationException) {
                 // cancel the lock when the creation future is cancelled
-                lockFuture.cancel();
-                return BoxedUnit.UNIT;
+                lockFuture.cancel(true);
             }
         });
         // lock the stream - fetch the last commit position on success
-        lockFuture.flatMap(new Function<Void, Future<AsyncLogReader>>() {
+        lockFuture.thenCompose(new Function<Void, CompletableFuture<AsyncLogReader>>() {
             @Override
-            public Future<AsyncLogReader> apply(Void complete) {
+            public CompletableFuture<AsyncLogReader> apply(Void complete) {
                 if (fromDLSN.isPresent()) {
-                    return Future.value((AsyncLogReader) reader);
+                    return FutureUtils.value(reader);
                 }
                 LOG.info("Reader {} @ {} reading last commit position from subscription store after acquired lock.",
                         subscriberId.get(), name);
                 // we acquired lock
                 final SubscriptionsStore subscriptionsStore = driver.getSubscriptionsStore(getStreamName());
                 return subscriptionsStore.getLastCommitPosition(subscriberId.get())
-                        .map(new ExceptionalFunction<DLSN, AsyncLogReader>() {
-                    @Override
-                    public AsyncLogReader applyE(DLSN lastCommitPosition) throws UnexpectedException {
-                        LOG.info("Reader {} @ {} positioned to last commit position {}.",
-                                new Object[] { subscriberId.get(), name, lastCommitPosition });
-                        reader.setStartDLSN(lastCommitPosition);
-                        return reader;
-                    }
-                });
+                        .thenCompose(lastCommitPosition -> {
+                            LOG.info("Reader {} @ {} positioned to last commit position {}.",
+                                    new Object[] { subscriberId.get(), name, lastCommitPosition });
+                            try {
+                                reader.setStartDLSN(lastCommitPosition);
+                            } catch (UnexpectedException e) {
+                                return FutureUtils.exception(e);
+                            }
+                            return FutureUtils.value(reader);
+                        });
             }
-        }).addEventListener(new FutureEventListener<AsyncLogReader>() {
+        }).whenComplete(new FutureEventListener<AsyncLogReader>() {
             @Override
             public void onSuccess(AsyncLogReader r) {
                 pendingReaders.remove(reader);
-                FutureUtils.setValue(createPromise, r);
+                FutureUtils.complete(createPromise, r);
             }
 
             @Override
             public void onFailure(final Throwable cause) {
                 pendingReaders.remove(reader);
-                reader.asyncClose().ensure(new AbstractFunction0<BoxedUnit>() {
-                    @Override
-                    public BoxedUnit apply() {
-                        FutureUtils.setException(createPromise, cause);
-                        return BoxedUnit.UNIT;
-                    }
-                });
+                FutureUtils.ensure(
+                    reader.asyncClose(),
+                    () -> FutureUtils.completeExceptionally(createPromise, cause));
             }
         });
         return createPromise;
@@ -833,7 +779,7 @@
         throws IOException {
         DLSN fromDLSN;
         try {
-            fromDLSN = FutureUtils.result(getDLSNNotLessThanTxId(fromTxnId));
+            fromDLSN = Utils.ioResult(getDLSNNotLessThanTxId(fromTxnId));
         } catch (LogEmptyException lee) {
             fromDLSN = DLSN.InitialDLSN;
         }
@@ -861,25 +807,25 @@
     @Override
     public LogRecordWithDLSN getLastLogRecord() throws IOException {
         checkClosedOrInError("getLastLogRecord");
-        return FutureUtils.result(getLastLogRecordAsync());
+        return Utils.ioResult(getLastLogRecordAsync());
     }
 
     @Override
     public long getFirstTxId() throws IOException {
         checkClosedOrInError("getFirstTxId");
-        return FutureUtils.result(getFirstRecordAsyncInternal()).getTransactionId();
+        return Utils.ioResult(getFirstRecordAsyncInternal()).getTransactionId();
     }
 
     @Override
     public long getLastTxId() throws IOException {
         checkClosedOrInError("getLastTxId");
-        return FutureUtils.result(getLastTxIdAsync());
+        return Utils.ioResult(getLastTxIdAsync());
     }
 
     @Override
     public DLSN getLastDLSN() throws IOException {
         checkClosedOrInError("getLastDLSN");
-        return FutureUtils.result(getLastLogRecordAsyncInternal(false, false)).getDlsn();
+        return Utils.ioResult(getLastLogRecordAsyncInternal(false, false)).getDlsn();
     }
 
     /**
@@ -888,15 +834,15 @@
      * @return latest log record
      */
     @Override
-    public Future<LogRecordWithDLSN> getLastLogRecordAsync() {
+    public CompletableFuture<LogRecordWithDLSN> getLastLogRecordAsync() {
         return getLastLogRecordAsyncInternal(false, false);
     }
 
-    private Future<LogRecordWithDLSN> getLastLogRecordAsyncInternal(final boolean recover,
+    private CompletableFuture<LogRecordWithDLSN> getLastLogRecordAsyncInternal(final boolean recover,
                                                                     final boolean includeEndOfStream) {
-        return processReaderOperation(new Function<BKLogReadHandler, Future<LogRecordWithDLSN>>() {
+        return processReaderOperation(new Function<BKLogReadHandler, CompletableFuture<LogRecordWithDLSN>>() {
             @Override
-            public Future<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
+            public CompletableFuture<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
                 return ledgerHandler.getLastLogRecordAsync(recover, includeEndOfStream);
             }
         });
@@ -908,9 +854,9 @@
      * @return latest transaction id
      */
     @Override
-    public Future<Long> getLastTxIdAsync() {
+    public CompletableFuture<Long> getLastTxIdAsync() {
         return getLastLogRecordAsyncInternal(false, false)
-                .map(RECORD_2_TXID_FUNCTION);
+                .thenApply(RECORD_2_TXID_FUNCTION);
     }
 
     /**
@@ -919,14 +865,14 @@
      * @return first dlsn in the stream
      */
     @Override
-    public Future<DLSN> getFirstDLSNAsync() {
-        return getFirstRecordAsyncInternal().map(RECORD_2_DLSN_FUNCTION);
+    public CompletableFuture<DLSN> getFirstDLSNAsync() {
+        return getFirstRecordAsyncInternal().thenApply(RECORD_2_DLSN_FUNCTION);
     }
 
-    private Future<LogRecordWithDLSN> getFirstRecordAsyncInternal() {
-        return processReaderOperation(new Function<BKLogReadHandler, Future<LogRecordWithDLSN>>() {
+    private CompletableFuture<LogRecordWithDLSN> getFirstRecordAsyncInternal() {
+        return processReaderOperation(new Function<BKLogReadHandler, CompletableFuture<LogRecordWithDLSN>>() {
             @Override
-            public Future<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
+            public CompletableFuture<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
                 return ledgerHandler.asyncGetFirstLogRecord();
             }
         });
@@ -938,9 +884,9 @@
      * @return latest transaction id
      */
     @Override
-    public Future<DLSN> getLastDLSNAsync() {
+    public CompletableFuture<DLSN> getLastDLSNAsync() {
         return getLastLogRecordAsyncInternal(false, false)
-                .map(RECORD_2_DLSN_FUNCTION);
+                .thenApply(RECORD_2_DLSN_FUNCTION);
     }
 
     /**
@@ -953,7 +899,7 @@
     @Override
     public long getLogRecordCount() throws IOException {
         checkClosedOrInError("getLogRecordCount");
-        return FutureUtils.result(getLogRecordCountAsync(DLSN.InitialDLSN));
+        return Utils.ioResult(getLogRecordCountAsync(DLSN.InitialDLSN));
     }
 
     /**
@@ -964,10 +910,10 @@
      * @throws IOException
      */
     @Override
-    public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN) {
-        return processReaderOperation(new Function<BKLogReadHandler, Future<Long>>() {
+    public CompletableFuture<Long> getLogRecordCountAsync(final DLSN beginDLSN) {
+        return processReaderOperation(new Function<BKLogReadHandler, CompletableFuture<Long>>() {
                     @Override
-                    public Future<Long> apply(BKLogReadHandler ledgerHandler) {
+                    public CompletableFuture<Long> apply(BKLogReadHandler ledgerHandler) {
                         return ledgerHandler.asyncGetLogRecordCount(beginDLSN);
                     }
                 });
@@ -991,7 +937,7 @@
         checkClosedOrInError("recoverInternal");
         BKLogWriteHandler ledgerHandler = createWriteHandler(true);
         try {
-            FutureUtils.result(ledgerHandler.recoverIncompleteLogSegments());
+            Utils.ioResult(ledgerHandler.recoverIncompleteLogSegments());
         } finally {
             Utils.closeQuietly(ledgerHandler);
         }
@@ -1004,7 +950,7 @@
      */
     @Override
     public void delete() throws IOException {
-        FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
+        Utils.ioResult(driver.getLogStreamMetadataStore(WRITER)
                 .deleteLog(uri, getStreamName()));
     }
 
@@ -1025,7 +971,7 @@
         BKLogWriteHandler ledgerHandler = createWriteHandler(true);
         try {
             LOG.info("Purging logs for {} older than {}", ledgerHandler.getFullyQualifiedName(), minTxIdToKeep);
-            FutureUtils.result(ledgerHandler.purgeLogSegmentsOlderThanTxnId(minTxIdToKeep));
+            Utils.ioResult(ledgerHandler.purgeLogSegmentsOlderThanTxnId(minTxIdToKeep));
         } finally {
             Utils.closeQuietly(ledgerHandler);
         }
@@ -1049,14 +995,11 @@
         }
 
         @Override
-        public Future<Void> asyncClose() {
+        public CompletableFuture<Void> asyncClose() {
             return Utils.closeSequence(executorService, true, readers.toArray(new AsyncLogReader[readers.size()]))
-                    .onSuccess(new AbstractFunction1<Void, BoxedUnit>() {
-                        @Override
-                        public BoxedUnit apply(Void value) {
-                            readers.clear();
-                            return BoxedUnit.UNIT;
-                        }
+                    .thenApply(value -> {
+                        readers.clear();
+                        return null;
                     });
         }
     };
@@ -1065,28 +1008,28 @@
      * Close the distributed log manager, freeing any resources it may hold.
      */
     @Override
-    public Future<Void> asyncClose() {
-        Promise<Void> closeFuture;
+    public CompletableFuture<Void> asyncClose() {
+        CompletableFuture<Void> closeFuture;
         BKLogReadHandler readHandlerToClose;
         synchronized (this) {
             if (null != closePromise) {
                 return closePromise;
             }
-            closeFuture = closePromise = new Promise<Void>();
+            closeFuture = closePromise = new CompletableFuture<Void>();
             readHandlerToClose = readHandlerForListener;
         }
 
-        Future<Void> closeResult = Utils.closeSequence(null, true,
+        CompletableFuture<Void> closeResult = Utils.closeSequence(null, true,
                 readHandlerToClose,
                 pendingReaders,
                 resourcesCloseable.or(AsyncCloseable.NULL));
-        closeResult.proxyTo(closeFuture);
+        FutureUtils.proxyTo(closeResult, closeFuture);
         return closeFuture;
     }
 
     @Override
     public void close() throws IOException {
-        FutureUtils.result(asyncClose());
+        Utils.ioResult(asyncClose());
     }
 
     @Override
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
index adb591f..60ad916 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
@@ -20,6 +20,8 @@
 import com.google.common.base.Optional;
 import com.google.common.base.Ticker;
 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.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
@@ -28,14 +30,11 @@
 import org.apache.distributedlog.injector.AsyncFailureInjector;
 import org.apache.distributedlog.io.AsyncCloseable;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
 import org.apache.distributedlog.namespace.NamespaceDriver;
 import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.util.FutureUtils;
-import org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor;
 import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.PermitLimiter;
-import org.apache.distributedlog.util.SchedulerUtils;
+import org.apache.distributedlog.common.util.PermitLimiter;
+import org.apache.distributedlog.common.util.SchedulerUtils;
 import org.apache.distributedlog.util.Utils;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.stats.StatsLogger;
@@ -52,7 +51,7 @@
 import static org.apache.distributedlog.util.DLUtils.validateAndNormalizeName;
 
 /**
- * BKDistributedLogNamespace is the default implementation of {@link DistributedLogNamespace}. It uses
+ * BKDistributedLogNamespace is the default implementation of {@link Namespace}. It uses
  * zookeeper for metadata storage and bookkeeper for data storage.
  * <h3>Metrics</h3>
  *
@@ -74,8 +73,6 @@
  * <ul>
  * <li> `scope`/factory/thread_pool/* : stats about the ordered scheduler used by this namespace.
  * See {@link OrderedScheduler}.
- * <li> `scope`/factory/readahead_thread_pool/* : stats about the readahead thread pool executor
- * used by this namespace. See {@link MonitoredScheduledThreadPoolExecutor}.
  * <li> `scope`/writeLimiter/* : stats about the global write limiter used by this namespace.
  * See {@link PermitLimiter}.
  * </ul>
@@ -84,7 +81,7 @@
  *
  * All the core stats about reader and writer are exposed under current scope via {@link BKDistributedLogManager}.
  */
-public class BKDistributedLogNamespace implements DistributedLogNamespace {
+public class BKDistributedLogNamespace implements Namespace {
     static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogNamespace.class);
 
     private final String clientId;
@@ -149,8 +146,8 @@
             throws InvalidStreamNameException, IOException {
         checkState();
         logName = validateAndNormalizeName(logName);
-        URI uri = FutureUtils.result(driver.getLogMetadataStore().createLog(logName));
-        FutureUtils.result(driver.getLogStreamMetadataStore(WRITER).getLog(uri, logName, true, true));
+        URI uri = Utils.ioResult(driver.getLogMetadataStore().createLog(logName));
+        Utils.ioResult(driver.getLogStreamMetadataStore(WRITER).getLog(uri, logName, true, true));
     }
 
     @Override
@@ -158,7 +155,7 @@
             throws InvalidStreamNameException, LogNotFoundException, IOException {
         checkState();
         logName = validateAndNormalizeName(logName);
-        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
+        Optional<URI> uri = Utils.ioResult(driver.getLogMetadataStore().getLogLocation(logName));
         if (!uri.isPresent()) {
             throw new LogNotFoundException("Log " + logName + " isn't found.");
         }
@@ -187,7 +184,7 @@
             throws InvalidStreamNameException, IOException {
         checkState();
         logName = validateAndNormalizeName(logName);
-        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
+        Optional<URI> uri = Utils.ioResult(driver.getLogMetadataStore().getLogLocation(logName));
         if (!uri.isPresent()) {
             throw new LogNotFoundException("Log " + logName + " isn't found.");
         }
@@ -202,10 +199,10 @@
     public boolean logExists(String logName)
         throws IOException, IllegalArgumentException {
         checkState();
-        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
+        Optional<URI> uri = Utils.ioResult(driver.getLogMetadataStore().getLogLocation(logName));
         if (uri.isPresent()) {
             try {
-                FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
+                Utils.ioResult(driver.getLogStreamMetadataStore(WRITER)
                         .logExists(uri.get(), logName));
                 return true;
             } catch (LogNotFoundException lnfe) {
@@ -219,7 +216,7 @@
     @Override
     public Iterator<String> getLogs() throws IOException {
         checkState();
-        return FutureUtils.result(driver.getLogMetadataStore().getLogs());
+        return Utils.ioResult(driver.getLogMetadataStore().getLogs());
     }
 
     @Override
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
index 3919e92..a7d0d25 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
@@ -18,6 +18,27 @@
 package org.apache.distributedlog;
 
 import com.google.common.base.Stopwatch;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+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.commons.lang3.tuple.Pair;
 import org.apache.distributedlog.callback.LogSegmentNamesListener;
 import org.apache.distributedlog.exceptions.LogEmptyException;
 import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
@@ -31,35 +52,12 @@
 import org.apache.distributedlog.logsegment.LogSegmentFilter;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
 import org.apache.distributedlog.metadata.LogStreamMetadataStore;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-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.commons.lang3.tuple.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
 /**
  * The base class about log handler on managing log segments.
  *
@@ -171,25 +169,27 @@
         return lockClientId;
     }
 
-    public Future<LogRecordWithDLSN> asyncGetFirstLogRecord() {
-        final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
-        streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
-                .addEventListener(new FutureEventListener<Void>() {
+    public CompletableFuture<LogRecordWithDLSN> asyncGetFirstLogRecord() {
+        final CompletableFuture<LogRecordWithDLSN> promise = new CompletableFuture<LogRecordWithDLSN>();
+        streamMetadataStore.logExists(
+            logMetadata.getUri(),
+            logMetadata.getLogName()
+        ).whenComplete(new FutureEventListener<Void>() {
             @Override
             public void onSuccess(Void value) {
                 readLogSegmentsFromStore(
                         LogSegmentMetadata.COMPARATOR,
                         LogSegmentFilter.DEFAULT_FILTER,
                         null
-                ).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+                ).whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
 
                     @Override
                     public void onSuccess(Versioned<List<LogSegmentMetadata>> ledgerList) {
                         if (ledgerList.getValue().isEmpty()) {
-                            promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+                            promise.completeExceptionally(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
                             return;
                         }
-                        Future<LogRecordWithDLSN> firstRecord = null;
+                        CompletableFuture<LogRecordWithDLSN> firstRecord = null;
                         for (LogSegmentMetadata ledger : ledgerList.getValue()) {
                             if (!ledger.isTruncated() && (ledger.getRecordCount() > 0 || ledger.isInProgress())) {
                                 firstRecord = asyncReadFirstUserRecord(ledger, DLSN.InitialDLSN);
@@ -197,43 +197,45 @@
                             }
                         }
                         if (null != firstRecord) {
-                            promise.become(firstRecord);
+                            FutureUtils.proxyTo(firstRecord, promise);
                         } else {
-                            promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+                            promise.completeExceptionally(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
                         }
                     }
 
                     @Override
                     public void onFailure(Throwable cause) {
-                        promise.setException(cause);
+                        promise.completeExceptionally(cause);
                     }
                 });
             }
 
             @Override
             public void onFailure(Throwable cause) {
-                promise.setException(cause);
+                promise.completeExceptionally(cause);
             }
         });
         return promise;
     }
 
-    public Future<LogRecordWithDLSN> getLastLogRecordAsync(final boolean recover, final boolean includeEndOfStream) {
-        final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
-        streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
-                .addEventListener(new FutureEventListener<Void>() {
+    public CompletableFuture<LogRecordWithDLSN> getLastLogRecordAsync(final boolean recover, final boolean includeEndOfStream) {
+        final CompletableFuture<LogRecordWithDLSN> promise = new CompletableFuture<LogRecordWithDLSN>();
+        streamMetadataStore.logExists(
+            logMetadata.getUri(),
+            logMetadata.getLogName()
+        ).whenComplete(new FutureEventListener<Void>() {
             @Override
             public void onSuccess(Void value) {
                 readLogSegmentsFromStore(
                         LogSegmentMetadata.DESC_COMPARATOR,
                         LogSegmentFilter.DEFAULT_FILTER,
                         null
-                ).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+                ).whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
 
                     @Override
                     public void onSuccess(Versioned<List<LogSegmentMetadata>> ledgerList) {
                         if (ledgerList.getValue().isEmpty()) {
-                            promise.setException(
+                            promise.completeExceptionally(
                                     new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
                             return;
                         }
@@ -247,49 +249,51 @@
 
                     @Override
                     public void onFailure(Throwable cause) {
-                        promise.setException(cause);
+                        promise.completeExceptionally(cause);
                     }
                 });
             }
 
             @Override
             public void onFailure(Throwable cause) {
-                promise.setException(cause);
+                promise.completeExceptionally(cause);
             }
         });
         return promise;
     }
 
     private void asyncGetLastLogRecord(final Iterator<LogSegmentMetadata> ledgerIter,
-                                       final Promise<LogRecordWithDLSN> promise,
+                                       final CompletableFuture<LogRecordWithDLSN> promise,
                                        final boolean fence,
                                        final boolean includeControlRecord,
                                        final boolean includeEndOfStream) {
         if (ledgerIter.hasNext()) {
             LogSegmentMetadata metadata = ledgerIter.next();
-            asyncReadLastRecord(metadata, fence, includeControlRecord, includeEndOfStream).addEventListener(
-                    new FutureEventListener<LogRecordWithDLSN>() {
-                        @Override
-                        public void onSuccess(LogRecordWithDLSN record) {
-                            if (null == record) {
-                                asyncGetLastLogRecord(ledgerIter, promise, fence, includeControlRecord, includeEndOfStream);
-                            } else {
-                                promise.setValue(record);
-                            }
-                        }
-
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            promise.setException(cause);
+            asyncReadLastRecord(
+                metadata, fence, includeControlRecord, includeEndOfStream
+            ).whenComplete(
+                new FutureEventListener<LogRecordWithDLSN>() {
+                    @Override
+                    public void onSuccess(LogRecordWithDLSN record) {
+                        if (null == record) {
+                            asyncGetLastLogRecord(ledgerIter, promise, fence, includeControlRecord, includeEndOfStream);
+                        } else {
+                            promise.complete(record);
                         }
                     }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                                                         promise.completeExceptionally(cause);
+                                                                                              }
+                }
             );
         } else {
-            promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+            promise.completeExceptionally(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
         }
     }
 
-    private Future<LogRecordWithDLSN> asyncReadFirstUserRecord(LogSegmentMetadata ledger, DLSN beginDLSN) {
+    private CompletableFuture<LogRecordWithDLSN> asyncReadFirstUserRecord(LogSegmentMetadata ledger, DLSN beginDLSN) {
         return ReadUtils.asyncReadFirstUserRecord(
                 getFullyQualifiedName(),
                 ledger,
@@ -307,15 +311,17 @@
      * beginDLSN and the second denoted by endPosition. Its up to the caller to ensure that endPosition refers to
      * position in the same ledger as beginDLSN.
      */
-    private Future<Long> asyncGetLogRecordCount(LogSegmentMetadata ledger, final DLSN beginDLSN, final long endPosition) {
-        return asyncReadFirstUserRecord(ledger, beginDLSN).map(new Function<LogRecordWithDLSN, Long>() {
-            public Long apply(final LogRecordWithDLSN beginRecord) {
-                long recordCount = 0;
-                if (null != beginRecord) {
-                    recordCount = endPosition + 1 - beginRecord.getLastPositionWithinLogSegment();
-                }
-                return recordCount;
+    private CompletableFuture<Long> asyncGetLogRecordCount(LogSegmentMetadata ledger,
+                                                           final DLSN beginDLSN,
+                                                           final long endPosition) {
+        return asyncReadFirstUserRecord(
+            ledger, beginDLSN
+        ).thenApply(beginRecord -> {
+            long recordCount = 0;
+            if (null != beginRecord) {
+                recordCount = endPosition + 1 - beginRecord.getLastPositionWithinLogSegment();
             }
+            return recordCount;
         });
     }
 
@@ -325,31 +331,29 @@
      * an interior entry. For the last entry, if it is inprogress, we need to recover it and find the last user
      * entry.
      */
-    private Future<Long> asyncGetLogRecordCount(final LogSegmentMetadata ledger, final DLSN beginDLSN) {
+    private CompletableFuture<Long> asyncGetLogRecordCount(final LogSegmentMetadata ledger, final DLSN beginDLSN) {
         if (ledger.isInProgress() && ledger.isDLSNinThisSegment(beginDLSN)) {
-            return asyncReadLastUserRecord(ledger).flatMap(new Function<LogRecordWithDLSN, Future<Long>>() {
-                public Future<Long> apply(final LogRecordWithDLSN endRecord) {
+            return asyncReadLastUserRecord(ledger).thenCompose(
+                (Function<LogRecordWithDLSN, CompletableFuture<Long>>) endRecord -> {
                     if (null != endRecord) {
-                        return asyncGetLogRecordCount(ledger, beginDLSN, endRecord.getLastPositionWithinLogSegment() /* end position */);
+                        return asyncGetLogRecordCount(
+                            ledger, beginDLSN, endRecord.getLastPositionWithinLogSegment() /* end position */);
                     } else {
-                        return Future.value((long) 0);
+                        return FutureUtils.value((long) 0);
                     }
-                }
-            });
+                });
         } else if (ledger.isInProgress()) {
-            return asyncReadLastUserRecord(ledger).map(new Function<LogRecordWithDLSN, Long>() {
-                public Long apply(final LogRecordWithDLSN endRecord) {
-                    if (null != endRecord) {
-                        return (long) endRecord.getLastPositionWithinLogSegment();
-                    } else {
-                        return (long) 0;
-                    }
+            return asyncReadLastUserRecord(ledger).thenApply(endRecord -> {
+                if (null != endRecord) {
+                    return (long) endRecord.getLastPositionWithinLogSegment();
+                } else {
+                    return (long) 0;
                 }
             });
         } else if (ledger.isDLSNinThisSegment(beginDLSN)) {
             return asyncGetLogRecordCount(ledger, beginDLSN, ledger.getRecordCount() /* end position */);
         } else {
-            return Future.value((long) ledger.getRecordCount());
+            return FutureUtils.value((long) ledger.getRecordCount());
         }
     }
 
@@ -359,29 +363,26 @@
      * @param beginDLSN dlsn marking the start of the range
      * @return the count of records present in the range
      */
-    public Future<Long> asyncGetLogRecordCount(final DLSN beginDLSN) {
+    public CompletableFuture<Long> asyncGetLogRecordCount(final DLSN beginDLSN) {
         return streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
-                .flatMap(new Function<Void, Future<Long>>() {
-            public Future<Long> apply(Void done) {
+                .thenCompose(new Function<Void, CompletableFuture<Long>>() {
+            public CompletableFuture<Long> apply(Void done) {
 
                 return readLogSegmentsFromStore(
                         LogSegmentMetadata.COMPARATOR,
                         LogSegmentFilter.DEFAULT_FILTER,
                         null
-                ).flatMap(new Function<Versioned<List<LogSegmentMetadata>>, Future<Long>>() {
-                    public Future<Long> apply(Versioned<List<LogSegmentMetadata>> ledgerList) {
+                ).thenCompose(new Function<Versioned<List<LogSegmentMetadata>>, CompletableFuture<Long>>() {
+                    public CompletableFuture<Long> apply(Versioned<List<LogSegmentMetadata>> ledgerList) {
 
-                        List<Future<Long>> futureCounts = new ArrayList<Future<Long>>(ledgerList.getValue().size());
+                        List<CompletableFuture<Long>> futureCounts =
+                          Lists.newArrayListWithExpectedSize(ledgerList.getValue().size());
                         for (LogSegmentMetadata ledger : ledgerList.getValue()) {
                             if (ledger.getLogSegmentSequenceNumber() >= beginDLSN.getLogSegmentSequenceNo()) {
                                 futureCounts.add(asyncGetLogRecordCount(ledger, beginDLSN));
                             }
                         }
-                        return Future.collect(futureCounts).map(new Function<List<Long>, Long>() {
-                            public Long apply(List<Long> counts) {
-                                return sum(counts);
-                            }
-                        });
+                        return FutureUtils.collect(futureCounts).thenApply(counts -> sum(counts));
                     }
                 });
             }
@@ -397,15 +398,15 @@
     }
 
     @Override
-    public Future<Void> asyncAbort() {
+    public CompletableFuture<Void> asyncAbort() {
         return asyncClose();
     }
 
-    public Future<LogRecordWithDLSN> asyncReadLastUserRecord(final LogSegmentMetadata l) {
+    public CompletableFuture<LogRecordWithDLSN> asyncReadLastUserRecord(final LogSegmentMetadata l) {
         return asyncReadLastRecord(l, false, false, false);
     }
 
-    public Future<LogRecordWithDLSN> asyncReadLastRecord(final LogSegmentMetadata l,
+    public CompletableFuture<LogRecordWithDLSN> asyncReadLastRecord(final LogSegmentMetadata l,
                                                          final boolean fence,
                                                          final boolean includeControl,
                                                          final boolean includeEndOfStream) {
@@ -422,7 +423,7 @@
                 numRecordsScanned,
                 scheduler,
                 entryStore
-        ).addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+        ).whenComplete(new FutureEventListener<LogRecordWithDLSN>() {
             @Override
             public void onSuccess(LogRecordWithDLSN value) {
                 recoverLastEntryStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
@@ -572,17 +573,17 @@
      * @param logSegmentNamesListener
      * @return future represents the result of log segments
      */
-    public Future<Versioned<List<LogSegmentMetadata>>> readLogSegmentsFromStore(
+    public CompletableFuture<Versioned<List<LogSegmentMetadata>>> readLogSegmentsFromStore(
             final Comparator<LogSegmentMetadata> comparator,
             final LogSegmentFilter segmentFilter,
             final LogSegmentNamesListener logSegmentNamesListener) {
-        final Promise<Versioned<List<LogSegmentMetadata>>> readResult =
-                new Promise<Versioned<List<LogSegmentMetadata>>>();
+        final CompletableFuture<Versioned<List<LogSegmentMetadata>>> readResult =
+                new CompletableFuture<Versioned<List<LogSegmentMetadata>>>();
         metadataStore.getLogSegmentNames(logMetadata.getLogSegmentsPath(), logSegmentNamesListener)
-                .addEventListener(new FutureEventListener<Versioned<List<String>>>() {
+                .whenComplete(new FutureEventListener<Versioned<List<String>>>() {
                     @Override
                     public void onFailure(Throwable cause) {
-                        FutureUtils.setException(readResult, cause);
+                        readResult.completeExceptionally(cause);
                     }
 
                     @Override
@@ -596,7 +597,7 @@
     protected void readLogSegmentsFromStore(final Versioned<List<String>> logSegmentNames,
                                             final Comparator<LogSegmentMetadata> comparator,
                                             final LogSegmentFilter segmentFilter,
-                                            final Promise<Versioned<List<LogSegmentMetadata>>> readResult) {
+                                            final CompletableFuture<Versioned<List<LogSegmentMetadata>>> readResult) {
         Set<String> segmentsReceived = new HashSet<String>();
         segmentsReceived.addAll(segmentFilter.filter(logSegmentNames.getValue()));
         Set<String> segmentsAdded;
@@ -619,12 +620,11 @@
             try {
                 segmentList = getCachedLogSegments(comparator);
             } catch (UnexpectedException e) {
-                FutureUtils.setException(readResult, e);
+                readResult.completeExceptionally(e);
                 return;
             }
 
-            FutureUtils.setValue(readResult,
-                    new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNames.getVersion()));
+            readResult.complete(new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNames.getVersion()));
             return;
         }
 
@@ -646,7 +646,7 @@
                 continue;
             }
             metadataStore.getLogSegment(logSegmentPath)
-                    .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+                    .whenComplete(new FutureEventListener<LogSegmentMetadata>() {
 
                         @Override
                         public void onSuccess(LogSegmentMetadata result) {
@@ -666,7 +666,7 @@
                             } else {
                                 // fail fast
                                 if (1 == numFailures.incrementAndGet()) {
-                                    FutureUtils.setException(readResult, cause);
+                                    readResult.completeExceptionally(cause);
                                     return;
                                 }
                             }
@@ -689,7 +689,7 @@
     private void completeReadLogSegmentsFromStore(final Set<String> removedSegments,
                                                   final Map<String, LogSegmentMetadata> addedSegments,
                                                   final Comparator<LogSegmentMetadata> comparator,
-                                                  final Promise<Versioned<List<LogSegmentMetadata>>> readResult,
+                                                  final CompletableFuture<Versioned<List<LogSegmentMetadata>>> readResult,
                                                   final Version logSegmentNamesVersion,
                                                   final AtomicInteger numChildren,
                                                   final AtomicInteger numFailures) {
@@ -705,11 +705,10 @@
         try {
             segmentList = getCachedLogSegments(comparator);
         } catch (UnexpectedException e) {
-            FutureUtils.setException(readResult, e);
+            readResult.completeExceptionally(e);
             return;
         }
-        FutureUtils.setValue(readResult,
-            new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNamesVersion));
+        readResult.complete(new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNamesVersion));
     }
 
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
index c6e2e07..ff6b527 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
@@ -17,15 +17,22 @@
  */
 package org.apache.distributedlog;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.TimeUnit;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
+import javax.annotation.Nullable;
+import org.apache.bookkeeper.stats.AlertStatsLogger;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.distributedlog.callback.LogSegmentListener;
 import org.apache.distributedlog.callback.LogSegmentNamesListener;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
@@ -40,29 +47,12 @@
 import org.apache.distributedlog.logsegment.LogSegmentFilter;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
 import org.apache.distributedlog.metadata.LogStreamMetadataStore;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.ExceptionalFunction;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
-import com.twitter.util.Try;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.SafeRunnable;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import javax.annotation.Nullable;
 
 /**
  * Log Handler for Readers.
@@ -112,7 +102,7 @@
 
     private final Optional<String> subscriberId;
     private DistributedLock readLock;
-    private Future<Void> lockAcquireFuture;
+    private CompletableFuture<Void> lockAcquireFuture;
 
     // notify the state change about the read handler
     protected final AsyncNotification readerStateNotification;
@@ -166,31 +156,23 @@
         return logMetadataForReader.getReadLockPath(subscriberId);
     }
 
-    <T> void satisfyPromiseAsync(final Promise<T> promise, final Try<T> result) {
-        scheduler.submit(new SafeRunnable() {
-            @Override
-            public void safeRun() {
-                promise.update(result);
-            }
-        });
-    }
-
-    Future<Void> checkLogStreamExists() {
+    CompletableFuture<Void> checkLogStreamExists() {
         return streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName());
     }
 
     /**
      * Elective stream lock--readers are not required to acquire the lock before using the stream.
      */
-    synchronized Future<Void> lockStream() {
+    synchronized CompletableFuture<Void> lockStream() {
         if (null == lockAcquireFuture) {
             lockAcquireFuture = streamMetadataStore.createReadLock(logMetadataForReader, subscriberId)
-                    .flatMap(new ExceptionalFunction<DistributedLock, Future<Void>>() {
-                        @Override
-                        public Future<Void> applyE(DistributedLock lock) throws Throwable {
+                    .thenCompose(lock -> {
+                        try {
                             BKLogReadHandler.this.readLock = lock;
                             LOG.info("acquiring readlock {} at {}", getLockClientId(), getReadLockPath());
                             return acquireLockOnExecutorThread(lock);
+                        } catch (LockingException le) {
+                            return FutureUtils.exception(le);
                         }
                     });
         }
@@ -201,33 +183,31 @@
      * Begin asynchronous lock acquire, but ensure that the returned future is satisfied on an
      * executor service thread.
      */
-    Future<Void> acquireLockOnExecutorThread(DistributedLock lock) throws LockingException {
-        final Future<? extends DistributedLock> acquireFuture = lock.asyncAcquire();
+    CompletableFuture<Void> acquireLockOnExecutorThread(DistributedLock lock) throws LockingException {
+        final CompletableFuture<? extends DistributedLock> acquireFuture = lock.asyncAcquire();
 
         // The future we return must be satisfied on an executor service thread. If we simply
         // return the future returned by asyncAcquire, user callbacks may end up running in
         // the lock state executor thread, which will cause deadlocks and introduce latency
         // etc.
-        final Promise<Void> threadAcquirePromise = new Promise<Void>();
-        threadAcquirePromise.setInterruptHandler(new Function<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable t) {
-                FutureUtils.cancel(acquireFuture);
-                return null;
+        final CompletableFuture<Void> threadAcquirePromise = new CompletableFuture<Void>();
+        threadAcquirePromise.whenComplete((value, cause) -> {
+            if (cause instanceof CancellationException) {
+                acquireFuture.cancel(true);
             }
         });
-        acquireFuture.addEventListener(new FutureEventListener<DistributedLock>() {
+        acquireFuture.whenCompleteAsync(new FutureEventListener<DistributedLock>() {
             @Override
             public void onSuccess(DistributedLock lock) {
                 LOG.info("acquired readlock {} at {}", getLockClientId(), getReadLockPath());
-                satisfyPromiseAsync(threadAcquirePromise, new Return<Void>(null));
+                threadAcquirePromise.complete(null);
             }
 
             @Override
             public void onFailure(Throwable cause) {
                 LOG.info("failed to acquire readlock {} at {}",
                         new Object[]{ getLockClientId(), getReadLockPath(), cause });
-                satisfyPromiseAsync(threadAcquirePromise, new Throw<Void>(cause));
+                threadAcquirePromise.completeExceptionally(cause);
             }
         });
         return threadAcquirePromise;
@@ -239,7 +219,7 @@
     void checkReadLock() throws DLIllegalStateException, LockingException {
         synchronized (this) {
             if ((null == lockAcquireFuture) ||
-                (!lockAcquireFuture.isDefined())) {
+                (!lockAcquireFuture.isDone())) {
                 throw new DLIllegalStateException("Attempt to check for lock before it has been acquired successfully");
             }
         }
@@ -247,27 +227,24 @@
         readLock.checkOwnership();
     }
 
-    public Future<Void> asyncClose() {
+    public CompletableFuture<Void> asyncClose() {
         DistributedLock lockToClose;
         synchronized (this) {
-            if (null != lockAcquireFuture && !lockAcquireFuture.isDefined()) {
-                FutureUtils.cancel(lockAcquireFuture);
+            if (null != lockAcquireFuture && !lockAcquireFuture.isDone()) {
+                lockAcquireFuture.cancel(true);
             }
             lockToClose = readLock;
         }
         return Utils.closeSequence(scheduler, lockToClose)
-                .flatMap(new AbstractFunction1<Void, Future<Void>>() {
-            @Override
-            public Future<Void> apply(Void result) {
+            .thenApply((value) -> {
                 // unregister the log segment listener
                 metadataStore.unregisterLogSegmentListener(logMetadata.getLogSegmentsPath(), BKLogReadHandler.this);
-                return Future.Void();
-            }
-        });
+                return null;
+            });
     }
 
     @Override
-    public Future<Void> asyncAbort() {
+    public CompletableFuture<Void> asyncAbort() {
         return asyncClose();
     }
 
@@ -277,18 +254,18 @@
      *
      * @return future represents the fetch result
      */
-    Future<Versioned<List<LogSegmentMetadata>>> asyncStartFetchLogSegments() {
-        Promise<Versioned<List<LogSegmentMetadata>>> promise =
-                new Promise<Versioned<List<LogSegmentMetadata>>>();
+    CompletableFuture<Versioned<List<LogSegmentMetadata>>> asyncStartFetchLogSegments() {
+        CompletableFuture<Versioned<List<LogSegmentMetadata>>> promise =
+                new CompletableFuture<Versioned<List<LogSegmentMetadata>>>();
         asyncStartFetchLogSegments(promise);
         return promise;
     }
 
-    void asyncStartFetchLogSegments(final Promise<Versioned<List<LogSegmentMetadata>>> promise) {
+    void asyncStartFetchLogSegments(final CompletableFuture<Versioned<List<LogSegmentMetadata>>> promise) {
         readLogSegmentsFromStore(
                 LogSegmentMetadata.COMPARATOR,
                 LogSegmentFilter.DEFAULT_FILTER,
-                this).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+                this).whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
             @Override
             public void onFailure(Throwable cause) {
                 if (cause instanceof LogNotFoundException ||
@@ -298,7 +275,7 @@
                     metadataException.compareAndSet(null, (IOException) cause);
                     // notify the reader that read handler is in error state
                     notifyReaderOnError(cause);
-                    FutureUtils.setException(promise, cause);
+                    FutureUtils.completeExceptionally(promise, cause);
                     return;
                 }
                 scheduler.schedule(new Runnable() {
@@ -312,7 +289,7 @@
             @Override
             public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
                 // no-op
-                FutureUtils.setValue(promise, segments);
+                FutureUtils.complete(promise, segments);
             }
         });
     }
@@ -332,9 +309,9 @@
             }
         }
 
-        Promise<Versioned<List<LogSegmentMetadata>>> readLogSegmentsPromise =
-                new Promise<Versioned<List<LogSegmentMetadata>>>();
-        readLogSegmentsPromise.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+        CompletableFuture<Versioned<List<LogSegmentMetadata>>> readLogSegmentsPromise =
+                new CompletableFuture<Versioned<List<LogSegmentMetadata>>>();
+        readLogSegmentsPromise.whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
             @Override
             public void onFailure(Throwable cause) {
                 if (cause instanceof LogNotFoundException ||
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
index 6b60c77..a4016c8 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
@@ -20,6 +20,8 @@
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -28,6 +30,7 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Stopwatch;
+import java.util.function.Function;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.exceptions.EndOfStreamException;
@@ -47,20 +50,15 @@
 import org.apache.distributedlog.lock.DistributedLock;
 import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
 import org.apache.distributedlog.logsegment.LogSegmentWriter;
-import org.apache.distributedlog.stats.BroadCastStatsLogger;
-import org.apache.distributedlog.stats.OpStatsListener;
+import org.apache.distributedlog.common.stats.BroadCastStatsLogger;
+import org.apache.distributedlog.common.stats.OpStatsListener;
 import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.PermitLimiter;
-import org.apache.distributedlog.util.SafeQueueingFuturePool;
+import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.util.SimplePermitLimiter;
-import org.apache.distributedlog.util.Sizable;
-import com.twitter.util.Function0;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.FuturePool;
-import com.twitter.util.Promise;
+import org.apache.distributedlog.common.util.Sizable;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
 import org.apache.bookkeeper.client.BKException;
@@ -73,10 +71,9 @@
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.util.MathUtils;
+import org.apache.distributedlog.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
 
 import static com.google.common.base.Charsets.UTF_8;
 import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
@@ -146,7 +143,7 @@
     final private AtomicReference<ScheduledFuture<?>> immFlushSchedFutureRef = new AtomicReference<ScheduledFuture<?>>(null);
     final private AtomicReference<Exception> scheduledFlushException = new AtomicReference<Exception>(null);
     private boolean enforceLock = true;
-    private Promise<Void> closeFuture = null;
+    private CompletableFuture<Void> closeFuture = null;
     private final boolean enableRecordCounts;
     private int positionWithinLogSegment = 0;
     private final long logSegmentSequenceNumber;
@@ -170,28 +167,17 @@
     private final OpStatsLogger addCompleteDeferredTime;
     private final Counter pendingWrites;
 
-    // add complete processing
-    private final SafeQueueingFuturePool<Void> addCompleteFuturePool;
-
     // Functions
-    private final AbstractFunction1<Integer, Future<Long>> GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC =
-            new AbstractFunction1<Integer, Future<Long>>() {
-                @Override
-                public Future<Long> apply(Integer transmitRc) {
-                    if (BKException.Code.OK == transmitRc) {
-                        return Future.value(getLastTxIdAcknowledged());
-                    } else {
-                        return Future.exception(new BKTransmitException("Failed to transmit entry", transmitRc));
-                    }
-                }
-            };
-    final AbstractFunction1<Long, Future<Long>> COMMIT_AFTER_FLUSH_FUNC =
-            new AbstractFunction1<Long, Future<Long>>() {
-                @Override
-                public Future<Long> apply(Long lastAckedTxId) {
-                    return commit();
-                }
-            };
+    private final Function<Integer, CompletableFuture<Long>> GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC =
+        transmitRc -> {
+            if (BKException.Code.OK == transmitRc) {
+                return FutureUtils.value(getLastTxIdAcknowledged());
+            } else {
+                return FutureUtils.exception(new BKTransmitException("Failed to transmit entry", transmitRc));
+            }
+        };
+    final Function<Long, CompletableFuture<Long>> COMMIT_AFTER_FLUSH_FUNC =
+        lastAckedTxId -> commit();
 
     private final AlertStatsLogger alertStatsLogger;
     private final WriteLimiter writeLimiter;
@@ -341,11 +327,6 @@
         }
 
         this.conf = conf;
-        if (null != scheduler) {
-            this.addCompleteFuturePool = new SafeQueueingFuturePool<Void>(scheduler.getFuturePool(streamName));
-        } else {
-            this.addCompleteFuturePool = null;
-        }
         assert(!this.immediateFlushEnabled || (null != this.scheduler));
         this.lastTransmit = Stopwatch.createStarted();
     }
@@ -360,11 +341,8 @@
     }
 
     @VisibleForTesting
-    FuturePool getFuturePool() {
-        if (null == scheduler) {
-            return null;
-        }
-        return scheduler.getFuturePool(streamName);
+    ScheduledExecutorService getFuturePool() {
+        return scheduler.chooseExecutor(streamName);
     }
 
     @VisibleForTesting
@@ -471,21 +449,15 @@
     }
 
     @Override
-    public Future<Void> asyncClose() {
+    public CompletableFuture<Void> asyncClose() {
         return closeInternal(false);
     }
 
     @Override
-    public Future<Void> asyncAbort() {
+    public CompletableFuture<Void> asyncAbort() {
         return closeInternal(true);
     }
 
-    private void flushAddCompletes() {
-        if (null != addCompleteFuturePool) {
-            addCompleteFuturePool.close();
-        }
-    }
-
     private synchronized void abortPacket(BKTransmitPacket packet) {
         long numRecords = 0;
         if (null != packet) {
@@ -495,7 +467,7 @@
             if (BKException.Code.OK == rc) {
                 rc = BKException.Code.InterruptedException;
             }
-            Throwable reason = new WriteCancelledException(streamName, FutureUtils.transmitException(rc));
+            Throwable reason = new WriteCancelledException(streamName, Utils.transmitException(rc));
             recordSet.abortTransmit(reason);
         }
         LOG.info("Stream {} aborted {} writes", fullyQualifiedLogSegment, numRecords);
@@ -509,21 +481,13 @@
         }
     }
 
-    private synchronized long getPendingAddCompleteCount() {
-        if (null != addCompleteFuturePool) {
-            return addCompleteFuturePool.size();
-        } else {
-            return 0;
-        }
-    }
-
-    private Future<Void> closeInternal(boolean abort) {
-        Promise<Void> closePromise;
+    private CompletableFuture<Void> closeInternal(boolean abort) {
+        CompletableFuture<Void> closePromise;
         synchronized (this) {
             if (null != closeFuture) {
                 return closeFuture;
             }
-            closePromise = closeFuture = new Promise<Void>();
+            closePromise = closeFuture = new CompletableFuture<Void>();
         }
 
         AtomicReference<Throwable> throwExc = new AtomicReference<Throwable>(null);
@@ -533,7 +497,7 @@
 
     private void closeInternal(final boolean abort,
                                final AtomicReference<Throwable> throwExc,
-                               final Promise<Void> closePromise) {
+                               final CompletableFuture<Void> closePromise) {
         // clean stats resources
         this.transmitOutstandingLogger.unregisterGauge("requests", transmitOutstandingGauge);
         this.writeLimiter.close();
@@ -560,7 +524,7 @@
         if (!abort && !isLogSegmentInError()) {
             this.enforceLock = false;
             LOG.info("Flushing before closing log segment {}", getFullyQualifiedLogSegment());
-            flushAndCommit().addEventListener(new FutureEventListener<Long>() {
+            flushAndCommit().whenComplete(new FutureEventListener<Long>() {
                 @Override
                 public void onSuccess(Long value) {
                     abortTransmitPacketOnClose(abort, throwExc, closePromise);
@@ -580,11 +544,11 @@
 
     private void abortTransmitPacketOnClose(final boolean abort,
                                             final AtomicReference<Throwable> throwExc,
-                                            final Promise<Void> closePromise) {
+                                            final CompletableFuture<Void> closePromise) {
         LOG.info("Closing BKPerStreamLogWriter (abort={}) for {} :" +
-                        " lastDLSN = {} outstandingTransmits = {} writesPendingTransmit = {} addCompletesPending = {}",
+                        " lastDLSN = {} outstandingTransmits = {} writesPendingTransmit = {}",
                 new Object[]{abort, fullyQualifiedLogSegment, getLastDLSN(),
-                        outstandingTransmits.get(), getWritesPendingTransmit(), getPendingAddCompleteCount()});
+                        outstandingTransmits.get(), getWritesPendingTransmit()});
 
         // Save the current packet to reset, leave a new empty packet to avoid a race with
         // addCompleteDeferredProcessing.
@@ -602,7 +566,6 @@
             packetPreviousSaved.addTransmitCompleteListener(new FutureEventListener<Integer>() {
                 @Override
                 public void onSuccess(Integer transmitResult) {
-                    flushAddCompletes();
                     abortPacket(packetCurrentSaved);
                 }
                 @Override
@@ -620,7 +583,7 @@
 
     private void closeLedgerOnClose(final boolean abort,
                                     final AtomicReference<Throwable> throwExc,
-                                    final Promise<Void> closePromise) {
+                                    final CompletableFuture<Void> closePromise) {
         // close the log segment if it isn't in error state, so all the outstanding addEntry(s) will callback.
         if (null == throwExc.get() && !isLogSegmentInError()) {
             // Synchronous closing the ledger handle, if we couldn't close a ledger handle successfully.
@@ -644,16 +607,16 @@
 
     private void completeClosePromise(final boolean abort,
                                       final AtomicReference<Throwable> throwExc,
-                                      final Promise<Void> closePromise) {
+                                      final CompletableFuture<Void> closePromise) {
         // If add entry failed because of closing ledger above, we don't need to fail the close operation
         if (!abort && null == throwExc.get() && shouldFailCompleteLogSegment()) {
             throwExc.set(new BKTransmitException("Closing an errored stream : ", transmitResult.get()));
         }
 
         if (null == throwExc.get()) {
-            FutureUtils.setValue(closePromise, null);
+            FutureUtils.complete(closePromise, null);
         } else {
-            FutureUtils.setException(closePromise, throwExc.get());
+            FutureUtils.completeExceptionally(closePromise, throwExc.get());
         }
     }
 
@@ -664,12 +627,12 @@
     }
 
     @Override
-    synchronized public Future<DLSN> asyncWrite(LogRecord record) {
+    synchronized public CompletableFuture<DLSN> asyncWrite(LogRecord record) {
         return asyncWrite(record, true);
     }
 
-    synchronized public Future<DLSN> asyncWrite(LogRecord record, boolean flush) {
-        Future<DLSN> result = null;
+    synchronized public CompletableFuture<DLSN> asyncWrite(LogRecord record, boolean flush) {
+        CompletableFuture<DLSN> result = null;
         try {
             if (record.isControl()) {
                 // we don't pack control records with user records together
@@ -677,7 +640,7 @@
                 try {
                     transmit();
                 } catch (IOException ioe) {
-                    return Future.exception(new WriteCancelledException(fullyQualifiedLogSegment, ioe));
+                    return FutureUtils.exception(new WriteCancelledException(fullyQualifiedLogSegment, ioe));
                 }
                 result = writeControlLogRecord(record);
                 transmit();
@@ -685,7 +648,7 @@
                 result = writeUserRecord(record);
                 if (!isDurableWriteEnabled) {
                     // we have no idea about the DLSN if durability is turned off.
-                    result = Future.value(DLSN.InvalidDLSN);
+                    result = FutureUtils.value(DLSN.InvalidDLSN);
                 }
                 if (flush) {
                     flushIfNeeded();
@@ -697,7 +660,7 @@
             if (null != result) {
                 LOG.error("Overriding first result with flush failure {}", result);
             }
-            result = Future.exception(ioe);
+            result = FutureUtils.exception(ioe);
 
             // Flush to ensure any prev. writes with flush=false are flushed despite failure.
             flushIfNeededNoThrow();
@@ -705,7 +668,7 @@
         return result;
     }
 
-    synchronized private Future<DLSN> writeUserRecord(LogRecord record) throws IOException {
+    synchronized private CompletableFuture<DLSN> writeUserRecord(LogRecord record) throws IOException {
         if (null != closeFuture) {
             throw new WriteException(fullyQualifiedLogSegment, BKException.getMessage(BKException.Code.LedgerClosedException));
         }
@@ -737,7 +700,7 @@
         // Internally generated log records don't increment the count
         // writeInternal will always set a count regardless of whether it was
         // incremented or not.
-        Future<DLSN> future = null;
+        CompletableFuture<DLSN> future = null;
         try {
             // increment the position for the record to write
             // if the record is failed to write, it would be decremented.
@@ -759,12 +722,9 @@
         }
 
         // Track outstanding requests and return the future.
-        return future.ensure(new Function0<BoxedUnit>() {
-            public BoxedUnit apply() {
-                pendingWrites.dec();
-                writeLimiter.release();
-                return null;
-            }
+        return FutureUtils.ensure(future, () -> {
+            pendingWrites.dec();
+            writeLimiter.release();
         });
     }
 
@@ -777,7 +737,7 @@
                 (transmitResult.get() != BKException.Code.LedgerClosedException);
     }
 
-    synchronized public Future<DLSN> writeInternal(LogRecord record)
+    synchronized public CompletableFuture<DLSN> writeInternal(LogRecord record)
             throws LogRecordTooLongException, LockingException, BKTransmitException,
                    WriteException, InvalidEnvelopedEntryException {
         int logRecordSize = record.getPersistentSize();
@@ -802,8 +762,8 @@
             record.setPositionWithinLogSegment(positionWithinLogSegment);
         }
 
-        Promise<DLSN> writePromise = new Promise<DLSN>();
-        writePromise.addEventListener(new OpStatsListener<DLSN>(writeTime));
+        CompletableFuture<DLSN> writePromise = new CompletableFuture<DLSN>();
+        writePromise.whenComplete(new OpStatsListener<DLSN>(writeTime));
         recordSetWriter.writeRecord(record, writePromise);
 
         if (record.getTransactionId() < lastTxId) {
@@ -818,7 +778,7 @@
         return writePromise;
     }
 
-    synchronized private Future<DLSN> writeControlLogRecord()
+    synchronized private CompletableFuture<DLSN> writeControlLogRecord()
             throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
                    LockingException, LogRecordTooLongException {
         LogRecord controlRec = new LogRecord(lastTxId, DistributedLogConstants.CONTROL_RECORD_CONTENT);
@@ -826,7 +786,7 @@
         return writeControlLogRecord(controlRec);
     }
 
-    synchronized private Future<DLSN> writeControlLogRecord(LogRecord record)
+    synchronized private CompletableFuture<DLSN> writeControlLogRecord(LogRecord record)
             throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
                    LockingException, LogRecordTooLongException {
         return writeInternal(record);
@@ -851,12 +811,12 @@
      * as read-only in the metadata. No appends to the
      * stream will be allowed after this point
      */
-    public Future<Long> markEndOfStream() {
+    public CompletableFuture<Long> markEndOfStream() {
         synchronized (this) {
             try {
                 writeEndOfStreamMarker();
             } catch (IOException e) {
-                return Future.exception(e);
+                return FutureUtils.exception(e);
             }
             streamEnded = true;
         }
@@ -905,60 +865,60 @@
     }
 
     @Override
-    public synchronized Future<Long> flush() {
+    public synchronized CompletableFuture<Long> flush() {
         try {
             checkStateBeforeTransmit();
         } catch (WriteException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         }
 
-        Future<Integer> transmitFuture;
+        CompletableFuture<Integer> transmitFuture;
         try {
             transmitFuture = transmit();
         } catch (BKTransmitException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         } catch (LockingException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         } catch (WriteException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         } catch (InvalidEnvelopedEntryException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         }
 
         if (null == transmitFuture) {
             if (null != packetPrevious) {
                 transmitFuture = packetPrevious.getTransmitFuture();
             }  else {
-                return Future.value(getLastTxIdAcknowledged());
+                return FutureUtils.value(getLastTxIdAcknowledged());
             }
         }
 
-        return transmitFuture.flatMap(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
+        return transmitFuture.thenCompose(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
     }
 
     @Override
-    public synchronized Future<Long> commit() {
+    public synchronized CompletableFuture<Long> commit() {
         // we don't pack control records with user records together
         // so transmit current output buffer if possible
-        Future<Integer> transmitFuture;
+        CompletableFuture<Integer> transmitFuture;
         try {
             try {
                 transmitFuture = transmit();
             } catch (IOException ioe) {
-                return Future.exception(ioe);
+                return FutureUtils.exception(ioe);
             }
             if (null == transmitFuture) {
                 writeControlLogRecord();
                 return flush();
             }
         } catch (IOException ioe) {
-            return Future.exception(ioe);
+            return FutureUtils.exception(ioe);
         }
-        return transmitFuture.flatMap(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
+        return transmitFuture.thenCompose(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
     }
 
-    Future<Long> flushAndCommit() {
-        return flush().flatMap(COMMIT_AFTER_FLUSH_FUNC);
+    CompletableFuture<Long> flushAndCommit() {
+        return flush().thenCompose(COMMIT_AFTER_FLUSH_FUNC);
     }
 
     void flushIfNeededNoThrow() {
@@ -1054,7 +1014,7 @@
      * @throws WriteException if failed to create the envelope for the data to transmit
      * @throws InvalidEnvelopedEntryException when built an invalid enveloped entry
      */
-    private Future<Integer> transmit()
+    private CompletableFuture<Integer> transmit()
         throws BKTransmitException, LockingException, WriteException, InvalidEnvelopedEntryException {
         EntryBuffer recordSetToTransmit;
         transmitLock.lock();
@@ -1183,10 +1143,11 @@
             }
         }
 
-        if (null != addCompleteFuturePool) {
+        if (null != scheduler) {
             final Stopwatch queuedTime = Stopwatch.createStarted();
-            addCompleteFuturePool.apply(new Function0<Void>() {
-                public Void apply() {
+            scheduler.submit(streamName, new Callable<Void>() {
+                @Override
+                public Void call() {
                     final Stopwatch deferredTime = Stopwatch.createStarted();
                     addCompleteQueuedTime.registerSuccessfulEvent(queuedTime.elapsed(TimeUnit.MICROSECONDS));
                     addCompleteDeferredProcessing(transmitPacket, entryId, effectiveRC.get());
@@ -1198,7 +1159,7 @@
                     return String.format("AddComplete(Stream=%s, entryId=%d, rc=%d)",
                             fullyQualifiedLogSegment, entryId, rc);
                 }
-            }).addEventListener(new FutureEventListener<Void>() {
+            }).whenComplete(new FutureEventListener<Void>() {
                 @Override
                 public void onSuccess(Void done) {
                 }
@@ -1278,7 +1239,7 @@
         if (BKException.Code.OK == transmitResult.get()) {
             recordSet.completeTransmit(logSegmentSequenceNumber, entryId);
         } else {
-            recordSet.abortTransmit(FutureUtils.transmitException(transmitResult.get()));
+            recordSet.abortTransmit(Utils.transmitException(transmitResult.get()));
         }
 
         if (cancelPendingPromises) {
@@ -1292,7 +1253,7 @@
             }
             packetCurrentSaved.getRecordSet().abortTransmit(
                     new WriteCancelledException(streamName,
-                            FutureUtils.transmitException(transmitResult.get())));
+                            Utils.transmitException(transmitResult.get())));
         }
     }
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
index fdb29f3..1293d00 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
@@ -17,9 +17,25 @@
  */
 package org.apache.distributedlog;
 
+import static org.apache.distributedlog.impl.ZKLogSegmentFilters.WRITE_HANDLE_FILTER;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+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.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
 import org.apache.distributedlog.exceptions.EndOfStreamException;
@@ -43,34 +59,14 @@
 import org.apache.distributedlog.util.Allocator;
 import org.apache.distributedlog.util.DLUtils;
 import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.util.FutureUtils;
-import org.apache.distributedlog.util.FutureUtils.FutureEventListenerRunnable;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Transaction;
-import org.apache.distributedlog.util.PermitLimiter;
+import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.distributedlog.impl.ZKLogSegmentFilters.WRITE_HANDLE_FILTER;
 
 /**
  * Log Handler for Writers.
@@ -108,7 +104,7 @@
     protected final boolean validateLogSegmentSequenceNumber;
     protected final int regionId;
     protected final RollingPolicy rollingPolicy;
-    protected Future<? extends DistributedLock> lockFuture = null;
+    protected CompletableFuture<? extends DistributedLock> lockFuture = null;
     protected final PermitLimiter writeLimiter;
     protected final FeatureProvider featureProvider;
     protected final DynamicDistributedLogConfiguration dynConf;
@@ -117,16 +113,16 @@
     protected final LinkedList<Long> inprogressLSSNs;
 
     // Fetch LogSegments State: write can continue without full list of log segments while truncation needs
-    private final Future<Versioned<List<LogSegmentMetadata>>> fetchForWrite;
-    private Future<Versioned<List<LogSegmentMetadata>>> fetchForTruncation;
+    private final CompletableFuture<Versioned<List<LogSegmentMetadata>>> fetchForWrite;
+    private CompletableFuture<Versioned<List<LogSegmentMetadata>>> fetchForTruncation;
 
     // Recover Functions
     private final RecoverLogSegmentFunction recoverLogSegmentFunction =
             new RecoverLogSegmentFunction();
-    private final AbstractFunction1<List<LogSegmentMetadata>, Future<Long>> recoverLogSegmentsFunction =
-            new AbstractFunction1<List<LogSegmentMetadata>, Future<Long>>() {
+    private final Function<List<LogSegmentMetadata>, CompletableFuture<Long>> recoverLogSegmentsFunction =
+            new Function<List<LogSegmentMetadata>, CompletableFuture<Long>>() {
                 @Override
-                public Future<Long> apply(List<LogSegmentMetadata> segmentList) {
+                public CompletableFuture<Long> apply(List<LogSegmentMetadata> segmentList) {
                     LOG.info("Initiating Recovery For {} : {}", getFullyQualifiedName(), segmentList);
                     // if lastLedgerRollingTimeMillis is not updated, we set it to now.
                     synchronized (BKLogWriteHandler.this) {
@@ -145,8 +141,11 @@
                         }
                     }
 
-                    return FutureUtils.processList(segmentList, recoverLogSegmentFunction, scheduler).map(
-                            GetLastTxIdFunction.INSTANCE);
+                    return FutureUtils.processList(
+                        segmentList,
+                        recoverLogSegmentFunction,
+                        scheduler
+                    ).thenApply(GetLastTxIdFunction.INSTANCE);
                 }
             };
 
@@ -232,30 +231,30 @@
         deleteOpStats = segmentsStatsLogger.getOpStatsLogger("delete");
     }
 
-    private Future<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFetch(
+    private CompletableFuture<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFetch(
             final Comparator<LogSegmentMetadata> comparator) {
-        final Promise<List<LogSegmentMetadata>> promise = new Promise<List<LogSegmentMetadata>>();
-        fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+        final CompletableFuture<List<LogSegmentMetadata>> promise = new CompletableFuture<List<LogSegmentMetadata>>();
+        fetchForWrite.whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
             @Override
             public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
+                promise.completeExceptionally(cause);
             }
 
             @Override
             public void onSuccess(Versioned<List<LogSegmentMetadata>> result) {
                 try {
-                    FutureUtils.setValue(promise, getCachedLogSegments(comparator));
+                    promise.complete(getCachedLogSegments(comparator));
                 } catch (UnexpectedException e) {
-                    FutureUtils.setException(promise, e);
+                    promise.completeExceptionally(e);
                 }
             }
         });
         return promise;
     }
 
-    private Future<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFullFetch(
+    private CompletableFuture<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFullFetch(
             final Comparator<LogSegmentMetadata> comparator) {
-        Future<Versioned<List<LogSegmentMetadata>>> result;
+        CompletableFuture<Versioned<List<LogSegmentMetadata>>> result;
         synchronized (this) {
             if (null == fetchForTruncation) {
                 fetchForTruncation = readLogSegmentsFromStore(
@@ -266,19 +265,19 @@
             result = fetchForTruncation;
         }
 
-        final Promise<List<LogSegmentMetadata>> promise = new Promise<List<LogSegmentMetadata>>();
-        result.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+        final CompletableFuture<List<LogSegmentMetadata>> promise = new CompletableFuture<List<LogSegmentMetadata>>();
+        result.whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
             @Override
             public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
+                FutureUtils.completeExceptionally(promise, cause);
             }
 
             @Override
             public void onSuccess(Versioned<List<LogSegmentMetadata>> result) {
                 try {
-                    FutureUtils.setValue(promise, getCachedLogSegments(comparator));
+                    FutureUtils.complete(promise, getCachedLogSegments(comparator));
                 } catch (UnexpectedException e) {
-                    FutureUtils.setException(promise, e);
+                    FutureUtils.completeExceptionally(promise, e);
                 }
             }
         });
@@ -374,7 +373,7 @@
      *
      * @return future represents the lock result
      */
-    Future<? extends DistributedLock> lockHandler() {
+    CompletableFuture<? extends DistributedLock> lockHandler() {
         if (null != lockFuture) {
             return lockFuture;
         }
@@ -382,11 +381,11 @@
         return lockFuture;
     }
 
-    Future<Void> unlockHandler() {
+    CompletableFuture<Void> unlockHandler() {
         if (null != lockFuture) {
             return lock.asyncClose();
         } else {
-            return Future.Void();
+            return FutureUtils.Void();
         }
     }
 
@@ -483,23 +482,23 @@
     }
 
     protected BKLogSegmentWriter doStartLogSegment(long txId, boolean bestEffort, boolean allowMaxTxID) throws IOException {
-        return FutureUtils.result(asyncStartLogSegment(txId, bestEffort, allowMaxTxID));
+        return Utils.ioResult(asyncStartLogSegment(txId, bestEffort, allowMaxTxID));
     }
 
-    protected Future<BKLogSegmentWriter> asyncStartLogSegment(final long txId,
+    protected CompletableFuture<BKLogSegmentWriter> asyncStartLogSegment(final long txId,
                                                               final boolean bestEffort,
                                                               final boolean allowMaxTxID) {
-        final Promise<BKLogSegmentWriter> promise = new Promise<BKLogSegmentWriter>();
+        final CompletableFuture<BKLogSegmentWriter> promise = new CompletableFuture<BKLogSegmentWriter>();
         try {
             lock.checkOwnershipAndReacquire();
         } catch (LockingException e) {
-            FutureUtils.setException(promise, e);
+            FutureUtils.completeExceptionally(promise, e);
             return promise;
         }
-        fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+        fetchForWrite.whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
             @Override
             public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
+                FutureUtils.completeExceptionally(promise, cause);
             }
 
             @Override
@@ -513,11 +512,11 @@
     protected void doStartLogSegment(final long txId,
                                      final boolean bestEffort,
                                      final boolean allowMaxTxID,
-                                     final Promise<BKLogSegmentWriter> promise) {
+                                     final CompletableFuture<BKLogSegmentWriter> promise) {
         // validate the tx id
         if ((txId < 0) ||
                 (!allowMaxTxID && (txId == DistributedLogConstants.MAX_TXID))) {
-            FutureUtils.setException(promise, new IOException("Invalid Transaction Id " + txId));
+            FutureUtils.completeExceptionally(promise, new IOException("Invalid Transaction Id " + txId));
             return;
         }
 
@@ -525,11 +524,11 @@
         if (txId < highestTxIdWritten) {
             if (highestTxIdWritten == DistributedLogConstants.MAX_TXID) {
                 LOG.error("We've already marked the stream as ended and attempting to start a new log segment");
-                FutureUtils.setException(promise, new EndOfStreamException("Writing to a stream after it has been marked as completed"));
+                FutureUtils.completeExceptionally(promise, new EndOfStreamException("Writing to a stream after it has been marked as completed"));
                 return;
             } else {
                 LOG.error("We've already seen TxId {} the max TXId is {}", txId, highestTxIdWritten);
-                FutureUtils.setException(promise, new TransactionIdOutOfOrderException(txId, highestTxIdWritten));
+                FutureUtils.completeExceptionally(promise, new TransactionIdOutOfOrderException(txId, highestTxIdWritten));
                 return;
             }
         }
@@ -554,7 +553,7 @@
         }
 
         logSegmentAllocator.tryObtain(txn, NULL_OP_LISTENER)
-                .addEventListener(new FutureEventListener<LogSegmentEntryWriter>() {
+                .whenComplete(new FutureEventListener<LogSegmentEntryWriter>() {
 
             @Override
             public void onSuccess(LogSegmentEntryWriter entryWriter) {
@@ -574,13 +573,13 @@
         });
     }
 
-    private void failStartLogSegment(Promise<BKLogSegmentWriter> promise,
+    private void failStartLogSegment(CompletableFuture<BKLogSegmentWriter> promise,
                                      boolean bestEffort,
                                      Throwable cause) {
         if (bestEffort) {
-            FutureUtils.setValue(promise, null);
+            FutureUtils.complete(promise, null);
         } else {
-            FutureUtils.setException(promise, cause);
+            FutureUtils.completeExceptionally(promise, cause);
         }
     }
 
@@ -591,7 +590,7 @@
                                             final long txId,
                                             final LogSegmentEntryWriter entryWriter,
                                             boolean bestEffort,
-                                            final Promise<BKLogSegmentWriter> promise) {
+                                            final CompletableFuture<BKLogSegmentWriter> promise) {
         final long logSegmentSeqNo;
         try {
             FailpointUtils.checkFailPoint(
@@ -626,12 +625,12 @@
         LOG.debug("Try storing MaxTxId in startLogSegment  {} {}", inprogressZnodePath, txId);
         storeMaxTxId(txn, maxTxId, txId);
 
-        txn.execute().addEventListener(FutureEventListenerRunnable.of(new FutureEventListener<Void>() {
+        txn.execute().whenCompleteAsync(new FutureEventListener<Void>() {
 
             @Override
             public void onSuccess(Void value) {
                 try {
-                    FutureUtils.setValue(promise, new BKLogSegmentWriter(
+                    FutureUtils.complete(promise, new BKLogSegmentWriter(
                             getFullyQualifiedName(),
                             l.getSegmentName(),
                             conf,
@@ -656,7 +655,7 @@
             public void onFailure(Throwable cause) {
                 failStartLogSegment(promise, false, cause);
             }
-        }, scheduler));
+        }, scheduler);
     }
 
     boolean shouldStartNewSegment(BKLogSegmentWriter writer) {
@@ -672,21 +671,21 @@
      * the firstTxId of the ledger matches firstTxId for the segment we are
      * trying to finalize.
      */
-    Future<LogSegmentMetadata> completeAndCloseLogSegment(final BKLogSegmentWriter writer) {
-        final Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
+    CompletableFuture<LogSegmentMetadata> completeAndCloseLogSegment(final BKLogSegmentWriter writer) {
+        final CompletableFuture<LogSegmentMetadata> promise = new CompletableFuture<LogSegmentMetadata>();
         completeAndCloseLogSegment(writer, promise);
         return promise;
     }
 
     private void completeAndCloseLogSegment(final BKLogSegmentWriter writer,
-                                            final Promise<LogSegmentMetadata> promise) {
-        writer.asyncClose().addEventListener(new FutureEventListener<Void>() {
+                                            final CompletableFuture<LogSegmentMetadata> promise) {
+        writer.asyncClose().whenComplete(new FutureEventListener<Void>() {
             @Override
             public void onSuccess(Void value) {
                 // in theory closeToFinalize should throw exception if a stream is in error.
                 // just in case, add another checking here to make sure we don't close log segment is a stream is in error.
                 if (writer.shouldFailCompleteLogSegment()) {
-                    FutureUtils.setException(promise,
+                    FutureUtils.completeExceptionally(promise,
                             new IOException("LogSegmentWriter for " + writer.getFullyQualifiedLogSegment() + " is already in error."));
                     return;
                 }
@@ -704,7 +703,7 @@
 
             @Override
             public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
+                FutureUtils.completeExceptionally(promise, cause);
             }
         });
     }
@@ -791,7 +790,7 @@
             int recordCount,
             long lastEntryId,
             long lastSlotId) throws IOException {
-        Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
+        CompletableFuture<LogSegmentMetadata> promise = new CompletableFuture<LogSegmentMetadata>();
         doCompleteAndCloseLogSegment(
                 inprogressZnodeName,
                 logSegmentSeqNo,
@@ -802,7 +801,7 @@
                 lastEntryId,
                 lastSlotId,
                 promise);
-        return FutureUtils.result(promise);
+        return Utils.ioResult(promise);
     }
 
     protected void doCompleteAndCloseLogSegment(final String inprogressZnodeName,
@@ -813,11 +812,11 @@
                                                 final int recordCount,
                                                 final long lastEntryId,
                                                 final long lastSlotId,
-                                                final Promise<LogSegmentMetadata> promise) {
-        fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+                                                final CompletableFuture<LogSegmentMetadata> promise) {
+        fetchForWrite.whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
             @Override
             public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
+                FutureUtils.completeExceptionally(promise, cause);
             }
 
             @Override
@@ -845,11 +844,11 @@
             int recordCount,
             long lastEntryId,
             long lastSlotId,
-            final Promise<LogSegmentMetadata> promise) {
+            final CompletableFuture<LogSegmentMetadata> promise) {
         try {
             lock.checkOwnershipAndReacquire();
         } catch (IOException ioe) {
-            FutureUtils.setException(promise, ioe);
+            FutureUtils.completeExceptionally(promise, ioe);
             return;
         }
 
@@ -858,7 +857,7 @@
 
         // validate log segment
         if (inprogressLogSegment.getLogSegmentId() != logSegmentId) {
-            FutureUtils.setException(promise, new IOException(
+            FutureUtils.completeExceptionally(promise, new IOException(
                 "Active ledger has different ID to inprogress. "
                     + inprogressLogSegment.getLogSegmentId() + " found, "
                     + logSegmentId + " expected"));
@@ -866,7 +865,7 @@
         }
         // validate the transaction id
         if (inprogressLogSegment.getFirstTxId() != firstTxId) {
-            FutureUtils.setException(promise, new IOException("Transaction id not as expected, "
+            FutureUtils.completeExceptionally(promise, new IOException("Transaction id not as expected, "
                 + inprogressLogSegment.getFirstTxId() + " found, " + firstTxId + " expected"));
             return;
         }
@@ -874,7 +873,7 @@
         if (validateLogSegmentSequenceNumber) {
             synchronized (inprogressLSSNs) {
                 if (inprogressLSSNs.isEmpty()) {
-                    FutureUtils.setException(promise, new UnexpectedException(
+                    FutureUtils.completeExceptionally(promise, new UnexpectedException(
                             "Didn't find matched inprogress log segments when completing inprogress "
                                     + inprogressLogSegment));
                     return;
@@ -886,7 +885,7 @@
                 // it should also be same as the least inprogress log segment sequence number tracked in {@link inprogressLSSNs}
                 if ((inprogressLogSegment.getLogSegmentSequenceNumber() != logSegmentSeqNo) ||
                         (leastInprogressLSSN != logSegmentSeqNo)) {
-                    FutureUtils.setException(promise, new UnexpectedException(
+                    FutureUtils.completeExceptionally(promise, new UnexpectedException(
                             "Didn't find matched inprogress log segments when completing inprogress "
                                     + inprogressLogSegment));
                     return;
@@ -906,7 +905,7 @@
             LOG.warn("Unexpected max ledger sequence number {} found while completing log segment {} for {}",
                     new Object[] { maxLogSegmentSequenceNo.getSequenceNumber(), logSegmentSeqNo, getFullyQualifiedName() });
             if (validateLogSegmentSequenceNumber) {
-                FutureUtils.setException(promise, new DLIllegalStateException("Unexpected max log segment sequence number "
+                FutureUtils.completeExceptionally(promise, new DLIllegalStateException("Unexpected max log segment sequence number "
                         + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName()
                         + ", expected " + (logSegmentSeqNo - 1)));
                 return;
@@ -919,7 +918,7 @@
         try {
             startSequenceId = computeStartSequenceId(inprogressLogSegment);
         } catch (IOException ioe) {
-            FutureUtils.setException(promise, ioe);
+            FutureUtils.completeExceptionally(promise, ioe);
             return;
         }
         // write completed ledger znode
@@ -946,50 +945,45 @@
         LOG.debug("Trying storing LastTxId in Finalize Path {} LastTxId {}", pathForCompletedLedger, lastTxId);
         storeMaxTxId(txn, maxTxId, lastTxId);
 
-        txn.execute().addEventListener(FutureEventListenerRunnable.of(new FutureEventListener<Void>() {
+        txn.execute().whenCompleteAsync(new FutureEventListener<Void>() {
             @Override
             public void onSuccess(Void value) {
                 LOG.info("Completed {} to {} for {} : {}",
                         new Object[] { inprogressZnodeName, completedLogSegment.getSegmentName(),
                                 getFullyQualifiedName(), completedLogSegment });
-                FutureUtils.setValue(promise, completedLogSegment);
+                FutureUtils.complete(promise, completedLogSegment);
             }
 
             @Override
             public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
+                FutureUtils.completeExceptionally(promise, cause);
             }
-        }, scheduler));
+        }, scheduler);
     }
 
-    public Future<Long> recoverIncompleteLogSegments() {
+    public CompletableFuture<Long> recoverIncompleteLogSegments() {
         try {
             FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_RecoverIncompleteLogSegments);
         } catch (IOException ioe) {
-            return Future.exception(ioe);
+            return FutureUtils.exception(ioe);
         }
-        return getCachedLogSegmentsAfterFirstFetch(LogSegmentMetadata.COMPARATOR).flatMap(recoverLogSegmentsFunction);
+        return getCachedLogSegmentsAfterFirstFetch(LogSegmentMetadata.COMPARATOR).thenCompose(recoverLogSegmentsFunction);
     }
 
-    class RecoverLogSegmentFunction extends Function<LogSegmentMetadata, Future<LogSegmentMetadata>> {
+    class RecoverLogSegmentFunction implements Function<LogSegmentMetadata, CompletableFuture<LogSegmentMetadata>> {
 
         @Override
-        public Future<LogSegmentMetadata> apply(final LogSegmentMetadata l) {
+        public CompletableFuture<LogSegmentMetadata> apply(final LogSegmentMetadata l) {
             if (!l.isInProgress()) {
-                return Future.value(l);
+                return FutureUtils.value(l);
             }
 
             LOG.info("Recovering last record in log segment {} for {}.", l, getFullyQualifiedName());
-            return asyncReadLastRecord(l, true, true, true).flatMap(
-                    new AbstractFunction1<LogRecordWithDLSN, Future<LogSegmentMetadata>>() {
-                        @Override
-                        public Future<LogSegmentMetadata> apply(LogRecordWithDLSN lastRecord) {
-                            return completeLogSegment(l, lastRecord);
-                        }
-                    });
+            return asyncReadLastRecord(l, true, true, true).thenCompose(
+                lastRecord -> completeLogSegment(l, lastRecord));
         }
 
-        private Future<LogSegmentMetadata> completeLogSegment(LogSegmentMetadata l,
+        private CompletableFuture<LogSegmentMetadata> completeLogSegment(LogSegmentMetadata l,
                                                               LogRecordWithDLSN lastRecord) {
             LOG.info("Recovered last record in log segment {} for {}.", l, getFullyQualifiedName());
 
@@ -1009,14 +1003,14 @@
                 LOG.error("Unrecoverable corruption has occurred in segment "
                     + l.toString() + " at path " + l.getZkPath()
                     + ". Unable to continue recovery.");
-                return Future.exception(new IOException("Unrecoverable corruption,"
+                return FutureUtils.exception(new IOException("Unrecoverable corruption,"
                     + " please check logs."));
             } else if (endTxId == DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID) {
                 // TODO: Empty ledger - Ideally we should just remove it?
                 endTxId = l.getFirstTxId();
             }
 
-            Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
+            CompletableFuture<LogSegmentMetadata> promise = new CompletableFuture<LogSegmentMetadata>();
             doCompleteAndCloseLogSegment(
                     l.getZNodeName(),
                     l.getLogSegmentSequenceNumber(),
@@ -1032,21 +1026,16 @@
 
     }
 
-    Future<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(final DLSN dlsn) {
+    CompletableFuture<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(final DLSN dlsn) {
         if (DLSN.InvalidDLSN == dlsn) {
             List<LogSegmentMetadata> emptyList = new ArrayList<LogSegmentMetadata>(0);
-            return Future.value(emptyList);
+            return FutureUtils.value(emptyList);
         }
-        return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
-                new AbstractFunction1<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
-                    @Override
-                    public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
-                        return setLogSegmentsOlderThanDLSNTruncated(logSegments, dlsn);
-                    }
-                });
+        return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).thenCompose(
+            logSegments -> setLogSegmentsOlderThanDLSNTruncated(logSegments, dlsn));
     }
 
-    private Future<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(List<LogSegmentMetadata> logSegments,
+    private CompletableFuture<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(List<LogSegmentMetadata> logSegments,
                                                                                   final DLSN dlsn) {
         LOG.debug("Setting truncation status on logs older than {} from {} for {}",
                 new Object[]{dlsn, logSegments, getFullyQualifiedName()});
@@ -1064,7 +1053,7 @@
                     if (null != partialTruncate) {
                         String logMsg = String.format("Potential metadata inconsistency for stream %s at segment %s", getFullyQualifiedName(), l);
                         LOG.error(logMsg);
-                        return Future.exception(new DLIllegalStateException(logMsg));
+                        return FutureUtils.exception(new DLIllegalStateException(logMsg));
                     }
                     LOG.info("{}: Partially truncating log segment {} older than {}.", new Object[] {getFullyQualifiedName(), l, dlsn});
                     partialTruncate = l;
@@ -1096,15 +1085,15 @@
         }
     }
 
-    Future<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTimestamp(final long minTimestampToKeep) {
+    CompletableFuture<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTimestamp(final long minTimestampToKeep) {
         if (minTimestampToKeep >= Utils.nowInMillis()) {
-            return Future.exception(new IllegalArgumentException(
+            return FutureUtils.exception(new IllegalArgumentException(
                     "Invalid timestamp " + minTimestampToKeep + " to purge logs for " + getFullyQualifiedName()));
         }
-        return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
-                new Function<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
+        return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).thenCompose(
+                new Function<List<LogSegmentMetadata>, CompletableFuture<List<LogSegmentMetadata>>>() {
             @Override
-            public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
+            public CompletableFuture<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
                 List<LogSegmentMetadata> purgeList = new ArrayList<LogSegmentMetadata>(logSegments.size());
 
                 int numCandidates = getNumCandidateLogSegmentsToPurge(logSegments);
@@ -1129,38 +1118,35 @@
         });
     }
 
-    Future<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTxnId(final long minTxIdToKeep) {
-        return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
-            new AbstractFunction1<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
-                @Override
-                public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
-                    int numLogSegmentsToProcess;
+    CompletableFuture<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTxnId(final long minTxIdToKeep) {
+        return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).thenCompose(
+            logSegments -> {
+                int numLogSegmentsToProcess;
 
-                    if (minTxIdToKeep < 0) {
-                        // we are deleting the log, we can remove whole log segments
-                        numLogSegmentsToProcess = logSegments.size();
-                    } else {
-                        numLogSegmentsToProcess = getNumCandidateLogSegmentsToPurge(logSegments);
-                    }
-                    List<LogSegmentMetadata> purgeList = Lists.newArrayListWithExpectedSize(numLogSegmentsToProcess);
-                    for (int iterator = 0; iterator < numLogSegmentsToProcess; iterator++) {
-                        LogSegmentMetadata l = logSegments.get(iterator);
-                        if ((minTxIdToKeep < 0) ||
-                            ((l.isTruncated() || !conf.getExplicitTruncationByApplication()) &&
-                            !l.isInProgress() && (l.getLastTxId() < minTxIdToKeep))) {
-                            purgeList.add(l);
-                        } else {
-                            // stop truncating log segments if we find either an inprogress or a partially
-                            // truncated log segment
-                            break;
-                        }
-                    }
-                    return deleteLogSegments(purgeList);
+                if (minTxIdToKeep < 0) {
+                    // we are deleting the log, we can remove whole log segments
+                    numLogSegmentsToProcess = logSegments.size();
+                } else {
+                    numLogSegmentsToProcess = getNumCandidateLogSegmentsToPurge(logSegments);
                 }
+                List<LogSegmentMetadata> purgeList = Lists.newArrayListWithExpectedSize(numLogSegmentsToProcess);
+                for (int iterator = 0; iterator < numLogSegmentsToProcess; iterator++) {
+                    LogSegmentMetadata l = logSegments.get(iterator);
+                    if ((minTxIdToKeep < 0) ||
+                        ((l.isTruncated() || !conf.getExplicitTruncationByApplication()) &&
+                        !l.isInProgress() && (l.getLastTxId() < minTxIdToKeep))) {
+                        purgeList.add(l);
+                    } else {
+                        // stop truncating log segments if we find either an inprogress or a partially
+                        // truncated log segment
+                        break;
+                    }
+                }
+                return deleteLogSegments(purgeList);
             });
     }
 
-    private Future<List<LogSegmentMetadata>> setLogSegmentTruncationStatus(
+    private CompletableFuture<List<LogSegmentMetadata>> setLogSegmentTruncationStatus(
             final List<LogSegmentMetadata> truncateList,
             LogSegmentMetadata partialTruncate,
             DLSN minActiveDLSN) {
@@ -1183,39 +1169,31 @@
             listAfterTruncated.add(newSegment);
         }
 
-        return updateTxn.execute().map(new AbstractFunction1<Void, List<LogSegmentMetadata>>() {
-            @Override
-            public List<LogSegmentMetadata> apply(Void value) {
-                for (int i = 0; i < listToTruncate.size(); i++) {
-                    removeLogSegmentFromCache(listToTruncate.get(i).getSegmentName());
-                    LogSegmentMetadata newSegment = listAfterTruncated.get(i);
-                    addLogSegmentToCache(newSegment.getSegmentName(), newSegment);
-                }
-                return listAfterTruncated;
+        return updateTxn.execute().thenApply(value -> {
+            for (int i = 0; i < listToTruncate.size(); i++) {
+                removeLogSegmentFromCache(listToTruncate.get(i).getSegmentName());
+                LogSegmentMetadata newSegment = listAfterTruncated.get(i);
+                addLogSegmentToCache(newSegment.getSegmentName(), newSegment);
             }
+            return listAfterTruncated;
         });
     }
 
-    private Future<List<LogSegmentMetadata>> deleteLogSegments(
+    private CompletableFuture<List<LogSegmentMetadata>> deleteLogSegments(
             final List<LogSegmentMetadata> logs) {
         if (LOG.isTraceEnabled()) {
             LOG.trace("Purging logs for {} : {}", getFullyQualifiedName(), logs);
         }
         return FutureUtils.processList(logs,
-                new Function<LogSegmentMetadata, Future<LogSegmentMetadata>>() {
-            @Override
-            public Future<LogSegmentMetadata> apply(LogSegmentMetadata segment) {
-                return deleteLogSegment(segment);
-            }
-        }, scheduler);
+            segment -> deleteLogSegment(segment), scheduler);
     }
 
-    private Future<LogSegmentMetadata> deleteLogSegment(
+    private CompletableFuture<LogSegmentMetadata> deleteLogSegment(
             final LogSegmentMetadata ledgerMetadata) {
         LOG.info("Deleting ledger {} for {}", ledgerMetadata, getFullyQualifiedName());
-        final Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
+        final CompletableFuture<LogSegmentMetadata> promise = new CompletableFuture<LogSegmentMetadata>();
         final Stopwatch stopwatch = Stopwatch.createStarted();
-        promise.addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+        promise.whenComplete(new FutureEventListener<LogSegmentMetadata>() {
             @Override
             public void onSuccess(LogSegmentMetadata segment) {
                 deleteOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
@@ -1227,10 +1205,10 @@
             }
         });
         entryStore.deleteLogSegment(ledgerMetadata)
-                .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+                .whenComplete(new FutureEventListener<LogSegmentMetadata>() {
             @Override
             public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
+                FutureUtils.completeExceptionally(promise, cause);
             }
 
             @Override
@@ -1242,14 +1220,14 @@
     }
 
     private void deleteLogSegmentMetadata(final LogSegmentMetadata segmentMetadata,
-                                          final Promise<LogSegmentMetadata> promise) {
+                                          final CompletableFuture<LogSegmentMetadata> promise) {
         Transaction<Object> deleteTxn = metadataStore.transaction();
         metadataStore.deleteLogSegment(deleteTxn, segmentMetadata, new Transaction.OpListener<Void>() {
             @Override
             public void onCommit(Void r) {
                 // purge log segment
                 removeLogSegmentFromCache(segmentMetadata.getZNodeName());
-                promise.setValue(segmentMetadata);
+                promise.complete(segmentMetadata);
             }
 
             @Override
@@ -1257,12 +1235,12 @@
                 if (t instanceof LogSegmentNotFoundException) {
                     // purge log segment
                     removeLogSegmentFromCache(segmentMetadata.getZNodeName());
-                    promise.setValue(segmentMetadata);
+                    promise.complete(segmentMetadata);
                     return;
                 } else {
                     LOG.error("Couldn't purge {} for {}: with error {}",
                             new Object[]{ segmentMetadata, getFullyQualifiedName(), t });
-                    promise.setException(t);
+                    promise.completeExceptionally(t);
                 }
             }
         });
@@ -1270,14 +1248,14 @@
     }
 
     @Override
-    public Future<Void> asyncClose() {
+    public CompletableFuture<Void> asyncClose() {
         return Utils.closeSequence(scheduler,
                 lock,
                 logSegmentAllocator);
     }
 
     @Override
-    public Future<Void> asyncAbort() {
+    public CompletableFuture<Void> asyncAbort() {
         return asyncClose();
     }
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
index bf89823..47301b5 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
@@ -20,17 +20,15 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Ticker;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.exceptions.EndOfStreamException;
 import org.apache.distributedlog.exceptions.IdleReaderException;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.versioning.Versioned;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
 
 import java.io.IOException;
 import java.util.LinkedList;
@@ -48,7 +46,7 @@
     private final AtomicReference<IOException> readerException =
             new AtomicReference<IOException>(null);
     private final int maxReadAheadWaitTime;
-    private Promise<Void> closeFuture;
+    private CompletableFuture<Void> closeFuture;
     private final Optional<Long> startTransactionId;
     private boolean positioned = false;
     private Entry.Reader currentEntry = null;
@@ -101,13 +99,10 @@
                     bkdlm.alertStatsLogger);
         readHandler.registerListener(readAheadReader);
         readHandler.asyncStartFetchLogSegments()
-                .map(new AbstractFunction1<Versioned<List<LogSegmentMetadata>>, BoxedUnit>() {
-                    @Override
-                    public BoxedUnit apply(Versioned<List<LogSegmentMetadata>> logSegments) {
-                        readAheadReader.addStateChangeNotification(BKSyncLogReader.this);
-                        readAheadReader.start(logSegments.getValue());
-                        return BoxedUnit.UNIT;
-                    }
+                .thenApply(logSegments -> {
+                    readAheadReader.addStateChangeNotification(BKSyncLogReader.this);
+                    readAheadReader.start(logSegments.getValue());
+                    return null;
                 });
     }
 
@@ -234,26 +229,28 @@
     }
 
     @Override
-    public Future<Void> asyncClose() {
-        Promise<Void> closePromise;
+    public CompletableFuture<Void> asyncClose() {
+        CompletableFuture<Void> closePromise;
         synchronized (this) {
             if (null != closeFuture) {
                 return closeFuture;
             }
-            closeFuture = closePromise = new Promise<Void>();
+            closeFuture = closePromise = new CompletableFuture<Void>();
         }
         readHandler.unregisterListener(readAheadReader);
         readAheadReader.removeStateChangeNotification(this);
-        Utils.closeSequence(bkdlm.getScheduler(), true,
-                readAheadReader,
-                readHandler
-        ).proxyTo(closePromise);
+        FutureUtils.proxyTo(
+            Utils.closeSequence(bkdlm.getScheduler(), true,
+                    readAheadReader,
+                    readHandler
+            ),
+            closePromise);
         return closePromise;
     }
 
     @Override
     public void close() throws IOException {
-        FutureUtils.result(asyncClose());
+        Utils.ioResult(asyncClose());
     }
 
     //
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java
index 7d33d12..15296b2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java
@@ -17,11 +17,11 @@
  */
 package org.apache.distributedlog;
 
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.util.FutureUtils;
-
 import java.io.IOException;
 import java.util.List;
+import org.apache.distributedlog.api.LogWriter;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.util.Utils;
 
 class BKSyncLogWriter extends BKAbstractLogWriter implements LogWriter {
 
@@ -59,7 +59,7 @@
      */
     @Override
     public void markEndOfStream() throws IOException {
-        FutureUtils.result(getLedgerWriter(DistributedLogConstants.MAX_TXID, true).markEndOfStream());
+        Utils.ioResult(getLedgerWriter(DistributedLogConstants.MAX_TXID, true).markEndOfStream());
         closeAndComplete();
     }
 
@@ -73,7 +73,7 @@
         long highestTransactionId = 0;
         BKLogSegmentWriter writer = getCachedLogWriter();
         if (null != writer) {
-            highestTransactionId = Math.max(highestTransactionId, FutureUtils.result(writer.flush()));
+            highestTransactionId = Math.max(highestTransactionId, Utils.ioResult(writer.flush()));
         }
         return highestTransactionId;
     }
@@ -93,7 +93,7 @@
         long highestTransactionId = 0;
         BKLogSegmentWriter writer = getCachedLogWriter();
         if (null != writer) {
-            highestTransactionId = Math.max(highestTransactionId, FutureUtils.result(writer.commit()));
+            highestTransactionId = Math.max(highestTransactionId, Utils.ioResult(writer.commit()));
             LOG.debug("FlushAndSync Completed");
         } else {
             LOG.debug("FlushAndSync Completed - Nothing to Flush");
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
index 6ed662b..3715327 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
@@ -17,30 +17,28 @@
  */
 package org.apache.distributedlog;
 
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 
 class BKTransmitPacket {
 
     private final EntryBuffer recordSet;
     private final long transmitTime;
-    private final Promise<Integer> transmitComplete;
+    private final CompletableFuture<Integer> transmitComplete;
 
     BKTransmitPacket(EntryBuffer recordSet) {
         this.recordSet = recordSet;
         this.transmitTime = System.nanoTime();
-        this.transmitComplete = new Promise<Integer>();
+        this.transmitComplete = new CompletableFuture<Integer>();
     }
 
     EntryBuffer getRecordSet() {
         return recordSet;
     }
 
-    Promise<Integer> getTransmitFuture() {
+    CompletableFuture<Integer> getTransmitFuture() {
         return transmitComplete;
     }
 
@@ -53,7 +51,7 @@
      *          transmit result code.
      */
     public void notifyTransmitComplete(int transmitResult) {
-        transmitComplete.setValue(transmitResult);
+        transmitComplete.complete(transmitResult);
     }
 
     /**
@@ -66,7 +64,7 @@
      * @see #awaitTransmitComplete(long, TimeUnit)
      */
     void addTransmitCompleteListener(FutureEventListener<Integer> transmitCompleteListener) {
-        transmitComplete.addEventListener(transmitCompleteListener);
+        transmitComplete.whenComplete(transmitCompleteListener);
     }
 
     /**
@@ -79,8 +77,7 @@
      */
     int awaitTransmitComplete(long timeout, TimeUnit unit)
         throws Exception {
-        return Await.result(transmitComplete,
-                Duration.fromTimeUnit(timeout, unit));
+        return FutureUtils.result(transmitComplete, timeout, unit);
     }
 
     public long getTransmitTime() {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
index a7b17f4..2ea3b5d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
@@ -18,17 +18,7 @@
 package org.apache.distributedlog;
 
 import com.google.common.base.Optional;
-import org.apache.distributedlog.ZooKeeperClient.Credentials;
-import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
-import org.apache.distributedlog.exceptions.AlreadyClosedException;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.net.NetUtils;
-import org.apache.distributedlog.util.ConfUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
+import java.util.concurrent.CompletableFuture;
 import org.apache.bookkeeper.client.AsyncCallback;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
@@ -41,6 +31,14 @@
 import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
 import org.apache.bookkeeper.zookeeper.RetryPolicy;
 import org.apache.commons.configuration.ConfigurationException;
+import org.apache.distributedlog.ZooKeeperClient.Credentials;
+import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.net.NetUtils;
+import org.apache.distributedlog.util.ConfUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.zookeeper.KeeperException;
 import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
 import org.jboss.netty.util.HashedWheelTimer;
@@ -198,52 +196,52 @@
     }
 
     // Util functions
-    public Future<LedgerHandle> createLedger(int ensembleSize,
-                                             int writeQuorumSize,
-                                             int ackQuorumSize) {
+    public CompletableFuture<LedgerHandle> createLedger(int ensembleSize,
+                                                        int writeQuorumSize,
+                                                        int ackQuorumSize) {
         BookKeeper bk;
         try {
             bk = get();
         } catch (IOException ioe) {
-            return Future.exception(ioe);
+            return FutureUtils.exception(ioe);
         }
-        final Promise<LedgerHandle> promise = new Promise<LedgerHandle>();
+        final CompletableFuture<LedgerHandle> promise = new CompletableFuture<LedgerHandle>();
         bk.asyncCreateLedger(ensembleSize, writeQuorumSize, ackQuorumSize,
                 BookKeeper.DigestType.CRC32, passwd, new AsyncCallback.CreateCallback() {
                     @Override
                     public void createComplete(int rc, LedgerHandle lh, Object ctx) {
                         if (BKException.Code.OK == rc) {
-                            promise.updateIfEmpty(new Return<LedgerHandle>(lh));
+                            promise.complete(lh);
                         } else {
-                            promise.updateIfEmpty(new Throw<LedgerHandle>(BKException.create(rc)));
+                            promise.completeExceptionally(BKException.create(rc));
                         }
                     }
                 }, null);
         return promise;
     }
 
-    public Future<Void> deleteLedger(long lid,
+    public CompletableFuture<Void> deleteLedger(long lid,
                                      final boolean ignoreNonExistentLedger) {
         BookKeeper bk;
         try {
             bk = get();
         } catch (IOException ioe) {
-            return Future.exception(ioe);
+            return FutureUtils.exception(ioe);
         }
-        final Promise<Void> promise = new Promise<Void>();
+        final CompletableFuture<Void> promise = new CompletableFuture<Void>();
         bk.asyncDeleteLedger(lid, new AsyncCallback.DeleteCallback() {
             @Override
             public void deleteComplete(int rc, Object ctx) {
                 if (BKException.Code.OK == rc) {
-                    promise.updateIfEmpty(new Return<Void>(null));
+                    promise.complete(null);
                 } else if (BKException.Code.NoSuchLedgerExistsException == rc) {
                     if (ignoreNonExistentLedger) {
-                        promise.updateIfEmpty(new Return<Void>(null));
+                        promise.complete(null);
                     } else {
-                        promise.updateIfEmpty(new Throw<Void>(BKException.create(rc)));
+                        promise.completeExceptionally(BKException.create(rc));
                     }
                 } else {
-                    promise.updateIfEmpty(new Throw<Void>(BKException.create(rc)));
+                    promise.completeExceptionally(BKException.create(rc));
                 }
             }
         }, null);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
index 0cb608f..3269f57 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
@@ -22,7 +22,7 @@
 import com.google.common.collect.Sets;
 import org.apache.distributedlog.bk.QuorumConfig;
 import org.apache.distributedlog.feature.DefaultFeatureProvider;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.net.DNSResolverForRacks;
 import org.apache.distributedlog.net.DNSResolverForRows;
 import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -1351,7 +1351,6 @@
      * <p>
      * The setting is only applied for v2 implementation.
      *
-     * @see org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor
      * @return number of resource release threads used by distributedlog namespace.
      */
     public int getNumResourceReleaseThreads() {
@@ -3048,7 +3047,7 @@
     /**
      * Whether to enable per stream stat or not.
      *
-     * @deprecated please use {@link DistributedLogNamespaceBuilder#perLogStatsLogger(StatsLogger)}
+     * @deprecated please use {@link NamespaceBuilder#perLogStatsLogger(StatsLogger)}
      * @return flag to enable per stream stat.
      */
     public boolean getEnablePerStreamStat() {
@@ -3058,7 +3057,7 @@
     /**
      * Set the flag to enable per stream stat or not.
      *
-     * @deprecated please use {@link DistributedLogNamespaceBuilder#perLogStatsLogger(StatsLogger)}
+     * @deprecated please use {@link NamespaceBuilder#perLogStatsLogger(StatsLogger)}
      * @param enabled
      *          flag to enable/disable per stream stat.
      * @return dl configuration.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java b/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
index 617282c..30cd499 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
@@ -19,10 +19,10 @@
 
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.exceptions.LogRecordTooLongException;
 import org.apache.distributedlog.exceptions.WriteException;
 import org.apache.distributedlog.io.CompressionCodec;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 
@@ -342,7 +342,7 @@
          * @throws LogRecordTooLongException if the record is too long
          * @throws WriteException when encountered exception writing the record
          */
-        void writeRecord(LogRecord record, Promise<DLSN> transmitPromise)
+        void writeRecord(LogRecord record, CompletableFuture<DLSN> transmitPromise)
                 throws LogRecordTooLongException, WriteException;
 
         /**
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
index aed47fc..09301aa 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
@@ -30,10 +30,10 @@
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
 
-import org.apache.distributedlog.annotations.DistributedLogAnnotations.Compression;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations.Compression;
 import org.apache.distributedlog.io.CompressionCodec;
 import org.apache.distributedlog.io.CompressionUtils;
-import org.apache.distributedlog.util.BitMaskUtils;
+import org.apache.distributedlog.common.util.BitMaskUtils;
 
 /**
  * An enveloped entry written to BookKeeper.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
index 54858d7..18645d4 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.distributedlog;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.Entry.Writer;
 import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
 import org.apache.distributedlog.exceptions.LogRecordTooLongException;
@@ -24,7 +25,6 @@
 import org.apache.distributedlog.exceptions.WriteException;
 import org.apache.distributedlog.io.Buffer;
 import org.apache.distributedlog.io.CompressionCodec;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,9 +46,9 @@
     private static class WriteRequest {
 
         private final int numRecords;
-        private final Promise<DLSN> promise;
+        private final CompletableFuture<DLSN> promise;
 
-        WriteRequest(int numRecords, Promise<DLSN> promise) {
+        WriteRequest(int numRecords, CompletableFuture<DLSN> promise) {
             this.numRecords = numRecords;
             this.promise = promise;
         }
@@ -89,7 +89,7 @@
 
     @Override
     public synchronized void writeRecord(LogRecord record,
-                                         Promise<DLSN> transmitPromise)
+                                         CompletableFuture<DLSN> transmitPromise)
             throws LogRecordTooLongException, WriteException {
         int logRecordSize = record.getPersistentSize();
         if (logRecordSize > MAX_LOGRECORD_SIZE) {
@@ -121,7 +121,7 @@
     private synchronized void satisfyPromises(long lssn, long entryId) {
         long nextSlotId = 0;
         for (WriteRequest request : writeRequests) {
-            request.promise.setValue(new DLSN(lssn, entryId, nextSlotId));
+            request.promise.complete(new DLSN(lssn, entryId, nextSlotId));
             nextSlotId += request.numRecords;
         }
         writeRequests.clear();
@@ -129,7 +129,7 @@
 
     private synchronized void cancelPromises(Throwable reason) {
         for (WriteRequest request : writeRequests) {
-            request.promise.setException(reason);
+            request.promise.completeExceptionally(reason);
         }
         writeRequests.clear();
     }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
index c5050ec..462ddaa 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
@@ -23,14 +23,13 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
 import org.apache.distributedlog.exceptions.UnsupportedMetadataVersionException;
 import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -590,21 +589,21 @@
                 .build();
     }
 
-    public static Future<LogSegmentMetadata> read(ZooKeeperClient zkc, String path) {
+    public static CompletableFuture<LogSegmentMetadata> read(ZooKeeperClient zkc, String path) {
         return read(zkc, path, false);
     }
 
-    public static Future<LogSegmentMetadata> read(ZooKeeperClient zkc, String path, final boolean skipMinVersionCheck) {
-        final Promise<LogSegmentMetadata> result = new Promise<LogSegmentMetadata>();
+    public static CompletableFuture<LogSegmentMetadata> read(ZooKeeperClient zkc, String path, final boolean skipMinVersionCheck) {
+        final CompletableFuture<LogSegmentMetadata> result = new CompletableFuture<LogSegmentMetadata>();
         try {
             zkc.get().getData(path, false, new AsyncCallback.DataCallback() {
                 @Override
                 public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
                     if (KeeperException.Code.OK.intValue() != rc) {
                         if (KeeperException.Code.NONODE.intValue() == rc) {
-                            FutureUtils.setException(result, new LogSegmentNotFoundException(path));
+                            FutureUtils.completeExceptionally(result, new LogSegmentNotFoundException(path));
                         } else {
-                            FutureUtils.setException(result,
+                            FutureUtils.completeExceptionally(result,
                                     new ZKException("Failed to read log segment metadata from " + path,
                                             KeeperException.Code.get(rc)));
                         }
@@ -612,17 +611,17 @@
                     }
                     try {
                         LogSegmentMetadata metadata = parseData(path, data, skipMinVersionCheck);
-                        FutureUtils.setValue(result, metadata);
+                        FutureUtils.complete(result, metadata);
                     } catch (IOException ie) {
                         LOG.error("Error on parsing log segment metadata from {} : ", path, ie);
-                        result.setException(ie);
+                        result.completeExceptionally(ie);
                     }
                 }
             }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            result.setException(FutureUtils.zkException(e, path));
+            result.completeExceptionally(Utils.zkException(e, path));
         } catch (InterruptedException e) {
-            result.setException(FutureUtils.zkException(e, path));
+            result.completeExceptionally(Utils.zkException(e, path));
         }
         return result;
     }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
index f94a6e0..386a9a1 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
@@ -21,6 +21,9 @@
 import com.google.common.base.Stopwatch;
 import com.google.common.base.Ticker;
 import com.google.common.collect.Lists;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.Function;
 import org.apache.distributedlog.callback.LogSegmentListener;
 import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
@@ -32,19 +35,13 @@
 import org.apache.distributedlog.logsegment.LogSegmentEntryReader;
 import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
 import org.apache.distributedlog.logsegment.LogSegmentFilter;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Function0;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Futures;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.versioning.Versioned;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Function1;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
 
 import java.io.IOException;
 import java.util.LinkedList;
@@ -52,7 +49,6 @@
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
@@ -75,12 +71,9 @@
     // Static Functions
     //
 
-    private static AbstractFunction1<LogSegmentEntryReader, BoxedUnit> START_READER_FUNC = new AbstractFunction1<LogSegmentEntryReader, BoxedUnit>() {
-        @Override
-        public BoxedUnit apply(LogSegmentEntryReader reader) {
-            reader.start();
-            return BoxedUnit.UNIT;
-        }
+    private static Function<LogSegmentEntryReader, Void> START_READER_FUNC = reader -> {
+        reader.start();
+        return null;
     };
 
     //
@@ -91,7 +84,7 @@
 
         private LogSegmentMetadata metadata;
         private final long startEntryId;
-        private Future<LogSegmentEntryReader> openFuture = null;
+        private CompletableFuture<LogSegmentEntryReader> openFuture = null;
         private LogSegmentEntryReader reader = null;
         private boolean isStarted = false;
         private boolean isClosed = false;
@@ -122,7 +115,7 @@
             if (null != openFuture) {
                 return;
             }
-            openFuture = entryStore.openReader(metadata, startEntryId).addEventListener(this);
+            openFuture = entryStore.openReader(metadata, startEntryId).whenComplete(this);
         }
 
         synchronized boolean isReaderStarted() {
@@ -137,16 +130,16 @@
             if (null != reader) {
                 reader.start();
             } else {
-                openFuture.onSuccess(START_READER_FUNC);
+                openFuture.thenApply(START_READER_FUNC);
             }
         }
 
-        synchronized Future<List<Entry.Reader>> readNext() {
+        synchronized CompletableFuture<List<Entry.Reader>> readNext() {
             if (null != reader) {
                 checkCatchingUpStatus(reader);
                 return reader.readNext(numReadAheadEntries);
             } else {
-                return openFuture.flatMap(readFunc);
+                return openFuture.thenCompose(readFunc);
             }
         }
 
@@ -155,14 +148,10 @@
                 reader.onLogSegmentMetadataUpdated(segment);
                 this.metadata = segment;
             } else {
-                openFuture.onSuccess(new AbstractFunction1<LogSegmentEntryReader, BoxedUnit>() {
-                    @Override
-                    public BoxedUnit apply(LogSegmentEntryReader reader) {
-                        reader.onLogSegmentMetadataUpdated(segment);
-                        synchronized (SegmentReader.this) {
-                            SegmentReader.this.metadata = segment;
-                        }
-                        return BoxedUnit.UNIT;
+                openFuture.thenAccept(reader1 -> {
+                    reader1.onLogSegmentMetadataUpdated(segment);
+                    synchronized (SegmentReader.this) {
+                        SegmentReader.this.metadata = segment;
                     }
                 });
             }
@@ -185,28 +174,21 @@
             return isClosed;
         }
 
-        synchronized Future<Void> close() {
+        synchronized CompletableFuture<Void> close() {
             if (null == openFuture) {
-                return Future.Void();
+                return FutureUtils.Void();
             }
-            return openFuture.flatMap(new AbstractFunction1<LogSegmentEntryReader, Future<Void>>() {
-                @Override
-                public Future<Void> apply(LogSegmentEntryReader reader) {
-                    return reader.asyncClose();
-                }
-            }).ensure(new Function0<BoxedUnit>() {
-                @Override
-                public BoxedUnit apply() {
+            return FutureUtils.ensure(
+                openFuture.thenCompose(reader1 -> reader1.asyncClose()),
+                () -> {
                     synchronized (SegmentReader.this) {
                         isClosed = true;
                     }
-                    return null;
-                }
-            });
+                });
         }
     }
 
-    private class ReadEntriesFunc extends AbstractFunction1<LogSegmentEntryReader, Future<List<Entry.Reader>>> {
+    private class ReadEntriesFunc implements Function<LogSegmentEntryReader, CompletableFuture<List<Entry.Reader>>> {
 
         private final int numEntries;
 
@@ -215,7 +197,7 @@
         }
 
         @Override
-        public Future<List<Entry.Reader>> apply(LogSegmentEntryReader reader) {
+        public CompletableFuture<List<Entry.Reader>> apply(LogSegmentEntryReader reader) {
             checkCatchingUpStatus(reader);
             return reader.readNext(numEntries);
         }
@@ -244,14 +226,8 @@
     //
     // Functions
     //
-    private final Function1<LogSegmentEntryReader, Future<List<Entry.Reader>>> readFunc;
-    private final Function0<BoxedUnit> removeClosedSegmentReadersFunc = new Function0<BoxedUnit>() {
-        @Override
-        public BoxedUnit apply() {
-            removeClosedSegmentReaders();
-            return BoxedUnit.UNIT;
-        }
-    };
+    private final Function<LogSegmentEntryReader, CompletableFuture<List<Entry.Reader>>> readFunc;
+    private final Runnable removeClosedSegmentReadersFunc = () -> removeClosedSegmentReaders();
 
     //
     // Resources
@@ -282,7 +258,7 @@
     private final AtomicBoolean started = new AtomicBoolean(false);
     private boolean isInitialized = false;
     private boolean readAheadPaused = false;
-    private Promise<Void> closePromise = null;
+    private CompletableFuture<Void> closePromise = null;
     // segment readers
     private long currentSegmentSequenceNumber;
     private SegmentReader currentSegmentReader;
@@ -344,15 +320,12 @@
 
     private ScheduledFuture<?> scheduleIdleReaderTaskIfNecessary() {
         if (idleWarnThresholdMillis < Integer.MAX_VALUE && idleWarnThresholdMillis > 0) {
-            return scheduler.scheduleAtFixedRate(streamName, new Runnable() {
-                @Override
-                public void run() {
-                    if (!isReaderIdle(idleWarnThresholdMillis, TimeUnit.MILLISECONDS)) {
-                        return;
-                    }
-                    // the readahead has been idle
-                    unsafeCheckIfReadAheadIsIdle();
+            return scheduler.scheduleAtFixedRate(streamName, () -> {
+                if (!isReaderIdle(idleWarnThresholdMillis, TimeUnit.MILLISECONDS)) {
+                    return;
                 }
+                // the readahead has been idle
+                unsafeCheckIfReadAheadIsIdle();
             }, idleWarnThresholdMillis, idleWarnThresholdMillis, TimeUnit.MILLISECONDS);
         }
         return null;
@@ -366,7 +339,7 @@
                     LogSegmentMetadata.COMPARATOR,
                     LogSegmentFilter.DEFAULT_FILTER,
                     null
-            ).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+            ).whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
                 @Override
                 public void onFailure(Throwable cause) {
                     // do nothing here since it would be retried on next idle reader check task
@@ -459,13 +432,13 @@
     }
 
     @Override
-    public Future<Void> asyncClose() {
-        final Promise<Void> closeFuture;
+    public CompletableFuture<Void> asyncClose() {
+        final CompletableFuture<Void> closeFuture;
         synchronized (this) {
             if (null != closePromise) {
                 return closePromise;
             }
-            closePromise = closeFuture = new Promise<Void>();
+            closePromise = closeFuture = new CompletableFuture<Void>();
         }
 
         // cancel the idle reader task
@@ -489,8 +462,8 @@
         return closeFuture;
     }
 
-    private void unsafeAsyncClose(Promise<Void> closePromise) {
-        List<Future<Void>> closeFutures = Lists.newArrayListWithExpectedSize(
+    private void unsafeAsyncClose(CompletableFuture<Void> closePromise) {
+        List<CompletableFuture<Void>> closeFutures = Lists.newArrayListWithExpectedSize(
                 segmentReaders.size() + segmentReadersToClose.size() + 1);
         if (null != currentSegmentReader) {
             segmentReadersToClose.add(currentSegmentReader);
@@ -505,7 +478,9 @@
         for (SegmentReader reader : segmentReadersToClose) {
             closeFutures.add(reader.close());
         }
-        Futures.collect(closeFutures).proxyTo(closePromise);
+        FutureUtils.proxyTo(
+            FutureUtils.collect(closeFutures).thenApply((value) -> null),
+            closePromise);
     }
 
     //
@@ -921,7 +896,9 @@
     private void unsafeMoveToNextLogSegment() {
         if (null != currentSegmentReader) {
             segmentReadersToClose.add(currentSegmentReader);
-            currentSegmentReader.close().ensure(removeClosedSegmentReadersFunc);
+            FutureUtils.ensure(
+                currentSegmentReader.close(),
+                removeClosedSegmentReadersFunc);
             logger.debug("close current segment reader {}", currentSegmentReader.getSegment());
             currentSegmentReader = null;
         }
@@ -971,7 +948,7 @@
     }
 
     private void unsafeReadNext(SegmentReader reader) {
-        reader.readNext().addEventListener(this);
+        reader.readNext().whenComplete(this);
     }
 
     @Override
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
index 9935d5f..bf4e140 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
@@ -31,14 +32,10 @@
 import org.apache.distributedlog.selector.FirstTxIdNotLessThanSelector;
 import org.apache.distributedlog.selector.LastRecordSelector;
 import org.apache.distributedlog.selector.LogRecordSelector;
-import org.apache.distributedlog.util.FutureUtils.FutureEventListenerRunnable;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.BoxedUnit;
 
 /**
  * Utility function for readers
@@ -78,7 +75,7 @@
      *          log segment entry store
      * @return a future with last record.
      */
-    public static Future<LogRecordWithDLSN> asyncReadLastRecord(
+    public static CompletableFuture<LogRecordWithDLSN> asyncReadLastRecord(
             final String streamName,
             final LogSegmentMetadata l,
             final boolean fence,
@@ -116,7 +113,7 @@
      *          threshold dlsn
      * @return a future with last record.
      */
-    public static Future<LogRecordWithDLSN> asyncReadFirstUserRecord(
+    public static CompletableFuture<LogRecordWithDLSN> asyncReadFirstUserRecord(
             final String streamName,
             final LogSegmentMetadata l,
             final int scanStartBatchSize,
@@ -243,14 +240,14 @@
      *          scan context
      * @return a future with the log record.
      */
-    private static Future<LogRecordWithDLSN> asyncReadRecordFromEntries(
+    private static CompletableFuture<LogRecordWithDLSN> asyncReadRecordFromEntries(
             final String streamName,
             final LogSegmentRandomAccessEntryReader reader,
             final LogSegmentMetadata metadata,
             final ExecutorService executorService,
             final ScanContext context,
             final LogRecordSelector selector) {
-        final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
+        final CompletableFuture<LogRecordWithDLSN> promise = new CompletableFuture<LogRecordWithDLSN>();
         final long startEntryId = context.curStartEntryId.get();
         final long endEntryId = context.curEndEntryId.get();
         if (LOG.isDebugEnabled()) {
@@ -271,7 +268,7 @@
                         } catch (IOException ioe) {
                             // exception is only thrown due to bad ledger entry, so it might be corrupted
                             // we shouldn't do anything beyond this point. throw the exception to application
-                            promise.setException(ioe);
+                            promise.completeExceptionally(ioe);
                             return;
                         }
                     }
@@ -282,16 +279,16 @@
                                 new Object[]{streamName, startEntryId, endEntryId,
                                         metadata, record});
                     }
-                    promise.setValue(record);
+                    promise.complete(record);
                 }
 
                 @Override
                 public void onFailure(final Throwable cause) {
-                    promise.setException(cause);
+                    promise.completeExceptionally(cause);
                 }
             };
         reader.readEntries(startEntryId, endEntryId)
-                .addEventListener(FutureEventListenerRunnable.of(readEntriesListener, executorService));
+                .whenCompleteAsync(readEntriesListener, executorService);
         return promise;
     }
 
@@ -343,7 +340,7 @@
             final LogSegmentRandomAccessEntryReader reader,
             final LogSegmentMetadata metadata,
             final ExecutorService executorService,
-            final Promise<LogRecordWithDLSN> promise,
+            final CompletableFuture<LogRecordWithDLSN> promise,
             final ScanContext context,
             final LogRecordSelector selector) {
         FutureEventListener<LogRecordWithDLSN> readEntriesListener =
@@ -356,12 +353,12 @@
                                         metadata, value});
                     }
                     if (null != value) {
-                        promise.setValue(value);
+                        promise.complete(value);
                         return;
                     }
                     if (!context.moveToNextRange()) {
                         // no entries to read again
-                        promise.setValue(null);
+                        promise.complete(null);
                         return;
                     }
                     // scan next range
@@ -376,11 +373,11 @@
 
                 @Override
                 public void onFailure(Throwable cause) {
-                    promise.setException(cause);
+                    promise.completeExceptionally(cause);
                 }
             };
         asyncReadRecordFromEntries(streamName, reader, metadata, executorService, context, selector)
-                .addEventListener(FutureEventListenerRunnable.of(readEntriesListener, executorService));
+                .whenCompleteAsync(readEntriesListener, executorService);
     }
 
     private static void asyncReadRecordFromLogSegment(
@@ -392,7 +389,7 @@
             final int scanMaxBatchSize,
             final boolean includeControl,
             final boolean includeEndOfStream,
-            final Promise<LogRecordWithDLSN> promise,
+            final CompletableFuture<LogRecordWithDLSN> promise,
             final AtomicInteger numRecordsScanned,
             final LogRecordSelector selector,
             final boolean backward,
@@ -402,7 +399,7 @@
             if (LOG.isDebugEnabled()) {
                 LOG.debug("Log segment {} is empty for {}.", new Object[] { metadata, streamName });
             }
-            promise.setValue(null);
+            promise.complete(null);
             return;
         }
         final ScanContext context = new ScanContext(
@@ -413,7 +410,7 @@
                                    promise, context, selector);
     }
 
-    private static Future<LogRecordWithDLSN> asyncReadRecord(
+    private static CompletableFuture<LogRecordWithDLSN> asyncReadRecord(
             final String streamName,
             final LogSegmentMetadata l,
             final boolean fence,
@@ -428,7 +425,7 @@
             final boolean backward,
             final long startEntryId) {
 
-        final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
+        final CompletableFuture<LogRecordWithDLSN> promise = new CompletableFuture<LogRecordWithDLSN>();
 
         FutureEventListener<LogSegmentRandomAccessEntryReader> openReaderListener =
             new FutureEventListener<LogSegmentRandomAccessEntryReader>() {
@@ -438,13 +435,7 @@
                         LOG.debug("{} Opened log segment {} for reading record",
                                 streamName, l);
                     }
-                    promise.ensure(new AbstractFunction0<BoxedUnit>() {
-                        @Override
-                        public BoxedUnit apply() {
-                            reader.asyncClose();
-                            return BoxedUnit.UNIT;
-                        }
-                    });
+                    promise.whenComplete((value, cause) -> reader.asyncClose());
                     if (LOG.isDebugEnabled()) {
                         LOG.debug("{} {} scanning {}.", new Object[]{
                                 (backward ? "backward" : "forward"), streamName, l});
@@ -458,11 +449,11 @@
 
                 @Override
                 public void onFailure(final Throwable cause) {
-                    promise.setException(cause);
+                    promise.completeExceptionally(cause);
                 }
             };
         entryStore.openRandomAccessReader(l, fence)
-                .addEventListener(FutureEventListenerRunnable.of(openReaderListener, executorService));
+                .whenCompleteAsync(openReaderListener, executorService);
         return promise;
     }
 
@@ -499,7 +490,7 @@
      *          how many number of entries to search in parallel
      * @return found log record. none if all transaction ids are less than provided <code>transactionId</code>.
      */
-    public static Future<Optional<LogRecordWithDLSN>> getLogRecordNotLessThanTxId(
+    public static CompletableFuture<Optional<LogRecordWithDLSN>> getLogRecordNotLessThanTxId(
             final String logName,
             final LogSegmentMetadata segment,
             final long transactionId,
@@ -511,30 +502,23 @@
                 // all log records whose transaction id is less than provided transactionId
                 // then return none
                 Optional<LogRecordWithDLSN> noneRecord = Optional.absent();
-                return Future.value(noneRecord);
+                return FutureUtils.value(noneRecord);
             }
         }
 
-        final Promise<Optional<LogRecordWithDLSN>> promise =
-                new Promise<Optional<LogRecordWithDLSN>>();
+        final CompletableFuture<Optional<LogRecordWithDLSN>> promise =
+                new CompletableFuture<Optional<LogRecordWithDLSN>>();
         final FutureEventListener<LogSegmentRandomAccessEntryReader> openReaderListener =
             new FutureEventListener<LogSegmentRandomAccessEntryReader>() {
                 @Override
                 public void onSuccess(final LogSegmentRandomAccessEntryReader reader) {
-                    promise.ensure(new AbstractFunction0<BoxedUnit>() {
-                        @Override
-                        public BoxedUnit apply() {
-                            reader.asyncClose();
-                            return BoxedUnit.UNIT;
-                        }
-
-                    });
+                    promise.whenComplete((value, cause) -> reader.asyncClose());
                     long lastEntryId = reader.getLastAddConfirmed();
                     if (lastEntryId < 0) {
                         // it means that the log segment is created but not written yet or an empty log segment.
                         // it is equivalent to 'all log records whose transaction id is less than provided transactionId'
                         Optional<LogRecordWithDLSN> nonRecord = Optional.absent();
-                        promise.setValue(nonRecord);
+                        promise.complete(nonRecord);
                         return;
                     }
                     // all log records whose transaction id is not less than provided transactionId
@@ -548,15 +532,15 @@
                                 executorService,
                                 new SingleEntryScanContext(0L),
                                 selector
-                        ).addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+                        ).whenComplete(new FutureEventListener<LogRecordWithDLSN>() {
                             @Override
                             public void onSuccess(LogRecordWithDLSN value) {
-                                promise.setValue(Optional.of(selector.result()));
+                                promise.complete(Optional.of(selector.result()));
                             }
 
                             @Override
                             public void onFailure(Throwable cause) {
-                                promise.setException(cause);
+                                promise.completeExceptionally(cause);
                             }
                         });
 
@@ -576,12 +560,12 @@
 
                 @Override
                 public void onFailure(final Throwable cause) {
-                    promise.setException(cause);
+                    promise.completeExceptionally(cause);
                 }
             };
 
         entryStore.openRandomAccessReader(segment, false)
-                .addEventListener(FutureEventListenerRunnable.of(openReaderListener, executorService));
+                .whenCompleteAsync(openReaderListener, executorService);
         return promise;
     }
 
@@ -617,12 +601,12 @@
             final List<Long> entriesToSearch,
             final int nWays,
             final Optional<LogRecordWithDLSN> prevFoundRecord,
-            final Promise<Optional<LogRecordWithDLSN>> promise) {
-        final List<Future<LogRecordWithDLSN>> searchResults =
+            final CompletableFuture<Optional<LogRecordWithDLSN>> promise) {
+        final List<CompletableFuture<LogRecordWithDLSN>> searchResults =
                 Lists.newArrayListWithExpectedSize(entriesToSearch.size());
         for (Long entryId : entriesToSearch) {
             LogRecordSelector selector = new FirstTxIdNotLessThanSelector(transactionId);
-            Future<LogRecordWithDLSN> searchResult = asyncReadRecordFromEntries(
+            CompletableFuture<LogRecordWithDLSN> searchResult = asyncReadRecordFromEntries(
                     logName,
                     reader,
                     segment,
@@ -649,11 +633,11 @@
 
                     @Override
                     public void onFailure(Throwable cause) {
-                        promise.setException(cause);
+                        promise.completeExceptionally(cause);
                     }
                 };
-        Future.collect(searchResults).addEventListener(
-                FutureEventListenerRunnable.of(processSearchResultsListener, executorService));
+        FutureUtils.collect(searchResults).whenCompleteAsync(
+                processSearchResultsListener, executorService);
     }
 
     /**
@@ -668,7 +652,7 @@
             final List<LogRecordWithDLSN> searchResults,
             final int nWays,
             final Optional<LogRecordWithDLSN> prevFoundRecord,
-            final Promise<Optional<LogRecordWithDLSN>> promise) {
+            final CompletableFuture<Optional<LogRecordWithDLSN>> promise) {
         int found = -1;
         for (int i = 0; i < searchResults.size(); i++) {
             LogRecordWithDLSN record = searchResults.get(i);
@@ -678,7 +662,7 @@
             }
         }
         if (found == -1) { // all log records' transaction id is less than provided transaction id
-            promise.setValue(prevFoundRecord);
+            promise.complete(prevFoundRecord);
             return;
         }
         // we found a log record
@@ -691,7 +675,7 @@
         if (foundRecord.getDlsn().getSlotId() != 0L
                 || found == 0
                 || foundRecord.getDlsn().getEntryId() == (searchResults.get(found - 1).getDlsn().getEntryId() + 1)) {
-            promise.setValue(Optional.of(foundRecord));
+            promise.complete(Optional.of(foundRecord));
             return;
         }
 
@@ -702,7 +686,7 @@
                 searchResults.get(found),
                 nWays);
         if (nextSearchBatch.isEmpty()) {
-            promise.setValue(prevFoundRecord);
+            promise.complete(prevFoundRecord);
             return;
         }
         getLogRecordNotLessThanTxIdFromEntries(
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
index a2109f4..04bb9e4 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
@@ -18,7 +18,7 @@
 package org.apache.distributedlog;
 
 import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.util.PermitLimiter;
+import org.apache.distributedlog.common.util.PermitLimiter;
 
 class WriteLimiter {
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
index 4e94984..c4939c0 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
@@ -19,12 +19,19 @@
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.distributedlog.DistributedLogManager;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.LogRecordWithDLSN;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.ReadUtils;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
 import org.apache.distributedlog.impl.acl.ZKAccessControl;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
@@ -35,21 +42,12 @@
 import org.apache.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
 import org.apache.distributedlog.metadata.MetadataUpdater;
 import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
 import org.apache.distributedlog.namespace.NamespaceDriver;
 import org.apache.distributedlog.thrift.AccessControlEntry;
 import org.apache.distributedlog.tools.DistributedLogTool;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.SchedulerUtils;
-import com.twitter.util.Await;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
+import org.apache.distributedlog.common.util.SchedulerUtils;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -95,11 +93,11 @@
      *          is confirmation needed before executing actual action.
      * @throws IOException
      */
-    public static void fixInprogressSegmentWithLowerSequenceNumber(final DistributedLogNamespace namespace,
+    public static void fixInprogressSegmentWithLowerSequenceNumber(final Namespace namespace,
                                                                    final MetadataUpdater metadataUpdater,
                                                                    final String streamName,
                                                                    final boolean verbose,
-                                                                   final boolean interactive) throws IOException {
+                                                                   final boolean interactive) throws Exception {
         DistributedLogManager dlm = namespace.openLog(streamName);
         try {
             List<LogSegmentMetadata> segments = dlm.getLogSegments();
@@ -193,21 +191,21 @@
     }
 
     public static void checkAndRepairDLNamespace(final URI uri,
-                                                 final DistributedLogNamespace namespace,
+                                                 final Namespace namespace,
                                                  final MetadataUpdater metadataUpdater,
                                                  final OrderedScheduler scheduler,
                                                  final boolean verbose,
-                                                 final boolean interactive) throws IOException {
+                                                 final boolean interactive) throws Exception {
         checkAndRepairDLNamespace(uri, namespace, metadataUpdater, scheduler, verbose, interactive, 1);
     }
 
     public static void checkAndRepairDLNamespace(final URI uri,
-                                                 final DistributedLogNamespace namespace,
+                                                 final Namespace namespace,
                                                  final MetadataUpdater metadataUpdater,
                                                  final OrderedScheduler scheduler,
                                                  final boolean verbose,
                                                  final boolean interactive,
-                                                 final int concurrency) throws IOException {
+                                                 final int concurrency) throws Exception {
         Preconditions.checkArgument(concurrency > 0, "Invalid concurrency " + concurrency + " found.");
         // 0. getting streams under a given uri.
         Iterator<String> streamsIter = namespace.getLogs();
@@ -247,7 +245,7 @@
     }
 
     private static Map<String, StreamCandidate> checkStreams(
-            final DistributedLogNamespace namespace,
+            final Namespace namespace,
             final Collection<String> streams,
             final OrderedScheduler scheduler,
             final int concurrency) throws IOException {
@@ -274,7 +272,7 @@
                         LOG.info("Checking stream {}.", stream);
                         candidate = checkStream(namespace, stream, scheduler);
                         LOG.info("Checked stream {} - {}.", stream, candidate);
-                    } catch (IOException e) {
+                    } catch (Throwable e) {
                         LOG.error("Error on checking stream {} : ", stream, e);
                         doneLatch.countDown();
                         break;
@@ -313,7 +311,7 @@
     }
 
     private static StreamCandidate checkStream(
-            final DistributedLogNamespace namespace,
+            final Namespace namespace,
             final String streamName,
             final OrderedScheduler scheduler) throws IOException {
         DistributedLogManager dlm = namespace.openLog(streamName);
@@ -322,14 +320,14 @@
             if (segments.isEmpty()) {
                 return null;
             }
-            List<Future<LogSegmentCandidate>> futures =
-                    new ArrayList<Future<LogSegmentCandidate>>(segments.size());
+            List<CompletableFuture<LogSegmentCandidate>> futures =
+                    new ArrayList<CompletableFuture<LogSegmentCandidate>>(segments.size());
             for (LogSegmentMetadata segment : segments) {
                 futures.add(checkLogSegment(namespace, streamName, segment, scheduler));
             }
             List<LogSegmentCandidate> segmentCandidates;
             try {
-                segmentCandidates = Await.result(Future.collect(futures));
+                segmentCandidates = FutureUtils.result(FutureUtils.collect(futures));
             } catch (Exception e) {
                 throw new IOException("Failed on checking stream " + streamName, e);
             }
@@ -348,13 +346,13 @@
         }
     }
 
-    private static Future<LogSegmentCandidate> checkLogSegment(
-            final DistributedLogNamespace namespace,
+    private static CompletableFuture<LogSegmentCandidate> checkLogSegment(
+            final Namespace namespace,
             final String streamName,
             final LogSegmentMetadata metadata,
             final OrderedScheduler scheduler) {
         if (metadata.isInProgress()) {
-            return Future.value(null);
+            return FutureUtils.value(null);
         }
 
         final LogSegmentEntryStore entryStore = namespace.getNamespaceDriver()
@@ -370,7 +368,7 @@
                 new AtomicInteger(0),
                 scheduler,
                 entryStore
-        ).map(new Function<LogRecordWithDLSN, LogSegmentCandidate>() {
+        ).thenApply(new Function<LogRecordWithDLSN, LogSegmentCandidate>() {
             @Override
             public LogSegmentCandidate apply(LogRecordWithDLSN record) {
                 if (null != record &&
@@ -388,7 +386,7 @@
     private static boolean repairStream(MetadataUpdater metadataUpdater,
                                         StreamCandidate streamCandidate,
                                         boolean verbose,
-                                        boolean interactive) throws IOException {
+                                        boolean interactive) throws Exception {
         if (verbose) {
             System.out.println("Stream " + streamCandidate.streamName + " : ");
             for (LogSegmentCandidate segmentCandidate : streamCandidate.segmentCandidates) {
@@ -863,7 +861,7 @@
         protected ZKAccessControl getZKAccessControl(ZooKeeperClient zkc, String zkPath) throws Exception {
             ZKAccessControl accessControl;
             try {
-                accessControl = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+                accessControl = FutureUtils.result(ZKAccessControl.read(zkc, zkPath, null));
             } catch (KeeperException.NoNodeException nne) {
                 accessControl = new ZKAccessControl(new AccessControlEntry(), zkPath);
             }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogReader.java
similarity index 85%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java
rename to distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogReader.java
index e3ace05..3838bf7 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogReader.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog;
-
-import org.apache.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
+package org.apache.distributedlog.api;
 
 import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.io.AsyncCloseable;
 
 public interface AsyncLogReader extends AsyncCloseable {
 
@@ -37,7 +37,7 @@
      *
      * @return A promise that when satisfied will contain the Log Record with its DLSN.
      */
-    public Future<LogRecordWithDLSN> readNext();
+    public CompletableFuture<LogRecordWithDLSN> readNext();
 
     /**
      * Read next <i>numEntries</i> entries. The future is only satisfied with non-empty list
@@ -48,7 +48,7 @@
      *          num entries
      * @return A promise that when satisfied will contain a non-empty list of records with their DLSN.
      */
-    public Future<List<LogRecordWithDLSN>> readBulk(int numEntries);
+    public CompletableFuture<List<LogRecordWithDLSN>> readBulk(int numEntries);
 
     /**
      * Read next <i>numEntries</i> entries in a given <i>waitTime</i>.
@@ -65,5 +65,5 @@
      *          wait time unit
      * @return A promise that when satisfied will contain a non-empty list of records with their DLSN.
      */
-    public Future<List<LogRecordWithDLSN>> readBulk(int numEntries, long waitTime, TimeUnit timeUnit);
+    public CompletableFuture<List<LogRecordWithDLSN>> readBulk(int numEntries, long waitTime, TimeUnit timeUnit);
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogWriter.java
similarity index 85%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
rename to distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogWriter.java
index 53b393b..9e12de2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogWriter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,14 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog;
+package org.apache.distributedlog.api;
 
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.io.AsyncAbortable;
 import org.apache.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.io.Closeable;
-import java.util.List;
 
 public interface AsyncLogWriter extends AsyncCloseable, AsyncAbortable {
 
@@ -40,7 +40,7 @@
      * @return A Future which contains a DLSN if the record was successfully written
      * or an exception if the write fails
      */
-    public Future<DLSN> write(LogRecord record);
+    public CompletableFuture<DLSN> write(LogRecord record);
 
     /**
      * Write log records to the stream in bulk. Each future in the list represents the result of
@@ -51,7 +51,7 @@
      * @return A Future which contains a list of Future DLSNs if the record was successfully written
      * or an exception if the operation fails.
      */
-    public Future<List<Future<DLSN>>> writeBulk(List<LogRecord> record);
+    public CompletableFuture<List<CompletableFuture<DLSN>>> writeBulk(List<LogRecord> record);
 
     /**
      * Truncate the log until <i>dlsn</i>.
@@ -61,7 +61,7 @@
      * @return A Future indicates whether the operation succeeds or not, or an exception
      * if the truncation fails.
      */
-    public Future<Boolean> truncate(DLSN dlsn);
+    public CompletableFuture<Boolean> truncate(DLSN dlsn);
 
     /**
      * Get the name of the stream this writer writes data to
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java
similarity index 87%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java
rename to distributedlog-core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java
index 7d33e9c..60f629d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,18 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog;
-
-import org.apache.distributedlog.callback.LogSegmentListener;
-import org.apache.distributedlog.io.AsyncCloseable;
-import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.subscription.SubscriptionStateStore;
-import org.apache.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.util.Future;
+package org.apache.distributedlog.api;
 
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.callback.LogSegmentListener;
+import org.apache.distributedlog.io.AsyncCloseable;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
 
 /**
  * A DistributedLogManager is responsible for managing a single place of storing
@@ -79,7 +82,7 @@
      *
      * @return result represents the open result
      */
-    public Future<AsyncLogWriter> openAsyncLogWriter();
+    public CompletableFuture<AsyncLogWriter> openAsyncLogWriter();
 
     /**
      * Begin writing to the log stream identified by the name
@@ -129,7 +132,7 @@
      *          transaction id to start reading from
      * @return async log reader
      */
-    public Future<AsyncLogReader> openAsyncLogReader(long fromTxnId);
+    public CompletableFuture<AsyncLogReader> openAsyncLogReader(long fromTxnId);
 
     /**
      * Open an async log reader to read records from a log starting from <code>fromDLSN</code>
@@ -138,7 +141,7 @@
      *          dlsn to start reading from
      * @return async log reader
      */
-    public Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN);
+    public CompletableFuture<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN);
 
     // @Deprecated
     public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException;
@@ -146,7 +149,7 @@
     // @Deprecated
     public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException;
 
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN);
+    public CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN);
 
     /**
      * Get a log reader with lock starting from <i>fromDLSN</i> and using <i>subscriberId</i>.
@@ -159,7 +162,7 @@
      *          subscriber id
      * @return async log reader
      */
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN, String subscriberId);
+    public CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN, String subscriberId);
 
     /**
      * Get a log reader using <i>subscriberId</i> with lock. The reader will start reading from
@@ -173,7 +176,7 @@
      *          subscriber id
      * @return async log reader
      */
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId);
+    public CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId);
 
     /**
      * Get the {@link DLSN} of first log record whose transaction id is not less than <code>transactionId</code>.
@@ -182,7 +185,7 @@
      *          transaction id
      * @return dlsn of first log record whose transaction id is not less than transactionId.
      */
-    public Future<DLSN> getDLSNNotLessThanTxId(long transactionId);
+    public CompletableFuture<DLSN> getDLSNNotLessThanTxId(long transactionId);
 
     /**
      * Get the last log record in the stream
@@ -222,28 +225,28 @@
      *
      * @return latest log record with DLSN
      */
-    public Future<LogRecordWithDLSN> getLastLogRecordAsync();
+    public CompletableFuture<LogRecordWithDLSN> getLastLogRecordAsync();
 
     /**
      * Get Latest Transaction Id in the log - async
      *
      * @return latest transaction id
      */
-    public Future<Long> getLastTxIdAsync();
+    public CompletableFuture<Long> getLastTxIdAsync();
 
     /**
      * Get first DLSN in the log.
      *
      * @return first dlsn in the stream
      */
-    public Future<DLSN> getFirstDLSNAsync();
+    public CompletableFuture<DLSN> getFirstDLSNAsync();
 
     /**
      * Get Latest DLSN in the log - async
      *
      * @return latest transaction id
      */
-    public Future<DLSN> getLastDLSNAsync();
+    public CompletableFuture<DLSN> getLastDLSNAsync();
 
     /**
      * Get the number of log records in the active portion of the log
@@ -261,7 +264,7 @@
      * @return future number of log records
      * @throws IOException
      */
-    public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN);
+    public CompletableFuture<Long> getLogRecordCountAsync(final DLSN beginDLSN);
 
     /**
      * Run recovery on the log.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogReader.java
similarity index 97%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java
rename to distributedlog-core/src/main/java/org/apache/distributedlog/api/LogReader.java
index baf3182..631a8a9 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogReader.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,8 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog;
+package org.apache.distributedlog.api;
 
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.LogRecordWithDLSN;
 import org.apache.distributedlog.io.AsyncCloseable;
 
 import java.io.Closeable;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogWriter.java
similarity index 96%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java
rename to distributedlog-core/src/main/java/org/apache/distributedlog/api/LogWriter.java
index 8a4a30b..46ad1f0 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogWriter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,8 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog;
+package org.apache.distributedlog.api;
 
+import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.io.Abortable;
 
 import java.io.Closeable;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/MetadataAccessor.java
similarity index 96%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java
rename to distributedlog-core/src/main/java/org/apache/distributedlog/api/MetadataAccessor.java
index 3d1d601..76ef700 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/MetadataAccessor.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog;
+package org.apache.distributedlog.api;
 
 import org.apache.distributedlog.io.AsyncCloseable;
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java
similarity index 93%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java
copy to distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java
index 4cbee98..818824d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.namespace;
+package org.apache.distributedlog.api.namespace;
 
 import com.google.common.annotations.Beta;
 import com.google.common.base.Optional;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.apache.distributedlog.acl.AccessControlManager;
 import org.apache.distributedlog.callback.NamespaceListener;
@@ -31,6 +31,7 @@
 import java.util.Iterator;
 
 import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.namespace.NamespaceDriver;
 
 /**
  * A namespace is the basic unit for managing a set of distributedlogs.
@@ -38,7 +39,7 @@
  * <h4>Namespace Interface</h4>
  *
  * <P>
- * The <code>DistributedLogNamespace</code> interface is implemented by different backend providers.
+ * The <code>Namespace</code> interface is implemented by different backend providers.
  * There are several components are required for an implementation:
  * <OL>
  *     <LI>Log Management -- manage logs in a given namespace. e.g. create/open/delete log, list of logs,
@@ -49,7 +50,7 @@
  *
  * <h4>Namespace Location</h4>
  *
- * At the highest level, a <code>DistributedLogNamespace</code> is located by a <code>URI</code>. The location
+ * At the highest level, a <code>Namespace</code> is located by a <code>URI</code>. The location
  * URI is in string form has the syntax
  *
  * <blockquote>
@@ -68,7 +69,7 @@
  * @since 0.3.32
  */
 @Beta
-public interface DistributedLogNamespace {
+public interface Namespace {
 
     /**
      * Get the namespace driver used by this namespace.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java
similarity index 85%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceBuilder.java
rename to distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java
index 2706201..45dc021 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceBuilder.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.namespace;