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;
+package org.apache.distributedlog.api.namespace;
 
 import com.google.common.base.Preconditions;
 import org.apache.distributedlog.BKDistributedLogNamespace;
@@ -25,10 +25,12 @@
 import org.apache.distributedlog.feature.CoreFeatureKeys;
 import org.apache.distributedlog.injector.AsyncFailureInjector;
 import org.apache.distributedlog.injector.AsyncRandomFailureInjector;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.namespace.NamespaceDriverManager;
 import org.apache.distributedlog.util.ConfUtils;
 import org.apache.distributedlog.util.DLUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.PermitLimiter;
+import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.util.SimplePermitLimiter;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.feature.FeatureProvider;
@@ -42,18 +44,18 @@
 import java.net.URI;
 
 /**
- * Builder to construct a <code>DistributedLogNamespace</code>.
+ * Builder to construct a <code>Namespace</code>.
  * The builder takes the responsibility of loading backend according to the uri.
  *
- * @see DistributedLogNamespace
+ * @see Namespace
  * @since 0.3.32
  */
-public class DistributedLogNamespaceBuilder {
+public class NamespaceBuilder {
 
-    private static final Logger logger = LoggerFactory.getLogger(DistributedLogNamespaceBuilder.class);
+    private static final Logger logger = LoggerFactory.getLogger(NamespaceBuilder.class);
 
-    public static DistributedLogNamespaceBuilder newBuilder() {
-        return new DistributedLogNamespaceBuilder();
+    public static NamespaceBuilder newBuilder() {
+        return new NamespaceBuilder();
     }
 
     private DistributedLogConfiguration _conf = null;
@@ -66,7 +68,7 @@
     private int _regionId = DistributedLogConstants.LOCAL_REGION_ID;
 
     // private constructor
-    private DistributedLogNamespaceBuilder() {}
+    private NamespaceBuilder() {}
 
     /**
      * DistributedLog Configuration used for the namespace.
@@ -75,7 +77,7 @@
      *          distributedlog configuration
      * @return namespace builder.
      */
-    public DistributedLogNamespaceBuilder conf(DistributedLogConfiguration conf) {
+    public NamespaceBuilder conf(DistributedLogConfiguration conf) {
         this._conf = conf;
         return this;
     }
@@ -86,7 +88,7 @@
      * @param dynConf dynamic distributedlog configuration
      * @return namespace builder
      */
-    public DistributedLogNamespaceBuilder dynConf(DynamicDistributedLogConfiguration dynConf) {
+    public NamespaceBuilder dynConf(DynamicDistributedLogConfiguration dynConf) {
         this._dynConf = dynConf;
         return this;
     }
@@ -96,10 +98,10 @@
      *
      * @param uri
      *          namespace location uri.
-     * @see DistributedLogNamespace
+     * @see Namespace
      * @return namespace builder.
      */
-    public DistributedLogNamespaceBuilder uri(URI uri) {
+    public NamespaceBuilder uri(URI uri) {
         this._uri = uri;
         return this;
     }
@@ -111,7 +113,7 @@
      *          stats logger
      * @return namespace builder.
      */
-    public DistributedLogNamespaceBuilder statsLogger(StatsLogger statsLogger) {
+    public NamespaceBuilder statsLogger(StatsLogger statsLogger) {
         this._statsLogger = statsLogger;
         return this;
     }
@@ -123,7 +125,7 @@
      *          stats logger for collecting per log stats
      * @return namespace builder.
      */
-    public DistributedLogNamespaceBuilder perLogStatsLogger(StatsLogger statsLogger) {
+    public NamespaceBuilder perLogStatsLogger(StatsLogger statsLogger) {
         this._perLogStatsLogger = statsLogger;
         return this;
     }
@@ -135,7 +137,7 @@
      *          feature provider to control availabilities of features.
      * @return namespace builder.
      */
-    public DistributedLogNamespaceBuilder featureProvider(FeatureProvider featureProvider) {
+    public NamespaceBuilder featureProvider(FeatureProvider featureProvider) {
         this._featureProvider = featureProvider;
         return this;
     }
@@ -147,7 +149,7 @@
      *          client id used for accessing the namespace
      * @return namespace builder.
      */
-    public DistributedLogNamespaceBuilder clientId(String clientId) {
+    public NamespaceBuilder clientId(String clientId) {
         this._clientId = clientId;
         return this;
     }
@@ -160,7 +162,7 @@
      *          region id.
      * @return namespace builder.
      */
-    public DistributedLogNamespaceBuilder regionId(int regionId) {
+    public NamespaceBuilder regionId(int regionId) {
         this._regionId = regionId;
         return this;
     }
@@ -185,7 +187,7 @@
      * @throws NullPointerException when there is null argument provided in the builder
      * @throws IOException when fail to build the backend
      */
-    public DistributedLogNamespace build()
+    public Namespace build()
             throws IllegalArgumentException, NullPointerException, IOException {
         // Check arguments
         Preconditions.checkNotNull(_conf, "No DistributedLog Configuration.");
@@ -224,14 +226,9 @@
         StatsLogger perLogStatsLogger = normalizePerLogStatsLogger(_statsLogger, _perLogStatsLogger, _conf);
 
         // build the scheduler
-        StatsLogger schedulerStatsLogger = _statsLogger.scope("factory").scope("thread_pool");
         OrderedScheduler scheduler = OrderedScheduler.newBuilder()
                 .name("DLM-" + normalizedUri.getPath())
                 .corePoolSize(_conf.getNumWorkerThreads())
-                .statsLogger(schedulerStatsLogger)
-                .perExecutorStatsLogger(schedulerStatsLogger)
-                .traceTaskExecution(_conf.getEnableTaskExecutionStats())
-                .traceTaskExecutionWarnTimeUs(_conf.getTaskExecutionWarnTimeMicros())
                 .build();
 
         // initialize the namespace driver
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/package-info.java
similarity index 87%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
copy to distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/package-info.java
index d81f8a4..fa8f288 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/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
+ * Namespace API and the builder to build namespace instance.
  */
-package org.apache.distributedlog.readahead;
+package org.apache.distributedlog.api.namespace;
\ No newline at end of file
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/package-info.java
similarity index 78%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
copy to distributedlog-core/src/main/java/org/apache/distributedlog/api/package-info.java
index d81f8a4..eca11fd 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/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,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /**
- * ReadAhead Mechanism for distributedlog streaming reads
+ * DistributedLog NEW API.
+ *
+ * <p>This is the new Java8 {@link java.util.concurrent.CompletableFuture} based API. It is
+ * <strong>experimental</strong> and still under developing.
  */
-package org.apache.distributedlog.readahead;
+package org.apache.distributedlog.api;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionStateStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionStateStore.java
similarity index 84%
rename from distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionStateStore.java
rename to distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionStateStore.java
index ebfc32a..bf4a8d3 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionStateStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionStateStore.java
@@ -15,14 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.subscription;
+package org.apache.distributedlog.api.subscription;
 
 import java.io.Closeable;
-
-import scala.runtime.BoxedUnit;
-
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DLSN;
-import com.twitter.util.Future;
 
 public interface SubscriptionStateStore extends Closeable {
     /**
@@ -30,7 +27,7 @@
      *
      * @return future represents the last commit position
      */
-    public Future<DLSN> getLastCommitPosition();
+    public CompletableFuture<DLSN> getLastCommitPosition();
 
     /**
      * Advances the position associated with the subscriber
@@ -38,5 +35,5 @@
      * @param newPosition - new commit position
      * @return future represents the advance result
      */
-    public Future<BoxedUnit> advanceCommitPosition(DLSN newPosition);
+    public CompletableFuture<Void> advanceCommitPosition(DLSN newPosition);
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionsStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionsStore.java
similarity index 82%
copy from distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionsStore.java
copy to distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionsStore.java
index 1974f1e..b6a0ed1 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionsStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionsStore.java
@@ -15,14 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.distributedlog.subscription;
-
-import org.apache.distributedlog.DLSN;
-import com.twitter.util.Future;
-import scala.runtime.BoxedUnit;
+package org.apache.distributedlog.api.subscription;
 
 import java.io.Closeable;
 import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.DLSN;
 
 /**
  * Store to manage subscriptions
@@ -36,14 +34,14 @@
      *          subscriber id
      * @return future representing last committed position.
      */
-    public Future<DLSN> getLastCommitPosition(String subscriberId);
+    public CompletableFuture<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();
+    public CompletableFuture<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);
+    public CompletableFuture<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);
+    public CompletableFuture<Boolean> deleteSubscriber(String subscriberId);
 
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java b/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
index 56a4f2e..34011b5 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
@@ -20,20 +20,20 @@
 import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.SettableFuture;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.BookKeeperClientBuilder;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.namespace.NamespaceDriver;
 import org.apache.distributedlog.util.DLUtils;
 import org.apache.bookkeeper.client.BKException;
@@ -45,6 +45,7 @@
 import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
 import org.apache.bookkeeper.zookeeper.RetryPolicy;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
@@ -84,13 +85,13 @@
         this.conf = conf;
     }
 
-    private ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) {
+    private ZooKeeperClient getZooKeeperClient(Namespace namespace) {
         NamespaceDriver driver = namespace.getNamespaceDriver();
         assert(driver instanceof BKNamespaceDriver);
         return ((BKNamespaceDriver) driver).getWriterZKC();
     }
 
-    private BookKeeperClient getBookKeeperClient(DistributedLogNamespace namespace) {
+    private BookKeeperClient getBookKeeperClient(Namespace namespace) {
         NamespaceDriver driver = namespace.getNamespaceDriver();
         assert(driver instanceof BKNamespaceDriver);
         return ((BKNamespaceDriver) driver).getReaderBKC();
@@ -169,7 +170,7 @@
         LedgerManager lm = BookKeeperAccessor.getLedgerManager(bkc.get());
 
         final Set<Long> ledgers = new HashSet<Long>();
-        final SettableFuture<Void> doneFuture = SettableFuture.create();
+        final CompletableFuture<Void> doneFuture = FutureUtils.createFuture();
 
         BookkeeperInternalCallbacks.Processor<Long> collector =
                 new BookkeeperInternalCallbacks.Processor<Long>() {
@@ -195,9 +196,9 @@
             @Override
             public void processResult(int rc, String path, Object ctx) {
                 if (BKException.Code.OK == rc) {
-                    doneFuture.set(null);
+                    doneFuture.complete(null);
                 } else {
-                    doneFuture.setException(BKException.create(rc));
+                    doneFuture.completeExceptionally(BKException.create(rc));
                 }
             }
         };
@@ -225,12 +226,12 @@
     private Set<Long> collectLedgersFromDL(List<URI> uris, List<List<String>> allocationPaths)
             throws IOException {
         final Set<Long> ledgers = new TreeSet<Long>();
-        List<DistributedLogNamespace> namespaces =
-                new ArrayList<DistributedLogNamespace>(uris.size());
+        List<Namespace> namespaces =
+                new ArrayList<Namespace>(uris.size());
         try {
             for (URI uri : uris) {
                 namespaces.add(
-                        DistributedLogNamespaceBuilder.newBuilder()
+                        NamespaceBuilder.newBuilder()
                                 .conf(conf)
                                 .uri(uri)
                                 .build());
@@ -240,8 +241,8 @@
             ExecutorService executor = Executors.newFixedThreadPool(uris.size());
             try {
                 int i = 0;
-                for (final DistributedLogNamespace namespace : namespaces) {
-                    final DistributedLogNamespace dlNamespace = namespace;
+                for (final Namespace namespace : namespaces) {
+                    final Namespace dlNamespace = namespace;
                     final URI uri = uris.get(i);
                     final List<String> aps = allocationPaths.get(i);
                     i++;
@@ -278,7 +279,7 @@
                 executor.shutdown();
             }
         } finally {
-            for (DistributedLogNamespace namespace : namespaces) {
+            for (Namespace namespace : namespaces) {
                 namespace.close();
             }
         }
@@ -286,7 +287,7 @@
     }
 
     private void collectLedgersFromAllocator(final URI uri,
-                                             final DistributedLogNamespace namespace,
+                                             final Namespace namespace,
                                              final List<String> allocationPaths,
                                              final Set<Long> ledgers) throws IOException {
         final LinkedBlockingQueue<String> poolQueue =
@@ -346,7 +347,7 @@
     }
 
     private void collectLedgersFromDL(final URI uri,
-                                      final DistributedLogNamespace namespace,
+                                      final Namespace namespace,
                                       final Set<Long> ledgers) throws IOException {
         logger.info("Enumerating {} to collect streams.", uri);
         Iterator<String> streams = namespace.getLogs();
@@ -366,7 +367,7 @@
         });
     }
 
-    private List<Long> collectLedgersFromStream(DistributedLogNamespace namespace,
+    private List<Long> collectLedgersFromStream(Namespace namespace,
                                                 String stream,
                                                 Set<Long> ledgers)
             throws IOException {
@@ -394,7 +395,7 @@
      */
     public Map<String, Long> calculateStreamSpaceUsage(final URI uri) throws IOException {
         logger.info("Collecting stream space usage for {}.", uri);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .build();
@@ -406,7 +407,7 @@
     }
 
     private Map<String, Long> calculateStreamSpaceUsage(
-            final URI uri, final DistributedLogNamespace namespace)
+            final URI uri, final Namespace namespace)
         throws IOException {
         Iterator<String> streams = namespace.getLogs();
         final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
@@ -432,7 +433,7 @@
         return streamSpaceUsageMap;
     }
 
-    private long calculateStreamSpaceUsage(final DistributedLogNamespace namespace,
+    private long calculateStreamSpaceUsage(final Namespace namespace,
                                            final String stream) throws IOException {
         DistributedLogManager dlm = namespace.openLog(stream);
         long totalBytes = 0;
@@ -504,7 +505,7 @@
 
         LedgerManager lm = BookKeeperAccessor.getLedgerManager(bkc.get());
 
-        final SettableFuture<Void> doneFuture = SettableFuture.create();
+        final CompletableFuture<Void> doneFuture = FutureUtils.createFuture();
         final BookKeeper bk = bkc.get();
 
         BookkeeperInternalCallbacks.Processor<Long> collector =
@@ -544,9 +545,9 @@
             @Override
             public void processResult(int rc, String path, Object ctx) {
                 if (BKException.Code.OK == rc) {
-                    doneFuture.set(null);
+                    doneFuture.complete(null);
                 } else {
-                    doneFuture.setException(BKException.create(rc));
+                    doneFuture.completeExceptionally(BKException.create(rc));
                 }
             }
         };
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java
index 6ea248b..ee33dc3 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java
@@ -17,9 +17,10 @@
  */
 package org.apache.distributedlog.bk;
 
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.Transaction;
 import org.apache.distributedlog.util.Transaction.OpListener;
-import com.twitter.util.Future;
 import org.apache.bookkeeper.client.LedgerHandle;
 
 import java.io.IOException;
@@ -57,8 +58,8 @@
     }
 
     @Override
-    public Future<Void> delete() {
-        return Future.exception(new UnsupportedOperationException("Can't delete an allocator by delegator"));
+    public CompletableFuture<Void> delete() {
+        return FutureUtils.exception(new UnsupportedOperationException("Can't delete an allocator by delegator"));
     }
 
     @Override
@@ -67,17 +68,17 @@
     }
 
     @Override
-    public Future<LedgerHandle> tryObtain(Transaction<Object> txn,
+    public CompletableFuture<LedgerHandle> tryObtain(Transaction<Object> txn,
                                           OpListener<LedgerHandle> listener) {
         return this.allocator.tryObtain(txn, listener);
     }
 
     @Override
-    public Future<Void> asyncClose() {
+    public CompletableFuture<Void> asyncClose() {
         if (ownAllocator) {
             return this.allocator.asyncClose();
         } else {
-            return Future.value(null);
+            return FutureUtils.value(null);
         }
     }
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java
index 4fff2f6..19c5546 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java
@@ -19,17 +19,15 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
-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.Transaction;
 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.client.LedgerHandle;
 import org.apache.bookkeeper.meta.ZkVersion;
 import org.apache.bookkeeper.util.ZkUtils;
@@ -40,7 +38,6 @@
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -320,25 +317,25 @@
     }
 
     @Override
-    public Future<LedgerHandle> tryObtain(final Transaction<Object> txn,
-                                          final Transaction.OpListener<LedgerHandle> listener) {
+    public CompletableFuture<LedgerHandle> tryObtain(final Transaction<Object> txn,
+                                                     final Transaction.OpListener<LedgerHandle> listener) {
         final SimpleLedgerAllocator allocator;
         synchronized (this) {
             if (allocatingList.isEmpty()) {
-                return Future.exception(new IOException("No ledger allocator available under " + poolPath + "."));
+                return FutureUtils.exception(new IOException("No ledger allocator available under " + poolPath + "."));
             } else {
                 allocator = allocatingList.removeFirst();
             }
         }
 
-        final Promise<LedgerHandle> tryObtainPromise = new Promise<LedgerHandle>();
+        final CompletableFuture<LedgerHandle> tryObtainPromise = new CompletableFuture<LedgerHandle>();
         final FutureEventListener<LedgerHandle> tryObtainListener = new FutureEventListener<LedgerHandle>() {
             @Override
             public void onSuccess(LedgerHandle lh) {
                 synchronized (LedgerAllocatorPool.this) {
                     obtainMap.put(lh, allocator);
                     reverseObtainMap.put(allocator, lh);
-                    tryObtainPromise.setValue(lh);
+                    tryObtainPromise.complete(lh);
                 }
             }
 
@@ -349,7 +346,7 @@
                 } catch (IOException ioe) {
                     logger.info("Failed to rescue allocator {}", allocator.allocatePath, ioe);
                 }
-                tryObtainPromise.setException(cause);
+                tryObtainPromise.completeExceptionally(cause);
             }
         };
 
@@ -365,7 +362,7 @@
                 abortObtain(allocator);
                 listener.onAbort(t);
             }
-        }).addEventListener(tryObtainListener);
+        }).whenComplete(tryObtainListener);
         return tryObtainPromise;
     }
 
@@ -399,7 +396,7 @@
     }
 
     @Override
-    public Future<Void> asyncClose() {
+    public CompletableFuture<Void> asyncClose() {
         List<LedgerAllocator> allocatorsToClose;
         synchronized (this) {
             allocatorsToClose = Lists.newArrayListWithExpectedSize(
@@ -414,21 +411,15 @@
                 allocatorsToClose.add(allocator);
             }
         }
-        return FutureUtils.processList(allocatorsToClose, new Function<LedgerAllocator, Future<Void>>() {
-            @Override
-            public Future<Void> apply(LedgerAllocator allocator) {
-                return allocator.asyncClose();
-            }
-        }, scheduledExecutorService).map(new AbstractFunction1<List<Void>, Void>() {
-            @Override
-            public Void apply(List<Void> values) {
-                return null;
-            }
-        });
+        return FutureUtils.processList(
+            allocatorsToClose,
+            allocator -> allocator.asyncClose(),
+            scheduledExecutorService
+        ).thenApply(values -> null);
     }
 
     @Override
-    public Future<Void> delete() {
+    public CompletableFuture<Void> delete() {
         List<LedgerAllocator> allocatorsToDelete;
         synchronized (this) {
             allocatorsToDelete = Lists.newArrayListWithExpectedSize(
@@ -443,16 +434,10 @@
                 allocatorsToDelete.add(allocator);
             }
         }
-        return FutureUtils.processList(allocatorsToDelete, new Function<LedgerAllocator, Future<Void>>() {
-            @Override
-            public Future<Void> apply(LedgerAllocator allocator) {
-                return allocator.delete();
-            }
-        }, scheduledExecutorService).flatMap(new AbstractFunction1<List<Void>, Future<Void>>() {
-            @Override
-            public Future<Void> apply(List<Void> values) {
-                return Utils.zkDelete(zkc, poolPath, new ZkVersion(-1));
-            }
-        });
+        return FutureUtils.processList(
+            allocatorsToDelete,
+            allocator -> allocator.delete(),
+            scheduledExecutorService
+        ).thenCompose(values -> Utils.zkDelete(zkc, poolPath, new ZkVersion(-1)));
     }
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java
index e0102f3..144b0a6 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java
@@ -18,19 +18,20 @@
 package org.apache.distributedlog.bk;
 
 import com.google.common.collect.Lists;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+import java.util.function.Function;
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.util.DLUtils;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.util.Transaction;
 import org.apache.distributedlog.util.Transaction.OpListener;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.Utils;
 import org.apache.distributedlog.zk.ZKTransaction;
 import org.apache.distributedlog.zk.ZKVersionedSetOp;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.meta.ZkVersion;
 import org.apache.bookkeeper.versioning.Version;
@@ -40,9 +41,6 @@
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
 
 import java.io.IOException;
 import java.util.LinkedList;
@@ -96,7 +94,7 @@
     // version
     ZkVersion version = new ZkVersion(-1);
     // outstanding allocation
-    Promise<LedgerHandle> allocatePromise;
+    CompletableFuture<LedgerHandle> allocatePromise;
     // outstanding tryObtain transaction
     Transaction<Object> tryObtainTxn = null;
     OpListener<LedgerHandle> tryObtainListener = null;
@@ -105,73 +103,71 @@
     // Allocated Ledger
     LedgerHandle allocatedLh = null;
 
-    Future<Void> closeFuture = null;
-    final LinkedList<Future<Void>> ledgerDeletions =
-            new LinkedList<Future<Void>>();
+    CompletableFuture<Void> closeFuture = null;
+    final LinkedList<CompletableFuture<Void>> ledgerDeletions =
+            new LinkedList<CompletableFuture<Void>>();
 
     // Ledger configuration
     private final QuorumConfigProvider quorumConfigProvider;
 
-    static Future<Versioned<byte[]>> getAndCreateAllocationData(final String allocatePath,
+    static CompletableFuture<Versioned<byte[]>> getAndCreateAllocationData(final String allocatePath,
                                                                 final ZooKeeperClient zkc) {
         return Utils.zkGetData(zkc, allocatePath, false)
-                .flatMap(new AbstractFunction1<Versioned<byte[]>, Future<Versioned<byte[]>>>() {
+                .thenCompose(new Function<Versioned<byte[]>, CompletionStage<Versioned<byte[]>>>() {
             @Override
-            public Future<Versioned<byte[]>> apply(Versioned<byte[]> result) {
+            public CompletableFuture<Versioned<byte[]>> apply(Versioned<byte[]> result) {
                 if (null != result && null != result.getVersion() && null != result.getValue()) {
-                    return Future.value(result);
+                    return FutureUtils.value(result);
                 }
                 return createAllocationData(allocatePath, zkc);
             }
         });
     }
 
-    private static Future<Versioned<byte[]>> createAllocationData(final String allocatePath,
+    private static CompletableFuture<Versioned<byte[]>> createAllocationData(final String allocatePath,
                                                                   final ZooKeeperClient zkc) {
         try {
-            final Promise<Versioned<byte[]>> promise = new Promise<Versioned<byte[]>>();
+            final CompletableFuture<Versioned<byte[]>> promise = new CompletableFuture<Versioned<byte[]>>();
             zkc.get().create(allocatePath, DistributedLogConstants.EMPTY_BYTES,
                     zkc.getDefaultACL(), CreateMode.PERSISTENT,
                     new org.apache.zookeeper.AsyncCallback.Create2Callback() {
                         @Override
                         public void processResult(int rc, String path, Object ctx, String name, Stat stat) {
                             if (KeeperException.Code.OK.intValue() == rc) {
-                                promise.setValue(new Versioned<byte[]>(DistributedLogConstants.EMPTY_BYTES,
+                                promise.complete(new Versioned<byte[]>(DistributedLogConstants.EMPTY_BYTES,
                                         new ZkVersion(stat.getVersion())));
                             } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
-                                Utils.zkGetData(zkc, allocatePath, false).proxyTo(promise);
+                                FutureUtils.proxyTo(
+                                  Utils.zkGetData(zkc, allocatePath, false),
+                                  promise
+                                );
                             } else {
-                                promise.setException(FutureUtils.zkException(
+                                promise.completeExceptionally(Utils.zkException(
                                         KeeperException.create(KeeperException.Code.get(rc)), allocatePath));
                             }
                         }
                     }, null);
             return promise;
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            return Future.exception(FutureUtils.zkException(e, allocatePath));
+            return FutureUtils.exception(Utils.zkException(e, allocatePath));
         } catch (InterruptedException e) {
-            return Future.exception(FutureUtils.zkException(e, allocatePath));
+            return FutureUtils.exception(Utils.zkException(e, allocatePath));
         }
     }
 
-    public static Future<SimpleLedgerAllocator> of(final String allocatePath,
+    public static CompletableFuture<SimpleLedgerAllocator> of(final String allocatePath,
                                                    final Versioned<byte[]> allocationData,
                                                    final QuorumConfigProvider quorumConfigProvider,
                                                    final ZooKeeperClient zkc,
                                                    final BookKeeperClient bkc) {
         if (null != allocationData && null != allocationData.getValue()
                 && null != allocationData.getVersion()) {
-            return Future.value(new SimpleLedgerAllocator(allocatePath, allocationData,
+            return FutureUtils.value(new SimpleLedgerAllocator(allocatePath, allocationData,
                     quorumConfigProvider, zkc, bkc));
         }
         return getAndCreateAllocationData(allocatePath, zkc)
-                .map(new AbstractFunction1<Versioned<byte[]>, SimpleLedgerAllocator>() {
-            @Override
-            public SimpleLedgerAllocator apply(Versioned<byte[]> allocationData) {
-                return new SimpleLedgerAllocator(allocatePath, allocationData,
-                        quorumConfigProvider, zkc, bkc);
-            }
-        });
+            .thenApply(allocationData1 -> new SimpleLedgerAllocator(allocatePath, allocationData1,
+                        quorumConfigProvider, zkc, bkc));
     }
 
     /**
@@ -240,14 +236,14 @@
     }
 
     @Override
-    public synchronized Future<LedgerHandle> tryObtain(final Transaction<Object> txn,
-                                                       final OpListener<LedgerHandle> listener) {
+    public synchronized CompletableFuture<LedgerHandle> tryObtain(final Transaction<Object> txn,
+                                                                  final OpListener<LedgerHandle> listener) {
         if (Phase.ERROR == phase) {
-            return Future.exception(new AllocationException(Phase.ERROR,
+            return FutureUtils.exception(new AllocationException(Phase.ERROR,
                     "Error on allocating ledger under " + allocatePath));
         }
         if (Phase.HANDING_OVER == phase || Phase.HANDED_OVER == phase || null != tryObtainTxn) {
-            return Future.exception(new ConcurrentObtainException(phase,
+            return FutureUtils.exception(new ConcurrentObtainException(phase,
                     "Ledger handle is handling over to another thread : " + phase));
         }
         tryObtainTxn = txn;
@@ -328,13 +324,13 @@
             return;
         }
         setPhase(Phase.ALLOCATING);
-        allocatePromise = new Promise<LedgerHandle>();
+        allocatePromise = new CompletableFuture<LedgerHandle>();
         QuorumConfig quorumConfig = quorumConfigProvider.getQuorumConfig();
         bkc.createLedger(
                 quorumConfig.getEnsembleSize(),
                 quorumConfig.getWriteQuorumSize(),
                 quorumConfig.getAckQuorumSize()
-        ).addEventListener(this);
+        ).whenComplete(this);
     }
 
     private synchronized void completeAllocation(LedgerHandle lh) {
@@ -347,11 +343,11 @@
         ZKVersionedSetOp commitOp = new ZKVersionedSetOp(zkSetDataOp, this);
         tryObtainTxn.addOp(commitOp);
         setPhase(Phase.HANDING_OVER);
-        FutureUtils.setValue(allocatePromise, lh);
+        allocatePromise.complete(lh);
     }
 
     private synchronized void failAllocation(Throwable cause) {
-        FutureUtils.setException(allocatePromise, cause);
+        allocatePromise.completeExceptionally(cause);
     }
 
     @Override
@@ -386,7 +382,7 @@
     private void markAsAllocated(final LedgerHandle lh) {
         byte[] data = DLUtils.logSegmentId2Bytes(lh.getId());
         Utils.zkSetData(zkc, allocatePath, data, getVersion())
-            .addEventListener(new FutureEventListener<ZkVersion>() {
+            .whenComplete(new FutureEventListener<ZkVersion>() {
                 @Override
                 public void onSuccess(ZkVersion version) {
                     // we only issue deleting ledger left from previous allocation when we could allocate first ledger
@@ -411,27 +407,20 @@
     }
 
     void deleteLedger(final long ledgerId) {
-        final Future<Void> deleteFuture = bkc.deleteLedger(ledgerId, true);
+        final CompletableFuture<Void> deleteFuture = bkc.deleteLedger(ledgerId, true);
         synchronized (ledgerDeletions) {
             ledgerDeletions.add(deleteFuture);
         }
-        deleteFuture.onFailure(new AbstractFunction1<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable cause) {
+        deleteFuture.whenComplete((value, cause) -> {
+            if (null != cause) {
                 LOG.error("Error deleting ledger {} for ledger allocator {}, retrying : ",
                         new Object[] { ledgerId, allocatePath, cause });
                 if (!isClosing()) {
                     deleteLedger(ledgerId);
                 }
-                return BoxedUnit.UNIT;
             }
-        }).ensure(new AbstractFunction0<BoxedUnit>() {
-            @Override
-            public BoxedUnit apply() {
-                synchronized (ledgerDeletions) {
-                    ledgerDeletions.remove(deleteFuture);
-                }
-                return BoxedUnit.UNIT;
+            synchronized (ledgerDeletions) {
+                ledgerDeletions.remove(deleteFuture);
             }
         });
     }
@@ -440,25 +429,25 @@
         return closeFuture != null;
     }
 
-    private Future<Void> closeInternal(boolean cleanup) {
-        Promise<Void> closePromise;
+    private CompletableFuture<Void> closeInternal(boolean cleanup) {
+        CompletableFuture<Void> closePromise;
         synchronized (this) {
             if (null != closeFuture) {
                 return closeFuture;
             }
-            closePromise = new Promise<Void>();
+            closePromise = new CompletableFuture<Void>();
             closeFuture = closePromise;
         }
         if (!cleanup) {
             LOG.info("Abort ledger allocator without cleaning up on {}.", allocatePath);
-            FutureUtils.setValue(closePromise, null);
+            closePromise.complete(null);
             return closePromise;
         }
         cleanupAndClose(closePromise);
         return closePromise;
     }
 
-    private void cleanupAndClose(final Promise<Void> closePromise) {
+    private void cleanupAndClose(final CompletableFuture<Void> closePromise) {
         LOG.info("Closing ledger allocator on {}.", allocatePath);
         final ZKTransaction txn = new ZKTransaction(zkc);
         // try obtain ledger handle
@@ -476,21 +465,21 @@
             }
 
             private void complete() {
-                FutureUtils.setValue(closePromise, null);
+                closePromise.complete(null);
                 LOG.info("Closed ledger allocator on {}.", allocatePath);
             }
-        }).addEventListener(new FutureEventListener<LedgerHandle>() {
+        }).whenComplete(new FutureEventListener<LedgerHandle>() {
             @Override
             public void onSuccess(LedgerHandle lh) {
                 // try obtain succeed
                 // if we could obtain the ledger handle, we have the responsibility to close it
                 deleteLedger(lh.getId());
                 // wait for deletion to be completed
-                List<Future<Void>> outstandingDeletions;
+                List<CompletableFuture<Void>> outstandingDeletions;
                 synchronized (ledgerDeletions) {
                     outstandingDeletions = Lists.newArrayList(ledgerDeletions);
                 }
-                Future.collect(outstandingDeletions).addEventListener(new FutureEventListener<List<Void>>() {
+                FutureUtils.collect(outstandingDeletions).whenComplete(new FutureEventListener<List<Void>>() {
                     @Override
                     public void onSuccess(List<Void> values) {
                         txn.execute();
@@ -499,7 +488,7 @@
                     @Override
                     public void onFailure(Throwable cause) {
                         LOG.debug("Fail to obtain the allocated ledger handle when closing the allocator : ", cause);
-                        FutureUtils.setValue(closePromise, null);
+                        closePromise.complete(null);
                     }
                 });
             }
@@ -507,7 +496,7 @@
             @Override
             public void onFailure(Throwable cause) {
                 LOG.debug("Fail to obtain the allocated ledger handle when closing the allocator : ", cause);
-                FutureUtils.setValue(closePromise, null);
+                closePromise.complete(null);
             }
         });
 
@@ -519,18 +508,13 @@
     }
 
     @Override
-    public Future<Void> asyncClose() {
+    public CompletableFuture<Void> asyncClose() {
         return closeInternal(false);
     }
 
     @Override
-    public Future<Void> delete() {
-        return closeInternal(true).flatMap(new AbstractFunction1<Void, Future<Void>>() {
-            @Override
-            public Future<Void> apply(Void value) {
-                return Utils.zkDelete(zkc, allocatePath, getVersion());
-            }
-        });
+    public CompletableFuture<Void> delete() {
+        return closeInternal(true).thenCompose(value -> Utils.zkDelete(zkc, allocatePath, getVersion()));
     }
 
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java
index c77778a..c69b7a5 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java
@@ -24,7 +24,6 @@
 import org.apache.distributedlog.DistributedLogConfiguration;
 
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.net.MalformedURLException;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -35,6 +34,11 @@
 
 import org.apache.commons.configuration.ConfigurationException;
 
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
+import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
+import org.apache.distributedlog.common.config.ConfigurationSubscription;
+import org.apache.distributedlog.common.config.FileConfigurationBuilder;
+import org.apache.distributedlog.common.config.PropertiesConfigurationBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
index ea7f4a7..9e760c5 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
@@ -19,6 +19,7 @@
 
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.bk.QuorumConfig;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
 
 import static org.apache.distributedlog.DistributedLogConfiguration.*;
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java
index 83cac22..f51302e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.feature;
 
-import org.apache.distributedlog.config.ConcurrentBaseConfiguration;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
 import org.apache.bookkeeper.feature.CacheableFeatureProvider;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.feature.FeatureProvider;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java
index 4689d51..201ed8a 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java
@@ -21,11 +21,11 @@
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.config.ConcurrentBaseConfiguration;
-import org.apache.distributedlog.config.ConfigurationListener;
-import org.apache.distributedlog.config.ConfigurationSubscription;
-import org.apache.distributedlog.config.FileConfigurationBuilder;
-import org.apache.distributedlog.config.PropertiesConfigurationBuilder;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
+import org.apache.distributedlog.common.config.ConfigurationListener;
+import org.apache.distributedlog.common.config.ConfigurationSubscription;
+import org.apache.distributedlog.common.config.FileConfigurationBuilder;
+import org.apache.distributedlog.common.config.PropertiesConfigurationBuilder;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.feature.SettableFeature;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/function/CloseAsyncCloseableFunction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/function/CloseAsyncCloseableFunction.java
deleted file mode 100644
index b1adf4a..0000000
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/function/CloseAsyncCloseableFunction.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.function;
-
-import org.apache.distributedlog.io.AsyncCloseable;
-import scala.Function0;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.BoxedUnit;
-
-/**
- * Function to close {@link org.apache.distributedlog.io.AsyncCloseable}
- */
-public class CloseAsyncCloseableFunction extends AbstractFunction0<BoxedUnit> {
-
-    /**
-     * Return a function to close an {@link AsyncCloseable}.
-     *
-     * @param closeable closeable to close
-     * @return function to close an {@link AsyncCloseable}
-     */
-    public static Function0<BoxedUnit> of(AsyncCloseable closeable) {
-        return new CloseAsyncCloseableFunction(closeable);
-    }
-
-    private final AsyncCloseable closeable;
-
-    private CloseAsyncCloseableFunction(AsyncCloseable closeable) {
-        this.closeable = closeable;
-    }
-
-    @Override
-    public BoxedUnit apply() {
-        closeable.asyncClose();
-        return BoxedUnit.UNIT;
-    }
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/function/DefaultValueMapFunction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/function/DefaultValueMapFunction.java
deleted file mode 100644
index 6360f2c..0000000
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/function/DefaultValueMapFunction.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.function;
-
-import scala.runtime.AbstractFunction1;
-
-/**
- * Map Function return default value
- */
-public class DefaultValueMapFunction<T, R> extends AbstractFunction1<T, R> {
-
-    public static <T, R> DefaultValueMapFunction<T, R> of(R defaultValue) {
-        return new DefaultValueMapFunction<T, R>(defaultValue);
-    }
-
-    private final R defaultValue;
-
-    private DefaultValueMapFunction(R defaultValue) {
-        this.defaultValue = defaultValue;
-    }
-
-    @Override
-    public R apply(T any) {
-        return defaultValue;
-    }
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetLastTxIdFunction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetLastTxIdFunction.java
index 1bf620c..00703e3 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetLastTxIdFunction.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetLastTxIdFunction.java
@@ -17,16 +17,15 @@
  */
 package org.apache.distributedlog.function;
 
+import java.util.List;
+import java.util.function.Function;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.LogSegmentMetadata;
-import scala.runtime.AbstractFunction1;
-
-import java.util.List;
 
 /**
  * Retrieve the last tx id from list of log segments
  */
-public class GetLastTxIdFunction extends AbstractFunction1<List<LogSegmentMetadata>, Long> {
+public class GetLastTxIdFunction implements Function<List<LogSegmentMetadata>, Long> {
 
     public static final GetLastTxIdFunction INSTANCE = new GetLastTxIdFunction();
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetVersionedValueFunction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetVersionedValueFunction.java
deleted file mode 100644
index 98164de..0000000
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetVersionedValueFunction.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.function;
-
-import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.bookkeeper.versioning.Versioned;
-import scala.Function1;
-import scala.runtime.AbstractFunction1;
-
-import java.util.List;
-
-/**
- * Function to get the versioned value from {@link org.apache.bookkeeper.versioning.Versioned}
- */
-public class GetVersionedValueFunction<T> extends AbstractFunction1<Versioned<T>, T> {
-
-    public static final Function1<Versioned<List<LogSegmentMetadata>>, List<LogSegmentMetadata>>
-            GET_LOGSEGMENT_LIST_FUNC = new GetVersionedValueFunction<List<LogSegmentMetadata>>();
-
-    @Override
-    public T apply(Versioned<T> versionedValue) {
-        return versionedValue.getValue();
-    }
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/function/VoidFunctions.java b/distributedlog-core/src/main/java/org/apache/distributedlog/function/VoidFunctions.java
deleted file mode 100644
index 79f9c32..0000000
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/function/VoidFunctions.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.function;
-
-import scala.runtime.AbstractFunction1;
-
-import java.util.List;
-
-public class VoidFunctions {
-
-    public static final AbstractFunction1<List<Void>, Void> LIST_TO_VOID_FUNC =
-            new AbstractFunction1<List<Void>, Void>() {
-                @Override
-                public Void apply(List<Void> list) {
-                    return null;
-                }
-            };
-
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java
index dbe5400..21fe227 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java
@@ -24,7 +24,7 @@
 import org.apache.distributedlog.BookKeeperClientBuilder;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.MetadataAccessor;
+import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
 import org.apache.distributedlog.acl.AccessControlManager;
@@ -47,7 +47,7 @@
 import org.apache.distributedlog.metadata.LogStreamMetadataStore;
 import org.apache.distributedlog.namespace.NamespaceDriver;
 import org.apache.distributedlog.namespace.NamespaceDriverManager;
-import org.apache.distributedlog.subscription.SubscriptionsStore;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
 import org.apache.bookkeeper.feature.FeatureProvider;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java
index 0761cfc..7069cbb 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java
@@ -19,23 +19,22 @@
 
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.callback.NamespaceListener;
 import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.metadata.LogMetadataStore;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
 
-import java.net.URI;
-import java.util.Iterator;
-import java.util.List;
-
 import static org.apache.distributedlog.util.DLUtils.*;
 
 /**
@@ -60,18 +59,18 @@
     }
 
     @Override
-    public Future<URI> createLog(String logName) {
-        return Future.value(namespace);
+    public CompletableFuture<URI> createLog(String logName) {
+        return FutureUtils.value(namespace);
     }
 
     @Override
-    public Future<Optional<URI>> getLogLocation(String logName) {
-        return Future.value(nsOptional);
+    public CompletableFuture<Optional<URI>> getLogLocation(String logName) {
+        return FutureUtils.value(nsOptional);
     }
 
     @Override
-    public Future<Iterator<String>> getLogs() {
-        final Promise<Iterator<String>> promise = new Promise<Iterator<String>>();
+    public CompletableFuture<Iterator<String>> getLogs() {
+        final CompletableFuture<Iterator<String>> promise = new CompletableFuture<Iterator<String>>();
         final String nsRootPath = namespace.getPath();
         try {
             final ZooKeeper zk = zkc.get();
@@ -89,30 +88,30 @@
                                             results.add(child);
                                         }
                                     }
-                                    promise.setValue(results.iterator());
+                                    promise.complete(results.iterator());
                                 } else if (KeeperException.Code.NONODE.intValue() == rc) {
                                     List<String> streams = Lists.newLinkedList();
-                                    promise.setValue(streams.iterator());
+                                    promise.complete(streams.iterator());
                                 } else {
-                                    promise.setException(new ZKException("Error reading namespace " + nsRootPath,
+                                    promise.completeExceptionally(new ZKException("Error reading namespace " + nsRootPath,
                                             KeeperException.Code.get(rc)));
                                 }
                             }
                         }, null);
                     } else if (KeeperException.Code.NONODE.intValue() == syncRc) {
                         List<String> streams = Lists.newLinkedList();
-                        promise.setValue(streams.iterator());
+                        promise.complete(streams.iterator());
                     } else {
-                        promise.setException(new ZKException("Error reading namespace " + nsRootPath,
+                        promise.completeExceptionally(new ZKException("Error reading namespace " + nsRootPath,
                                 KeeperException.Code.get(syncRc)));
                     }
                 }
             }, null);
             zkc.get();
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (InterruptedException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         }
         return promise;
     }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java
index b9cb374..f747045 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog.impl;
 
 import com.google.common.collect.ImmutableList;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.ZooKeeperClient;
@@ -29,17 +30,15 @@
 import org.apache.distributedlog.metadata.LogMetadataForWriter;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
 import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Transaction;
 import org.apache.distributedlog.util.Transaction.OpListener;
+import org.apache.distributedlog.util.Utils;
 import org.apache.distributedlog.zk.DefaultZKOp;
 import org.apache.distributedlog.zk.ZKOp;
 import org.apache.distributedlog.zk.ZKTransaction;
 import org.apache.distributedlog.zk.ZKVersionedSetOp;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.meta.ZkVersion;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
@@ -116,7 +115,7 @@
         @Override
         public void run() {
             if (null != store.listeners.get(logSegmentsPath)) {
-                store.zkGetLogSegmentNames(logSegmentsPath, store).addEventListener(this);
+                store.zkGetLogSegmentNames(logSegmentsPath, store).whenComplete(this);
             } else {
                 logger.debug("Log segments listener for {} has been removed.", logSegmentsPath);
             }
@@ -350,18 +349,18 @@
     }
 
     @Override
-    public Future<LogSegmentMetadata> getLogSegment(String logSegmentPath) {
+    public CompletableFuture<LogSegmentMetadata> getLogSegment(String logSegmentPath) {
         return LogSegmentMetadata.read(zkc, logSegmentPath, skipMinVersionCheck);
     }
 
-    Future<Versioned<List<String>>> zkGetLogSegmentNames(String logSegmentsPath, Watcher watcher) {
-        Promise<Versioned<List<String>>> result = new Promise<Versioned<List<String>>>();
+    CompletableFuture<Versioned<List<String>>> zkGetLogSegmentNames(String logSegmentsPath, Watcher watcher) {
+        CompletableFuture<Versioned<List<String>>> result = new CompletableFuture<Versioned<List<String>>>();
         try {
             zkc.get().getChildren(logSegmentsPath, watcher, this, result);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            result.setException(FutureUtils.zkException(e, logSegmentsPath));
+            result.completeExceptionally(Utils.zkException(e, logSegmentsPath));
         } catch (InterruptedException e) {
-            result.setException(FutureUtils.zkException(e, logSegmentsPath));
+            result.completeExceptionally(Utils.zkException(e, logSegmentsPath));
         }
         return result;
     }
@@ -369,21 +368,21 @@
     @Override
     @SuppressWarnings("unchecked")
     public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
-        Promise<Versioned<List<String>>> result = ((Promise<Versioned<List<String>>>) ctx);
+        CompletableFuture<Versioned<List<String>>> result = ((CompletableFuture<Versioned<List<String>>>) ctx);
         if (KeeperException.Code.OK.intValue() == rc) {
             /** cversion: the number of changes to the children of this znode **/
             ZkVersion zkVersion = new ZkVersion(stat.getCversion());
-            result.setValue(new Versioned(children, zkVersion));
+            result.complete(new Versioned(children, zkVersion));
         } else if (KeeperException.Code.NONODE.intValue() == rc) {
-            result.setException(new LogNotFoundException("Log " + path + " not found"));
+            result.completeExceptionally(new LogNotFoundException("Log " + path + " not found"));
         } else {
-            result.setException(new ZKException("Failed to get log segments from " + path,
+            result.completeExceptionally(new ZKException("Failed to get log segments from " + path,
                     KeeperException.Code.get(rc)));
         }
     }
 
     @Override
-    public Future<Versioned<List<String>>> getLogSegmentNames(String logSegmentsPath,
+    public CompletableFuture<Versioned<List<String>>> getLogSegmentNames(String logSegmentsPath,
                                                               LogSegmentNamesListener listener) {
         Watcher zkWatcher;
         if (null == listener) {
@@ -422,9 +421,9 @@
                 closeLock.readLock().unlock();
             }
         }
-        Future<Versioned<List<String>>> getLogSegmentNamesResult = zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
+        CompletableFuture<Versioned<List<String>>> getLogSegmentNamesResult = zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
         if (null != listener) {
-            getLogSegmentNamesResult.addEventListener(new ReadLogSegmentsTask(logSegmentsPath, this));
+            getLogSegmentNamesResult.whenComplete(new ReadLogSegmentsTask(logSegmentsPath, this));
         }
         return zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
     }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java
index 551cc44..b3fe456 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java
@@ -21,17 +21,16 @@
 import java.net.URI;
 
 import com.google.common.annotations.VisibleForTesting;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.MetadataAccessor;
+import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
-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.StatsLogger;
 import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
 import org.apache.bookkeeper.zookeeper.RetryPolicy;
@@ -45,7 +44,7 @@
 public class ZKMetadataAccessor implements MetadataAccessor {
     static final Logger LOG = LoggerFactory.getLogger(ZKMetadataAccessor.class);
     protected final String name;
-    protected Promise<Void> closePromise;
+    protected CompletableFuture<Void> closePromise;
     protected final URI uri;
     // zookeeper clients
     // NOTE: The actual zookeeper client is initialized lazily when it is referenced by
@@ -213,13 +212,13 @@
      * @return future represents the close result.
      */
     @Override
-    public Future<Void> asyncClose() {
-        Promise<Void> closeFuture;
+    public CompletableFuture<Void> asyncClose() {
+        CompletableFuture<Void> closeFuture;
         synchronized (this) {
             if (null != closePromise) {
                 return closePromise;
             }
-            closeFuture = closePromise = new Promise<Void>();
+            closeFuture = closePromise = new CompletableFuture<Void>();
         }
         // NOTE: ownWriterZKC and ownReaderZKC are mostly used by tests
         //       the managers created by the namespace - whose zkc will be closed by namespace
@@ -233,13 +232,13 @@
         } catch (Exception e) {
             LOG.warn("Exception while closing distributed log manager", e);
         }
-        FutureUtils.setValue(closeFuture, null);
+        FutureUtils.complete(closeFuture, null);
         return closeFuture;
     }
 
     @Override
     public void close() throws IOException {
-        FutureUtils.result(asyncClose());
+        Utils.ioResult(asyncClose());
     }
 
     public synchronized void checkClosedOrInError(String operation) throws AlreadyClosedException {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java
index 63a81bd..e4a175c 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java
@@ -19,10 +19,9 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.thrift.AccessControlEntry;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TJSONProtocol;
 import org.apache.thrift.transport.TMemoryBuffer;
@@ -100,8 +99,8 @@
         return accessControlEntry;
     }
 
-    public Future<ZKAccessControl> create(ZooKeeperClient zkc) {
-        final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
+    public CompletableFuture<ZKAccessControl> create(ZooKeeperClient zkc) {
+        final CompletableFuture<ZKAccessControl> promise = new CompletableFuture<ZKAccessControl>();
         try {
             zkc.get().create(zkPath, serialize(accessControlEntry), zkc.getDefaultACL(), CreateMode.PERSISTENT,
                     new AsyncCallback.StringCallback() {
@@ -109,48 +108,48 @@
                         public void processResult(int rc, String path, Object ctx, String name) {
                             if (KeeperException.Code.OK.intValue() == rc) {
                                 ZKAccessControl.this.zkVersion = 0;
-                                promise.setValue(ZKAccessControl.this);
+                                promise.complete(ZKAccessControl.this);
                             } else {
-                                promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                                promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                             }
                         }
                     }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (InterruptedException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (IOException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         }
         return promise;
     }
 
-    public Future<ZKAccessControl> update(ZooKeeperClient zkc) {
-        final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
+    public CompletableFuture<ZKAccessControl> update(ZooKeeperClient zkc) {
+        final CompletableFuture<ZKAccessControl> promise = new CompletableFuture<ZKAccessControl>();
         try {
             zkc.get().setData(zkPath, serialize(accessControlEntry), zkVersion, new AsyncCallback.StatCallback() {
                 @Override
                 public void processResult(int rc, String path, Object ctx, Stat stat) {
                     if (KeeperException.Code.OK.intValue() == rc) {
                         ZKAccessControl.this.zkVersion = stat.getVersion();
-                        promise.setValue(ZKAccessControl.this);
+                        promise.complete(ZKAccessControl.this);
                     } else {
-                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                        promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                     }
                 }
             }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (InterruptedException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (IOException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         }
         return promise;
     }
 
-    public static Future<ZKAccessControl> read(final ZooKeeperClient zkc, final String zkPath, Watcher watcher) {
-        final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
+    public static CompletableFuture<ZKAccessControl> read(final ZooKeeperClient zkc, final String zkPath, Watcher watcher) {
+        final CompletableFuture<ZKAccessControl> promise = new CompletableFuture<ZKAccessControl>();
 
         try {
             zkc.get().getData(zkPath, watcher, new AsyncCallback.DataCallback() {
@@ -159,25 +158,25 @@
                     if (KeeperException.Code.OK.intValue() == rc) {
                         try {
                             AccessControlEntry ace = deserialize(zkPath, data);
-                            promise.setValue(new ZKAccessControl(ace, zkPath, stat.getVersion()));
+                            promise.complete(new ZKAccessControl(ace, zkPath, stat.getVersion()));
                         } catch (IOException ioe) {
-                            promise.setException(ioe);
+                            promise.completeExceptionally(ioe);
                         }
                     } else {
-                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                        promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                     }
                 }
             }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (InterruptedException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         }
         return promise;
     }
 
-    public static Future<Void> delete(final ZooKeeperClient zkc, final String zkPath) {
-        final Promise<Void> promise = new Promise<Void>();
+    public static CompletableFuture<Void> delete(final ZooKeeperClient zkc, final String zkPath) {
+        final CompletableFuture<Void> promise = new CompletableFuture<Void>();
 
         try {
             zkc.get().delete(zkPath, -1, new AsyncCallback.VoidCallback() {
@@ -185,16 +184,16 @@
                 public void processResult(int rc, String path, Object ctx) {
                     if (KeeperException.Code.OK.intValue() == rc ||
                             KeeperException.Code.NONODE.intValue() == rc) {
-                        promise.setValue(null);
+                        promise.complete(null);
                     } else {
-                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                        promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                     }
                 }
             }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (InterruptedException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         }
         return promise;
     }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java
index be8db64..3dbde6a 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java
@@ -18,16 +18,15 @@
 package org.apache.distributedlog.impl.acl;
 
 import com.google.common.collect.Sets;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.acl.AccessControlManager;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.thrift.AccessControlEntry;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.util.ZkUtils;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -76,7 +75,7 @@
         this.scheduledExecutorService = scheduledExecutorService;
         this.streamEntries = new ConcurrentHashMap<String, ZKAccessControl>();
         try {
-            Await.result(fetchDefaultAccessControlEntry());
+            FutureUtils.result(fetchDefaultAccessControlEntry());
         } catch (Throwable t) {
             if (t instanceof InterruptedException) {
                 throw new DLInterruptedException("Interrupted on getting default access control entry for " + zkRootPath, t);
@@ -90,7 +89,7 @@
         }
 
         try {
-            Await.result(fetchAccessControlEntries());
+            FutureUtils.result(fetchAccessControlEntries());
         } catch (Throwable t) {
             if (t instanceof InterruptedException) {
                 throw new DLInterruptedException("Interrupted on getting access control entries for " + zkRootPath, t);
@@ -140,19 +139,19 @@
         closed = true;
     }
 
-    private Future<Void> fetchAccessControlEntries() {
-        final Promise<Void> promise = new Promise<Void>();
+    private CompletableFuture<Void> fetchAccessControlEntries() {
+        final CompletableFuture<Void> promise = new CompletableFuture<Void>();
         fetchAccessControlEntries(promise);
         return promise;
     }
 
-    private void fetchAccessControlEntries(final Promise<Void> promise) {
+    private void fetchAccessControlEntries(final CompletableFuture<Void> promise) {
         try {
             zkc.get().getChildren(zkRootPath, this, new AsyncCallback.Children2Callback() {
                 @Override
                 public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
                     if (KeeperException.Code.OK.intValue() != rc) {
-                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                        promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                         return;
                     }
                     Set<String> streamsReceived = new HashSet<String>();
@@ -166,7 +165,7 @@
                         }
                     }
                     if (streamsReceived.isEmpty()) {
-                        promise.setValue(null);
+                        promise.complete(null);
                         return;
                     }
                     final AtomicInteger numPendings = new AtomicInteger(streamsReceived.size());
@@ -174,7 +173,7 @@
                     for (String s : streamsReceived) {
                         final String streamName = s;
                         ZKAccessControl.read(zkc, zkRootPath + "/" + streamName, null)
-                                .addEventListener(new FutureEventListener<ZKAccessControl>() {
+                                .whenComplete(new FutureEventListener<ZKAccessControl>() {
 
                                     @Override
                                     public void onSuccess(ZKAccessControl accessControl) {
@@ -193,7 +192,7 @@
                                             streamEntries.remove(streamName);
                                         } else {
                                             if (1 == numFailures.incrementAndGet()) {
-                                                promise.setException(cause);
+                                                promise.completeExceptionally(cause);
                                             }
                                         }
                                         complete();
@@ -201,7 +200,7 @@
 
                                     private void complete() {
                                         if (0 == numPendings.decrementAndGet() && numFailures.get() == 0) {
-                                            promise.setValue(null);
+                                            promise.complete(null);
                                         }
                                     }
                                 });
@@ -209,28 +208,28 @@
                 }
             }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (InterruptedException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         }
     }
 
-    private Future<ZKAccessControl> fetchDefaultAccessControlEntry() {
-        final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
+    private CompletableFuture<ZKAccessControl> fetchDefaultAccessControlEntry() {
+        final CompletableFuture<ZKAccessControl> promise = new CompletableFuture<ZKAccessControl>();
         fetchDefaultAccessControlEntry(promise);
         return promise;
     }
 
-    private void fetchDefaultAccessControlEntry(final Promise<ZKAccessControl> promise) {
+    private void fetchDefaultAccessControlEntry(final CompletableFuture<ZKAccessControl> promise) {
         ZKAccessControl.read(zkc, zkRootPath, this)
-            .addEventListener(new FutureEventListener<ZKAccessControl>() {
+            .whenComplete(new FutureEventListener<ZKAccessControl>() {
                 @Override
                 public void onSuccess(ZKAccessControl accessControl) {
                     logger.info("Default Access Control will be changed from {} to {}",
                                 ZKAccessControlManager.this.defaultAccessControl,
                                 accessControl);
                     ZKAccessControlManager.this.defaultAccessControl = accessControl;
-                    promise.setValue(accessControl);
+                    promise.complete(accessControl);
                 }
 
                 @Override
@@ -239,21 +238,21 @@
                         logger.info("Default Access Control is missing, creating one for {} ...", zkRootPath);
                         createDefaultAccessControlEntryIfNeeded(promise);
                     } else {
-                        promise.setException(cause);
+                        promise.completeExceptionally(cause);
                     }
                 }
             });
     }
 
-    private void createDefaultAccessControlEntryIfNeeded(final Promise<ZKAccessControl> promise) {
+    private void createDefaultAccessControlEntryIfNeeded(final CompletableFuture<ZKAccessControl> promise) {
         ZooKeeper zk;
         try {
             zk = zkc.get();
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
             return;
         } catch (InterruptedException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
             return;
         }
         ZkUtils.asyncCreateFullPathOptimistic(zk, zkRootPath, new byte[0], zkc.getDefaultACL(),
@@ -264,7 +263,7 @@
                     logger.info("Created zk path {} for default ACL.", zkRootPath);
                     fetchDefaultAccessControlEntry(promise);
                 } else {
-                    promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                    promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                 }
             }
         }, null);
@@ -277,7 +276,7 @@
         scheduledExecutorService.schedule(new Runnable() {
             @Override
             public void run() {
-                fetchDefaultAccessControlEntry().addEventListener(new FutureEventListener<ZKAccessControl>() {
+                fetchDefaultAccessControlEntry().whenComplete(new FutureEventListener<ZKAccessControl>() {
                     @Override
                     public void onSuccess(ZKAccessControl value) {
                         // no-op
@@ -305,7 +304,7 @@
         scheduledExecutorService.schedule(new Runnable() {
             @Override
             public void run() {
-                fetchAccessControlEntries().addEventListener(new FutureEventListener<Void>() {
+                fetchAccessControlEntries().whenComplete(new FutureEventListener<Void>() {
                     @Override
                     public void onSuccess(Void value) {
                         // no-op
@@ -328,10 +327,10 @@
         scheduledExecutorService.schedule(new Runnable() {
             @Override
             public void run() {
-                fetchDefaultAccessControlEntry().addEventListener(new FutureEventListener<ZKAccessControl>() {
+                fetchDefaultAccessControlEntry().whenComplete(new FutureEventListener<ZKAccessControl>() {
                     @Override
                     public void onSuccess(ZKAccessControl value) {
-                        fetchAccessControlEntries().addEventListener(new FutureEventListener<Void>() {
+                        fetchAccessControlEntries().whenComplete(new FutureEventListener<Void>() {
                             @Override
                             public void onSuccess(Void value) {
                                 // no-op
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
index 5d7af9d..17515c3 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
@@ -23,6 +23,7 @@
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.callback.NamespaceListener;
@@ -32,12 +33,10 @@
 import org.apache.distributedlog.impl.ZKNamespaceWatcher;
 import org.apache.distributedlog.metadata.LogMetadataStore;
 import org.apache.distributedlog.namespace.NamespaceWatcher;
-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.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -49,8 +48,6 @@
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
 
 import java.io.IOException;
 import java.net.URI;
@@ -80,8 +77,9 @@
  * NOTE: current federated namespace isn't optimized for deletion/creation. so don't use it in the workloads
  *       that have lots of creations or deletions.
  */
-public class FederatedZKLogMetadataStore extends NamespaceWatcher implements LogMetadataStore, Watcher, Runnable,
-        FutureEventListener<Set<URI>> {
+public class FederatedZKLogMetadataStore
+        extends NamespaceWatcher
+        implements LogMetadataStore, Watcher, Runnable, FutureEventListener<Set<URI>> {
 
     static final Logger logger = LoggerFactory.getLogger(FederatedZKLogMetadataStore.class);
 
@@ -100,7 +98,7 @@
      * @throws KeeperException
      */
     public static void createFederatedNamespace(URI namespace, ZooKeeperClient zkc)
-            throws InterruptedException, ZooKeeperClient.ZooKeeperConnectionException, KeeperException {
+            throws IOException, KeeperException {
         String zkSubNamespacesPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES;
         Utils.zkCreateFullPathOptimistic(zkc, zkSubNamespacesPath, new byte[0],
                 zkc.getDefaultACL(), CreateMode.PERSISTENT);
@@ -112,7 +110,7 @@
     class SubNamespace implements NamespaceListener {
         final URI uri;
         final ZKNamespaceWatcher watcher;
-        Promise<Set<String>> logsFuture = new Promise<Set<String>>();
+        CompletableFuture<Set<String>> logsFuture = new CompletableFuture<Set<String>>();
 
         SubNamespace(URI uri) {
             this.uri = uri;
@@ -124,7 +122,7 @@
             this.watcher.watchNamespaceChanges();
         }
 
-        synchronized Future<Set<String>> getLogs() {
+        synchronized CompletableFuture<Set<String>> getLogs() {
             return logsFuture;
         }
 
@@ -134,16 +132,16 @@
             Set<String> oldLogs = Sets.newHashSet();
 
             // update the sub namespace cache
-            Promise<Set<String>> newLogsPromise;
+            CompletableFuture<Set<String>> newLogsPromise;
             synchronized (this) {
-                if (logsFuture.isDefined()) { // the promise is already satisfied
+                if (logsFuture.isDone()) { // the promise is already satisfied
                     try {
                         oldLogs = FutureUtils.result(logsFuture);
-                    } catch (IOException e) {
+                    } catch (Exception e) {
                         logger.error("Unexpected exception when getting logs from a satisified future of {} : ",
                                 uri, e);
                     }
-                    logsFuture = new Promise<Set<String>>();
+                    logsFuture = new CompletableFuture<Set<String>>();
                 }
 
                 // update the reverse cache
@@ -163,7 +161,7 @@
                 }
                 newLogsPromise = logsFuture;
             }
-            newLogsPromise.setValue(newLogs);
+            newLogsPromise.complete(newLogs);
 
             // notify namespace changes
             notifyOnNamespaceChanges();
@@ -203,7 +201,16 @@
         this.maxLogsPerSubnamespace = conf.getFederatedMaxLogsPerSubnamespace();
 
         // fetch the sub namespace
-        Set<URI> uris = FutureUtils.result(fetchSubNamespaces(this));
+        Set<URI> uris;
+        try {
+            uris = FutureUtils.result(fetchSubNamespaces(this));
+        } catch (Exception e) {
+            if (e instanceof IOException) {
+                throw (IOException) e;
+            } else {
+                throw new IOException(e);
+            }
+        }
         for (URI uri : uris) {
             SubNamespace subNs = new SubNamespace(uri);
             if (null == subNamespaces.putIfAbsent(uri, subNs)) {
@@ -228,21 +235,21 @@
         }
     }
 
-    private <T> Future<T> postStateCheck(Future<T> future) {
-        final Promise<T> postCheckedPromise = new Promise<T>();
-        future.addEventListener(new FutureEventListener<T>() {
+    private <T> CompletableFuture<T> postStateCheck(CompletableFuture<T> future) {
+        final CompletableFuture<T> postCheckedPromise = new CompletableFuture<T>();
+        future.whenComplete(new FutureEventListener<T>() {
             @Override
             public void onSuccess(T value) {
                 if (duplicatedLogFound.get()) {
-                    postCheckedPromise.setException(new UnexpectedException("Duplicate log found under " + namespace));
+                    postCheckedPromise.completeExceptionally(new UnexpectedException("Duplicate log found under " + namespace));
                 } else {
-                    postCheckedPromise.setValue(value);
+                    postCheckedPromise.complete(value);
                 }
             }
 
             @Override
             public void onFailure(Throwable cause) {
-                postCheckedPromise.setException(cause);
+                postCheckedPromise.completeExceptionally(cause);
             }
         });
         return postCheckedPromise;
@@ -273,13 +280,13 @@
                 namespace.getFragment());
     }
 
-    Future<Set<URI>> getCachedSubNamespaces() {
+    CompletableFuture<Set<URI>> getCachedSubNamespaces() {
         Set<URI> nsSet = subNamespaces.keySet();
-        return Future.value(nsSet);
+        return FutureUtils.value(nsSet);
     }
 
-    Future<Set<URI>> fetchSubNamespaces(final Watcher watcher) {
-        final Promise<Set<URI>> promise = new Promise<Set<URI>>();
+    CompletableFuture<Set<URI>> fetchSubNamespaces(final Watcher watcher) {
+        final CompletableFuture<Set<URI>> promise = new CompletableFuture<Set<URI>>();
         try {
             zkc.get().sync(this.zkSubnamespacesPath, new AsyncCallback.VoidCallback() {
                 @Override
@@ -287,27 +294,27 @@
                     if (Code.OK.intValue() == rc) {
                         fetchSubNamespaces(watcher, promise);
                     } else {
-                        promise.setException(KeeperException.create(Code.get(rc)));
+                        promise.completeExceptionally(KeeperException.create(Code.get(rc)));
                     }
                 }
             }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (InterruptedException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         }
         return promise;
     }
 
     private void fetchSubNamespaces(Watcher watcher,
-                                    final Promise<Set<URI>> promise) {
+                                    final CompletableFuture<Set<URI>> promise) {
         try {
             zkc.get().getChildren(this.zkSubnamespacesPath, watcher,
                     new AsyncCallback.Children2Callback() {
                         @Override
                         public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
                             if (Code.NONODE.intValue() == rc) {
-                                promise.setException(new UnexpectedException(
+                                promise.completeExceptionally(new UnexpectedException(
                                         "The subnamespaces don't exist for the federated namespace " + namespace));
                             } else if (Code.OK.intValue() == rc) {
                                 Set<URI> subnamespaces = Sets.newHashSet();
@@ -318,26 +325,26 @@
                                     }
                                 } catch (URISyntaxException use) {
                                     logger.error("Invalid sub namespace uri found : ", use);
-                                    promise.setException(new UnexpectedException(
+                                    promise.completeExceptionally(new UnexpectedException(
                                             "Invalid sub namespace uri found in " + namespace, use));
                                     return;
                                 }
                                 // update the sub namespaces set before update version
                                 setZkSubnamespacesVersion(stat.getVersion());
-                                promise.setValue(subnamespaces);
+                                promise.complete(subnamespaces);
                             }
                         }
                     }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (InterruptedException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         }
     }
 
     @Override
     public void run() {
-        fetchSubNamespaces(this).addEventListener(this);
+        fetchSubNamespaces(this).whenComplete(this);
     }
 
     @Override
@@ -370,7 +377,7 @@
         }
         if (Event.EventType.NodeChildrenChanged == watchedEvent.getType()) {
             // fetch the namespace
-            fetchSubNamespaces(this).addEventListener(this);
+            fetchSubNamespaces(this).whenComplete(this);
         }
     }
 
@@ -378,27 +385,27 @@
     // Log Related Methods
     //
 
-    private <A> Future<A> duplicatedLogException(String logName) {
-        return Future.exception(new UnexpectedException("Duplicated log " + logName
+    private <A> CompletableFuture<A> duplicatedLogException(String logName) {
+        return FutureUtils.exception(new UnexpectedException("Duplicated log " + logName
                 + " found in namespace " + namespace));
     }
 
     @Override
-    public Future<URI> createLog(final String logName) {
+    public CompletableFuture<URI> createLog(final String logName) {
         if (duplicatedLogFound.get()) {
             return duplicatedLogException(duplicatedLogName.get());
         }
-        Promise<URI> createPromise = new Promise<URI>();
+        CompletableFuture<URI> createPromise = new CompletableFuture<URI>();
         doCreateLog(logName, createPromise);
         return postStateCheck(createPromise);
     }
 
-    void doCreateLog(final String logName, final Promise<URI> createPromise) {
-        getLogLocation(logName).addEventListener(new FutureEventListener<Optional<URI>>() {
+    void doCreateLog(final String logName, final CompletableFuture<URI> createPromise) {
+        getLogLocation(logName).whenComplete(new FutureEventListener<Optional<URI>>() {
             @Override
             public void onSuccess(Optional<URI> uriOptional) {
                 if (uriOptional.isPresent()) {
-                    createPromise.setException(new LogExistsException("Log " + logName + " already exists in " + uriOptional.get()));
+                    createPromise.completeExceptionally(new LogExistsException("Log " + logName + " already exists in " + uriOptional.get()));
                 } else {
                     getCachedSubNamespacesAndCreateLog(logName, createPromise);
                 }
@@ -406,14 +413,14 @@
 
             @Override
             public void onFailure(Throwable cause) {
-                createPromise.setException(cause);
+                createPromise.completeExceptionally(cause);
             }
         });
     }
 
     private void getCachedSubNamespacesAndCreateLog(final String logName,
-                                                    final Promise<URI> createPromise) {
-        getCachedSubNamespaces().addEventListener(new FutureEventListener<Set<URI>>() {
+                                                    final CompletableFuture<URI> createPromise) {
+        getCachedSubNamespaces().whenComplete(new FutureEventListener<Set<URI>>() {
             @Override
             public void onSuccess(Set<URI> uris) {
                 findSubNamespaceToCreateLog(logName, uris, createPromise);
@@ -421,14 +428,14 @@
 
             @Override
             public void onFailure(Throwable cause) {
-                createPromise.setException(cause);
+                createPromise.completeExceptionally(cause);
             }
         });
     }
 
     private void fetchSubNamespacesAndCreateLog(final String logName,
-                                                final Promise<URI> createPromise) {
-        fetchSubNamespaces(null).addEventListener(new FutureEventListener<Set<URI>>() {
+                                                final CompletableFuture<URI> createPromise) {
+        fetchSubNamespaces(null).whenComplete(new FutureEventListener<Set<URI>>() {
             @Override
             public void onSuccess(Set<URI> uris) {
                 findSubNamespaceToCreateLog(logName, uris, createPromise);
@@ -436,26 +443,26 @@
 
             @Override
             public void onFailure(Throwable cause) {
-                createPromise.setException(cause);
+                createPromise.completeExceptionally(cause);
             }
         });
     }
 
     private void findSubNamespaceToCreateLog(final String logName,
                                              final Set<URI> uris,
-                                             final Promise<URI> createPromise) {
+                                             final CompletableFuture<URI> createPromise) {
         final List<URI> uriList = Lists.newArrayListWithExpectedSize(uris.size());
-        List<Future<Set<String>>> futureList = Lists.newArrayListWithExpectedSize(uris.size());
+        List<CompletableFuture<Set<String>>> futureList = Lists.newArrayListWithExpectedSize(uris.size());
         for (URI uri : uris) {
             SubNamespace subNs = subNamespaces.get(uri);
             if (null == subNs) {
-                createPromise.setException(new UnexpectedException("No sub namespace " + uri + " found"));
+                createPromise.completeExceptionally(new UnexpectedException("No sub namespace " + uri + " found"));
                 return;
             }
             futureList.add(subNs.getLogs());
             uriList.add(uri);
         }
-        Future.collect(futureList).addEventListener(new FutureEventListener<List<Set<String>>>() {
+        FutureUtils.collect(futureList).whenComplete(new FutureEventListener<List<Set<String>>>() {
             @Override
             public void onSuccess(List<Set<String>> resultList) {
                 for (int i = resultList.size() - 1; i >= 0; i--) {
@@ -467,7 +474,7 @@
                     }
                 }
                 // All sub namespaces are full
-                createSubNamespace().addEventListener(new FutureEventListener<URI>() {
+                createSubNamespace().whenComplete(new FutureEventListener<URI>() {
                     @Override
                     public void onSuccess(URI uri) {
                         // the new namespace will be propagated to the namespace cache by the namespace listener
@@ -479,14 +486,14 @@
 
                     @Override
                     public void onFailure(Throwable cause) {
-                        createPromise.setException(cause);
+                        createPromise.completeExceptionally(cause);
                     }
                 });
             }
 
             @Override
             public void onFailure(Throwable cause) {
-                createPromise.setException(cause);
+                createPromise.completeExceptionally(cause);
             }
         });
     }
@@ -499,8 +506,8 @@
         return SUB_NAMESPACE_PREFIX + parts[parts.length - 1];
     }
 
-    Future<URI> createSubNamespace() {
-        final Promise<URI> promise = new Promise<URI>();
+    CompletableFuture<URI> createSubNamespace() {
+        final CompletableFuture<URI> promise = new CompletableFuture<URI>();
 
         final String nsPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES + "/" + SUB_NAMESPACE_PREFIX;
         try {
@@ -512,21 +519,21 @@
                                 try {
                                     URI newUri = getSubNamespaceURI(getNamespaceFromZkPath(name));
                                     logger.info("Created sub namespace {}", newUri);
-                                    promise.setValue(newUri);
+                                    promise.complete(newUri);
                                 } catch (UnexpectedException ue) {
-                                    promise.setException(ue);
+                                    promise.completeExceptionally(ue);
                                 } catch (URISyntaxException e) {
-                                    promise.setException(new UnexpectedException("Invalid namespace " + name + " is created."));
+                                    promise.completeExceptionally(new UnexpectedException("Invalid namespace " + name + " is created."));
                                 }
                             } else {
-                                promise.setException(KeeperException.create(Code.get(rc)));
+                                promise.completeExceptionally(KeeperException.create(Code.get(rc)));
                             }
                         }
                     }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         } catch (InterruptedException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         }
 
         return promise;
@@ -545,22 +552,22 @@
      */
     private void createLogInNamespace(final URI uri,
                                       final String logName,
-                                      final Promise<URI> createPromise) {
+                                      final CompletableFuture<URI> createPromise) {
         // TODO: rewrite this after we bump to zk 3.5, where we will have asynchronous version of multi
         scheduler.submit(new Runnable() {
             @Override
             public void run() {
                 try {
                     createLogInNamespaceSync(uri, logName);
-                    createPromise.setValue(uri);
+                    createPromise.complete(uri);
                 } catch (InterruptedException e) {
-                    createPromise.setException(e);
+                    createPromise.completeExceptionally(e);
                 } catch (IOException e) {
-                    createPromise.setException(e);
+                    createPromise.completeExceptionally(e);
                 } catch (KeeperException.BadVersionException bve) {
                     fetchSubNamespacesAndCreateLog(logName, createPromise);
                 } catch (KeeperException e) {
-                    createPromise.setException(e);
+                    createPromise.completeExceptionally(e);
                 }
             }
         });
@@ -617,39 +624,35 @@
     }
 
     @Override
-    public Future<Optional<URI>> getLogLocation(final String logName) {
+    public CompletableFuture<Optional<URI>> getLogLocation(final String logName) {
         if (duplicatedLogFound.get()) {
             return duplicatedLogException(duplicatedLogName.get());
         }
         URI location = log2Locations.get(logName);
         if (null != location) {
-            return postStateCheck(Future.value(Optional.of(location)));
+            return postStateCheck(FutureUtils.value(Optional.of(location)));
         }
         if (!forceCheckLogExistence) {
             Optional<URI> result = Optional.absent();
-            return Future.value(result);
+            return FutureUtils.value(result);
         }
-        return postStateCheck(fetchLogLocation(logName).onSuccess(
-                new AbstractFunction1<Optional<URI>, BoxedUnit>() {
-                    @Override
-                    public BoxedUnit apply(Optional<URI> uriOptional) {
-                        if (uriOptional.isPresent()) {
-                            log2Locations.putIfAbsent(logName, uriOptional.get());
-                        }
-                        return BoxedUnit.UNIT;
-                    }
-                }));
+        return postStateCheck(fetchLogLocation(logName).thenApply((uriOptional) -> {
+            if (uriOptional.isPresent()) {
+                log2Locations.putIfAbsent(logName, uriOptional.get());
+            }
+            return uriOptional;
+        }));
     }
 
-    private Future<Optional<URI>> fetchLogLocation(final String logName) {
-        final Promise<Optional<URI>> fetchPromise = new Promise<Optional<URI>>();
+    private CompletableFuture<Optional<URI>> fetchLogLocation(final String logName) {
+        final CompletableFuture<Optional<URI>> fetchPromise = new CompletableFuture<Optional<URI>>();
 
         Set<URI> uris = subNamespaces.keySet();
-        List<Future<Optional<URI>>> fetchFutures = Lists.newArrayListWithExpectedSize(uris.size());
+        List<CompletableFuture<Optional<URI>>> fetchFutures = Lists.newArrayListWithExpectedSize(uris.size());
         for (URI uri : uris) {
             fetchFutures.add(fetchLogLocation(uri, logName));
         }
-        Future.collect(fetchFutures).addEventListener(new FutureEventListener<List<Optional<URI>>>() {
+        FutureUtils.collect(fetchFutures).whenComplete(new FutureEventListener<List<Optional<URI>>>() {
             @Override
             public void onSuccess(List<Optional<URI>> fetchResults) {
                 Optional<URI> result = Optional.absent();
@@ -660,7 +663,7 @@
                                     new Object[] { logName, result.get(), fetchResult.get() });
                             duplicatedLogName.compareAndSet(null, logName);
                             duplicatedLogFound.set(true);
-                            fetchPromise.setException(new UnexpectedException("Log " + logName
+                            fetchPromise.completeExceptionally(new UnexpectedException("Log " + logName
                                     + " is found in multiple sub namespaces : "
                                     + result.get() + " & " + fetchResult.get()));
                             return;
@@ -669,62 +672,57 @@
                         result = fetchResult;
                     }
                 }
-                fetchPromise.setValue(result);
+                fetchPromise.complete(result);
             }
 
             @Override
             public void onFailure(Throwable cause) {
-                fetchPromise.setException(cause);
+                fetchPromise.completeExceptionally(cause);
             }
         });
         return fetchPromise;
     }
 
-    private Future<Optional<URI>> fetchLogLocation(final URI uri, String logName) {
-        final Promise<Optional<URI>> fetchPromise = new Promise<Optional<URI>>();
+    private CompletableFuture<Optional<URI>> fetchLogLocation(final URI uri, String logName) {
+        final CompletableFuture<Optional<URI>> fetchPromise = new CompletableFuture<Optional<URI>>();
         final String logRootPath = uri.getPath() + "/" + logName;
         try {
             zkc.get().exists(logRootPath, false, new AsyncCallback.StatCallback() {
                 @Override
                 public void processResult(int rc, String path, Object ctx, Stat stat) {
                     if (Code.OK.intValue() == rc) {
-                        fetchPromise.setValue(Optional.of(uri));
+                        fetchPromise.complete(Optional.of(uri));
                     } else if (Code.NONODE.intValue() == rc) {
-                        fetchPromise.setValue(Optional.<URI>absent());
+                        fetchPromise.complete(Optional.<URI>absent());
                     } else {
-                        fetchPromise.setException(KeeperException.create(Code.get(rc)));
+                        fetchPromise.completeExceptionally(KeeperException.create(Code.get(rc)));
                     }
                 }
             }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            fetchPromise.setException(e);
+            fetchPromise.completeExceptionally(e);
         } catch (InterruptedException e) {
-            fetchPromise.setException(e);
+            fetchPromise.completeExceptionally(e);
         }
         return fetchPromise;
     }
 
     @Override
-    public Future<Iterator<String>> getLogs() {
+    public CompletableFuture<Iterator<String>> getLogs() {
         if (duplicatedLogFound.get()) {
             return duplicatedLogException(duplicatedLogName.get());
         }
-        return postStateCheck(retrieveLogs().map(
-                new AbstractFunction1<List<Set<String>>, Iterator<String>>() {
-                    @Override
-                    public Iterator<String> apply(List<Set<String>> resultList) {
-                        return getIterator(resultList);
-                    }
-                }));
+        return postStateCheck(retrieveLogs().thenApply(
+            resultList -> getIterator(resultList)));
     }
 
-    private Future<List<Set<String>>> retrieveLogs() {
+    private CompletableFuture<List<Set<String>>> retrieveLogs() {
         Collection<SubNamespace> subNss = subNamespaces.values();
-        List<Future<Set<String>>> logsList = Lists.newArrayListWithExpectedSize(subNss.size());
+        List<CompletableFuture<Set<String>>> logsList = Lists.newArrayListWithExpectedSize(subNss.size());
         for (SubNamespace subNs : subNss) {
             logsList.add(subNs.getLogs());
         }
-        return Future.collect(logsList);
+        return FutureUtils.collect(logsList);
     }
 
     private Iterator<String> getIterator(List<Set<String>> resultList) {
@@ -747,13 +745,9 @@
     }
 
     private void notifyOnNamespaceChanges() {
-        retrieveLogs().onSuccess(new AbstractFunction1<List<Set<String>>, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(List<Set<String>> resultList) {
-                for (NamespaceListener listener : listeners) {
-                    listener.onStreamsChanged(getIterator(resultList));
-                }
-                return BoxedUnit.UNIT;
+        retrieveLogs().thenAccept(resultList -> {
+            for (NamespaceListener listener : listeners) {
+                listener.onStreamsChanged(getIterator(resultList));
             }
         });
     }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
index 8f9913e..e45c755 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
@@ -17,25 +17,23 @@
  */
 package org.apache.distributedlog.impl.logsegment;
 
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
 import org.apache.distributedlog.bk.LedgerAllocator;
 import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
 import org.apache.distributedlog.util.Allocator;
 import org.apache.distributedlog.util.Transaction;
-import com.twitter.util.Future;
 import org.apache.bookkeeper.client.LedgerHandle;
-import scala.Function1;
-import scala.runtime.AbstractFunction1;
-
-import java.io.IOException;
 
 /**
  * Allocate log segments
  */
 class BKLogSegmentAllocator implements Allocator<LogSegmentEntryWriter, Object> {
 
-    private static class NewLogSegmentEntryWriterFn extends AbstractFunction1<LedgerHandle, LogSegmentEntryWriter> {
+    private static class NewLogSegmentEntryWriterFn implements Function<LedgerHandle, LogSegmentEntryWriter> {
 
-        static final Function1<LedgerHandle, LogSegmentEntryWriter> INSTANCE =
+        static final Function<LedgerHandle, LogSegmentEntryWriter> INSTANCE =
                 new NewLogSegmentEntryWriterFn();
 
         private NewLogSegmentEntryWriterFn() {}
@@ -58,8 +56,8 @@
     }
 
     @Override
-    public Future<LogSegmentEntryWriter> tryObtain(Transaction<Object> txn,
-                                                   final Transaction.OpListener<LogSegmentEntryWriter> listener) {
+    public CompletableFuture<LogSegmentEntryWriter> tryObtain(Transaction<Object> txn,
+                                                              final Transaction.OpListener<LogSegmentEntryWriter> listener) {
         return allocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
             @Override
             public void onCommit(LedgerHandle lh) {
@@ -70,16 +68,16 @@
             public void onAbort(Throwable t) {
                 listener.onAbort(t);
             }
-        }).map(NewLogSegmentEntryWriterFn.INSTANCE);
+        }).thenApply(NewLogSegmentEntryWriterFn.INSTANCE);
     }
 
     @Override
-    public Future<Void> asyncClose() {
+    public CompletableFuture<Void> asyncClose() {
         return allocator.asyncClose();
     }
 
     @Override
-    public Future<Void> delete() {
+    public CompletableFuture<Void> delete() {
         return allocator.delete();
     }
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
index 034b23e..0bb91ae 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
@@ -19,6 +19,7 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.Entry;
 import org.apache.distributedlog.LogSegmentMetadata;
@@ -29,10 +30,8 @@
 import org.apache.distributedlog.exceptions.ReadCancelledException;
 import org.apache.distributedlog.injector.AsyncFailureInjector;
 import org.apache.distributedlog.logsegment.LogSegmentEntryReader;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.client.AsyncCallback;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
@@ -87,7 +86,7 @@
             return done;
         }
 
-        void setValue(LedgerEntry entry) {
+        void complete(LedgerEntry entry) {
             synchronized (this) {
                 if (done) {
                     return;
@@ -98,7 +97,7 @@
             setDone(true);
         }
 
-        void setException(int rc) {
+        void completeExceptionally(int rc) {
             synchronized (this) {
                 if (done) {
                     return;
@@ -152,16 +151,16 @@
             while (entries.hasMoreElements()) {
                 // more entries are returned
                 if (null != entry) {
-                    setException(BKException.Code.UnexpectedConditionException);
+                    completeExceptionally(BKException.Code.UnexpectedConditionException);
                     return;
                 }
                 entry = entries.nextElement();
             }
             if (null == entry || entry.getEntryId() != entryId) {
-                setException(BKException.Code.UnexpectedConditionException);
+                completeExceptionally(BKException.Code.UnexpectedConditionException);
                 return;
             }
-            setValue(entry);
+            complete(entry);
         }
 
         @Override
@@ -186,7 +185,7 @@
                 return;
             }
             if (null != entry && this.entryId == entryId) {
-                setValue(entry);
+                complete(entry);
                 return;
             }
             // the long poll is timeout or interrupted; we will retry it again.
@@ -215,7 +214,7 @@
                         nextReadBackoffTime,
                         TimeUnit.MILLISECONDS);
             } else {
-                setException(rc);
+                completeExceptionally(rc);
             }
             return false;
         }
@@ -229,7 +228,7 @@
     private class PendingReadRequest {
         private final int numEntries;
         private final List<Entry.Reader> entries;
-        private final Promise<List<Entry.Reader>> promise;
+        private final CompletableFuture<List<Entry.Reader>> promise;
 
         PendingReadRequest(int numEntries) {
             this.numEntries = numEntries;
@@ -238,15 +237,15 @@
             } else {
                 this.entries = new ArrayList<Entry.Reader>();
             }
-            this.promise = new Promise<List<Entry.Reader>>();
+            this.promise = new CompletableFuture<List<Entry.Reader>>();
         }
 
-        Promise<List<Entry.Reader>> getPromise() {
+        CompletableFuture<List<Entry.Reader>> getPromise() {
             return promise;
         }
 
-        void setException(Throwable throwable) {
-            FutureUtils.setException(promise, throwable);
+        void completeExceptionally(Throwable throwable) {
+            FutureUtils.completeExceptionally(promise, throwable);
         }
 
         void addEntry(Entry.Reader entry) {
@@ -254,7 +253,7 @@
         }
 
         void complete() {
-            FutureUtils.setValue(promise, entries);
+            FutureUtils.complete(promise, entries);
             onEntriesConsumed(entries.size());
         }
 
@@ -277,7 +276,7 @@
     private final int numPrefetchEntries;
     private final int maxPrefetchEntries;
     // state
-    private Promise<Void> closePromise = null;
+    private CompletableFuture<Void> closePromise = null;
     private LogSegmentMetadata metadata;
     private LedgerHandle lh;
     private final List<LedgerHandle> openLedgerHandles;
@@ -457,7 +456,7 @@
         if (isBeyondLastAddConfirmed()) {
             // if the reader is already caught up, let's fail the reader immediately
             // as we need to pull the latest metadata of this log segment.
-            setException(new BKTransmitException("Failed to open ledger for reading log segment " + getSegment(), rc),
+            completeExceptionally(new BKTransmitException("Failed to open ledger for reading log segment " + getSegment(), rc),
                     true);
             return;
         }
@@ -488,7 +487,7 @@
      * @param throwable exception indicating the error
      * @param isBackground is the reader set exception by background reads or foreground reads
      */
-    private void setException(Throwable throwable, boolean isBackground) {
+    private void completeExceptionally(Throwable throwable, boolean isBackground) {
         lastException.compareAndSet(null, throwable);
         if (isBackground) {
             notifyReaders();
@@ -510,7 +509,7 @@
             readQueue.clear();
         }
         for (PendingReadRequest request : requestsToCancel) {
-            request.setException(throwExc);
+            request.completeExceptionally(throwExc);
         }
     }
 
@@ -630,11 +629,11 @@
     }
 
     @Override
-    public Future<List<Entry.Reader>> readNext(int numEntries) {
+    public CompletableFuture<List<Entry.Reader>> readNext(int numEntries) {
         final PendingReadRequest readRequest = new PendingReadRequest(numEntries);
 
         if (checkClosedOrInError()) {
-            readRequest.setException(lastException.get());
+            readRequest.completeExceptionally(lastException.get());
         } else {
             boolean wasQueueEmpty;
             synchronized (readQueue) {
@@ -682,9 +681,9 @@
             // mark the reader in error and abort all pending reads since
             // we don't know the last consumed read
             if (null == lastException.get()) {
-                if (nextRequest.getPromise().isInterrupted().isDefined()) {
-                    setException(new DLInterruptedException("Interrupted on reading log segment "
-                            + getSegment() + " : " + nextRequest.getPromise().isInterrupted().get()), false);
+                if (nextRequest.getPromise().isCancelled()) {
+                    completeExceptionally(new DLInterruptedException("Interrupted on reading log segment "
+                            + getSegment() + " : " + nextRequest.getPromise().isCancelled()), false);
                 }
             }
 
@@ -707,11 +706,11 @@
                 } else {
                     DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at reading from "
                             + getSegment());
-                    nextRequest.setException(ise);
+                    nextRequest.completeExceptionally(ise);
                     if (null != request) {
-                        request.setException(ise);
+                        request.completeExceptionally(ise);
                     }
-                    setException(ise, false);
+                    completeExceptionally(ise, false);
                 }
             } else {
                 if (0 == scheduleCountLocal) {
@@ -732,7 +731,7 @@
             }
             // reach end of log segment
             if (hitEndOfLogSegment) {
-                setException(new EndOfLogSegmentException(getSegment().getZNodeName()), false);
+                completeExceptionally(new EndOfLogSegmentException(getSegment().getZNodeName()), false);
                 return;
             }
             if (null == entry) {
@@ -742,7 +741,7 @@
             if (!entry.isDone()) {
                 // we already reached end of the log segment
                 if (isEndOfLogSegment(entry.getEntryId())) {
-                    setException(new EndOfLogSegmentException(getSegment().getZNodeName()), false);
+                    completeExceptionally(new EndOfLogSegmentException(getSegment().getZNodeName()), false);
                 }
                 return;
             }
@@ -751,13 +750,13 @@
                 if (entry != removedEntry) {
                     DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at reading from "
                             + getSegment());
-                    setException(ise, false);
+                    completeExceptionally(ise, false);
                     return;
                 }
                 try {
                     nextRequest.addEntry(processReadEntry(entry.getEntry()));
                 } catch (IOException e) {
-                    setException(e, false);
+                    completeExceptionally(e, false);
                     return;
                 }
             } else if (skipBrokenEntries && BKException.Code.DigestMatchException == entry.getRc()) {
@@ -766,7 +765,7 @@
                 readAheadEntries.poll();
                 continue;
             } else {
-                setException(new BKTransmitException("Encountered issue on reading entry " + entry.getEntryId()
+                completeExceptionally(new BKTransmitException("Encountered issue on reading entry " + entry.getEntryId()
                         + " @ log segment " + getSegment(), entry.getRc()), false);
                 return;
             }
@@ -812,26 +811,29 @@
     }
 
     @Override
-    public Future<Void> asyncClose() {
-        final Promise<Void> closeFuture;
+    public CompletableFuture<Void> asyncClose() {
+        final CompletableFuture<Void> closeFuture;
         ReadCancelledException exception;
         LedgerHandle[] lhsToClose;
         synchronized (this) {
             if (null != closePromise) {
                 return closePromise;
             }
-            closeFuture = closePromise = new Promise<Void>();
+            closeFuture = closePromise = new CompletableFuture<Void>();
             lhsToClose = openLedgerHandles.toArray(new LedgerHandle[openLedgerHandles.size()]);
             // set the exception to cancel pending and subsequent reads
             exception = new ReadCancelledException(getSegment().getZNodeName(), "Reader was closed");
-            setException(exception, false);
+            completeExceptionally(exception, false);
         }
 
         // cancel all pending reads
         cancelAllPendingReads(exception);
 
         // close all the open ledger
-        BKUtils.closeLedgers(lhsToClose).proxyTo(closeFuture);
+        FutureUtils.proxyTo(
+            BKUtils.closeLedgers(lhsToClose),
+            closeFuture
+        );
         return closeFuture;
     }
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
index 4dd4c12..14ebf4a 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
@@ -17,6 +17,7 @@
  */
 package org.apache.distributedlog.impl.logsegment;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.LogSegmentMetadata;
@@ -35,10 +36,8 @@
 import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
 import org.apache.distributedlog.metadata.LogMetadataForWriter;
 import org.apache.distributedlog.util.Allocator;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.client.AsyncCallback;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
@@ -65,13 +64,13 @@
 
         private final LogSegmentMetadata segment;
         private final long startEntryId;
-        private final Promise<LogSegmentEntryReader> openPromise;
+        private final CompletableFuture<LogSegmentEntryReader> openPromise;
 
         OpenReaderRequest(LogSegmentMetadata segment,
                           long startEntryId) {
             this.segment = segment;
             this.startEntryId = startEntryId;
-            this.openPromise = new Promise<LogSegmentEntryReader>();
+            this.openPromise = new CompletableFuture<LogSegmentEntryReader>();
         }
 
     }
@@ -79,11 +78,11 @@
     private static class DeleteLogSegmentRequest {
 
         private final LogSegmentMetadata segment;
-        private final Promise<LogSegmentMetadata> deletePromise;
+        private final CompletableFuture<LogSegmentMetadata> deletePromise;
 
         DeleteLogSegmentRequest(LogSegmentMetadata segment) {
             this.segment = segment;
-            this.deletePromise = new Promise<LogSegmentMetadata>();
+            this.deletePromise = new CompletableFuture<LogSegmentMetadata>();
         }
 
     }
@@ -119,13 +118,13 @@
     }
 
     @Override
-    public Future<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment) {
+    public CompletableFuture<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment) {
         DeleteLogSegmentRequest request = new DeleteLogSegmentRequest(segment);
         BookKeeper bk;
         try {
             bk = this.bkc.get();
         } catch (IOException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         }
         bk.asyncDeleteLedger(segment.getLogSegmentId(), this, request);
         return request.deletePromise;
@@ -141,11 +140,11 @@
             logger.error("Couldn't delete ledger {} from bookkeeper for {} : {}",
                     new Object[]{ deleteRequest.segment.getLogSegmentId(), deleteRequest.segment,
                             BKException.getMessage(rc) });
-            FutureUtils.setException(deleteRequest.deletePromise,
+            FutureUtils.completeExceptionally(deleteRequest.deletePromise,
                     new BKTransmitException("Couldn't delete log segment " + deleteRequest.segment, rc));
             return;
         }
-        FutureUtils.setValue(deleteRequest.deletePromise, deleteRequest.segment);
+        FutureUtils.complete(deleteRequest.deletePromise, deleteRequest.segment);
     }
 
     //
@@ -186,13 +185,13 @@
     //
 
     @Override
-    public Future<LogSegmentEntryReader> openReader(LogSegmentMetadata segment,
+    public CompletableFuture<LogSegmentEntryReader> openReader(LogSegmentMetadata segment,
                                                     long startEntryId) {
         BookKeeper bk;
         try {
             bk = this.bkc.get();
         } catch (IOException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         }
         OpenReaderRequest request = new OpenReaderRequest(segment, startEntryId);
         if (segment.isInProgress()) {
@@ -217,7 +216,7 @@
     public void openComplete(int rc, LedgerHandle lh, Object ctx) {
         OpenReaderRequest request = (OpenReaderRequest) ctx;
         if (BKException.Code.OK != rc) {
-            FutureUtils.setException(
+            FutureUtils.completeExceptionally(
                     request.openPromise,
                     new BKTransmitException("Failed to open ledger handle for log segment " + request.segment, rc));
             return;
@@ -233,28 +232,28 @@
                     conf,
                     statsLogger,
                     failureInjector);
-            FutureUtils.setValue(request.openPromise, reader);
+            FutureUtils.complete(request.openPromise, reader);
         } catch (IOException e) {
-            FutureUtils.setException(request.openPromise, e);
+            FutureUtils.completeExceptionally(request.openPromise, e);
         }
 
     }
 
     @Override
-    public Future<LogSegmentRandomAccessEntryReader> openRandomAccessReader(final LogSegmentMetadata segment,
+    public CompletableFuture<LogSegmentRandomAccessEntryReader> openRandomAccessReader(final LogSegmentMetadata segment,
                                                                             final boolean fence) {
         final BookKeeper bk;
         try {
             bk = this.bkc.get();
         } catch (IOException e) {
-            return Future.exception(e);
+            return FutureUtils.exception(e);
         }
-        final Promise<LogSegmentRandomAccessEntryReader> openPromise = new Promise<LogSegmentRandomAccessEntryReader>();
+        final CompletableFuture<LogSegmentRandomAccessEntryReader> openPromise = new CompletableFuture<LogSegmentRandomAccessEntryReader>();
         AsyncCallback.OpenCallback openCallback = new AsyncCallback.OpenCallback() {
             @Override
             public void openComplete(int rc, LedgerHandle lh, Object ctx) {
                 if (BKException.Code.OK != rc) {
-                    FutureUtils.setException(
+                    FutureUtils.completeExceptionally(
                             openPromise,
                             new BKTransmitException("Failed to open ledger handle for log segment " + segment, rc));
                     return;
@@ -263,7 +262,7 @@
                         segment,
                         lh,
                         conf);
-                FutureUtils.setValue(openPromise, reader);
+                FutureUtils.complete(openPromise, reader);
             }
         };
         if (segment.isInProgress() && !fence) {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
index d7b331b..254345e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
@@ -18,14 +18,13 @@
 package org.apache.distributedlog.impl.logsegment;
 
 import com.google.common.collect.Lists;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.Entry;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.LedgerEntry;
@@ -49,7 +48,7 @@
     // state
     private final LogSegmentMetadata metadata;
     private final LedgerHandle lh;
-    private Promise<Void> closePromise = null;
+    private CompletableFuture<Void> closePromise = null;
 
     BKLogSegmentRandomAccessEntryReader(LogSegmentMetadata metadata,
                                         LedgerHandle lh,
@@ -68,8 +67,8 @@
     }
 
     @Override
-    public Future<List<Entry.Reader>> readEntries(long startEntryId, long endEntryId) {
-        Promise<List<Entry.Reader>> promise = new Promise<List<Entry.Reader>>();
+    public CompletableFuture<List<Entry.Reader>> readEntries(long startEntryId, long endEntryId) {
+        CompletableFuture<List<Entry.Reader>> promise = new CompletableFuture<List<Entry.Reader>>();
         lh.asyncReadEntries(startEntryId, endEntryId, this, promise);
         return promise;
     }
@@ -86,34 +85,37 @@
 
     @Override
     public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> entries, Object ctx) {
-        Promise<List<Entry.Reader>> promise = (Promise<List<Entry.Reader>>) ctx;
+        CompletableFuture<List<Entry.Reader>> promise = (CompletableFuture<List<Entry.Reader>>) ctx;
         if (BKException.Code.OK == rc) {
             List<Entry.Reader> entryList = Lists.newArrayList();
             while (entries.hasMoreElements()) {
                 try {
                     entryList.add(processReadEntry(entries.nextElement()));
                 } catch (IOException ioe) {
-                    FutureUtils.setException(promise, ioe);
+                    FutureUtils.completeExceptionally(promise, ioe);
                     return;
                 }
             }
-            FutureUtils.setValue(promise, entryList);
+            FutureUtils.complete(promise, entryList);
         } else {
-            FutureUtils.setException(promise,
+            FutureUtils.completeExceptionally(promise,
                     new BKTransmitException("Failed to read entries :", rc));
         }
     }
 
     @Override
-    public Future<Void> asyncClose() {
-        final Promise<Void> closeFuture;
+    public CompletableFuture<Void> asyncClose() {
+        final CompletableFuture<Void> closeFuture;
         synchronized (this) {
             if (null != closePromise) {
                 return closePromise;
             }
-            closeFuture = closePromise = new Promise<Void>();
+            closeFuture = closePromise = new CompletableFuture<Void>();
         }
-        BKUtils.closeLedgers(lh).proxyTo(closeFuture);
+        FutureUtils.proxyTo(
+            BKUtils.closeLedgers(lh),
+            closeFuture
+        );
         return closeFuture;
     }
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java
index 3c02740..82ba775 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java
@@ -18,11 +18,9 @@
 package org.apache.distributedlog.impl.logsegment;
 
 import com.google.common.collect.Lists;
-import org.apache.distributedlog.function.VoidFunctions;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Futures;
-import com.twitter.util.Promise;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.common.functions.VoidFunctions;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.bookkeeper.client.AsyncCallback;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.LedgerHandle;
@@ -40,15 +38,15 @@
      * @param lh ledger handle
      * @return future represents close result.
      */
-    public static Future<Void> closeLedger(LedgerHandle lh) {
-        final Promise<Void> closePromise = new Promise<Void>();
+    public static CompletableFuture<Void> closeLedger(LedgerHandle lh) {
+        final CompletableFuture<Void> closePromise = new CompletableFuture<Void>();
         lh.asyncClose(new AsyncCallback.CloseCallback() {
             @Override
             public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
                 if (BKException.Code.OK != rc) {
-                    FutureUtils.setException(closePromise, BKException.create(rc));
+                    FutureUtils.completeExceptionally(closePromise, BKException.create(rc));
                 } else {
-                    FutureUtils.setValue(closePromise, null);
+                    FutureUtils.complete(closePromise, null);
                 }
             }
         }, null);
@@ -61,12 +59,12 @@
      * @param lhs a list of ledgers
      * @return future represents close results.
      */
-    public static Future<Void> closeLedgers(LedgerHandle ... lhs) {
-        List<Future<Void>> closeResults = Lists.newArrayListWithExpectedSize(lhs.length);
+    public static CompletableFuture<Void> closeLedgers(LedgerHandle ... lhs) {
+        List<CompletableFuture<Void>> closeResults = Lists.newArrayListWithExpectedSize(lhs.length);
         for (LedgerHandle lh : lhs) {
             closeResults.add(closeLedger(lh));
         }
-        return Futures.collect(closeResults).map(VoidFunctions.LIST_TO_VOID_FUNC);
+        return FutureUtils.collect(closeResults).thenApply(VoidFunctions.LIST_TO_VOID_FUNC);
     }
 
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
index 30f9dd4..9b02462 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
@@ -20,10 +20,12 @@
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.DLException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
 import org.apache.distributedlog.exceptions.InvalidStreamNameException;
 import org.apache.distributedlog.exceptions.LockCancelledException;
@@ -42,18 +44,14 @@
 import org.apache.distributedlog.metadata.LogMetadataForReader;
 import org.apache.distributedlog.metadata.LogMetadataForWriter;
 import org.apache.distributedlog.util.DLUtils;
-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 org.apache.distributedlog.zk.LimitedPermitManager;
 import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.PermitManager;
+import org.apache.distributedlog.common.util.PermitManager;
 import org.apache.distributedlog.util.Transaction;
 import org.apache.distributedlog.util.Utils;
 import org.apache.distributedlog.zk.ZKTransaction;
-import com.twitter.util.ExceptionalFunction;
-import com.twitter.util.ExceptionalFunction0;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.meta.ZkVersion;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.versioning.Versioned;
@@ -69,10 +67,7 @@
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
 
-import java.io.File;
 import java.io.IOException;
 import java.net.URI;
 import java.util.List;
@@ -120,14 +115,9 @@
 
     private synchronized OrderedScheduler getLockStateExecutor(boolean createIfNull) {
         if (createIfNull && null == lockStateExecutor) {
-            StatsLogger lockStateStatsLogger = statsLogger.scope("lock_scheduler");
             lockStateExecutor = OrderedScheduler.newBuilder()
                     .name("DLM-LockState")
                     .corePoolSize(conf.getNumLockStateThreads())
-                    .statsLogger(lockStateStatsLogger)
-                    .perExecutorStatsLogger(lockStateStatsLogger)
-                    .traceTaskExecution(conf.getEnableTaskExecutionStats())
-                    .traceTaskExecutionWarnTimeUs(conf.getTaskExecutionWarnTimeMicros())
                     .build();
         }
         return lockStateExecutor;
@@ -174,21 +164,21 @@
     }
 
     @Override
-    public Future<Void> logExists(URI uri, final String logName) {
+    public CompletableFuture<Void> logExists(URI uri, final String logName) {
         final String logSegmentsPath = LogMetadata.getLogSegmentsPath(
                 uri, logName, conf.getUnpartitionedStreamName());
-        final Promise<Void> promise = new Promise<Void>();
+        final CompletableFuture<Void> promise = new CompletableFuture<Void>();
         try {
             final ZooKeeper zk = zooKeeperClient.get();
             zk.sync(logSegmentsPath, new AsyncCallback.VoidCallback() {
                 @Override
                 public void processResult(int syncRc, String path, Object syncCtx) {
                     if (KeeperException.Code.NONODE.intValue() == syncRc) {
-                        promise.setException(new LogNotFoundException(
+                        promise.completeExceptionally(new LogNotFoundException(
                                 String.format("Log %s does not exist or has been deleted", logName)));
                         return;
                     } else if (KeeperException.Code.OK.intValue() != syncRc){
-                        promise.setException(new ZKException("Error on checking log existence for " + logName,
+                        promise.completeExceptionally(new ZKException("Error on checking log existence for " + logName,
                                 KeeperException.create(KeeperException.Code.get(syncRc))));
                         return;
                     }
@@ -196,12 +186,12 @@
                         @Override
                         public void processResult(int rc, String path, Object ctx, Stat stat) {
                             if (KeeperException.Code.OK.intValue() == rc) {
-                                promise.setValue(null);
+                                promise.complete(null);
                             } else if (KeeperException.Code.NONODE.intValue() == rc) {
-                                promise.setException(new LogNotFoundException(
+                                promise.completeExceptionally(new LogNotFoundException(
                                         String.format("Log %s does not exist or has been deleted", logName)));
                             } else {
-                                promise.setException(new ZKException("Error on checking log existence for " + logName,
+                                promise.completeExceptionally(new ZKException("Error on checking log existence for " + logName,
                                         KeeperException.create(KeeperException.Code.get(rc))));
                             }
                         }
@@ -211,10 +201,10 @@
 
         } catch (InterruptedException ie) {
             LOG.error("Interrupted while reading {}", logSegmentsPath, ie);
-            promise.setException(new DLInterruptedException("Interrupted while checking "
+            promise.completeExceptionally(new DLInterruptedException("Interrupted while checking "
                     + logSegmentsPath, ie));
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
+            promise.completeExceptionally(e);
         }
         return promise;
     }
@@ -237,15 +227,13 @@
     // Create Read Lock
     //
 
-    private Future<Void> ensureReadLockPathExist(final LogMetadata logMetadata,
+    private CompletableFuture<Void> ensureReadLockPathExist(final LogMetadata logMetadata,
                                                  final String readLockPath) {
-        final Promise<Void> promise = new Promise<Void>();
-        promise.setInterruptHandler(new com.twitter.util.Function<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable t) {
-                FutureUtils.setException(promise, new LockCancelledException(readLockPath,
-                        "Could not ensure read lock path", t));
-                return null;
+        final CompletableFuture<Void> promise = new CompletableFuture<Void>();
+        promise.whenComplete((value, cause) -> {
+            if (cause instanceof CancellationException) {
+                FutureUtils.completeExceptionally(promise, new LockCancelledException(readLockPath,
+                        "Could not ensure read lock path", cause));
             }
         });
         Optional<String> parentPathShouldNotCreate = Optional.of(logMetadata.getLogRootPath());
@@ -255,21 +243,21 @@
                     @Override
                     public void processResult(final int rc, final String path, Object ctx, String name) {
                         if (KeeperException.Code.NONODE.intValue() == rc) {
-                            FutureUtils.setException(promise, new LogNotFoundException(
+                            FutureUtils.completeExceptionally(promise, new LogNotFoundException(
                                     String.format("Log %s does not exist or has been deleted",
                                             logMetadata.getFullyQualifiedName())));
                         } else if (KeeperException.Code.OK.intValue() == rc) {
-                            FutureUtils.setValue(promise, null);
+                            FutureUtils.complete(promise, null);
                             LOG.trace("Created path {}.", path);
                         } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
-                            FutureUtils.setValue(promise, null);
+                            FutureUtils.complete(promise, null);
                             LOG.trace("Path {} is already existed.", path);
                         } else if (DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE == rc) {
-                            FutureUtils.setException(promise, new ZooKeeperClient.ZooKeeperConnectionException(path));
+                            FutureUtils.completeExceptionally(promise, new ZooKeeperClient.ZooKeeperConnectionException(path));
                         } else if (DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE == rc) {
-                            FutureUtils.setException(promise, new DLInterruptedException(path));
+                            FutureUtils.completeExceptionally(promise, new DLInterruptedException(path));
                         } else {
-                            FutureUtils.setException(promise, KeeperException.create(KeeperException.Code.get(rc)));
+                            FutureUtils.completeExceptionally(promise, KeeperException.create(KeeperException.Code.get(rc)));
                         }
                     }
                 }, null);
@@ -277,28 +265,19 @@
     }
 
     @Override
-    public Future<DistributedLock> createReadLock(final LogMetadataForReader metadata,
+    public CompletableFuture<DistributedLock> createReadLock(final LogMetadataForReader metadata,
                                                   Optional<String> readerId) {
         final String readLockPath = metadata.getReadLockPath(readerId);
-        return ensureReadLockPathExist(metadata, readLockPath).flatMap(
-                new ExceptionalFunction<Void, Future<DistributedLock>>() {
-            @Override
-            public Future<DistributedLock> applyE(Void value) throws Throwable {
-                // Unfortunately this has a blocking call which we should not execute on the
-                // ZK completion thread
-                return scheduler.apply(new ExceptionalFunction0<DistributedLock>() {
-                    @Override
-                    public DistributedLock applyE() throws Throwable {
-                        return new ZKDistributedLock(
-                            getLockStateExecutor(true),
-                            getLockFactory(true),
-                            readLockPath,
-                            conf.getLockTimeoutMilliSeconds(),
-                            statsLogger.scope("read_lock"));
-                    }
-                });
-            }
-        });
+        return ensureReadLockPathExist(metadata, readLockPath)
+            .thenApplyAsync((value) -> {
+                DistributedLock lock = new ZKDistributedLock(
+                    getLockStateExecutor(true),
+                    getLockFactory(true),
+                    readLockPath,
+                    conf.getLockTimeoutMilliSeconds(),
+                    statsLogger.scope("read_lock"));
+                return lock;
+            }, scheduler.chooseExecutor(readLockPath));
     }
 
     //
@@ -329,7 +308,7 @@
             (byte) (i)};
     }
 
-    static Future<List<Versioned<byte[]>>> checkLogMetadataPaths(ZooKeeper zk,
+    static CompletableFuture<List<Versioned<byte[]>>> checkLogMetadataPaths(ZooKeeper zk,
                                                                  String logRootPath,
                                                                  boolean ownAllocator) {
         // Note re. persistent lock state initialization: the read lock persistent state (path) is
@@ -344,7 +323,7 @@
         final String allocationPath = logRootPath + ALLOCATION_PATH;
 
         int numPaths = ownAllocator ? MetadataIndex.ALLOCATION + 1 : MetadataIndex.LOGSEGMENTS + 1;
-        List<Future<Versioned<byte[]>>> checkFutures = Lists.newArrayListWithExpectedSize(numPaths);
+        List<CompletableFuture<Versioned<byte[]>>> checkFutures = Lists.newArrayListWithExpectedSize(numPaths);
         checkFutures.add(Utils.zkGetData(zk, logRootParentPath, false));
         checkFutures.add(Utils.zkGetData(zk, logRootPath, false));
         checkFutures.add(Utils.zkGetData(zk, maxTxIdPath, false));
@@ -356,7 +335,7 @@
             checkFutures.add(Utils.zkGetData(zk, allocationPath, false));
         }
 
-        return Future.collect(checkFutures);
+        return FutureUtils.collect(checkFutures);
     }
 
     static boolean pathExists(Versioned<byte[]> metadata) {
@@ -374,7 +353,7 @@
                                       final List<ACL> acl,
                                       final boolean ownAllocator,
                                       final boolean createIfNotExists,
-                                      final Promise<List<Versioned<byte[]>>> promise) {
+                                      final CompletableFuture<List<Versioned<byte[]>>> promise) {
         final List<byte[]> pathsToCreate = Lists.newArrayListWithExpectedSize(metadatas.size());
         final List<Op> zkOps = Lists.newArrayListWithExpectedSize(metadatas.size());
         CreateMode createMode = CreateMode.PERSISTENT;
@@ -447,11 +426,11 @@
         }
         if (zkOps.isEmpty()) {
             // nothing missed
-            promise.setValue(metadatas);
+            promise.complete(metadatas);
             return;
         }
         if (!createIfNotExists) {
-            promise.setException(new LogNotFoundException("Log " + logRootPath + " not found"));
+            promise.completeExceptionally(new LogNotFoundException("Log " + logRootPath + " not found"));
             return;
         }
 
@@ -469,9 +448,9 @@
                             finalMetadatas.add(new Versioned<byte[]>(dataCreated, new ZkVersion(0)));
                         }
                     }
-                    promise.setValue(finalMetadatas);
+                    promise.complete(finalMetadatas);
                 } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
-                    promise.setException(new LogExistsException("Someone just created log "
+                    promise.completeExceptionally(new LogExistsException("Someone just created log "
                             + logRootPath));
                 } else {
                     if (LOG.isDebugEnabled()) {
@@ -488,7 +467,7 @@
                         LOG.debug("Failed to create log, full rc list = {}", resultCodeList);
                     }
 
-                    promise.setException(new ZKException("Failed to create log " + logRootPath,
+                    promise.completeExceptionally(new ZKException("Failed to create log " + logRootPath,
                             KeeperException.Code.get(rc)));
                 }
             }
@@ -538,7 +517,7 @@
         }
     }
 
-    static Future<LogMetadataForWriter> getLog(final URI uri,
+    static CompletableFuture<LogMetadataForWriter> getLog(final URI uri,
                                                final String logName,
                                                final String logIdentifier,
                                                final ZooKeeperClient zooKeeperClient,
@@ -549,42 +528,47 @@
             PathUtils.validatePath(logRootPath);
         } catch (IllegalArgumentException e) {
             LOG.error("Illegal path value {} for stream {}", new Object[]{logRootPath, logName, e});
-            return Future.exception(new InvalidStreamNameException(logName, "Log name is invalid"));
+            return FutureUtils.exception(new InvalidStreamNameException(logName, "Log name is invalid"));
         }
 
         try {
             final ZooKeeper zk = zooKeeperClient.get();
             return checkLogMetadataPaths(zk, logRootPath, ownAllocator)
-                    .flatMap(new AbstractFunction1<List<Versioned<byte[]>>, Future<List<Versioned<byte[]>>>>() {
+                    .thenCompose(new Function<List<Versioned<byte[]>>, CompletableFuture<List<Versioned<byte[]>>>>() {
                         @Override
-                        public Future<List<Versioned<byte[]>>> apply(List<Versioned<byte[]>> metadatas) {
-                            Promise<List<Versioned<byte[]>>> promise =
-                                    new Promise<List<Versioned<byte[]>>>();
+                        public CompletableFuture<List<Versioned<byte[]>>> apply(List<Versioned<byte[]>> metadatas) {
+                            CompletableFuture<List<Versioned<byte[]>>> promise =
+                                    new CompletableFuture<List<Versioned<byte[]>>>();
                             createMissingMetadata(zk, logRootPath, metadatas, zooKeeperClient.getDefaultACL(),
                                     ownAllocator, createIfNotExists, promise);
                             return promise;
                         }
-                    }).map(new ExceptionalFunction<List<Versioned<byte[]>>, LogMetadataForWriter>() {
+                    }).thenCompose(new Function<List<Versioned<byte[]>>, CompletableFuture<LogMetadataForWriter>>() {
                         @Override
-                        public LogMetadataForWriter applyE(List<Versioned<byte[]>> metadatas) throws DLException {
-                            return processLogMetadatas(
-                                    uri,
-                                    logName,
-                                    logIdentifier,
-                                    metadatas,
-                                    ownAllocator);
+                        public CompletableFuture<LogMetadataForWriter> apply(List<Versioned<byte[]>> metadatas) {
+                            try {
+                                return FutureUtils.value(
+                                    processLogMetadatas(
+                                        uri,
+                                        logName,
+                                        logIdentifier,
+                                        metadatas,
+                                        ownAllocator));
+                            } catch (UnexpectedException e) {
+                                return FutureUtils.exception(e);
+                            }
                         }
                     });
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            return Future.exception(new ZKException("Encountered zookeeper connection issue on creating log " + logName,
+            return FutureUtils.exception(new ZKException("Encountered zookeeper connection issue on creating log " + logName,
                     KeeperException.Code.CONNECTIONLOSS));
         } catch (InterruptedException e) {
-            return Future.exception(new DLInterruptedException("Interrupted on creating log " + logName, e));
+            return FutureUtils.exception(new DLInterruptedException("Interrupted on creating log " + logName, e));
         }
     }
 
     @Override
-    public Future<LogMetadataForWriter> getLog(final URI uri,
+    public CompletableFuture<LogMetadataForWriter> getLog(final URI uri,
                                                final String logName,
                                                final boolean ownAllocator,
                                                final boolean createIfNotExists) {
@@ -602,30 +586,30 @@
     //
 
     @Override
-    public Future<Void> deleteLog(URI uri, final String logName) {
-        final Promise<Void> promise = new Promise<Void>();
+    public CompletableFuture<Void> deleteLog(URI uri, final String logName) {
+        final CompletableFuture<Void> promise = new CompletableFuture<Void>();
         try {
             String streamPath = LogMetadata.getLogStreamPath(uri, logName);
             ZKUtil.deleteRecursive(zooKeeperClient.get(), streamPath, new AsyncCallback.VoidCallback() {
                 @Override
                 public void processResult(int rc, String path, Object ctx) {
                     if (KeeperException.Code.OK.intValue() != rc) {
-                        FutureUtils.setException(promise,
+                        FutureUtils.completeExceptionally(promise,
                                 new ZKException("Encountered zookeeper issue on deleting log stream "
                                         + logName, KeeperException.Code.get(rc)));
                         return;
                     }
-                    FutureUtils.setValue(promise, null);
+                    FutureUtils.complete(promise, null);
                 }
             }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            FutureUtils.setException(promise, new ZKException("Encountered zookeeper issue on deleting log stream "
+            FutureUtils.completeExceptionally(promise, new ZKException("Encountered zookeeper issue on deleting log stream "
                     + logName, KeeperException.Code.CONNECTIONLOSS));
         } catch (InterruptedException e) {
-            FutureUtils.setException(promise, new DLInterruptedException("Interrupted while deleting log stream "
+            FutureUtils.completeExceptionally(promise, new DLInterruptedException("Interrupted while deleting log stream "
                     + logName));
         } catch (KeeperException e) {
-            FutureUtils.setException(promise, new ZKException("Encountered zookeeper issue on deleting log stream "
+            FutureUtils.completeExceptionally(promise, new ZKException("Encountered zookeeper issue on deleting log stream "
                     + logName, e));
         }
         return promise;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
index 64abb77..302c666 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
@@ -17,27 +17,22 @@
  */
 package org.apache.distributedlog.impl.subscription;
 
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.distributedlog.subscription.SubscriptionStateStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
-
 import com.google.common.base.Charsets;
-
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.api.subscription.SubscriptionStateStore;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
-
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.util.Utils;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ZKSubscriptionStateStore implements SubscriptionStateStore {
 
@@ -60,16 +55,16 @@
      * Get the last committed position stored for this subscription
      */
     @Override
-    public Future<DLSN> getLastCommitPosition() {
+    public CompletableFuture<DLSN> getLastCommitPosition() {
         if (null != lastCommittedPosition.get()) {
-            return Future.value(lastCommittedPosition.get());
+            return FutureUtils.value(lastCommittedPosition.get());
         } else {
             return getLastCommitPositionFromZK();
         }
     }
 
-    Future<DLSN> getLastCommitPositionFromZK() {
-        final Promise<DLSN> result = new Promise<DLSN>();
+    CompletableFuture<DLSN> getLastCommitPositionFromZK() {
+        final CompletableFuture<DLSN> result = new CompletableFuture<DLSN>();
         try {
             logger.debug("Reading last commit position from path {}", zkPath);
             zooKeeperClient.get().getData(zkPath, false, new AsyncCallback.DataCallback() {
@@ -77,25 +72,25 @@
                 public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
                     logger.debug("Read last commit position from path {}: rc = {}", zkPath, rc);
                     if (KeeperException.Code.NONODE.intValue() == rc) {
-                        result.setValue(DLSN.NonInclusiveLowerBound);
+                        result.complete(DLSN.NonInclusiveLowerBound);
                     } else if (KeeperException.Code.OK.intValue() != rc) {
-                        result.setException(KeeperException.create(KeeperException.Code.get(rc), path));
+                        result.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path));
                     } else {
                         try {
                             DLSN dlsn = DLSN.deserialize(new String(data, Charsets.UTF_8));
-                            result.setValue(dlsn);
+                            result.complete(dlsn);
                         } catch (Exception t) {
                             logger.warn("Invalid last commit position found from path {}", zkPath, t);
                             // invalid dlsn recorded in subscription state store
-                            result.setValue(DLSN.NonInclusiveLowerBound);
+                            result.complete(DLSN.NonInclusiveLowerBound);
                         }
                     }
                 }
             }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
-            result.setException(zkce);
+            result.completeExceptionally(zkce);
         } catch (InterruptedException ie) {
-            result.setException(new DLInterruptedException("getLastCommitPosition was interrupted", ie));
+            result.completeExceptionally(new DLInterruptedException("getLastCommitPosition was interrupted", ie));
         }
         return result;
     }
@@ -106,7 +101,7 @@
      * @param newPosition - new commit position
      */
     @Override
-    public Future<BoxedUnit> advanceCommitPosition(DLSN newPosition) {
+    public CompletableFuture<Void> advanceCommitPosition(DLSN newPosition) {
         if (null == lastCommittedPosition.get() ||
             (newPosition.compareTo(lastCommittedPosition.get()) > 0)) {
             lastCommittedPosition.set(newPosition);
@@ -115,7 +110,7 @@
                 zooKeeperClient.getDefaultACL(),
                 CreateMode.PERSISTENT);
         } else {
-            return Future.Done();
+            return FutureUtils.Void();
         }
     }
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java
index d75f5fc..0392264 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java
@@ -17,30 +17,26 @@
  */
 package org.apache.distributedlog.impl.subscription;
 
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.subscription.SubscriptionStateStore;
-import org.apache.distributedlog.subscription.SubscriptionsStore;
-import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.api.subscription.SubscriptionStateStore;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.util.Utils;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
 
 /**
  * ZooKeeper Based Subscriptions Store.
@@ -82,72 +78,62 @@
     }
 
     @Override
-    public Future<DLSN> getLastCommitPosition(String subscriberId) {
+    public CompletableFuture<DLSN> getLastCommitPosition(String subscriberId) {
         return getSubscriber(subscriberId).getLastCommitPosition();
     }
 
     @Override
-    public Future<Map<String, DLSN>> getLastCommitPositions() {
-        final Promise<Map<String, DLSN>> result = new Promise<Map<String, DLSN>>();
+    public CompletableFuture<Map<String, DLSN>> getLastCommitPositions() {
+        final CompletableFuture<Map<String, DLSN>> result = new CompletableFuture<Map<String, DLSN>>();
         try {
             this.zkc.get().getChildren(this.zkPath, false, new AsyncCallback.Children2Callback() {
                 @Override
                 public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
                     if (KeeperException.Code.NONODE.intValue() == rc) {
-                        result.setValue(new HashMap<String, DLSN>());
+                        result.complete(new HashMap<String, DLSN>());
                     } else if (KeeperException.Code.OK.intValue() != rc) {
-                        result.setException(KeeperException.create(KeeperException.Code.get(rc), path));
+                        result.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path));
                     } else {
                         getLastCommitPositions(result, children);
                     }
                 }
             }, null);
         } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
-            result.setException(zkce);
+            result.completeExceptionally(zkce);
         } catch (InterruptedException ie) {
-            result.setException(new DLInterruptedException("getLastCommitPositions was interrupted", ie));
+            result.completeExceptionally(new DLInterruptedException("getLastCommitPositions was interrupted", ie));
         }
         return result;
     }
 
-    private void getLastCommitPositions(final Promise<Map<String, DLSN>> result,
+    private void getLastCommitPositions(final CompletableFuture<Map<String, DLSN>> result,
                                         List<String> subscribers) {
-        List<Future<Pair<String, DLSN>>> futures =
-                new ArrayList<Future<Pair<String, DLSN>>>(subscribers.size());
+        List<CompletableFuture<Pair<String, DLSN>>> futures =
+                new ArrayList<CompletableFuture<Pair<String, DLSN>>>(subscribers.size());
         for (String s : subscribers) {
             final String subscriber = s;
-            Future<Pair<String, DLSN>> future =
+            CompletableFuture<Pair<String, DLSN>> future =
                 // Get the last commit position from zookeeper
-                getSubscriber(subscriber).getLastCommitPositionFromZK().map(
-                        new AbstractFunction1<DLSN, Pair<String, DLSN>>() {
-                            @Override
-                            public Pair<String, DLSN> apply(DLSN dlsn) {
-                                return Pair.of(subscriber, dlsn);
-                            }
-                        });
+                getSubscriber(subscriber).getLastCommitPositionFromZK().thenApply(
+                    dlsn -> Pair.of(subscriber, dlsn));
             futures.add(future);
         }
-        Future.collect(futures).foreach(
-            new AbstractFunction1<List<Pair<String, DLSN>>, BoxedUnit>() {
-                @Override
-                public BoxedUnit apply(List<Pair<String, DLSN>> subscriptions) {
-                    Map<String, DLSN> subscriptionMap = new HashMap<String, DLSN>();
-                    for (Pair<String, DLSN> pair : subscriptions) {
-                        subscriptionMap.put(pair.getLeft(), pair.getRight());
-                    }
-                    result.setValue(subscriptionMap);
-                    return BoxedUnit.UNIT;
-                }
-            });
+        FutureUtils.collect(futures).thenAccept((subscriptions) -> {
+            Map<String, DLSN> subscriptionMap = new HashMap<String, DLSN>();
+            for (Pair<String, DLSN> pair : subscriptions) {
+                subscriptionMap.put(pair.getLeft(), pair.getRight());
+            }
+            result.complete(subscriptionMap);
+        });
     }
 
     @Override
-    public Future<BoxedUnit> advanceCommitPosition(String subscriberId, DLSN newPosition) {
+    public CompletableFuture<Void> advanceCommitPosition(String subscriberId, DLSN newPosition) {
         return getSubscriber(subscriberId).advanceCommitPosition(newPosition);
     }
 
     @Override
-    public Future<Boolean> deleteSubscriber(String subscriberId) {
+    public CompletableFuture<Boolean> deleteSubscriber(String subscriberId) {
         subscribers.remove(subscriberId);
         String path = getSubscriberZKPath(subscriberId);
         return Utils.zkDeleteIfNotExist(zkc, path, new ZkVersion(-1));
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java
index 95165ef..ae01bf7 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java
@@ -17,16 +17,10 @@
  */
 package org.apache.distributedlog.limiter;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-
-import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.limiter.GuavaRateLimiter;
-import org.apache.distributedlog.limiter.RateLimiter;
-
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.StatsLogger;
-
+import org.apache.distributedlog.exceptions.OverCapacityException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLock.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLock.java
index 986678c..156d6dd 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLock.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLock.java
@@ -17,9 +17,9 @@
  */
 package org.apache.distributedlog.lock;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
 
 /**
  * Interface for distributed locking
@@ -31,7 +31,7 @@
      *
      * @return future represents the acquire result.
      */
-    Future<? extends DistributedLock> asyncAcquire();
+    CompletableFuture<? extends DistributedLock> asyncAcquire();
 
     /**
      * Check if hold lock. If it doesn't, then re-acquire the lock.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java
index b70098e..1cb3364 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java
@@ -17,10 +17,13 @@
  */
 package org.apache.distributedlog.lock;
 
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import com.twitter.util.Timer;
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,11 +36,11 @@
 
     private final String lockId;
     private final String currentOwner;
-    private final Future<Boolean> acquireFuture;
+    private final CompletableFuture<Boolean> acquireFuture;
 
     public LockWaiter(String lockId,
                       String currentOwner,
-                      Future<Boolean> acquireFuture) {
+                      CompletableFuture<Boolean> acquireFuture) {
         this.lockId = lockId;
         this.currentOwner = currentOwner;
         this.acquireFuture = acquireFuture;
@@ -64,12 +67,13 @@
     /**
      * Return the future representing the waiting result.
      *
-     * <p>If the future is interrupted (e.g. {@link Future#within(Duration, Timer)}),
+     * <p>If the future is interrupted
+     * (e.g. {@link FutureUtils#within(CompletableFuture, long, TimeUnit, Throwable, OrderedScheduler, Object)}),
      * the waiter will automatically clean up its waiting state.
      *
      * @return the future representing the acquire result.
      */
-    public Future<Boolean> getAcquireFuture() {
+    public CompletableFuture<Boolean> getAcquireFuture() {
         return acquireFuture;
     }
 
@@ -81,12 +85,12 @@
     public boolean waitForAcquireQuietly() {
         boolean success = false;
         try {
-            success = Await.result(acquireFuture);
-        } catch (InterruptedException ie) {
+            success = Utils.ioResult(acquireFuture);
+        } catch (DLInterruptedException ie) {
             Thread.currentThread().interrupt();
         } catch (LockTimeoutException lte) {
             logger.debug("Timeout on lock acquiring", lte);
-        } catch (Exception e) {
+        } catch (IOException e) {
             logger.error("Caught exception waiting for lock acquired", e);
         }
         return success;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java
index 88abffa..7f770ad 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java
@@ -17,8 +17,9 @@
  */
 package org.apache.distributedlog.lock;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.exceptions.LockingException;
-import com.twitter.util.Future;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 
 /**
  * An implementation of {@link DistributedLock} which does nothing.
@@ -30,8 +31,8 @@
     private NopDistributedLock() {}
 
     @Override
-    public Future<? extends DistributedLock> asyncAcquire() {
-        return Future.value(this);
+    public CompletableFuture<? extends DistributedLock> asyncAcquire() {
+        return FutureUtils.value(this);
     }
 
     @Override
@@ -45,7 +46,7 @@
     }
 
     @Override
-    public Future<Void> asyncClose() {
-        return Future.Void();
+    public CompletableFuture<Void> asyncClose() {
+        return FutureUtils.Void();
     }
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLock.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLock.java
index 8aec2c0..3a46a13 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLock.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLock.java
@@ -17,12 +17,10 @@
  */
 package org.apache.distributedlog.lock;
 
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.util.Future;
-import scala.runtime.BoxedUnit;
-
-import java.util.concurrent.TimeUnit;
 
 /**
  * One time lock.
@@ -71,7 +69,7 @@
      * <i>tryLock</i> here is effectively the combination of following asynchronous calls.
      * <pre>
      *     ZKDistributedLock lock = ...;
-     *     Future<LockWaiter> attemptFuture = lock.asyncTryLock(...);
+     *     CompletableFuture<LockWaiter> attemptFuture = lock.asyncTryLock(...);
      *
      *     boolean acquired = waiter.waitForAcquireQuietly();
      *     if (acquired) {
@@ -106,7 +104,7 @@
      * @return lock waiter representing this attempt of acquiring lock.
      * @see #tryLock(long, TimeUnit)
      */
-    Future<LockWaiter> asyncTryLock(long timeout, TimeUnit unit);
+    CompletableFuture<LockWaiter> asyncTryLock(long timeout, TimeUnit unit);
 
     /**
      * Release a claimed lock.
@@ -121,6 +119,6 @@
      * @return future representing the result of unlock operation.
      * @see #unlock()
      */
-    Future<BoxedUnit> asyncUnlock();
+    CompletableFuture<Void> asyncUnlock();
 
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLockFactory.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLockFactory.java
index a68f2d8..9d3159e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLockFactory.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLockFactory.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.lock;
 
-import com.twitter.util.Future;
+import java.util.concurrent.CompletableFuture;
 
 /**
  * Factory to create {@link SessionLock}
@@ -33,6 +33,6 @@
      *          lock context
      * @return future represents the creation result.
      */
-    Future<SessionLock> createLock(String lockPath, DistributedLockContext context);
+    CompletableFuture<SessionLock> createLock(String lockPath, DistributedLockContext context);
 
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java
index a8eb482..77151df 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java
@@ -19,28 +19,24 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Stopwatch;
-import com.twitter.concurrent.AsyncSemaphore;
-import org.apache.distributedlog.exceptions.LockingException;
-import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.util.FutureUtils;
-import org.apache.distributedlog.util.FutureUtils.OrderedFutureEventListener;
-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 java.io.IOException;
+import java.util.Optional;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.common.concurrent.AsyncSemaphore;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.BoxedUnit;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * Distributed lock, using ZooKeeper.
@@ -78,22 +74,22 @@
     private final long lockTimeout;
     private final DistributedLockContext lockContext = new DistributedLockContext();
 
-    private final AsyncSemaphore lockSemaphore = new AsyncSemaphore(1);
+    private final AsyncSemaphore lockSemaphore = new AsyncSemaphore(1, Optional.empty());
     // We have two lock acquire futures:
     // 1. lock acquire future: for the initial acquire op
     // 2. lock reacquire future: for reacquire necessary when session expires, lock is closed
-    private Future<ZKDistributedLock> lockAcquireFuture = null;
-    private Future<ZKDistributedLock> lockReacquireFuture = null;
+    private CompletableFuture<ZKDistributedLock> lockAcquireFuture = null;
+    private CompletableFuture<ZKDistributedLock> lockReacquireFuture = null;
     // following variable tracking the status of acquire process
     //   => create (internalLock) => tryLock (tryLockFuture) => waitForAcquire (lockWaiter)
     private SessionLock internalLock = null;
-    private Future<LockWaiter> tryLockFuture = null;
+    private CompletableFuture<LockWaiter> tryLockFuture = null;
     private LockWaiter lockWaiter = null;
     // exception indicating if the reacquire failed
     private LockingException lockReacquireException = null;
     // closeFuture
     private volatile boolean closed = false;
-    private Future<Void> closeFuture = null;
+    private CompletableFuture<Void> closeFuture = null;
 
     // A counter to track how many re-acquires happened during a lock's life cycle.
     private final AtomicInteger reacquireCount = new AtomicInteger(0);
@@ -136,25 +132,19 @@
      * Asynchronously acquire the lock. Technically the try phase of this operation--which adds us to the waiter
      * list--is executed synchronously, but the lock wait itself doesn't block.
      */
-    public synchronized Future<ZKDistributedLock> asyncAcquire() {
+    public synchronized CompletableFuture<ZKDistributedLock> asyncAcquire() {
         if (null != lockAcquireFuture) {
-            return Future.exception(new UnexpectedException("Someone is already acquiring/acquired lock " + lockPath));
+            return FutureUtils.exception(new UnexpectedException("Someone is already acquiring/acquired lock " + lockPath));
         }
-        final Promise<ZKDistributedLock> promise =
-                new Promise<ZKDistributedLock>(new Function<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable cause) {
-                lockStateExecutor.submit(lockPath, new Runnable() {
-                    @Override
-                    public void run() {
-                        asyncClose();
-                    }
-                });
-                return BoxedUnit.UNIT;
+        final CompletableFuture<ZKDistributedLock> promise = FutureUtils.createFuture();
+        promise.whenComplete((zkDistributedLock, throwable) -> {
+            if (null == throwable || !(throwable instanceof CancellationException)) {
+                return;
             }
+            lockStateExecutor.submit(lockPath, () -> asyncClose());
         });
         final Stopwatch stopwatch = Stopwatch.createStarted();
-        promise.addEventListener(new FutureEventListener<ZKDistributedLock>() {
+        promise.whenComplete(new FutureEventListener<ZKDistributedLock>() {
             @Override
             public void onSuccess(ZKDistributedLock lock) {
                 acquireStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
@@ -176,41 +166,39 @@
         return promise;
     }
 
-    void doAsyncAcquireWithSemaphore(final Promise<ZKDistributedLock> acquirePromise,
+    void doAsyncAcquireWithSemaphore(final CompletableFuture<ZKDistributedLock> acquirePromise,
                                      final long lockTimeout) {
-        lockSemaphore.acquireAndRun(new AbstractFunction0<Future<ZKDistributedLock>>() {
-            @Override
-            public Future<ZKDistributedLock> apply() {
-                doAsyncAcquire(acquirePromise, lockTimeout);
-                return acquirePromise;
-            }
+        lockSemaphore.acquireAndRun(() -> {
+            doAsyncAcquire(acquirePromise, lockTimeout);
+            return acquirePromise;
         });
     }
 
-    void doAsyncAcquire(final Promise<ZKDistributedLock> acquirePromise,
+    void doAsyncAcquire(final CompletableFuture<ZKDistributedLock> acquirePromise,
                         final long lockTimeout) {
         LOG.trace("Async Lock Acquire {}", lockPath);
         try {
             checkLockState();
         } catch (IOException ioe) {
-            FutureUtils.setException(acquirePromise, ioe);
+            FutureUtils.completeExceptionally(acquirePromise, ioe);
             return;
         }
 
         if (haveLock()) {
             // it already hold the lock
-            FutureUtils.setValue(acquirePromise, this);
+            FutureUtils.complete(acquirePromise, this);
             return;
         }
 
-        lockFactory.createLock(lockPath, lockContext).addEventListener(OrderedFutureEventListener.of(
-                new FutureEventListener<SessionLock>() {
+        lockFactory
+            .createLock(lockPath, lockContext)
+            .whenCompleteAsync(new FutureEventListener<SessionLock>() {
             @Override
             public void onSuccess(SessionLock lock) {
                 synchronized (ZKDistributedLock.this) {
                     if (closed) {
                         LOG.info("Skipping tryLocking lock {} since it is already closed", lockPath);
-                        FutureUtils.setException(acquirePromise, newLockClosedException());
+                        FutureUtils.completeExceptionally(acquirePromise, newLockClosedException());
                         return;
                     }
                 }
@@ -223,62 +211,64 @@
 
             @Override
             public void onFailure(Throwable cause) {
-                FutureUtils.setException(acquirePromise, cause);
+                FutureUtils.completeExceptionally(acquirePromise, cause);
             }
-        }, lockStateExecutor, lockPath));
+        }, lockStateExecutor.chooseExecutor(lockPath));
     }
 
     void asyncTryLock(SessionLock lock,
-                      final Promise<ZKDistributedLock> acquirePromise,
+                      final CompletableFuture<ZKDistributedLock> acquirePromise,
                       final long lockTimeout) {
         if (null != tryLockFuture) {
-            tryLockFuture.cancel();
+            tryLockFuture.cancel(true);
         }
         tryLockFuture = lock.asyncTryLock(lockTimeout, TimeUnit.MILLISECONDS);
-        tryLockFuture.addEventListener(OrderedFutureEventListener.of(
-                new FutureEventListener<LockWaiter>() {
-                    @Override
-                    public void onSuccess(LockWaiter waiter) {
-                        synchronized (ZKDistributedLock.this) {
-                            if (closed) {
-                                LOG.info("Skipping acquiring lock {} since it is already closed", lockPath);
-                                waiter.getAcquireFuture().raise(new LockingException(lockPath, "lock is already closed."));
-                                FutureUtils.setException(acquirePromise, newLockClosedException());
-                                return;
-                            }
+        tryLockFuture.whenCompleteAsync(
+            new FutureEventListener<LockWaiter>() {
+                @Override
+                public void onSuccess(LockWaiter waiter) {
+                    synchronized (ZKDistributedLock.this) {
+                        if (closed) {
+                            LOG.info("Skipping acquiring lock {} since it is already closed", lockPath);
+                            waiter
+                                .getAcquireFuture()
+                                .completeExceptionally(new LockingException(lockPath, "lock is already closed."));
+                            FutureUtils.completeExceptionally(acquirePromise, newLockClosedException());
+                            return;
                         }
-                        tryLockFuture = null;
-                        lockWaiter = waiter;
-                        waitForAcquire(waiter, acquirePromise);
                     }
+                    tryLockFuture = null;
+                    lockWaiter = waiter;
+                    waitForAcquire(waiter, acquirePromise);
+                }
 
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        FutureUtils.setException(acquirePromise, cause);
-                    }
-                }, lockStateExecutor, lockPath));
+                @Override
+                public void onFailure(Throwable cause) {
+                    FutureUtils.completeExceptionally(acquirePromise, cause);
+                }
+            }, lockStateExecutor.chooseExecutor(lockPath));
     }
 
     void waitForAcquire(final LockWaiter waiter,
-                        final Promise<ZKDistributedLock> acquirePromise) {
-        waiter.getAcquireFuture().addEventListener(OrderedFutureEventListener.of(
-                new FutureEventListener<Boolean>() {
-                    @Override
-                    public void onSuccess(Boolean acquired) {
-                        LOG.info("{} acquired lock {}", waiter, lockPath);
-                        if (acquired) {
-                            FutureUtils.setValue(acquirePromise, ZKDistributedLock.this);
-                        } else {
-                            FutureUtils.setException(acquirePromise,
-                                    new OwnershipAcquireFailedException(lockPath, waiter.getCurrentOwner()));
-                        }
+                        final CompletableFuture<ZKDistributedLock> acquirePromise) {
+        waiter.getAcquireFuture().whenCompleteAsync(
+            new FutureEventListener<Boolean>() {
+                @Override
+                public void onSuccess(Boolean acquired) {
+                    LOG.info("{} acquired lock {}", waiter, lockPath);
+                    if (acquired) {
+                        FutureUtils.complete(acquirePromise, ZKDistributedLock.this);
+                    } else {
+                        FutureUtils.completeExceptionally(acquirePromise,
+                                new OwnershipAcquireFailedException(lockPath, waiter.getCurrentOwner()));
                     }
+                }
 
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        FutureUtils.setException(acquirePromise, cause);
-                    }
-                }, lockStateExecutor, lockPath));
+                @Override
+                public void onFailure(Throwable cause) {
+                    FutureUtils.completeExceptionally(acquirePromise, cause);
+                }
+            }, lockStateExecutor.chooseExecutor(lockPath));
     }
 
     /**
@@ -300,7 +290,7 @@
      * @throws LockingException     if the lock attempt fails
      */
     public synchronized void checkOwnershipAndReacquire() throws LockingException {
-        if (null == lockAcquireFuture || !lockAcquireFuture.isDefined()) {
+        if (null == lockAcquireFuture || !lockAcquireFuture.isDone()) {
             throw new LockingException(lockPath, "check ownership before acquiring");
         }
 
@@ -322,7 +312,7 @@
      * @throws LockingException     if the lock attempt fails
      */
     public synchronized void checkOwnership() throws LockingException {
-        if (null == lockAcquireFuture || !lockAcquireFuture.isDefined()) {
+        if (null == lockAcquireFuture || !lockAcquireFuture.isDone()) {
             throw new LockingException(lockPath, "check ownership before acquiring");
         }
         if (!haveLock()) {
@@ -336,12 +326,12 @@
     }
 
     @VisibleForTesting
-    synchronized Future<ZKDistributedLock> getLockReacquireFuture() {
+    synchronized CompletableFuture<ZKDistributedLock> getLockReacquireFuture() {
         return lockReacquireFuture;
     }
 
     @VisibleForTesting
-    synchronized Future<ZKDistributedLock> getLockAcquireFuture() {
+    synchronized CompletableFuture<ZKDistributedLock> getLockAcquireFuture() {
         return lockAcquireFuture;
     }
 
@@ -360,71 +350,65 @@
     }
 
     void closeWaiter(final LockWaiter waiter,
-                     final Promise<Void> closePromise) {
+                     final CompletableFuture<Void> closePromise) {
         if (null == waiter) {
             interruptTryLock(tryLockFuture, closePromise);
         } else {
-            waiter.getAcquireFuture().addEventListener(OrderedFutureEventListener.of(
-                    new FutureEventListener<Boolean>() {
-                        @Override
-                        public void onSuccess(Boolean value) {
-                            unlockInternalLock(closePromise);
-                        }
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            unlockInternalLock(closePromise);
-                        }
-                    }, lockStateExecutor, lockPath));
-            FutureUtils.cancel(waiter.getAcquireFuture());
+            waiter.getAcquireFuture().whenCompleteAsync(
+                new FutureEventListener<Boolean>() {
+                    @Override
+                    public void onSuccess(Boolean value) {
+                        unlockInternalLock(closePromise);
+                    }
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        unlockInternalLock(closePromise);
+                    }
+                }, lockStateExecutor.chooseExecutor(lockPath));
+            waiter.getAcquireFuture().cancel(true);
         }
     }
 
-    void interruptTryLock(final Future<LockWaiter> tryLockFuture,
-                          final Promise<Void> closePromise) {
+    void interruptTryLock(final CompletableFuture<LockWaiter> tryLockFuture,
+                          final CompletableFuture<Void> closePromise) {
         if (null == tryLockFuture) {
             unlockInternalLock(closePromise);
         } else {
-            tryLockFuture.addEventListener(OrderedFutureEventListener.of(
-                    new FutureEventListener<LockWaiter>() {
-                        @Override
-                        public void onSuccess(LockWaiter waiter) {
-                            closeWaiter(waiter, closePromise);
-                        }
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            unlockInternalLock(closePromise);
-                        }
-                    }, lockStateExecutor, lockPath));
-            FutureUtils.cancel(tryLockFuture);
+            tryLockFuture.whenCompleteAsync(
+                new FutureEventListener<LockWaiter>() {
+                    @Override
+                    public void onSuccess(LockWaiter waiter) {
+                        closeWaiter(waiter, closePromise);
+                    }
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        unlockInternalLock(closePromise);
+                    }
+                }, lockStateExecutor.chooseExecutor(lockPath));
+            tryLockFuture.cancel(true);
         }
     }
 
-    synchronized void unlockInternalLock(final Promise<Void> closePromise) {
+    synchronized void unlockInternalLock(final CompletableFuture<Void> closePromise) {
         if (internalLock == null) {
-            FutureUtils.setValue(closePromise, null);
+            FutureUtils.complete(closePromise, null);
         } else {
-            internalLock.asyncUnlock().ensure(new AbstractFunction0<BoxedUnit>() {
-                @Override
-                public BoxedUnit apply() {
-                    FutureUtils.setValue(closePromise, null);
-                    return BoxedUnit.UNIT;
-                }
-            });
+            internalLock.asyncUnlock().whenComplete((value, cause) -> closePromise.complete(null));
         }
     }
 
     @Override
-    public Future<Void> asyncClose() {
-        final Promise<Void> closePromise;
+    public CompletableFuture<Void> asyncClose() {
+        final CompletableFuture<Void> closePromise;
         synchronized (this) {
             if (closed) {
                 return closeFuture;
             }
             closed = true;
-            closeFuture = closePromise = new Promise<Void>();
+            closeFuture = closePromise = new CompletableFuture<Void>();
         }
-        final Promise<Void> closeWaiterFuture = new Promise<Void>();
-        closeWaiterFuture.addEventListener(OrderedFutureEventListener.of(new FutureEventListener<Void>() {
+        final CompletableFuture<Void> closeWaiterFuture = new CompletableFuture<Void>();
+        closeWaiterFuture.whenCompleteAsync(new FutureEventListener<Void>() {
             @Override
             public void onSuccess(Void value) {
                 complete();
@@ -435,9 +419,9 @@
             }
 
             private void complete() {
-                FutureUtils.setValue(closePromise, null);
+                FutureUtils.complete(closePromise, null);
             }
-        }, lockStateExecutor, lockPath));
+        }, lockStateExecutor.chooseExecutor(lockPath));
         lockStateExecutor.submit(lockPath, new Runnable() {
             @Override
             public void run() {
@@ -449,7 +433,7 @@
 
     void internalReacquireLock(final AtomicInteger numRetries,
                                final long lockTimeout,
-                               final Promise<ZKDistributedLock> reacquirePromise) {
+                               final CompletableFuture<ZKDistributedLock> reacquirePromise) {
         lockStateExecutor.submit(lockPath, new Runnable() {
             @Override
             public void run() {
@@ -460,25 +444,25 @@
 
     void doInternalReacquireLock(final AtomicInteger numRetries,
                                  final long lockTimeout,
-                                 final Promise<ZKDistributedLock> reacquirePromise) {
+                                 final CompletableFuture<ZKDistributedLock> reacquirePromise) {
         internalTryRetries.inc();
-        Promise<ZKDistributedLock> tryPromise = new Promise<ZKDistributedLock>();
-        tryPromise.addEventListener(new FutureEventListener<ZKDistributedLock>() {
+        CompletableFuture<ZKDistributedLock> tryPromise = new CompletableFuture<ZKDistributedLock>();
+        tryPromise.whenComplete(new FutureEventListener<ZKDistributedLock>() {
             @Override
             public void onSuccess(ZKDistributedLock lock) {
-                FutureUtils.setValue(reacquirePromise, lock);
+                FutureUtils.complete(reacquirePromise, lock);
             }
 
             @Override
             public void onFailure(Throwable cause) {
                 if (cause instanceof OwnershipAcquireFailedException) {
                     // the lock has been acquired by others
-                    FutureUtils.setException(reacquirePromise, cause);
+                    FutureUtils.completeExceptionally(reacquirePromise, cause);
                 } else {
                     if (numRetries.getAndDecrement() > 0 && !closed) {
                         internalReacquireLock(numRetries, lockTimeout, reacquirePromise);
                     } else {
-                        FutureUtils.setException(reacquirePromise, cause);
+                        FutureUtils.completeExceptionally(reacquirePromise, cause);
                     }
                 }
             }
@@ -486,9 +470,9 @@
         doAsyncAcquireWithSemaphore(tryPromise, 0);
     }
 
-    private Future<ZKDistributedLock> reacquireLock(boolean throwLockAcquireException) throws LockingException {
+    private CompletableFuture<ZKDistributedLock> reacquireLock(boolean throwLockAcquireException) throws LockingException {
         final Stopwatch stopwatch = Stopwatch.createStarted();
-        Promise<ZKDistributedLock> lockPromise;
+        CompletableFuture<ZKDistributedLock> lockPromise;
         synchronized (this) {
             if (closed) {
                 throw newLockClosedException();
@@ -504,8 +488,8 @@
                 return lockReacquireFuture;
             }
             LOG.info("reacquiring lock at {}", lockPath);
-            lockReacquireFuture = lockPromise = new Promise<ZKDistributedLock>();
-            lockReacquireFuture.addEventListener(new FutureEventListener<ZKDistributedLock>() {
+            lockReacquireFuture = lockPromise = new CompletableFuture<ZKDistributedLock>();
+            lockReacquireFuture.whenComplete(new FutureEventListener<ZKDistributedLock>() {
                 @Override
                 public void onSuccess(ZKDistributedLock lock) {
                     // if re-acquire successfully, clear the state.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java
index e2699e7..9fdcbf1 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java
@@ -17,8 +17,22 @@
  */
 package org.apache.distributedlog.lock;
 
+import static com.google.common.base.Charsets.UTF_8;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Stopwatch;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.util.FailpointUtils;
 import org.apache.distributedlog.exceptions.LockingException;
@@ -27,18 +41,10 @@
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
 import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.stats.OpStatsListener;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.stats.OpStatsListener;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Function0;
-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.TimeoutException;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.OpStatsLogger;
@@ -54,20 +60,6 @@
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
-import java.net.URLEncoder;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static com.google.common.base.Charsets.UTF_8;
 
 /**
  * A lock under a given zookeeper session. This is a one-time lock.
@@ -276,7 +268,7 @@
     private StateManagement lockState;
     private final DistributedLockContext lockContext;
 
-    private final Promise<Boolean> acquireFuture;
+    private final CompletableFuture<Boolean> acquireFuture;
     private String currentId;
     private String currentNode;
     private String watchedNode;
@@ -342,15 +334,14 @@
         this.unlockStats = statsLogger.getOpStatsLogger("unlock");
 
         // Attach interrupt handler to acquire future so clients can abort the future.
-        this.acquireFuture = new Promise<Boolean>(new com.twitter.util.Function<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable t) {
+        this.acquireFuture = FutureUtils.createFuture();
+        this.acquireFuture.whenComplete((value, cause) -> {
+            if (null != cause) {
                 // This will set the lock state to closed, and begin to cleanup the zk lock node.
                 // We have to be careful not to block here since doing so blocks the ordered lock
                 // state executor which can cause deadlocks depending on how futures are chained.
-                ZKSessionLock.this.asyncUnlock(t);
+                ZKSessionLock.this.asyncUnlock(cause);
                 // Note re. logging and exceptions: errors are already logged by unlockAsync.
-                return BoxedUnit.UNIT;
             }
         });
     }
@@ -433,7 +424,7 @@
      * @param promise
      *          promise
      */
-    protected <T> void executeLockAction(final int lockEpoch, final LockAction func, final Promise<T> promise) {
+    protected <T> void executeLockAction(final int lockEpoch, final LockAction func, final CompletableFuture<T> promise) {
         lockStateExecutor.submit(lockPath, new SafeRunnable() {
             @Override
             public void safeRun() {
@@ -453,7 +444,7 @@
                         LOG.trace("{} skipped executing lock action '{}' for lock {}, since epoch is changed from {} to {}.",
                                 new Object[]{lockId, func.getActionName(), lockPath, lockEpoch, currentEpoch});
                     }
-                    promise.setException(new EpochChangedException(lockPath, lockEpoch, currentEpoch));
+                    promise.completeExceptionally(new EpochChangedException(lockPath, lockEpoch, currentEpoch));
                 }
             }
         });
@@ -516,7 +507,7 @@
      *          node name
      * @return client id and its ephemeral owner.
      */
-    static Future<Pair<String, Long>> asyncParseClientID(ZooKeeper zkClient, String lockPath, String nodeName) {
+    static CompletableFuture<Pair<String, Long>> asyncParseClientID(ZooKeeper zkClient, String lockPath, String nodeName) {
         String[] parts = nodeName.split("_");
         // member_<clientid>_s<owner_session>_
         if (4 == parts.length && parts[2].startsWith("s")) {
@@ -524,19 +515,19 @@
             String clientId;
             try {
                 clientId = URLDecoder.decode(parts[1], UTF_8.name());
-                return Future.value(Pair.of(clientId, sessionOwner));
+                return FutureUtils.value(Pair.of(clientId, sessionOwner));
             } catch (UnsupportedEncodingException e) {
                 // if failed to parse client id, we have to get client id by zookeeper#getData.
             }
         }
-        final Promise<Pair<String, Long>> promise = new Promise<Pair<String, Long>>();
+        final CompletableFuture<Pair<String, Long>> promise = new CompletableFuture<Pair<String, Long>>();
         zkClient.getData(lockPath + "/" + nodeName, false, new AsyncCallback.DataCallback() {
             @Override
             public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
                 if (KeeperException.Code.OK.intValue() != rc) {
-                    promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                    promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                 } else {
-                    promise.setValue(Pair.of(deserializeClientId(data), stat.getEphemeralOwner()));
+                    promise.complete(Pair.of(deserializeClientId(data), stat.getEphemeralOwner()));
                 }
             }
         }, null);
@@ -544,8 +535,8 @@
     }
 
     @Override
-    public Future<LockWaiter> asyncTryLock(final long timeout, final TimeUnit unit) {
-        final Promise<String> result = new Promise<String>();
+    public CompletableFuture<LockWaiter> asyncTryLock(final long timeout, final TimeUnit unit) {
+        final CompletableFuture<String> result = new CompletableFuture<String>();
         final boolean wait = DistributedLogConstants.LOCK_IMMEDIATE != timeout;
         if (wait) {
             asyncTryLock(wait, result);
@@ -559,11 +550,11 @@
                         @Override
                         public void safeRun() {
                             if (!lockState.inState(State.INIT)) {
-                                result.setException(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
+                                result.completeExceptionally(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
                                 return;
                             }
                             if (KeeperException.Code.OK.intValue() != rc) {
-                                result.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                                result.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                                 return;
                             }
 
@@ -571,25 +562,20 @@
 
                             Collections.sort(children, MEMBER_COMPARATOR);
                             if (children.size() > 0) {
-                                asyncParseClientID(zk, lockPath, children.get(0)).addEventListener(
+                                asyncParseClientID(zk, lockPath, children.get(0)).whenCompleteAsync(
                                         new FutureEventListener<Pair<String, Long>>() {
                                             @Override
                                             public void onSuccess(Pair<String, Long> owner) {
                                                 if (!checkOrClaimLockOwner(owner, result)) {
-                                                    acquireFuture.updateIfEmpty(new Return<Boolean>(false));
+                                                    acquireFuture.complete(false);
                                                 }
                                             }
 
                                             @Override
                                             public void onFailure(final Throwable cause) {
-                                                lockStateExecutor.submit(lockPath, new SafeRunnable() {
-                                                    @Override
-                                                    public void safeRun() {
-                                                        result.setException(cause);
-                                                    }
-                                                });
+                                                result.completeExceptionally(cause);
                                             }
-                                        });
+                                        }, lockStateExecutor.chooseExecutor(lockPath));
                             } else {
                                 asyncTryLock(wait, result);
                             }
@@ -599,14 +585,9 @@
             }, null);
         }
 
-        final Promise<Boolean> waiterAcquireFuture = new Promise<Boolean>(new com.twitter.util.Function<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable t) {
-                acquireFuture.raise(t);
-                return BoxedUnit.UNIT;
-            }
-        });
-        return result.map(new AbstractFunction1<String, LockWaiter>() {
+        final CompletableFuture<Boolean> waiterAcquireFuture = FutureUtils.createFuture();
+        waiterAcquireFuture.whenComplete((value, cause) -> acquireFuture.completeExceptionally(cause));
+        return result.thenApply(new Function<String, LockWaiter>() {
             @Override
             public LockWaiter apply(final String currentOwner) {
                 final Exception acquireException = new OwnershipAcquireFailedException(lockPath, currentOwner);
@@ -617,7 +598,7 @@
                         acquireException,
                         lockStateExecutor,
                         lockPath
-                ).addEventListener(new FutureEventListener<Boolean>() {
+                ).whenComplete(new FutureEventListener<Boolean>() {
 
                     @Override
                     public void onSuccess(Boolean acquired) {
@@ -631,17 +612,17 @@
 
                     private void completeOrFail(final Throwable acquireCause) {
                         if (isLockHeld()) {
-                            waiterAcquireFuture.setValue(true);
+                            waiterAcquireFuture.complete(true);
                         } else {
-                            asyncUnlock().addEventListener(new FutureEventListener<BoxedUnit>() {
+                            asyncUnlock().whenComplete(new FutureEventListener<Void>() {
                                 @Override
-                                public void onSuccess(BoxedUnit value) {
-                                    waiterAcquireFuture.setException(acquireCause);
+                                public void onSuccess(Void value) {
+                                    waiterAcquireFuture.completeExceptionally(acquireCause);
                                 }
 
                                 @Override
                                 public void onFailure(Throwable cause) {
-                                    waiterAcquireFuture.setException(acquireCause);
+                                    waiterAcquireFuture.completeExceptionally(acquireCause);
                                 }
                             });
                         }
@@ -656,12 +637,12 @@
     }
 
     private boolean checkOrClaimLockOwner(final Pair<String, Long> currentOwner,
-                                          final Promise<String> result) {
+                                          final CompletableFuture<String> result) {
         if (lockId.compareTo(currentOwner) != 0 && !lockContext.hasLockId(currentOwner)) {
             lockStateExecutor.submit(lockPath, new SafeRunnable() {
                 @Override
                 public void safeRun() {
-                    result.setValue(currentOwner.getLeft());
+                    result.complete(currentOwner.getLeft());
                 }
             });
             return false;
@@ -672,7 +653,7 @@
             @Override
             public void execute() {
                 if (!lockState.inState(State.INIT)) {
-                    result.setException(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
+                    result.completeExceptionally(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
                     return;
                 }
                 asyncTryLock(false, result);
@@ -693,12 +674,12 @@
      * @param result
      *          promise to satisfy with current lock owner
      */
-    private void asyncTryLock(boolean wait, final Promise<String> result) {
-        final Promise<String> lockResult = new Promise<String>();
-        lockResult.addEventListener(new FutureEventListener<String>() {
+    private void asyncTryLock(boolean wait, final CompletableFuture<String> result) {
+        final CompletableFuture<String> lockResult = new CompletableFuture<String>();
+        lockResult.whenComplete(new FutureEventListener<String>() {
             @Override
             public void onSuccess(String currentOwner) {
-                result.setValue(currentOwner);
+                result.complete(currentOwner);
             }
 
             @Override
@@ -707,7 +688,7 @@
                 if (lockCause instanceof LockStateChangedException) {
                     LOG.info("skipping cleanup for {} at {} after encountering lock " +
                             "state change exception : ", new Object[] { lockId, lockPath, lockCause });
-                    result.setException(lockCause);
+                    result.completeExceptionally(lockCause);
                     return;
                 }
                 if (LOG.isDebugEnabled()) {
@@ -716,15 +697,15 @@
                 }
 
                 // If we encountered any exception we should cleanup
-                Future<BoxedUnit> unlockResult = asyncUnlock();
-                unlockResult.addEventListener(new FutureEventListener<BoxedUnit>() {
+                CompletableFuture<Void> unlockResult = asyncUnlock();
+                unlockResult.whenComplete(new FutureEventListener<Void>() {
                     @Override
-                    public void onSuccess(BoxedUnit value) {
-                        result.setException(lockCause);
+                    public void onSuccess(Void value) {
+                        result.completeExceptionally(lockCause);
                     }
                     @Override
                     public void onFailure(Throwable cause) {
-                        result.setException(lockCause);
+                        result.completeExceptionally(lockCause);
                     }
                 });
             }
@@ -734,7 +715,7 @@
 
     /**
      * Try lock. If wait is true, it would wait and watch sibling to acquire lock when
-     * the sibling is dead. <i>acquireFuture</i> will be notified either it locked successfully
+     * the sibling is dead. <i>acquireCompletableFuture</i> will be notified either it locked successfully
      * or the lock failed. The promise will only satisfy with current lock owner.
      *
      * NOTE: the <i>promise</i> is only satisfied on <i>lockStateExecutor</i>, so any
@@ -745,12 +726,12 @@
      * @param promise
      *          promise to satisfy with current lock owner.
      */
-    private void asyncTryLockWithoutCleanup(final boolean wait, final Promise<String> promise) {
+    private void asyncTryLockWithoutCleanup(final boolean wait, final CompletableFuture<String> promise) {
         executeLockAction(epoch.get(), new LockAction() {
             @Override
             public void execute() {
                 if (!lockState.inState(State.INIT)) {
-                    promise.setException(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
+                    promise.completeExceptionally(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
                     return;
                 }
                 lockState.transition(State.PREPARING);
@@ -776,7 +757,7 @@
                                     public void execute() {
                                         if (KeeperException.Code.OK.intValue() != rc) {
                                             KeeperException ke = KeeperException.create(KeeperException.Code.get(rc));
-                                            promise.setException(ke);
+                                            promise.completeExceptionally(ke);
                                             return;
                                         }
 
@@ -797,14 +778,12 @@
                                         if (lockState.isExpiredOrClosing()) {
                                             // Delete node attempt may have come after PREPARING but before create node, in which case
                                             // we'd be left with a dangling node unless we clean up.
-                                            Promise<BoxedUnit> deletePromise = new Promise<BoxedUnit>();
+                                            CompletableFuture<Void> deletePromise = new CompletableFuture<Void>();
                                             deleteLockNode(deletePromise);
-                                            deletePromise.ensure(new Function0<BoxedUnit>() {
-                                                public BoxedUnit apply() {
-                                                    promise.setException(new LockClosedException(lockPath, lockId, lockState.getState()));
-                                                    return BoxedUnit.UNIT;
-                                                }
-                                            });
+                                            FutureUtils.ensure(
+                                                deletePromise,
+                                                () -> promise.completeExceptionally(
+                                                    new LockClosedException(lockPath, lockId, lockState.getState())));
                                             return;
                                         }
 
@@ -830,7 +809,7 @@
     @Override
     public void tryLock(long timeout, TimeUnit unit) throws LockingException {
         final Stopwatch stopwatch = Stopwatch.createStarted();
-        Future<LockWaiter> tryFuture = asyncTryLock(timeout, unit);
+        CompletableFuture<LockWaiter> tryFuture = asyncTryLock(timeout, unit);
         LockWaiter waiter = waitForTry(stopwatch, tryFuture);
         boolean acquired = waiter.waitForAcquireQuietly();
         if (!acquired) {
@@ -838,13 +817,13 @@
         }
     }
 
-    synchronized LockWaiter waitForTry(Stopwatch stopwatch, Future<LockWaiter> tryFuture)
+    synchronized LockWaiter waitForTry(Stopwatch stopwatch, CompletableFuture<LockWaiter> tryFuture)
             throws LockingException {
         boolean success = false;
         boolean stateChanged = false;
         LockWaiter waiter;
         try {
-            waiter = Await.result(tryFuture, Duration.fromMilliseconds(lockOpTimeout));
+            waiter = FutureUtils.result(tryFuture, lockOpTimeout, TimeUnit.MILLISECONDS);
             success = true;
         } catch (LockStateChangedException ex) {
             stateChanged = true;
@@ -873,12 +852,12 @@
     }
 
     @Override
-    public Future<BoxedUnit> asyncUnlock() {
+    public CompletableFuture<Void> asyncUnlock() {
         return asyncUnlock(new LockClosedException(lockPath, lockId, lockState.getState()));
     }
 
-    Future<BoxedUnit> asyncUnlock(final Throwable cause) {
-        final Promise<BoxedUnit> promise = new Promise<BoxedUnit>();
+    CompletableFuture<Void> asyncUnlock(final Throwable cause) {
+        final CompletableFuture<Void> promise = new CompletableFuture<Void>();
 
         // Use lock executor here rather than lock action, because we want this opertaion to be applied
         // whether the epoch has changed or not. The member node is EPHEMERAL_SEQUENTIAL so there's no
@@ -886,9 +865,9 @@
         lockStateExecutor.submit(lockPath, new SafeRunnable() {
             @Override
             public void safeRun() {
-                acquireFuture.updateIfEmpty(new Throw<Boolean>(cause));
+                acquireFuture.completeExceptionally(cause);
                 unlockInternal(promise);
-                promise.addEventListener(new OpStatsListener<BoxedUnit>(unlockStats));
+                promise.whenComplete(new OpStatsListener<Void>(unlockStats));
             }
         });
 
@@ -897,9 +876,9 @@
 
     @Override
     public void unlock() {
-        Future<BoxedUnit> unlockResult = asyncUnlock();
+        CompletableFuture<Void> unlockResult = asyncUnlock();
         try {
-            Await.result(unlockResult, Duration.fromMilliseconds(lockOpTimeout));
+            FutureUtils.result(unlockResult, lockOpTimeout, TimeUnit.MILLISECONDS);
         } catch (TimeoutException toe) {
             // This shouldn't happen unless we lose a watch, and may result in a leaked lock.
             LOG.error("Timeout unlocking {} owned by {} : ", new Object[] { lockPath, lockId, toe });
@@ -921,13 +900,13 @@
                     new Object[] { lockPath, System.currentTimeMillis(),
                             lockEpoch, ZKSessionLock.this.epoch.get() });
         }
-        acquireFuture.updateIfEmpty(new Return<Boolean>(true));
+        acquireFuture.complete(true);
     }
 
     /**
      * NOTE: unlockInternal should only after try lock.
      */
-    private void unlockInternal(final Promise<BoxedUnit> promise) {
+    private void unlockInternal(final CompletableFuture<Void> promise) {
 
         // already closed or expired, nothing to cleanup
         this.epoch.incrementAndGet();
@@ -936,7 +915,7 @@
         }
 
         if (lockState.inState(State.CLOSED)) {
-            promise.setValue(BoxedUnit.UNIT);
+            promise.complete(null);
             return;
         }
 
@@ -951,39 +930,34 @@
             // Nothing to cleanup if INIT (never tried) or EXPIRED (ephemeral node
             // auto-removed)
             lockState.transition(State.CLOSED);
-            promise.setValue(BoxedUnit.UNIT);
+            promise.complete(null);
             return;
         }
 
         // In any other state, we should clean up the member node
-        Promise<BoxedUnit> deletePromise = new Promise<BoxedUnit>();
+        CompletableFuture<Void> deletePromise = new CompletableFuture<Void>();
         deleteLockNode(deletePromise);
 
         // Set the state to closed after we've cleaned up
-        deletePromise.addEventListener(new FutureEventListener<BoxedUnit>() {
+        deletePromise.whenCompleteAsync(new FutureEventListener<Void>() {
             @Override
-            public void onSuccess(BoxedUnit complete) {
-                lockStateExecutor.submit(lockPath, new SafeRunnable() {
-                    @Override
-                    public void safeRun() {
-                        lockState.transition(State.CLOSED);
-                        promise.setValue(BoxedUnit.UNIT);
-                    }
-                });
+            public void onSuccess(Void complete) {
+                lockState.transition(State.CLOSED);
+                promise.complete(null);
             }
             @Override
             public void onFailure(Throwable cause) {
                 // Delete failure is quite serious (causes lock leak) and should be
                 // handled better
                 LOG.error("lock node delete failed {} {}", lockId, lockPath);
-                promise.setValue(BoxedUnit.UNIT);
+                promise.complete(null);
             }
-        });
+        }, lockStateExecutor.chooseExecutor(lockPath));
     }
 
-    private void deleteLockNode(final Promise<BoxedUnit> promise) {
+    private void deleteLockNode(final CompletableFuture<Void> promise) {
         if (null == currentNode) {
-            promise.setValue(BoxedUnit.UNIT);
+            promise.complete(null);
             return;
         }
 
@@ -1005,7 +979,7 @@
                         }
 
                         FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockUnlockCleanup);
-                        promise.setValue(BoxedUnit.UNIT);
+                        promise.complete(null);
                     }
                 });
             }
@@ -1041,8 +1015,8 @@
 
                 // if session expired, just notify the waiter. as the lock acquire doesn't succeed.
                 // we don't even need to clean up the lock as the znode will disappear after session expired
-                acquireFuture.updateIfEmpty(new Throw<Boolean>(
-                        new LockSessionExpiredException(lockPath, lockId, lockState.getState())));
+                acquireFuture.completeExceptionally(
+                    new LockSessionExpiredException(lockPath, lockId, lockState.getState()));
 
                 // session expired, ephemeral node is gone.
                 currentNode = null;
@@ -1088,9 +1062,9 @@
         });
     }
 
-    private Future<String> checkLockOwnerAndWaitIfPossible(final LockWatcher lockWatcher,
+    private CompletableFuture<String> checkLockOwnerAndWaitIfPossible(final LockWatcher lockWatcher,
                                                            final boolean wait) {
-        final Promise<String> promise = new Promise<String>();
+        final CompletableFuture<String> promise = new CompletableFuture<String>();
         checkLockOwnerAndWaitIfPossible(lockWatcher, wait, promise);
         return promise;
     }
@@ -1107,7 +1081,7 @@
      */
     private void checkLockOwnerAndWaitIfPossible(final LockWatcher lockWatcher,
                                                  final boolean wait,
-                                                 final Promise<String> promise) {
+                                                 final CompletableFuture<String> promise) {
         zk.getChildren(lockPath, false, new AsyncCallback.Children2Callback() {
             @Override
             public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
@@ -1134,22 +1108,22 @@
                                     final boolean wait,
                                     final int getChildrenRc,
                                     final List<String> children,
-                                    final Promise<String> promise) {
+                                    final CompletableFuture<String> promise) {
         executeLockAction(lockWatcher.epoch, new LockAction() {
             @Override
             public void execute() {
                 if (!lockState.inState(State.PREPARED)) { // e.g. lock closed or session expired after prepared
-                    promise.setException(new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
+                    promise.completeExceptionally(new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
                     return;
                 }
 
                 if (KeeperException.Code.OK.intValue() != getChildrenRc) {
-                    promise.setException(KeeperException.create(KeeperException.Code.get(getChildrenRc)));
+                    promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(getChildrenRc)));
                     return;
                 }
                 if (children.isEmpty()) {
                     LOG.error("Error, member list is empty for lock {}.", lockPath);
-                    promise.setException(new UnexpectedException("Empty member list for lock " + lockPath));
+                    promise.completeExceptionally(new UnexpectedException("Empty member list for lock " + lockPath));
                     return;
                 }
 
@@ -1164,10 +1138,10 @@
                 if (memberIndex == 0) {
                     LOG.info("{} acquired the lock {}.", cid, lockPath);
                     claimOwnership(lockWatcher.epoch);
-                    promise.setValue(cid);
+                    promise.complete(cid);
                 } else if (memberIndex > 0) { // we are in the member list but we didn't hold the lock
                     // get ownership of current owner
-                    asyncParseClientID(zk, lockPath, children.get(0)).addEventListener(new FutureEventListener<Pair<String, Long>>() {
+                    asyncParseClientID(zk, lockPath, children.get(0)).whenComplete(new FutureEventListener<Pair<String, Long>>() {
                         @Override
                         public void onSuccess(Pair<String, Long> currentOwner) {
                             watchLockOwner(lockWatcher, wait,
@@ -1179,7 +1153,7 @@
                             executeLockAction(lockWatcher.epoch, new LockAction() {
                                 @Override
                                 public void execute() {
-                                    promise.setException(cause);
+                                    promise.completeExceptionally(cause);
                                 }
 
                                 @Override
@@ -1192,7 +1166,7 @@
                 } else {
                     LOG.error("Member {} doesn't exist in the members list {} for lock {}.",
                             new Object[]{ cid, children, lockPath});
-                    promise.setException(
+                    promise.completeExceptionally(
                             new UnexpectedException("Member " + cid + " doesn't exist in member list " +
                                     children + " for lock " + lockPath));
                 }
@@ -1229,7 +1203,7 @@
                                 final String siblingNode,
                                 final String ownerNode,
                                 final Pair<String, Long> currentOwner,
-                                final Promise<String> promise) {
+                                final CompletableFuture<String> promise) {
         executeLockAction(lockWatcher.epoch, new LockAction() {
             @Override
             public void execute() {
@@ -1270,7 +1244,7 @@
                                 @Override
                                 public void execute() {
                                     if (!lockState.inState(State.PREPARED)) {
-                                        promise.setException(new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
+                                        promise.completeExceptionally(new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
                                         return;
                                     }
 
@@ -1280,17 +1254,17 @@
                                             LOG.info("LockWatcher {} claimed ownership for {} after set watcher on {}.",
                                                     new Object[]{ myNode, lockPath, ownerNode });
                                             claimOwnership(lockWatcher.epoch);
-                                            promise.setValue(currentOwner.getLeft());
+                                            promise.complete(currentOwner.getLeft());
                                         } else {
                                             // watch sibling successfully
                                             lockState.transition(State.WAITING);
-                                            promise.setValue(currentOwner.getLeft());
+                                            promise.complete(currentOwner.getLeft());
                                         }
                                     } else if (KeeperException.Code.NONODE.intValue() == rc) {
                                         // sibling just disappeared, it might be the chance to claim ownership
                                         checkLockOwnerAndWaitIfPossible(lockWatcher, wait, promise);
                                     } else {
-                                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                                        promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                                     }
                                 }
 
@@ -1305,7 +1279,7 @@
                         }
                     }, null);
                 } else {
-                    promise.setValue(currentOwner.getLeft());
+                    promise.complete(currentOwner.getLeft());
                 }
             }
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java
index 3cb25f0..223a3a4 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java
@@ -17,20 +17,17 @@
  */
 package org.apache.distributedlog.lock;
 
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
-import org.apache.bookkeeper.stats.StatsLogger;
-import scala.runtime.BoxedUnit;
-
 import java.io.IOException;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.bookkeeper.stats.StatsLogger;
 
 /**
  * Factory to create zookeeper based locks.
@@ -65,16 +62,14 @@
     }
 
     @Override
-    public Future<SessionLock> createLock(String lockPath,
-                                          DistributedLockContext context) {
+    public CompletableFuture<SessionLock> createLock(String lockPath,
+                                                     DistributedLockContext context) {
         AtomicInteger numRetries = new AtomicInteger(lockCreationRetries);
         final AtomicReference<Throwable> interruptedException = new AtomicReference<Throwable>(null);
-        Promise<SessionLock> createPromise =
-                new Promise<SessionLock>(new com.twitter.util.Function<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable t) {
-                interruptedException.set(t);
-                return BoxedUnit.UNIT;
+        CompletableFuture<SessionLock> createPromise = FutureUtils.createFuture();
+        createPromise.whenComplete((value, cause) -> {
+            if (null != cause && cause instanceof CancellationException) {
+                interruptedException.set(cause);
             }
         });
         createLock(
@@ -91,13 +86,13 @@
                     final DistributedLockContext context,
                     final AtomicReference<Throwable> interruptedException,
                     final AtomicInteger numRetries,
-                    final Promise<SessionLock> createPromise,
+                    final CompletableFuture<SessionLock> createPromise,
                     final long delayMs) {
         lockStateExecutor.schedule(lockPath, new Runnable() {
             @Override
             public void run() {
                 if (null != interruptedException.get()) {
-                    createPromise.updateIfEmpty(new Throw<SessionLock>(interruptedException.get()));
+                    createPromise.completeExceptionally(interruptedException.get());
                     return;
                 }
                 try {
@@ -109,14 +104,14 @@
                             lockOpTimeout,
                             lockStatsLogger,
                             context);
-                    createPromise.updateIfEmpty(new Return<SessionLock>(lock));
+                    createPromise.complete(lock);
                 } catch (DLInterruptedException dlie) {
                     // if the creation is interrupted, throw the exception without retrie.
-                    createPromise.updateIfEmpty(new Throw<SessionLock>(dlie));
+                    createPromise.completeExceptionally(dlie);
                     return;
                 } catch (IOException e) {
                     if (numRetries.getAndDecrement() < 0) {
-                        createPromise.updateIfEmpty(new Throw<SessionLock>(e));
+                        createPromise.completeExceptionally(e);
                         return;
                     }
                     createLock(
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryReader.java
index 8440509..1b292e3 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryReader.java
@@ -18,12 +18,11 @@
 package org.apache.distributedlog.logsegment;
 
 import com.google.common.annotations.Beta;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.Entry;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.util.List;
 
 /**
  * An interface class to read the enveloped entry (serialized bytes of
@@ -87,7 +86,7 @@
      * @throws {@link org.apache.distributedlog.exceptions.EndOfLogSegmentException} when
      *          read entries beyond the end of a <i>closed</i> log segment.
      */
-    Future<List<Entry.Reader>> readNext(int numEntries);
+    CompletableFuture<List<Entry.Reader>> readNext(int numEntries);
 
     /**
      * Return the last add confirmed entry id (LAC).
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java
index 40be67b..ab2d898 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java
@@ -18,12 +18,11 @@
 package org.apache.distributedlog.logsegment;
 
 import com.google.common.annotations.Beta;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.metadata.LogMetadataForWriter;
 import org.apache.distributedlog.util.Allocator;
-import org.apache.distributedlog.util.Transaction;
-import com.twitter.util.Future;
 
 import java.io.IOException;
 
@@ -39,7 +38,7 @@
      * @param segment log segment metadata
      * @return future represent the delete result
      */
-    Future<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment);
+    CompletableFuture<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment);
 
     /**
      * Create a new log segment allocator for allocating log segment entry writers.
@@ -58,7 +57,7 @@
      * @param startEntryId the start entry id
      * @return future represent the opened reader
      */
-    Future<LogSegmentEntryReader> openReader(LogSegmentMetadata segment,
+    CompletableFuture<LogSegmentEntryReader> openReader(LogSegmentMetadata segment,
                                              long startEntryId);
 
     /**
@@ -68,6 +67,6 @@
      * @param fence the flag to fence log segment
      * @return future represent the opened random access reader
      */
-    Future<LogSegmentRandomAccessEntryReader> openRandomAccessReader(LogSegmentMetadata segment,
+    CompletableFuture<LogSegmentRandomAccessEntryReader> openRandomAccessReader(LogSegmentMetadata segment,
                                                                      boolean fence);
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryWriter.java
index 660592e..70f0da0 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryWriter.java
@@ -19,7 +19,7 @@
 
 import com.google.common.annotations.Beta;
 import org.apache.distributedlog.Entry;
-import org.apache.distributedlog.util.Sizable;
+import org.apache.distributedlog.common.util.Sizable;
 import org.apache.bookkeeper.client.AsyncCallback;
 
 /**
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataStore.java
index fdf72b1..a58264c 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataStore.java
@@ -18,15 +18,15 @@
 package org.apache.distributedlog.logsegment;
 
 import com.google.common.annotations.Beta;
+import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.callback.LogSegmentNamesListener;
 import org.apache.distributedlog.metadata.LogMetadata;
 import org.apache.distributedlog.metadata.LogMetadataForWriter;
 import org.apache.distributedlog.util.Transaction;
 import org.apache.distributedlog.util.Transaction.OpListener;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
 
 import java.io.Closeable;
 import java.util.List;
@@ -135,7 +135,7 @@
      *          path to store log segment metadata
      * @return future of the retrieved log segment metadata
      */
-    Future<LogSegmentMetadata> getLogSegment(String logSegmentPath);
+    CompletableFuture<LogSegmentMetadata> getLogSegment(String logSegmentPath);
 
     /**
      * Retrieve the list of log segments under <code>logSegmentsPath</code> and register a <i>listener</i>
@@ -147,8 +147,8 @@
      *          log segment listener on log segment changes
      * @return future of the retrieved list of log segment names
      */
-    Future<Versioned<List<String>>> getLogSegmentNames(String logSegmentsPath,
-                                                       LogSegmentNamesListener listener);
+    CompletableFuture<Versioned<List<String>>> getLogSegmentNames(String logSegmentsPath,
+                                                                  LogSegmentNamesListener listener);
 
     /**
      * Unregister a log segment <code>listener</code> on log segment changes under <code>logSegmentsPath</code>.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java
index 948ce30..23c784e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java
@@ -17,11 +17,10 @@
  */
 package org.apache.distributedlog.logsegment;
 
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.Entry;
 import org.apache.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.util.List;
 
 /**
  * An interface class to read entries {@link org.apache.distributedlog.Entry}
@@ -36,7 +35,7 @@
      * @param endEntryId end entry id
      * @return A promise that when satisfied will contain a list of entries of [startEntryId, endEntryId].
      */
-    Future<List<Entry.Reader>> readEntries(long startEntryId, long endEntryId);
+    CompletableFuture<List<Entry.Reader>> readEntries(long startEntryId, long endEntryId);
 
     /**
      * Return the last add confirmed entry id (LAC).
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentWriter.java
index 39c94f4..c483403 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentWriter.java
@@ -18,15 +18,14 @@
 package org.apache.distributedlog.logsegment;
 
 import com.google.common.annotations.Beta;
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.io.AsyncAbortable;
 import org.apache.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.io.IOException;
 
 /**
  * An interface class to write log records into a log segment.
@@ -53,7 +52,7 @@
      * @throws BKTransmitException if failed to transmit data to bk
      * @throws org.apache.distributedlog.exceptions.WriteException if failed to write to bk
      */
-    public Future<DLSN> asyncWrite(LogRecord record);
+    public CompletableFuture<DLSN> asyncWrite(LogRecord record);
 
     /**
      * This isn't a simple synchronous version of {@code asyncWrite}. It has different semantic.
@@ -74,7 +73,7 @@
      *
      * @return future representing the transmit result with last acknowledged transaction id.
      */
-    public Future<Long> flush();
+    public CompletableFuture<Long> flush();
 
     /**
      * Commit the current acknowledged data. It is the consequent operation of {@link #flush()},
@@ -82,6 +81,6 @@
      *
      * @return future representing the commit result.
      */
-    public Future<Long> commit();
+    public CompletableFuture<Long> commit();
 
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/RollingPolicy.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/RollingPolicy.java
index ce98ac9..4844ad4 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/RollingPolicy.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/RollingPolicy.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.logsegment;
 
-import org.apache.distributedlog.util.Sizable;
+import org.apache.distributedlog.common.util.Sizable;
 
 public interface RollingPolicy {
     /**
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/SizeBasedRollingPolicy.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/SizeBasedRollingPolicy.java
index 544f58e..1c3794a 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/SizeBasedRollingPolicy.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/SizeBasedRollingPolicy.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.logsegment;
 
-import org.apache.distributedlog.util.Sizable;
+import org.apache.distributedlog.common.util.Sizable;
 
 public class SizeBasedRollingPolicy implements RollingPolicy {
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java
index 141f139..80c09be 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.logsegment;
 
-import org.apache.distributedlog.util.Sizable;
+import org.apache.distributedlog.common.util.Sizable;
 import org.apache.distributedlog.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java
index 7339d55..948f2bf 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog.metadata;
 
 import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
 import org.apache.distributedlog.util.Utils;
 import org.apache.distributedlog.ZooKeeperClient;
@@ -144,13 +145,9 @@
         byte[] data = serialize();
         try {
             Utils.zkCreateFullPathOptimistic(zkc, uri.getPath(), data,
-                    zkc.getDefaultACL(), CreateMode.PERSISTENT);
-        } catch (KeeperException e) {
-            throw new IOException("Fail to write dl metadata " + new String(data, UTF_8)
-                    +  " to uri " + uri, e);
-        } catch (InterruptedException e) {
-            throw new IOException("Interrupted when writing dl metadata " + new String(data, UTF_8)
-                    + " to uri " + uri, e);
+                zkc.getDefaultACL(), CreateMode.PERSISTENT);
+        } catch (KeeperException ke) {
+            throw new ZKException("Encountered zookeeper exception on creating dl metadata", ke);
         } finally {
             zkc.close();
         }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
index 6b835b9..fe52804 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
@@ -17,10 +17,11 @@
  */
 package org.apache.distributedlog.metadata;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.Transaction;
-import com.twitter.util.Future;
 
 public class DryrunLogSegmentMetadataStoreUpdater extends LogSegmentMetadataStoreUpdater {
 
@@ -38,8 +39,8 @@
             }
 
             @Override
-            public Future<Void> execute() {
-                return Future.Void();
+            public CompletableFuture<Void> execute() {
+                return FutureUtils.Void();
             }
 
             @Override
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataStore.java
index f8fd777..8135678 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataStore.java
@@ -19,11 +19,10 @@
 
 import com.google.common.annotations.Beta;
 import com.google.common.base.Optional;
-import org.apache.distributedlog.callback.NamespaceListener;
-import com.twitter.util.Future;
-
 import java.net.URI;
 import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.callback.NamespaceListener;
 
 /**
  * Interface for log metadata store.
@@ -38,7 +37,7 @@
      *          name of the log
      * @return namespace location that stores this stream.
      */
-    Future<URI> createLog(String logName);
+    CompletableFuture<URI> createLog(String logName);
 
     /**
      * Get the location of the log.
@@ -47,14 +46,14 @@
      *          name of the log
      * @return namespace location that stores this stream.
      */
-    Future<Optional<URI>> getLogLocation(String logName);
+    CompletableFuture<Optional<URI>> getLogLocation(String logName);
 
     /**
      * Retrieves logs from the namespace.
      *
      * @return iterator of logs of the namespace.
      */
-    Future<Iterator<String>> getLogs();
+    CompletableFuture<Iterator<String>> getLogs();
 
     /**
      * Register a namespace listener on streams changes.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
index ee9b00e..71a1f98 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
@@ -18,16 +18,15 @@
 package org.apache.distributedlog.metadata;
 
 import com.google.common.base.Preconditions;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.LogRecordWithDLSN;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
 import org.apache.distributedlog.util.Transaction;
-import com.twitter.util.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
 
 public class LogSegmentMetadataStoreUpdater implements MetadataUpdater {
 
@@ -57,8 +56,8 @@
     }
 
     @Override
-    public Future<LogSegmentMetadata> updateLastRecord(LogSegmentMetadata segment,
-                                                       LogRecordWithDLSN record) {
+    public CompletableFuture<LogSegmentMetadata> updateLastRecord(LogSegmentMetadata segment,
+                                                                  LogRecordWithDLSN record) {
         DLSN dlsn = record.getDlsn();
         Preconditions.checkState(!segment.isInProgress(),
                 "Updating last dlsn for an inprogress log segment isn't supported.");
@@ -73,7 +72,7 @@
     }
 
     @Override
-    public Future<LogSegmentMetadata> changeSequenceNumber(LogSegmentMetadata segment,
+    public CompletableFuture<LogSegmentMetadata> changeSequenceNumber(LogSegmentMetadata segment,
                                                            long logSegmentSeqNo) {
         String newZkPath = segment.getZkPath()
                 .replace(formatLogSegmentSequenceNumber(segment.getLogSegmentSequenceNumber()),
@@ -92,7 +91,7 @@
      * @return new log segment
      */
     @Override
-    public Future<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment) {
+    public CompletableFuture<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment) {
         final LogSegmentMetadata newSegment = segment.mutator()
             .setTruncationStatus(LogSegmentMetadata.TruncationStatus.ACTIVE)
             .build();
@@ -106,7 +105,7 @@
      * @return new log segment
      */
     @Override
-    public Future<LogSegmentMetadata> setLogSegmentTruncated(LogSegmentMetadata segment) {
+    public CompletableFuture<LogSegmentMetadata> setLogSegmentTruncated(LogSegmentMetadata segment) {
         final LogSegmentMetadata newSegment = segment.mutator()
             .setTruncationStatus(LogSegmentMetadata.TruncationStatus.TRUNCATED)
             .build();
@@ -130,7 +129,7 @@
      * @return new log segment
      */
     @Override
-    public Future<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment, DLSN minActiveDLSN) {
+    public CompletableFuture<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment, DLSN minActiveDLSN) {
         final LogSegmentMetadata newSegment = segment.mutator()
             .setTruncationStatus(LogSegmentMetadata.TruncationStatus.PARTIALLY_TRUNCATED)
             .setMinActiveDLSN(minActiveDLSN)
@@ -150,28 +149,18 @@
         return newSegment;
     }
 
-    protected Future<LogSegmentMetadata> updateSegmentMetadata(final LogSegmentMetadata segment) {
+    protected CompletableFuture<LogSegmentMetadata> updateSegmentMetadata(final LogSegmentMetadata segment) {
         Transaction<Object> txn = transaction();
         metadataStore.updateLogSegment(txn, segment);
-        return txn.execute().map(new AbstractFunction1<Void, LogSegmentMetadata>() {
-            @Override
-            public LogSegmentMetadata apply(Void value) {
-                return segment;
-            }
-        });
+        return txn.execute().thenApply((value) -> segment);
     }
 
-    protected Future<LogSegmentMetadata> addNewSegmentAndDeleteOldSegment(
+    protected CompletableFuture<LogSegmentMetadata> addNewSegmentAndDeleteOldSegment(
             final LogSegmentMetadata newSegment, LogSegmentMetadata oldSegment) {
         LOG.info("old segment {} new segment {}", oldSegment, newSegment);
         Transaction<Object> txn = transaction();
         addNewSegmentAndDeleteOldSegment(txn, newSegment, oldSegment);
-        return txn.execute().map(new AbstractFunction1<Void, LogSegmentMetadata>() {
-            @Override
-            public LogSegmentMetadata apply(Void value) {
-                return newSegment;
-            }
-        });
+        return txn.execute().thenApply((value) -> newSegment);
     }
 
     protected void addNewSegmentAndDeleteOldSegment(Transaction<Object> txn,
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java
index f1e8f06..37ecab4 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java
@@ -19,11 +19,11 @@
 
 import com.google.common.annotations.Beta;
 import com.google.common.base.Optional;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.lock.DistributedLock;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
-import org.apache.distributedlog.util.PermitManager;
+import org.apache.distributedlog.common.util.PermitManager;
 import org.apache.distributedlog.util.Transaction;
-import com.twitter.util.Future;
 
 import java.io.Closeable;
 import java.net.URI;
@@ -47,10 +47,10 @@
      *
      * @param uri the location of the log stream
      * @param logName the name of the log stream
-     * @return future represents the existence of a log stream. {@link org.apache.distributedlog.LogNotFoundException}
-     *         is thrown if the log doesn't exist
+     * @return future represents the existence of a log stream.
+     *         {@link org.apache.distributedlog.exceptions.LogNotFoundException} is thrown if the log doesn't exist
      */
-    Future<Void> logExists(URI uri, String logName);
+    CompletableFuture<Void> logExists(URI uri, String logName);
 
     /**
      * Create the read lock for the log stream.
@@ -59,7 +59,7 @@
      * @param readerId the reader id used for lock
      * @return the read lock
      */
-    Future<DistributedLock> createReadLock(LogMetadataForReader metadata,
+    CompletableFuture<DistributedLock> createReadLock(LogMetadataForReader metadata,
                                            Optional<String> readerId);
 
     /**
@@ -79,7 +79,7 @@
      * @param createIfNotExists flag to create the stream if it doesn't exist
      * @return the metadata of the log
      */
-    Future<LogMetadataForWriter> getLog(URI uri,
+    CompletableFuture<LogMetadataForWriter> getLog(URI uri,
                                         String streamName,
                                         boolean ownAllocator,
                                         boolean createIfNotExists);
@@ -91,7 +91,7 @@
      * @param streamName the name of the log stream
      * @return future represents the result of the deletion.
      */
-    Future<Void> deleteLog(URI uri, String streamName);
+    CompletableFuture<Void> deleteLog(URI uri, String streamName);
 
     /**
      * Get the log segment metadata store.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataUpdater.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataUpdater.java
index 06a0600..793a2c9 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataUpdater.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataUpdater.java
@@ -17,11 +17,11 @@
  */
 package org.apache.distributedlog.metadata;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.LogRecordWithDLSN;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.util.Transaction;
-import com.twitter.util.Future;
 
 /**
  * An updater to update metadata. It contains utility functions on mutating metadata.
@@ -44,8 +44,8 @@
      *          correct last record.
      * @return new log segment
      */
-    Future<LogSegmentMetadata> updateLastRecord(LogSegmentMetadata segment,
-                                                LogRecordWithDLSN record);
+    CompletableFuture<LogSegmentMetadata> updateLastRecord(LogSegmentMetadata segment,
+                                                           LogRecordWithDLSN record);
 
     /**
      * Change ledger sequence number of <i>segment</i> to given <i>logSegmentSeqNo</i>.
@@ -56,7 +56,7 @@
      *          ledger sequence number to change.
      * @return new log segment
      */
-    Future<LogSegmentMetadata> changeSequenceNumber(LogSegmentMetadata segment,
+    CompletableFuture<LogSegmentMetadata> changeSequenceNumber(LogSegmentMetadata segment,
                                                     long logSegmentSeqNo);
 
     /**
@@ -66,7 +66,7 @@
      *          log segment to change truncation status to active.
      * @return new log segment
      */
-    Future<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment);
+    CompletableFuture<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment);
 
     /**
      * Change the truncation status of a <i>log segment</i> to truncated
@@ -75,7 +75,7 @@
      *          log segment to change truncation status to truncated.
      * @return new log segment
      */
-    Future<LogSegmentMetadata> setLogSegmentTruncated(LogSegmentMetadata segment);
+    CompletableFuture<LogSegmentMetadata> setLogSegmentTruncated(LogSegmentMetadata segment);
 
     /**
      * Change the truncation status of a <i>log segment</i> to truncated. The operation won't be executed
@@ -98,7 +98,7 @@
      *          DLSN within the log segment before which log has been truncated
      * @return new log segment
      */
-    Future<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment,
+    CompletableFuture<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment,
                                                                DLSN minActiveDLSN);
 
     /**
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java
index 2f5adc6..cf970ef 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java
@@ -18,16 +18,15 @@
 package org.apache.distributedlog.namespace;
 
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.MetadataAccessor;
+import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.acl.AccessControlManager;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.InvalidStreamNameException;
 import org.apache.distributedlog.injector.AsyncFailureInjector;
 import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
-import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
 import org.apache.distributedlog.metadata.LogMetadataStore;
 import org.apache.distributedlog.metadata.LogStreamMetadataStore;
-import org.apache.distributedlog.subscription.SubscriptionsStore;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.bookkeeper.feature.FeatureProvider;
 import org.apache.bookkeeper.stats.StatsLogger;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/rate/SampledMovingAverageRate.java b/distributedlog-core/src/main/java/org/apache/distributedlog/rate/SampledMovingAverageRate.java
deleted file mode 100644
index 0b3ccac..0000000
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/rate/SampledMovingAverageRate.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.rate;
-
-import com.twitter.common.stats.Rate;
-import com.twitter.util.TimerTask;
-import com.twitter.util.Timer;
-import com.twitter.util.Time;
-import java.util.concurrent.atomic.AtomicLong;
-
-class SampledMovingAverageRate implements MovingAverageRate {
-    private final Rate rate;
-    private final AtomicLong total;
-
-    private double value;
-
-    public SampledMovingAverageRate(int intervalSecs) {
-        this.total = new AtomicLong(0);
-        this.rate = Rate.of("Ignore", total)
-            .withWindowSize(intervalSecs)
-            .build();
-        this.value = 0;
-    }
-
-    @Override
-    public double get() {
-        return value;
-    }
-
-    @Override
-    public void add(long amount) {
-        total.getAndAdd(amount);
-    }
-
-    @Override
-    public void inc() {
-        add(1);
-    }
-
-    void sample() {
-        value = rate.doSample();
-    }
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/stats/BKExceptionStatsLogger.java b/distributedlog-core/src/main/java/org/apache/distributedlog/stats/BKExceptionStatsLogger.java
deleted file mode 100644
index 199aa4c..0000000
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/stats/BKExceptionStatsLogger.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.stats;
-
-import org.apache.bookkeeper.client.BKException.Code;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * A Util to logger stats on bk exceptions.
- */
-public class BKExceptionStatsLogger {
-
-    public static String getMessage(int code) {
-        switch (code) {
-            case Code.OK:
-                return "OK";
-            case Code.ReadException:
-                return "ReadException";
-            case Code.QuorumException:
-                return "QuorumException";
-            case Code.NoBookieAvailableException:
-                return "NoBookieAvailableException";
-            case Code.DigestNotInitializedException:
-                return "DigestNotInitializedException";
-            case Code.DigestMatchException:
-                return "DigestMatchException";
-            case Code.NotEnoughBookiesException:
-                return "NotEnoughBookiesException";
-            case Code.NoSuchLedgerExistsException:
-                return "NoSuchLedgerExistsException";
-            case Code.BookieHandleNotAvailableException:
-                return "BookieHandleNotAvailableException";
-            case Code.ZKException:
-                return "ZKException";
-            case Code.LedgerRecoveryException:
-                return "LedgerRecoveryException";
-            case Code.LedgerClosedException:
-                return "LedgerClosedException";
-            case Code.WriteException:
-                return "WriteException";
-            case Code.NoSuchEntryException:
-                return "NoSuchEntryException";
-            case Code.IncorrectParameterException:
-                return "IncorrectParameterException";
-            case Code.InterruptedException:
-                return "InterruptedException";
-            case Code.ProtocolVersionException:
-                return "ProtocolVersionException";
-            case Code.MetadataVersionException:
-                return "MetadataVersionException";
-            case Code.LedgerFencedException:
-                return "LedgerFencedException";
-            case Code.UnauthorizedAccessException:
-                return "UnauthorizedAccessException";
-            case Code.UnclosedFragmentException:
-                return "UnclosedFragmentException";
-            case Code.WriteOnReadOnlyBookieException:
-                return "WriteOnReadOnlyBookieException";
-            case Code.IllegalOpException:
-                return "IllegalOpException";
-            default:
-                return "UnexpectedException";
-        }
-    }
-
-    private final StatsLogger parentLogger;
-    private final Map<Integer, Counter> exceptionCounters;
-
-    public BKExceptionStatsLogger(StatsLogger parentLogger) {
-        this.parentLogger = parentLogger;
-        this.exceptionCounters = new HashMap<Integer, Counter>();
-    }
-
-    public Counter getExceptionCounter(int rc) {
-        Counter counter = exceptionCounters.get(rc);
-        if (null != counter) {
-            return counter;
-        }
-        // TODO: it would be better to have BKException.Code.get(rc)
-        synchronized (exceptionCounters) {
-            counter = exceptionCounters.get(rc);
-            if (null != counter) {
-                return counter;
-            }
-            counter = parentLogger.getCounter(getMessage(rc));
-            exceptionCounters.put(rc, counter);
-        }
-        return counter;
-    }
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/thrift/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/thrift/package-info.java
index efee3ca..ffe2303 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/thrift/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/thrift/package-info.java
@@ -18,4 +18,4 @@
 /**
  * Generated thrift code.
  */
-package org.apache.distributedlog.thrift;
\ No newline at end of file
+package org.apache.distributedlog.thrift;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
index 2c27088..64229d1 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
@@ -55,13 +55,15 @@
 import com.google.common.base.Preconditions;
 import org.apache.distributedlog.BKDistributedLogNamespace;
 import org.apache.distributedlog.Entry;
-import org.apache.distributedlog.MetadataAccessor;
+import org.apache.distributedlog.api.MetadataAccessor;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.callback.NamespaceListener;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.Utils;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
@@ -86,16 +88,16 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.RateLimiter;
-import org.apache.distributedlog.AsyncLogReader;
-import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.BookKeeperClientBuilder;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
-import org.apache.distributedlog.LogReader;
+import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.LogRecordWithDLSN;
 import org.apache.distributedlog.LogSegmentMetadata;
@@ -107,9 +109,7 @@
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
 import org.apache.distributedlog.metadata.MetadataUpdater;
 import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import org.apache.distributedlog.util.SchedulerUtils;
-import com.twitter.util.Await;
-import com.twitter.util.FutureEventListener;
+import org.apache.distributedlog.common.util.SchedulerUtils;
 
 import static com.google.common.base.Charsets.UTF_8;
 
@@ -163,7 +163,7 @@
         protected URI uri;
         protected String zkAclId = null;
         protected boolean force = false;
-        protected DistributedLogNamespace namespace = null;
+        protected Namespace namespace = null;
 
         protected PerDLCommand(String name, String description) {
             super(name, description);
@@ -252,9 +252,9 @@
             this.force = force;
         }
 
-        protected DistributedLogNamespace getNamespace() throws IOException {
+        protected Namespace getNamespace() throws IOException {
             if (null == this.namespace) {
-                this.namespace = DistributedLogNamespaceBuilder.newBuilder()
+                this.namespace = NamespaceBuilder.newBuilder()
                         .uri(getUri())
                         .conf(getConf())
                         .build();
@@ -464,7 +464,7 @@
             return 0;
         }
 
-        protected void printStreams(DistributedLogNamespace namespace) throws Exception {
+        protected void printStreams(Namespace namespace) throws Exception {
             Iterator<String> streams = namespace.getLogs();
             System.out.println("Streams under " + getUri() + " : ");
             System.out.println("--------------------------------");
@@ -536,7 +536,7 @@
             System.out.println("");
         }
 
-        protected void watchAndReportChanges(DistributedLogNamespace namespace) throws Exception {
+        protected void watchAndReportChanges(Namespace namespace) throws Exception {
             namespace.registerNamespaceListener(this);
         }
     }
@@ -783,7 +783,7 @@
             return truncateStreams(getNamespace());
         }
 
-        private int truncateStreams(final DistributedLogNamespace namespace) throws Exception {
+        private int truncateStreams(final Namespace namespace) throws Exception {
             Iterator<String> streamCollection = namespace.getLogs();
             final List<String> streams = new ArrayList<String>();
             while (streamCollection.hasNext()) {
@@ -827,7 +827,7 @@
             return 0;
         }
 
-        private void truncateStreams(DistributedLogNamespace namespace, List<String> streams,
+        private void truncateStreams(Namespace namespace, List<String> streams,
                                      int tid, int numStreamsPerThreads) throws IOException {
             int startIdx = tid * numStreamsPerThreads;
             int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
@@ -957,7 +957,7 @@
         }
 
         private void printHeader(DistributedLogManager dlm) throws Exception {
-            DLSN firstDlsn = Await.result(dlm.getFirstDLSNAsync());
+            DLSN firstDlsn = FutureUtils.result(dlm.getFirstDLSNAsync());
             boolean endOfStreamMarked = dlm.isEndOfStreamMarked();
             DLSN lastDlsn = dlm.getLastDLSN();
             long firstTxnId = dlm.getFirstTxId();
@@ -1121,7 +1121,7 @@
         }
 
         long countToLastRecord(DistributedLogManager dlm) throws Exception {
-            return Await.result(dlm.getLogRecordCountAsync(startDLSN)).longValue();
+            return FutureUtils.result(dlm.getLogRecordCountAsync(startDLSN)).longValue();
         }
 
         @Override
@@ -1439,7 +1439,7 @@
                 AsyncLogReader reader;
                 Object startOffset;
                 try {
-                    DLSN lastDLSN = Await.result(dlm.getLastDLSNAsync());
+                    DLSN lastDLSN = FutureUtils.result(dlm.getLastDLSNAsync());
                     System.out.println("Last DLSN : " + lastDLSN);
                     if (null == fromDLSN) {
                         reader = dlm.getAsyncLogReader(fromTxnId);
@@ -1468,7 +1468,7 @@
 
         private void dumpRecords(AsyncLogReader reader) throws Exception {
             int numRead = 0;
-            LogRecord record = Await.result(reader.readNext());
+            LogRecord record = FutureUtils.result(reader.readNext());
             while (record != null) {
                 // dump the record
                 dumpRecord(record);
@@ -1476,7 +1476,7 @@
                 if (numRead >= count) {
                     break;
                 }
-                record = Await.result(reader.readNext());
+                record = FutureUtils.result(reader.readNext());
             }
             if (numRead == 0) {
                 System.out.println("No records.");
@@ -2641,18 +2641,18 @@
             return truncateStream(getNamespace(), getStreamName(), dlsn);
         }
 
-        private int truncateStream(final DistributedLogNamespace namespace, String streamName, DLSN dlsn) throws Exception {
+        private int truncateStream(final Namespace namespace, String streamName, DLSN dlsn) throws Exception {
             DistributedLogManager dlm = namespace.openLog(streamName);
             try {
                 long totalRecords = dlm.getLogRecordCount();
-                long recordsAfterTruncate = Await.result(dlm.getLogRecordCountAsync(dlsn));
+                long recordsAfterTruncate = FutureUtils.result(dlm.getLogRecordCountAsync(dlsn));
                 long recordsToTruncate = totalRecords - recordsAfterTruncate;
                 if (!getForce() && !IOUtils.confirmPrompt("Do you want to truncate " + streamName + " at dlsn " + dlsn + " (" + recordsToTruncate + " records)?")) {
                     return 0;
                 } else {
                     AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
                     try {
-                        if (!Await.result(writer.truncate(dlsn))) {
+                        if (!FutureUtils.result(writer.truncate(dlsn))) {
                             System.out.println("Failed to truncate.");
                         }
                         return 0;
@@ -2764,7 +2764,7 @@
             return deleteSubscriber(getNamespace());
         }
 
-        private int deleteSubscriber(final DistributedLogNamespace namespace) throws Exception {
+        private int deleteSubscriber(final Namespace namespace) throws Exception {
             Iterator<String> streamCollection = namespace.getLogs();
             final List<String> streams = new ArrayList<String>();
             while (streamCollection.hasNext()) {
@@ -2809,7 +2809,7 @@
             return 0;
         }
 
-        private void deleteSubscriber(DistributedLogNamespace namespace, List<String> streams,
+        private void deleteSubscriber(Namespace namespace, List<String> streams,
                                       int tid, int numStreamsPerThreads) throws Exception {
             int startIdx = tid * numStreamsPerThreads;
             int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
@@ -2818,7 +2818,7 @@
                 DistributedLogManager dlm = namespace.openLog(s);
                 final CountDownLatch countDownLatch = new CountDownLatch(1);
                 dlm.getSubscriptionsStore().deleteSubscriber(subscriberId)
-                    .addEventListener(new FutureEventListener<Boolean>() {
+                    .whenComplete(new FutureEventListener<Boolean>() {
                         @Override
                         public void onFailure(Throwable cause) {
                             System.out.println("Failed to delete subscriber for stream " + s);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Allocator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Allocator.java
index c0da29a..a9b81e2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Allocator.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Allocator.java
@@ -17,12 +17,11 @@
  */
 package org.apache.distributedlog.util;
 
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.io.AsyncCloseable;
 import org.apache.distributedlog.io.AsyncDeleteable;
 import org.apache.distributedlog.util.Transaction.OpListener;
-import com.twitter.util.Future;
-
-import java.io.IOException;
 
 /**
  * A common interface to allocate <i>I</i> under transaction <i>T</i>.
@@ -47,7 +46,7 @@
  * final Transaction<T> txn = ...;
  *
  * // Try obtain object I
- * Future<I> tryObtainFuture = allocator.tryObtain(txn, new OpListener<I>() {
+ * CompletableFuture<I> tryObtainFuture = allocator.tryObtain(txn, new OpListener<I>() {
  *     public void onCommit(I resource) {
  *         // the obtain succeed, process with the resource
  *     }
@@ -97,6 +96,6 @@
      *          transaction.
      * @return future result returning <i>I</i> that would be obtained under transaction <code>txn</code>.
      */
-    Future<I> tryObtain(Transaction<T> txn, OpListener<I> listener);
+    CompletableFuture<I> tryObtain(Transaction<T> txn, OpListener<I> listener);
 
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/ConfUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/ConfUtils.java
index 41c2be7..aa7cefe 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/ConfUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/ConfUtils.java
@@ -18,7 +18,7 @@
 package org.apache.distributedlog.util;
 
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.config.ConcurrentConstConfiguration;
+import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.commons.configuration.Configuration;
 
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/MonitoredFuturePool.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/MonitoredFuturePool.java
deleted file mode 100644
index 3372476..0000000
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/MonitoredFuturePool.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.util;
-
-import com.google.common.base.Stopwatch;
-
-import com.twitter.util.FuturePool;
-import com.twitter.util.FuturePool$;
-import com.twitter.util.Future;
-
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import scala.runtime.BoxedUnit;
-import scala.Function0;
-
-/**
- * {@link FuturePool} with exposed stats. This class is exposing following stats for helping understanding
- * the healthy of this thread pool executor.
- * <h3>Metrics</h3>
- * Stats are only exposed when <code>traceTaskExecution</code> is true.
- * <ul>
- * <li>task_pending_time: opstats. measuring the characteristics about the time that tasks spent on waiting
- * being executed.
- * <li>task_execution_time: opstats. measuring the characteristics about the time that tasks spent on executing.
- * <li>task_enqueue_time: opstats. measuring the characteristics about the time that tasks spent on submitting.
- * <li>tasks_pending: gauge. how many tasks are pending in this future pool.
- * </ul>
- */
-public class MonitoredFuturePool implements FuturePool {
-    static final Logger LOG = LoggerFactory.getLogger(MonitoredFuturePool.class);
-
-    private final FuturePool futurePool;
-
-    private final StatsLogger statsLogger;
-    private final OpStatsLogger taskPendingTime;
-    private final OpStatsLogger taskExecutionTime;
-    private final OpStatsLogger taskEnqueueTime;
-    private final Counter taskPendingCounter;
-
-    private final boolean traceTaskExecution;
-    private final long traceTaskExecutionWarnTimeUs;
-
-    class TimedFunction0<T> extends com.twitter.util.Function0<T> {
-        private final Function0<T> function0;
-        private Stopwatch pendingStopwatch = Stopwatch.createStarted();
-
-        TimedFunction0(Function0<T> function0) {
-            this.function0 = function0;
-            this.pendingStopwatch = Stopwatch.createStarted();
-        }
-
-        @Override
-        public T apply() {
-            taskPendingTime.registerSuccessfulEvent(pendingStopwatch.elapsed(TimeUnit.MICROSECONDS));
-            Stopwatch executionStopwatch = Stopwatch.createStarted();
-            T result = function0.apply();
-            taskExecutionTime.registerSuccessfulEvent(executionStopwatch.elapsed(TimeUnit.MICROSECONDS));
-            long elapsed = executionStopwatch.elapsed(TimeUnit.MICROSECONDS);
-            if (elapsed > traceTaskExecutionWarnTimeUs) {
-                LOG.info("{} took too long {} microseconds", function0.toString(), elapsed);
-            }
-            return result;
-        }
-    }
-
-    /**
-     * Create a future pool with stats exposed.
-     *
-     * @param futurePool underlying future pool to execute futures
-     * @param statsLogger stats logger to receive exposed stats
-     * @param traceTaskExecution flag to enable/disable exposing stats about task execution
-     * @param traceTaskExecutionWarnTimeUs flag to enable/disable logging slow tasks
-     *                                     whose execution time is above this value
-     */
-    public MonitoredFuturePool(FuturePool futurePool,
-                               StatsLogger statsLogger,
-                               boolean traceTaskExecution,
-                               long traceTaskExecutionWarnTimeUs) {
-        this.futurePool = futurePool;
-        this.traceTaskExecution = traceTaskExecution;
-        this.traceTaskExecutionWarnTimeUs = traceTaskExecutionWarnTimeUs;
-        this.statsLogger = statsLogger;
-        this.taskPendingTime = statsLogger.getOpStatsLogger("task_pending_time");
-        this.taskExecutionTime = statsLogger.getOpStatsLogger("task_execution_time");
-        this.taskEnqueueTime = statsLogger.getOpStatsLogger("task_enqueue_time");
-        this.taskPendingCounter = statsLogger.getCounter("tasks_pending");
-    }
-
-    @Override
-    public <T> Future<T> apply(Function0<T> function0) {
-        if (traceTaskExecution) {
-            taskPendingCounter.inc();
-            Stopwatch taskEnqueueStopwatch = Stopwatch.createStarted();
-            Future<T> futureResult = futurePool.apply(new TimedFunction0<T>(function0));
-            taskEnqueueTime.registerSuccessfulEvent(taskEnqueueStopwatch.elapsed(TimeUnit.MICROSECONDS));
-            futureResult.ensure(new com.twitter.util.Function0<BoxedUnit>() {
-                @Override
-                public BoxedUnit apply() {
-                    taskPendingCounter.dec();
-                    return null;
-                }
-            });
-            return futureResult;
-        } else {
-            return futurePool.apply(function0);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/MonitoredScheduledThreadPoolExecutor.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/MonitoredScheduledThreadPoolExecutor.java
deleted file mode 100644
index 3121a19..0000000
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/MonitoredScheduledThreadPoolExecutor.java
+++ /dev/null
@@ -1,257 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.util;
-
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.MathUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-
-/**
- * {@link ScheduledThreadPoolExecutor} with exposed stats. This class is exposing following stats for
- * helping understanding the healthy of this thread pool executor.
- * <h3>Metrics</h3>
- * <ul>
- * <li>pending_tasks: gauge. how many tasks are pending in this executor.
- * <li>completed_tasks: gauge. how many tasks are completed in this executor.
- * <li>total_tasks: gauge. how many tasks are submitted to this executor.
- * <li>task_pending_time: opstats. measuring the characteristics about the time that tasks spent on
- * waiting being executed.
- * <li>task_execution_time: opstats. measuring the characteristics about the time that tasks spent on
- * executing.
- * </ul>
- */
-public class MonitoredScheduledThreadPoolExecutor extends ScheduledThreadPoolExecutor {
-    static final Logger LOG = LoggerFactory.getLogger(MonitoredScheduledThreadPoolExecutor.class);
-
-    private class TimedRunnable implements Runnable {
-
-        final Runnable runnable;
-        final long enqueueNanos;
-
-        TimedRunnable(Runnable runnable) {
-            this.runnable = runnable;
-            this.enqueueNanos = MathUtils.nowInNano();
-        }
-
-        @Override
-        public void run() {
-            long startNanos = MathUtils.nowInNano();
-            long pendingMicros = TimeUnit.NANOSECONDS.toMicros(startNanos - enqueueNanos);
-            taskPendingStats.registerSuccessfulEvent(pendingMicros);
-            try {
-                runnable.run();
-            } finally {
-                long executionMicros = TimeUnit.NANOSECONDS.toMicros(MathUtils.nowInNano() - startNanos);
-                taskExecutionStats.registerSuccessfulEvent(executionMicros);
-            }
-        }
-
-        @Override
-        public String toString() {
-            return runnable.toString();
-        }
-
-        @Override
-        public int hashCode() {
-            return runnable.hashCode();
-        }
-    }
-
-    private class TimedCallable<T> implements Callable<T> {
-
-        final Callable<T> task;
-        final long enqueueNanos;
-
-        TimedCallable(Callable<T> task) {
-            this.task = task;
-            this.enqueueNanos = MathUtils.nowInNano();
-        }
-
-        @Override
-        public T call() throws Exception {
-            long startNanos = MathUtils.nowInNano();
-            long pendingMicros = TimeUnit.NANOSECONDS.toMicros(startNanos - enqueueNanos);
-            taskPendingStats.registerSuccessfulEvent(pendingMicros);
-            try {
-                return task.call();
-            } finally {
-                long executionMicros = TimeUnit.NANOSECONDS.toMicros(MathUtils.nowInNano() - startNanos);
-                taskExecutionStats.registerSuccessfulEvent(executionMicros);
-            }
-        }
-    }
-
-    protected final boolean traceTaskExecution;
-    protected final OpStatsLogger taskExecutionStats;
-    protected final OpStatsLogger taskPendingStats;
-    protected final StatsLogger statsLogger;
-    // Gauges and their labels
-    private static final String pendingTasksGaugeLabel = "pending_tasks";
-    private final Gauge<Number> pendingTasksGauge;
-    private static final String completedTasksGaugeLabel = "completed_tasks";
-    protected final Gauge<Number> completedTasksGauge;
-    private static final String totalTasksGaugeLabel = "total_tasks";
-    protected final Gauge<Number> totalTasksGauge;
-
-    public MonitoredScheduledThreadPoolExecutor(int corePoolSize,
-                                                ThreadFactory threadFactory,
-                                                StatsLogger statsLogger,
-                                                boolean traceTaskExecution) {
-        super(corePoolSize, threadFactory);
-        this.traceTaskExecution = traceTaskExecution;
-        this.statsLogger = statsLogger;
-        this.taskPendingStats = this.statsLogger.getOpStatsLogger("task_pending_time");
-        this.taskExecutionStats = this.statsLogger.getOpStatsLogger("task_execution_time");
-        this.pendingTasksGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return getQueue().size();
-            }
-        };
-        this.completedTasksGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return getCompletedTaskCount();
-            }
-        };
-        this.totalTasksGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return getTaskCount();
-            }
-        };
-
-        // outstanding tasks
-        this.statsLogger.registerGauge(pendingTasksGaugeLabel, pendingTasksGauge);
-        // completed tasks
-        this.statsLogger.registerGauge(completedTasksGaugeLabel, completedTasksGauge);
-        // total tasks
-        this.statsLogger.registerGauge(totalTasksGaugeLabel, pendingTasksGauge);
-    }
-
-    private Runnable timedRunnable(Runnable r) {
-        return traceTaskExecution ? new TimedRunnable(r) : r;
-    }
-
-    private <T> Callable<T> timedCallable(Callable<T> task) {
-        return traceTaskExecution ? new TimedCallable<T>(task) : task;
-    }
-
-    @Override
-    public Future<?> submit(Runnable task) {
-        return super.submit(timedRunnable(task));
-    }
-
-    @Override
-    public <T> Future<T> submit(Runnable task, T result) {
-        return super.submit(timedRunnable(task), result);
-    }
-
-    @Override
-    public <T> Future<T> submit(Callable<T> task) {
-        return super.submit(timedCallable(task));
-    }
-
-    @Override
-    protected void afterExecute(Runnable r, Throwable t) {
-        super.afterExecute(r, t);
-        Throwable hiddenThrowable = extractThrowable(r);
-        if (hiddenThrowable != null)
-            logAndHandle(hiddenThrowable, true);
-
-        // The executor re-throws exceptions thrown by the task to the uncaught exception handler
-        // so we don't need to pass the exception to the handler explicitly
-        if (null != t) {
-            logAndHandle(t, false);
-        }
-    }
-
-    /**
-     * The executor re-throws exceptions thrown by the task to the uncaught exception handler
-     * so we only need to do anything if uncaught exception handler has not been se
-     */
-    private void logAndHandle(Throwable t, boolean passToHandler) {
-        if (Thread.getDefaultUncaughtExceptionHandler() == null) {
-            LOG.error("Unhandled exception on thread {}", Thread.currentThread().getName(), t);
-        }
-        else {
-            LOG.info("Unhandled exception on thread {}", Thread.currentThread().getName(), t);
-            if (passToHandler) {
-                Thread.getDefaultUncaughtExceptionHandler().uncaughtException(Thread.currentThread(), t);
-            }
-        }
-    }
-
-
-    /**
-     * Extract the exception (throwable) inside the ScheduledFutureTask
-     * @param runnable - The runable that was executed
-     * @return exception enclosed in the Runnable if any; null otherwise
-     */
-    private Throwable extractThrowable(Runnable runnable) {
-        // Check for exceptions wrapped by FutureTask.
-        // We do this by calling get(), which will cause it to throw any saved exception.
-        // Check for isDone to prevent blocking
-        if ((runnable instanceof Future<?>) && ((Future<?>) runnable).isDone()) {
-            try {
-                ((Future<?>) runnable).get();
-            } catch (CancellationException e) {
-                LOG.debug("Task {} cancelled", runnable, e.getCause());
-            } catch (InterruptedException e) {
-                LOG.debug("Task {} was interrupted", runnable, e);
-            } catch (ExecutionException e) {
-                return e.getCause();
-            }
-        }
-
-        return null;
-    }
-
-    void unregisterGauges() {
-        this.statsLogger.unregisterGauge(pendingTasksGaugeLabel, pendingTasksGauge);
-        this.statsLogger.unregisterGauge(completedTasksGaugeLabel, completedTasksGauge);
-        this.statsLogger.unregisterGauge(totalTasksGaugeLabel, totalTasksGauge);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java
deleted file mode 100644
index ad1ba4e..0000000
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java
+++ /dev/null
@@ -1,490 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.util;
-
-import com.google.common.base.Objects;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.distributedlog.stats.BroadCastStatsLogger;
-import com.twitter.util.ExecutorServiceFuturePool;
-import com.twitter.util.FuturePool;
-import com.twitter.util.Time;
-import com.twitter.util.Timer;
-import com.twitter.util.TimerTask;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.MathUtils;
-import scala.Function0;
-
-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.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;
-
-/**
- * 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 MonitoredScheduledThreadPoolExecutor}s. Each
- * {@link MonitoredScheduledThreadPoolExecutor} 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#apply(Object, Function0)} will be submitted to a dedicated executor based on
- * the hash value of submit <i>key</i>.
- *
- * <h3>Metrics</h3>
- *
- * <h4>Per Executor Metrics</h4>
- *
- * Metrics about individual executors are exposed via {@link Builder#perExecutorStatsLogger}
- * under <i>`scope`/`name`-executor-`id`-0</i>. `name` is the scheduler name provided by {@link Builder#name}
- * while `id` is the index of this executor in the pool. And corresponding stats of future pool of
- * that executor are exposed under <i>`scope`/`name`-executor-`id`-0/futurepool</i>.
- * <p>
- * See {@link MonitoredScheduledThreadPoolExecutor} and {@link MonitoredFuturePool} for per executor metrics
- * exposed.
- *
- * <h4>Aggregated Metrics</h4>
- * <ul>
- * <li>task_pending_time: opstats. measuring the characteristics about the time that tasks spent on
- * waiting being executed.
- * <li>task_execution_time: opstats. measuring the characteristics about the time that tasks spent on
- * executing.
- * <li>futurepool/task_pending_time: opstats. measuring the characteristics about the time that tasks spent
- * on waiting in future pool being executed.
- * <li>futurepool/task_execution_time: opstats. measuring the characteristics about the time that tasks spent
- * on executing.
- * <li>futurepool/task_enqueue_time: opstats. measuring the characteristics about the time that tasks spent on
- * submitting to future pool.
- * <li>futurepool/tasks_pending: gauge. how many tasks are pending in this future pool.
- * </ul>
- */
-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;
-        private boolean traceTaskExecution = false;
-        private long traceTaskExecutionWarnTimeUs = Long.MAX_VALUE;
-        private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
-        private StatsLogger perExecutorStatsLogger = NullStatsLogger.INSTANCE;
-
-        /**
-         * 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;
-        }
-
-        /**
-         * Enable/Disable exposing task execution stats.
-         *
-         * @param trace
-         *          flag to enable/disable exposing task execution stats.
-         * @return scheduler builder
-         */
-        public Builder traceTaskExecution(boolean trace) {
-            this.traceTaskExecution = trace;
-            return this;
-        }
-
-        /**
-         * Enable/Disable logging slow tasks whose execution time is above <code>timeUs</code>.
-         *
-         * @param timeUs
-         *          slow task execution time threshold in us.
-         * @return scheduler builder.
-         */
-        public Builder traceTaskExecutionWarnTimeUs(long timeUs) {
-            this.traceTaskExecutionWarnTimeUs = timeUs;
-            return this;
-        }
-
-        /**
-         * Expose the aggregated stats over <code>statsLogger</code>.
-         *
-         * @param statsLogger
-         *          stats logger to receive aggregated stats.
-         * @return scheduler builder
-         */
-        public Builder statsLogger(StatsLogger statsLogger) {
-            this.statsLogger = statsLogger;
-            return this;
-        }
-
-        /**
-         * Expose stats of individual executors over <code>perExecutorStatsLogger</code>.
-         * Each executor's stats will be exposed under a sub-scope `name`-executor-`id`-0.
-         * `name` is the scheduler name, while `id` is the index of the scheduler in the pool.
-         *
-         * @param perExecutorStatsLogger
-         *          stats logger to receive per executor stats.
-         * @return scheduler builder
-         */
-        public Builder perExecutorStatsLogger(StatsLogger perExecutorStatsLogger) {
-            this.perExecutorStatsLogger = perExecutorStatsLogger;
-            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,
-                    traceTaskExecution,
-                    traceTaskExecutionWarnTimeUs,
-                    statsLogger,
-                    perExecutorStatsLogger);
-        }
-
-    }
-
-    protected final String name;
-    protected final int corePoolSize;
-    protected final MonitoredScheduledThreadPoolExecutor[] executors;
-    protected final MonitoredFuturePool[] futurePools;
-    protected final Random random;
-
-    private OrderedScheduler(String name,
-                             int corePoolSize,
-                             ThreadFactory threadFactory,
-                             boolean traceTaskExecution,
-                             long traceTaskExecutionWarnTimeUs,
-                             StatsLogger statsLogger,
-                             StatsLogger perExecutorStatsLogger) {
-        this.name = name;
-        this.corePoolSize = corePoolSize;
-        this.executors = new MonitoredScheduledThreadPoolExecutor[corePoolSize];
-        this.futurePools = new MonitoredFuturePool[corePoolSize];
-        for (int i = 0; i < corePoolSize; i++) {
-            ThreadFactory tf = new ThreadFactoryBuilder()
-                    .setNameFormat(name + "-executor-" + i + "-%d")
-                    .setThreadFactory(threadFactory)
-                    .build();
-            StatsLogger broadcastStatsLogger =
-                    BroadCastStatsLogger.masterslave(perExecutorStatsLogger.scope("executor-" + i), statsLogger);
-            executors[i] = new MonitoredScheduledThreadPoolExecutor(
-                    1, tf, broadcastStatsLogger, traceTaskExecution);
-            futurePools[i] = new MonitoredFuturePool(
-                    new ExecutorServiceFuturePool(executors[i]),
-                    broadcastStatsLogger.scope("futurepool"),
-                    traceTaskExecution,
-                    traceTaskExecutionWarnTimeUs);
-        }
-        this.random = new Random(System.currentTimeMillis());
-    }
-
-    protected MonitoredScheduledThreadPoolExecutor chooseExecutor() {
-        return corePoolSize == 1 ? executors[0] : executors[random.nextInt(corePoolSize)];
-    }
-
-    protected MonitoredScheduledThreadPoolExecutor chooseExecutor(Object key) {
-        return corePoolSize == 1 ? executors[0] :
-                executors[MathUtils.signSafeMod(Objects.hashCode(key), corePoolSize)];
-    }
-
-    protected FuturePool chooseFuturePool(Object key) {
-        return corePoolSize == 1 ? futurePools[0] :
-                futurePools[MathUtils.signSafeMod(Objects.hashCode(key), corePoolSize)];
-    }
-
-    protected FuturePool chooseFuturePool() {
-        return corePoolSize == 1 ? futurePools[0] : futurePools[random.nextInt(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 (MonitoredScheduledThreadPoolExecutor executor : executors) {
-            // Unregister gauges
-            executor.unregisterGauges();
-            executor.shutdown();
-        }
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public List<Runnable> shutdownNow() {
-        List<Runnable> runnables = new ArrayList<Runnable>();
-        for (MonitoredScheduledThreadPoolExecutor executor : executors) {
-            runnables.addAll(executor.shutdownNow());
-        }
-        return runnables;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public boolean isShutdown() {
-        for (MonitoredScheduledThreadPoolExecutor executor : executors) {
-            if (!executor.isShutdown()) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public boolean isTerminated() {
-        for (MonitoredScheduledThreadPoolExecutor executor : executors) {
-            if (!executor.isTerminated()) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public boolean awaitTermination(long timeout, TimeUnit unit)
-            throws InterruptedException {
-        for (MonitoredScheduledThreadPoolExecutor 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
-
-    /**
-     * Return a future pool used by <code>key</code>.
-     *
-     * @param key
-     *          key to order in the future pool
-     * @return future pool
-     */
-    public FuturePool getFuturePool(Object key) {
-        return chooseFuturePool(key);
-    }
-
-    /**
-     * Execute the <code>function</code> in the executor that assigned by <code>key</code>.
-     *
-     * @see com.twitter.util.Future
-     * @param key key of the <i>function</i> to run
-     * @param function function to run
-     * @return future representing the result of the <i>function</i>
-     */
-    public <T> com.twitter.util.Future<T> apply(Object key, Function0<T> function) {
-        return chooseFuturePool(key).apply(function);
-    }
-
-    /**
-     * Execute the <code>function</code> by the scheduler. It would be submitted to any executor randomly.
-     *
-     * @param function function to run
-     * @return future representing the result of the <i>function</i>
-     */
-    public <T> com.twitter.util.Future<T> apply(Function0<T> function) {
-        return chooseFuturePool().apply(function);
-    }
-
-    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);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/SafeQueueingFuturePool.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/SafeQueueingFuturePool.java
deleted file mode 100644
index a467d26..0000000
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/SafeQueueingFuturePool.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.util;
-
-import com.google.common.base.Preconditions;
-
-import com.twitter.util.Function0;
-import com.twitter.util.FuturePool;
-import com.twitter.util.Future;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import scala.runtime.BoxedUnit;
-
-/**
- * Acts like a future pool, but collects failed apply calls into a queue to be applied
- * in-order on close. This happens either in the close thread or after close is called,
- * in the last operation to complete execution.
- * Ops submitted after close will not be scheduled, so its important to ensure no more
- * ops will be applied once close has been called.
- */
-public class SafeQueueingFuturePool<T> {
-
-    static final Logger LOG = LoggerFactory.getLogger(SafeQueueingFuturePool.class);
-
-    private boolean closed;
-    private int outstanding;
-    private ConcurrentLinkedQueue<Function0<T>> queue;
-    private FuturePool orderedFuturePool;
-
-    public SafeQueueingFuturePool(FuturePool orderedFuturePool) {
-        this.closed = false;
-        this.outstanding = 0;
-        this.queue = new ConcurrentLinkedQueue<Function0<T>>();
-        this.orderedFuturePool = orderedFuturePool;
-    }
-
-    public synchronized Future<T> apply(final Function0<T> fn) {
-        Preconditions.checkNotNull(fn);
-        if (closed) {
-            return Future.exception(new RejectedExecutionException("Operation submitted to closed SafeQueueingFuturePool"));
-        }
-        ++outstanding;
-        queue.add(fn);
-        Future<T> result = orderedFuturePool.apply(new Function0<T>() {
-            @Override
-            public T apply() {
-                return queue.poll().apply();
-            }
-            @Override
-            public String toString() {
-                return fn.toString();
-            }
-        }).ensure(new Function0<BoxedUnit>() {
-            public BoxedUnit apply() {
-                if (decrOutstandingAndCheckDone()) {
-                    applyAll();
-                }
-                return null;
-            }
-        });
-        return result;
-    }
-
-    private synchronized boolean decrOutstandingAndCheckDone() {
-        return --outstanding == 0 && closed;
-    }
-
-    public void close() {
-        final boolean done;
-        synchronized (this) {
-            if (closed) {
-                return;
-            }
-            closed = true;
-            done = (outstanding == 0);
-        }
-        if (done) {
-            applyAll();
-        }
-    }
-
-    private void applyAll() {
-        if (!queue.isEmpty()) {
-            LOG.info("Applying {} items", queue.size());
-        }
-        while (!queue.isEmpty()) {
-            queue.poll().apply();
-        }
-    }
-
-    public synchronized int size() {
-        return queue.size();
-    }
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java
index 767ddf6..3697b3f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.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
@@ -25,8 +25,7 @@
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.Gauge;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.distributedlog.common.util.PermitLimiter;
 
 /**
  * Simple counter based {@link PermitLimiter}.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/TimeSequencer.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/TimeSequencer.java
index 69dfdbe..5bc5af2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/TimeSequencer.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/TimeSequencer.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog.util;
 
 import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.common.util.Sequencer;
 
 /**
  * Time based sequencer. It generated non-decreasing transaction id using milliseconds.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Transaction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Transaction.java
index 3a623dc..d90a7f8 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Transaction.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Transaction.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,7 +18,7 @@
 package org.apache.distributedlog.util;
 
 import com.google.common.annotations.Beta;
-import com.twitter.util.Future;
+import java.util.concurrent.CompletableFuture;
 
 /**
  * Util class represents a transaction
@@ -44,7 +44,7 @@
     }
 
     /**
-     * Listener on the result of an {@link org.apache.distributedlog.util.Transaction.Op}.
+     * Listener on the result of an {@link Transaction.Op}.
      *
      * @param <OpResult>
      */
@@ -77,12 +77,12 @@
 
     /**
      * Execute the current transaction. If the transaction succeed, all operations will be
-     * committed (via {@link org.apache.distributedlog.util.Transaction.Op#commit(Object)}.
+     * committed (via {@link Transaction.Op#commit(Object)}.
      * Otherwise, all operations will be aborted (via {@link Op#abort(Throwable, Object)}).
      *
      * @return future representing the result of transaction execution.
      */
-    Future<Void> execute();
+    CompletableFuture<Void> execute();
 
     /**
      * Abort current transaction. If this is called and the transaction haven't been executed by
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Utils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Utils.java
index 347f041..99a4155 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Utils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Utils.java
@@ -20,8 +20,11 @@
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import javax.annotation.Nullable;
 
@@ -29,18 +32,18 @@
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
 import com.google.common.io.Closeables;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.BKException;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
 import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.function.VoidFunctions;
+import org.apache.distributedlog.common.functions.VoidFunctions;
+import org.apache.distributedlog.io.AsyncAbortable;
 import org.apache.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
 import org.apache.bookkeeper.meta.ZkVersion;
 import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.zookeeper.ZooKeeper;
@@ -49,17 +52,13 @@
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
 
 /**
  * Basic Utilities.
  */
+@Slf4j
 public class Utils {
 
-    private static final Logger logger = LoggerFactory.getLogger(Utils.class);
-
     /**
      * Current time from some arbitrary time base in the past, counting in
      * nanoseconds, and not affected by settimeofday or similar system clock
@@ -115,16 +114,15 @@
         String path,
         byte[] data,
         final List<ACL> acl,
-        final CreateMode createMode)
-        throws ZooKeeperClient.ZooKeeperConnectionException, KeeperException, InterruptedException {
+        final CreateMode createMode) throws IOException, KeeperException {
         try {
-            Await.result(zkAsyncCreateFullPathOptimistic(zkc, path, data, acl, createMode));
+            FutureUtils.result(zkAsyncCreateFullPathOptimistic(zkc, path, data, acl, createMode));
         } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
             throw zkce;
         } catch (KeeperException ke) {
             throw ke;
         } catch (InterruptedException ie) {
-            throw ie;
+            throw new DLInterruptedException("Interrupted on create zookeeper path " + path, ie);
         } catch (RuntimeException rte) {
             throw rte;
         } catch (Exception exc) {
@@ -208,7 +206,7 @@
      * @param acl Acl of the zk path
      * @param createMode Create mode of zk path
      */
-    public static Future<BoxedUnit> zkAsyncCreateFullPathOptimistic(
+    public static CompletableFuture<Void> zkAsyncCreateFullPathOptimistic(
         final ZooKeeperClient zkc,
         final String pathToCreate,
         final byte[] data,
@@ -234,14 +232,14 @@
      * @param acl Acl of the zk path
      * @param createMode Create mode of zk path
      */
-    public static Future<BoxedUnit> zkAsyncCreateFullPathOptimistic(
+    public static CompletableFuture<Void> zkAsyncCreateFullPathOptimistic(
         final ZooKeeperClient zkc,
         final String pathToCreate,
         final Optional<String> parentPathShouldNotCreate,
         final byte[] data,
         final List<ACL> acl,
         final CreateMode createMode) {
-        final Promise<BoxedUnit> result = new Promise<BoxedUnit>();
+        final CompletableFuture<Void> result = new CompletableFuture<Void>();
 
         zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
                 data, acl, createMode, new AsyncCallback.StringCallback() {
@@ -263,13 +261,13 @@
      * @param acl Acl of the zk path
      * @param createMode Create mode of zk path
      */
-    public static Future<BoxedUnit> zkAsyncCreateFullPathOptimisticAndSetData(
+    public static CompletableFuture<Void> zkAsyncCreateFullPathOptimisticAndSetData(
         final ZooKeeperClient zkc,
         final String pathToCreate,
         final byte[] data,
         final List<ACL> acl,
         final CreateMode createMode) {
-        final Promise<BoxedUnit> result = new Promise<BoxedUnit>();
+        final CompletableFuture<Void> result = new CompletableFuture<Void>();
 
         try {
             zkc.get().setData(pathToCreate, data, -1, new AsyncCallback.StatCallback() {
@@ -291,32 +289,32 @@
                 }
             }, result);
         } catch (Exception exc) {
-            result.setException(exc);
+            result.completeExceptionally(exc);
         }
 
         return result;
     }
 
-    private static void handleKeeperExceptionCode(int rc, String pathOrMessage, Promise<BoxedUnit> result) {
+    private static void handleKeeperExceptionCode(int rc, String pathOrMessage, CompletableFuture<Void> result) {
         if (KeeperException.Code.OK.intValue() == rc) {
-            result.setValue(BoxedUnit.UNIT);
+            result.complete(null);
         } else if (DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE == rc) {
-            result.setException(new ZooKeeperClient.ZooKeeperConnectionException(pathOrMessage));
+            result.completeExceptionally(new ZooKeeperClient.ZooKeeperConnectionException(pathOrMessage));
         } else if (DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE == rc) {
-            result.setException(new DLInterruptedException(pathOrMessage));
+            result.completeExceptionally(new DLInterruptedException(pathOrMessage));
         } else {
-            result.setException(KeeperException.create(KeeperException.Code.get(rc), pathOrMessage));
+            result.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), pathOrMessage));
         }
     }
 
-    public static Future<Versioned<byte[]>> zkGetData(ZooKeeperClient zkc, String path, boolean watch) {
+    public static CompletableFuture<Versioned<byte[]>> zkGetData(ZooKeeperClient zkc, String path, boolean watch) {
         ZooKeeper zk;
         try {
             zk = zkc.get();
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
+            return FutureUtils.exception(zkException(e, path));
         } catch (InterruptedException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
+            return FutureUtils.exception(zkException(e, path));
         }
         return zkGetData(zk, path, watch);
     }
@@ -330,35 +328,35 @@
      *          whether to watch the path
      * @return future representing the versioned value. null version or null value means path doesn't exist.
      */
-    public static Future<Versioned<byte[]>> zkGetData(ZooKeeper zk, String path, boolean watch) {
-        final Promise<Versioned<byte[]>> promise = new Promise<Versioned<byte[]>>();
+    public static CompletableFuture<Versioned<byte[]>> zkGetData(ZooKeeper zk, String path, boolean watch) {
+        final CompletableFuture<Versioned<byte[]>> promise = new CompletableFuture<Versioned<byte[]>>();
         zk.getData(path, watch, new AsyncCallback.DataCallback() {
             @Override
             public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
                 if (KeeperException.Code.OK.intValue() == rc) {
                     if (null == stat) {
-                        promise.setValue(new Versioned<byte[]>(null, null));
+                        promise.complete(new Versioned<byte[]>(null, null));
                     } else {
-                        promise.setValue(new Versioned<byte[]>(data, new ZkVersion(stat.getVersion())));
+                        promise.complete(new Versioned<byte[]>(data, new ZkVersion(stat.getVersion())));
                     }
                 } else if (KeeperException.Code.NONODE.intValue() == rc) {
-                    promise.setValue(new Versioned<byte[]>(null, null));
+                    promise.complete(new Versioned<byte[]>(null, null));
                 } else {
-                    promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                    promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                 }
             }
         }, null);
         return promise;
     }
 
-    public static Future<ZkVersion> zkSetData(ZooKeeperClient zkc, String path, byte[] data, ZkVersion version) {
+    public static CompletableFuture<ZkVersion> zkSetData(ZooKeeperClient zkc, String path, byte[] data, ZkVersion version) {
         ZooKeeper zk;
         try {
             zk = zkc.get();
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
+            return FutureUtils.exception(zkException(e, path));
         } catch (InterruptedException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
+            return FutureUtils.exception(zkException(e, path));
         }
         return zkSetData(zk, path, data, version);
     }
@@ -376,31 +374,31 @@
      *          version used to set data
      * @return future representing the version after this operation.
      */
-    public static Future<ZkVersion> zkSetData(ZooKeeper zk, String path, byte[] data, ZkVersion version) {
-        final Promise<ZkVersion> promise = new Promise<ZkVersion>();
+    public static CompletableFuture<ZkVersion> zkSetData(ZooKeeper zk, String path, byte[] data, ZkVersion version) {
+        final CompletableFuture<ZkVersion> promise = new CompletableFuture<ZkVersion>();
         zk.setData(path, data, version.getZnodeVersion(), new AsyncCallback.StatCallback() {
             @Override
             public void processResult(int rc, String path, Object ctx, Stat stat) {
                 if (KeeperException.Code.OK.intValue() == rc) {
-                    promise.updateIfEmpty(new Return<ZkVersion>(new ZkVersion(stat.getVersion())));
+                    promise.complete(new ZkVersion(stat.getVersion()));
                     return;
                 }
-                promise.updateIfEmpty(new Throw<ZkVersion>(
-                        KeeperException.create(KeeperException.Code.get(rc))));
+                promise.completeExceptionally(
+                        KeeperException.create(KeeperException.Code.get(rc)));
                 return;
             }
         }, null);
         return promise;
     }
 
-    public static Future<Void> zkDelete(ZooKeeperClient zkc, String path, ZkVersion version) {
+    public static CompletableFuture<Void> zkDelete(ZooKeeperClient zkc, String path, ZkVersion version) {
         ZooKeeper zk;
         try {
             zk = zkc.get();
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
+            return FutureUtils.exception(zkException(e, path));
         } catch (InterruptedException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
+            return FutureUtils.exception(zkException(e, path));
         }
         return zkDelete(zk, path, version);
     }
@@ -416,17 +414,17 @@
      *          version used to set data
      * @return future representing the version after this operation.
      */
-    public static Future<Void> zkDelete(ZooKeeper zk, String path, ZkVersion version) {
-        final Promise<Void> promise = new Promise<Void>();
+    public static CompletableFuture<Void> zkDelete(ZooKeeper zk, String path, ZkVersion version) {
+        final CompletableFuture<Void> promise = new CompletableFuture<Void>();
         zk.delete(path, version.getZnodeVersion(), new AsyncCallback.VoidCallback() {
             @Override
             public void processResult(int rc, String path, Object ctx) {
                 if (KeeperException.Code.OK.intValue() == rc) {
-                    promise.updateIfEmpty(new Return<Void>(null));
+                    promise.complete(null);
                     return;
                 }
-                promise.updateIfEmpty(new Throw<Void>(
-                        KeeperException.create(KeeperException.Code.get(rc))));
+                promise.completeExceptionally(
+                        KeeperException.create(KeeperException.Code.get(rc)));
                 return;
             }
         }, null);
@@ -446,35 +444,35 @@
      * false if the node doesn't exist, otherwise future will throw exception
      *
      */
-    public static Future<Boolean> zkDeleteIfNotExist(ZooKeeperClient zkc, String path, ZkVersion version) {
+    public static CompletableFuture<Boolean> zkDeleteIfNotExist(ZooKeeperClient zkc, String path, ZkVersion version) {
         ZooKeeper zk;
         try {
             zk = zkc.get();
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
+            return FutureUtils.exception(zkException(e, path));
         } catch (InterruptedException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
+            return FutureUtils.exception(zkException(e, path));
         }
-        final Promise<Boolean> promise = new Promise<Boolean>();
+        final CompletableFuture<Boolean> promise = new CompletableFuture<Boolean>();
         zk.delete(path, version.getZnodeVersion(), new AsyncCallback.VoidCallback() {
             @Override
             public void processResult(int rc, String path, Object ctx) {
                 if (KeeperException.Code.OK.intValue() == rc ) {
-                    promise.setValue(true);
+                    promise.complete(true);
                 } else if (KeeperException.Code.NONODE.intValue() == rc) {
-                    promise.setValue(false);
+                    promise.complete(false);
                 } else {
-                    promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                    promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
                 }
             }
         }, null);
         return promise;
     }
 
-    public static Future<Void> asyncClose(@Nullable AsyncCloseable closeable,
+    public static CompletableFuture<Void> asyncClose(@Nullable AsyncCloseable closeable,
                                           boolean swallowIOException) {
         if (null == closeable) {
-            return Future.Void();
+            return FutureUtils.Void();
         } else if (swallowIOException) {
             return FutureUtils.ignore(closeable.asyncClose());
         } else {
@@ -548,7 +546,7 @@
         if (null == closeable) {
             return;
         }
-        FutureUtils.result(closeable.asyncClose());
+        Utils.ioResult(closeable.asyncClose());
     }
 
     /**
@@ -562,7 +560,7 @@
             return;
         }
         try {
-            FutureUtils.result(closeable.asyncClose());
+            Utils.ioResult(closeable.asyncClose());
         } catch (IOException e) {
             // no-op. the exception is swallowed.
         }
@@ -575,7 +573,7 @@
      *          closeables to close
      * @return future represents the close future
      */
-    public static Future<Void> closeSequence(ExecutorService executorService,
+    public static CompletableFuture<Void> closeSequence(ExecutorService executorService,
                                              AsyncCloseable... closeables) {
         return closeSequence(executorService, false, closeables);
     }
@@ -588,7 +586,7 @@
      * @param closeables list of closeables
      * @return future represents the close future.
      */
-    public static Future<Void> closeSequence(ExecutorService executorService,
+    public static CompletableFuture<Void> closeSequence(ExecutorService executorService,
                                              boolean ignoreCloseError,
                                              AsyncCloseable... closeables) {
         List<AsyncCloseable> closeableList = Lists.newArrayListWithExpectedSize(closeables.length);
@@ -602,7 +600,8 @@
         return FutureUtils.processList(
                 closeableList,
                 ignoreCloseError ? AsyncCloseable.CLOSE_FUNC_IGNORE_ERRORS : AsyncCloseable.CLOSE_FUNC,
-                executorService).map(VoidFunctions.LIST_TO_VOID_FUNC);
+                executorService
+        ).thenApply(VoidFunctions.LIST_TO_VOID_FUNC);
     }
 
     /**
@@ -636,4 +635,112 @@
         return path.substring(0, lastIndex);
     }
 
+    /**
+     * Convert the <i>throwable</i> to zookeeper related exceptions.
+     *
+     * @param throwable cause
+     * @param path zookeeper path
+     * @return zookeeper related exceptions
+     */
+    public static Throwable zkException(Throwable throwable, String path) {
+        if (throwable instanceof KeeperException) {
+            return new ZKException("Encountered zookeeper exception on " + path, (KeeperException) throwable);
+        } else if (throwable instanceof ZooKeeperClient.ZooKeeperConnectionException) {
+            return new ZKException("Encountered zookeeper connection loss on " + path,
+                    KeeperException.Code.CONNECTIONLOSS);
+        } else if (throwable instanceof InterruptedException) {
+            return new DLInterruptedException("Interrupted on operating " + path, throwable);
+        } else {
+            return new UnexpectedException("Encountered unexpected exception on operatiing " + path, throwable);
+        }
+    }
+
+    /**
+     * Create transmit exception from transmit result.
+     *
+     * @param transmitResult
+     *          transmit result (basically bk exception code)
+     * @return transmit exception
+     */
+    public static BKTransmitException transmitException(int transmitResult) {
+        return new BKTransmitException("Failed to write to bookkeeper; Error is ("
+            + transmitResult + ") "
+            + BKException.getMessage(transmitResult), transmitResult);
+    }
+
+    /**
+     * A specific version of {@link FutureUtils#result(CompletableFuture)} to handle known exception issues.
+     */
+    public static <T> T ioResult(CompletableFuture<T> result) throws IOException {
+        return FutureUtils.result(
+            result,
+            (cause) -> {
+                if (cause instanceof IOException) {
+                    return (IOException) cause;
+                } else if (cause instanceof KeeperException) {
+                    return new ZKException("Encountered zookeeper exception on waiting result",
+                        (KeeperException) cause);
+                } else if (cause instanceof BKException) {
+                    return new BKTransmitException("Encountered bookkeeper exception on waiting result",
+                        ((BKException) cause).getCode());
+                } else if (cause instanceof InterruptedException) {
+                    return new DLInterruptedException("Interrupted on waiting result", cause);
+                } else {
+                    return new IOException("Encountered exception on waiting result", cause);
+                }
+            });
+    }
+
+    /**
+     * A specific version of {@link FutureUtils#result(CompletableFuture, long, TimeUnit)}
+     * to handle known exception issues.
+     */
+    public static <T> T ioResult(CompletableFuture<T> result, long timeout, TimeUnit timeUnit)
+            throws IOException, TimeoutException {
+        return FutureUtils.result(
+            result,
+            (cause) -> {
+                if (cause instanceof IOException) {
+                    return (IOException) cause;
+                } else if (cause instanceof KeeperException) {
+                    return new ZKException("Encountered zookeeper exception on waiting result",
+                        (KeeperException) cause);
+                } else if (cause instanceof BKException) {
+                    return new BKTransmitException("Encountered bookkeeper exception on waiting result",
+                        ((BKException) cause).getCode());
+                } else if (cause instanceof InterruptedException) {
+                    return new DLInterruptedException("Interrupted on waiting result", cause);
+                } else {
+                    return new IOException("Encountered exception on waiting result", cause);
+                }
+            },
+            timeout,
+            timeUnit);
+    }
+
+    /**
+     * Abort async <i>abortable</i>
+     *
+     * @param abortable the {@code AsyncAbortable} object to be aborted, or null, in which case this method
+     *                  does nothing.
+     * @param swallowIOException if true, don't propagate IO exceptions thrown by the {@code abort} methods
+     * @throws IOException if {@code swallowIOException} is false and {@code abort} throws an {@code IOException}
+     */
+    public static void abort(@Nullable AsyncAbortable abortable,
+                             boolean swallowIOException)
+            throws IOException {
+        if (null == abortable) {
+            return;
+        }
+        try {
+            ioResult(abortable.asyncAbort());
+        } catch (Exception ioe) {
+            if (swallowIOException) {
+                log.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe);
+            } else {
+                throw ioe;
+            }
+        }
+    }
+
 }
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java
index 9a61c1c..1dd702f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.zk;
 
-import org.apache.distributedlog.util.PermitManager;
+import org.apache.distributedlog.common.util.PermitManager;
 import org.apache.bookkeeper.stats.Gauge;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java
index a5da9c0..aeabbfa 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java
@@ -18,18 +18,17 @@
 package org.apache.distributedlog.zk;
 
 import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
+import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.OpResult;
 
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 /**
  * ZooKeeper Transaction
  */
@@ -38,14 +37,14 @@
     private final ZooKeeperClient zkc;
     private final List<ZKOp> ops;
     private final List<org.apache.zookeeper.Op> zkOps;
-    private final Promise<Void> result;
+    private final CompletableFuture<Void> result;
     private final AtomicBoolean done = new AtomicBoolean(false);
 
     public ZKTransaction(ZooKeeperClient zkc) {
         this.zkc = zkc;
         this.ops = Lists.newArrayList();
         this.zkOps = Lists.newArrayList();
-        this.result = new Promise<Void>();
+        this.result = new CompletableFuture<Void>();
     }
 
     @Override
@@ -60,16 +59,16 @@
     }
 
     @Override
-    public Future<Void> execute() {
+    public CompletableFuture<Void> execute() {
         if (!done.compareAndSet(false, true)) {
             return result;
         }
         try {
             zkc.get().multi(zkOps, this, result);
         } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            result.setException(FutureUtils.zkException(e, ""));
+            result.completeExceptionally(Utils.zkException(e, ""));
         } catch (InterruptedException e) {
-            result.setException(FutureUtils.zkException(e, ""));
+            result.completeExceptionally(Utils.zkException(e, ""));
         }
         return result;
     }
@@ -82,7 +81,7 @@
         for (int i = 0; i < ops.size(); i++) {
             ops.get(i).abortOpResult(cause, null);
         }
-        FutureUtils.setException(result, cause);
+        FutureUtils.completeExceptionally(result, cause);
     }
 
     @Override
@@ -91,13 +90,13 @@
             for (int i = 0; i < ops.size(); i++) {
                 ops.get(i).commitOpResult(results.get(i));
             }
-            FutureUtils.setValue(result, null);
+            FutureUtils.complete(result, null);
         } else {
             KeeperException ke = KeeperException.create(KeeperException.Code.get(rc));
             for (int i = 0; i < ops.size(); i++) {
                 ops.get(i).abortOpResult(ke, null != results ? results.get(i) : null);
             }
-            FutureUtils.setException(result, ke);
+            FutureUtils.completeExceptionally(result, ke);
         }
     }
 }
diff --git a/distributedlog-core/src/main/resources/findbugsExclude.xml b/distributedlog-core/src/main/resources/findbugsExclude.xml
index 80adec8..40920db 100644
--- a/distributedlog-core/src/main/resources/findbugsExclude.xml
+++ b/distributedlog-core/src/main/resources/findbugsExclude.xml
@@ -37,4 +37,77 @@
     <Method name="run" />
     <Bug pattern="JLM_JSR166_UTILCONCURRENT_MONITORENTER" />
   </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.BKLogReadHandler$1" />
+    <Method name="onSuccess" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.BookKeeperClient$2" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.ReadUtils" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.ReadUtils$2" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.auditor.DLAuditor$2" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.auditor.DLAuditor$8" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.bk.SimpleLedgerAllocator$4" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.bk.SimpleLedgerAllocator$4$1" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.bk.SimpleLedgerAllocator$5" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.impl.acl.ZKAccessControl$4" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.impl.acl.ZKAccessControlManager$1" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.impl.acl.ZKAccessControlManager$1$1" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore$1$1" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.lock.ZKSessionLock" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.lock.ZKSessionLock$12" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.lock.ZKSessionLock$13$1" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.util.Utils" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
+  <Match>
+    <Class name="org.apache.distributedlog.util.Utils$6" />
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION" />
+  </Match>
 </FindBugsFilter>
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java b/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
index 96d2d1c..d821b05 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
@@ -17,20 +17,21 @@
  */
 package org.apache.distributedlog;
 
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.api.MetadataAccessor;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
 import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
 import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.namespace.NamespaceDriver;
 import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.util.FutureUtils;
-import org.apache.distributedlog.util.PermitLimiter;
-import org.apache.distributedlog.util.RetryPolicyUtils;
+import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.conf.ServerConfiguration;
@@ -78,7 +79,7 @@
             new HashMap<Long, LogSegmentMetadata>(children.size());
         for (String child : children) {
             LogSegmentMetadata segment =
-                    FutureUtils.result(LogSegmentMetadata.read(zkc, ledgerPath + "/" + child));
+                    Utils.ioResult(LogSegmentMetadata.read(zkc, ledgerPath + "/" + child));
             LOG.info("Read segment {} : {}", child, segment);
             segments.put(segment.getLogSegmentSequenceNumber(), segment);
         }
@@ -92,7 +93,7 @@
     public static DistributedLogManager createNewDLM(String name,
                                                      DistributedLogConfiguration conf,
                                                      URI uri) throws Exception {
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf).uri(uri).build();
         return namespace.openLog(name);
     }
@@ -102,7 +103,7 @@
                                                       URI uri) throws Exception {
         // TODO: Metadata Accessor seems to be a legacy object which only used by kestrel
         //       (we might consider deprecating this)
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf).uri(uri).build();
         return namespace.getNamespaceDriver().getMetadataAccessor(name);
     }
@@ -113,7 +114,7 @@
             List<LogSegmentMetadata> logSegmentList = dlm.getLogSegments();
             LogSegmentMetadata lastSegment = logSegmentList.get(logSegmentList.size() - 1);
             LogSegmentEntryStore entryStore = dlm.getNamespaceDriver().getLogSegmentEntryStore(NamespaceDriver.Role.READER);
-            Utils.close(FutureUtils.result(entryStore.openRandomAccessReader(lastSegment, true)));
+            Utils.close(Utils.ioResult(entryStore.openRandomAccessReader(lastSegment, true)));
         } finally {
             dlm.close();
         }
@@ -313,12 +314,12 @@
         for (int i = 0; i < controlEntries; ++i) {
             LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
             record.setControl();
-            Await.result(out.write(record));
+            Utils.ioResult(out.write(record));
             txid += txidStep;
         }
         for (int i = 0; i < userEntries; ++i) {
             LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
-            Await.result(out.write(record));
+            Utils.ioResult(out.write(record));
             txid += txidStep;
         }
         Utils.close(out);
@@ -339,7 +340,7 @@
             throws Exception {
         BKDistributedLogManager dlm = (BKDistributedLogManager) manager;
         BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
-        FutureUtils.result(writeHandler.lockHandler());
+        Utils.ioResult(writeHandler.lockHandler());
         // Start a log segment with a given ledger seq number.
         BookKeeperClient bkc = getBookKeeperClient(dlm);
         LedgerHandle lh = bkc.get().createLedger(conf.getEnsembleSize(), conf.getWriteQuorumSize(),
@@ -377,12 +378,12 @@
             for (long j = 1; j <= segmentSize; j++) {
                 writer.write(DLMTestUtil.getLogRecordInstance(txid++));
             }
-            FutureUtils.result(writer.flushAndCommit());
+            Utils.ioResult(writer.flushAndCommit());
         }
         if (completeLogSegment) {
-            FutureUtils.result(writeHandler.completeAndCloseLogSegment(writer));
+            Utils.ioResult(writeHandler.completeAndCloseLogSegment(writer));
         }
-        FutureUtils.result(writeHandler.unlockHandler());
+        Utils.ioResult(writeHandler.unlockHandler());
     }
 
     public static void injectLogSegmentWithLastDLSN(DistributedLogManager manager, DistributedLogConfiguration conf,
@@ -390,7 +391,7 @@
                                                     boolean recordWrongLastDLSN) throws Exception {
         BKDistributedLogManager dlm = (BKDistributedLogManager) manager;
         BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
-        FutureUtils.result(writeHandler.lockHandler());
+        Utils.ioResult(writeHandler.lockHandler());
         // Start a log segment with a given ledger seq number.
         BookKeeperClient bkc = getBookKeeperClient(dlm);
         LedgerHandle lh = bkc.get().createLedger(conf.getEnsembleSize(), conf.getWriteQuorumSize(),
@@ -425,14 +426,14 @@
         long txid = startTxID;
         DLSN wrongDLSN = null;
         for (long j = 1; j <= segmentSize; j++) {
-            DLSN dlsn = Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(txid++)));
+            DLSN dlsn = Utils.ioResult(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(txid++)));
             if (j == (segmentSize - 1)) {
                 wrongDLSN = dlsn;
             }
         }
         assertNotNull(wrongDLSN);
         if (recordWrongLastDLSN) {
-            FutureUtils.result(writer.asyncClose());
+            Utils.ioResult(writer.asyncClose());
             writeHandler.completeAndCloseLogSegment(
                     writeHandler.inprogressZNodeName(writer.getLogSegmentId(), writer.getStartTxId(), writer.getLogSegmentSequenceNumber()),
                     writer.getLogSegmentSequenceNumber(),
@@ -443,9 +444,9 @@
                     wrongDLSN.getEntryId(),
                     wrongDLSN.getSlotId());
         } else {
-            FutureUtils.result(writeHandler.completeAndCloseLogSegment(writer));
+            Utils.ioResult(writeHandler.completeAndCloseLogSegment(writer));
         }
-        FutureUtils.result(writeHandler.unlockHandler());
+        Utils.ioResult(writeHandler.unlockHandler());
     }
 
     public static void updateSegmentMetadata(ZooKeeperClient zkc, LogSegmentMetadata segment) throws Exception {
@@ -469,18 +470,18 @@
         return conf;
     }
 
-    public static <T> void validateFutureFailed(Future<T> future, Class exClass) {
+    public static <T> void validateFutureFailed(CompletableFuture<T> future, Class exClass) {
         try {
-            Await.result(future);
+            Utils.ioResult(future);
         } catch (Exception ex) {
             LOG.info("Expected: {} Actual: {}", exClass.getName(), ex.getClass().getName());
             assertTrue("exceptions types equal", exClass.isInstance(ex));
         }
     }
 
-    public static <T> T validateFutureSucceededAndGetResult(Future<T> future) throws Exception {
+    public static <T> T validateFutureSucceededAndGetResult(CompletableFuture<T> future) throws Exception {
         try {
-            return Await.result(future, Duration.fromSeconds(10));
+            return Utils.ioResult(future, 10, TimeUnit.SECONDS);
         } catch (Exception ex) {
             fail("unexpected exception " + ex.getClass().getName());
             throw ex;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java b/distributedlog-core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java
index 2dbef02..126d337 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java
@@ -17,10 +17,11 @@
  */
 package org.apache.distributedlog;
 
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.exceptions.LogEmptyException;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.apache.distributedlog.exceptions.LogReadException;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -123,19 +124,19 @@
         for (long i = 0; i < 3; i++) {
             BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
             for (long j = 1; j < segmentSize; j++) {
-                FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+                Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
             }
             if (recover) {
-                FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+                Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
                 TimeUnit.MILLISECONDS.sleep(300);
                 writer.abort();
                 LOG.debug("Recovering Segments");
                 BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
-                FutureUtils.result(blplm.recoverIncompleteLogSegments());
-                FutureUtils.result(blplm.asyncClose());
+                Utils.ioResult(blplm.recoverIncompleteLogSegments());
+                Utils.ioResult(blplm.asyncClose());
                 LOG.debug("Recovered Segments");
             } else {
-                FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+                Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
                 writer.closeAndComplete();
             }
             TimeUnit.MILLISECONDS.sleep(300);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java
index 922d89e..ae77522 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java
@@ -17,17 +17,14 @@
  */
 package org.apache.distributedlog;
 
-import java.io.ByteArrayInputStream;
-import java.net.URI;
 import java.util.Arrays;
 
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
 
 import org.apache.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
index b5498ba..fc1f241 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
@@ -20,8 +20,11 @@
 import java.io.ByteArrayInputStream;
 import java.net.URI;
 
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.exceptions.BKTransmitException;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.Utils;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
@@ -29,9 +32,6 @@
 import org.apache.distributedlog.exceptions.EndOfStreamException;
 import org.apache.distributedlog.exceptions.WriteException;
 import org.apache.distributedlog.util.FailpointUtils;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -90,14 +90,14 @@
         // happen very quickly. But we can test that the mechanics of the future write and api are basically
         // correct.
         AppendOnlyStreamWriter writer = dlmwriter.getAppendOnlyStreamWriter();
-        Future<DLSN> dlsnFuture = writer.write(DLMTestUtil.repeatString("abc", 11).getBytes());
+        CompletableFuture<DLSN> dlsnFuture = writer.write(DLMTestUtil.repeatString("abc", 11).getBytes());
 
         // The real problem is the fsync completes before writes are submitted, so it never takes effect.
         Thread.sleep(1000);
-        assertFalse(dlsnFuture.isDefined());
+        assertFalse(dlsnFuture.isDone());
         writer.force(false);
         // Must not throw.
-        Await.result(dlsnFuture, Duration.fromSeconds(5));
+        Utils.ioResult(dlsnFuture, 5, TimeUnit.SECONDS);
         writer.close();
         dlmwriter.close();
 
@@ -124,11 +124,11 @@
         // happen very quickly. But we can test that the mechanics of the future write and api are basically
         // correct.
         AppendOnlyStreamWriter writer = dlmwriter.getAppendOnlyStreamWriter();
-        Future<DLSN> dlsnFuture = writer.write(byteStream);
+        CompletableFuture<DLSN> dlsnFuture = writer.write(byteStream);
         Thread.sleep(100);
 
         // Write hasn't been persisted, position better not be updated.
-        assertFalse(dlsnFuture.isDefined());
+        assertFalse(dlsnFuture.isDone());
         assertEquals(0, writer.position());
         writer.force(false);
         // Position guaranteed to be accurate after writer.force().
@@ -167,7 +167,7 @@
         // Much much less than the flush time, small enough not to slow down tests too much, just
         // gives a little more confidence.
         Thread.sleep(500);
-        Future<DLSN> dlsnFuture = writer.write(byteStream);
+        CompletableFuture<DLSN> dlsnFuture = writer.write(byteStream);
         assertEquals(0, writer.position());
 
         writer.close();
@@ -188,7 +188,7 @@
         AppendOnlyStreamWriter writer = dlmwriter.getAppendOnlyStreamWriter();
         assertEquals(0, writer.position());
 
-        Await.result(writer.write(byteStream));
+        Utils.ioResult(writer.write(byteStream));
         Thread.sleep(100); // let WriteCompleteListener have time to run
         assertEquals(33, writer.position());
 
@@ -205,12 +205,12 @@
         BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
 
         URI uri = createDLMURI("/" + name);
-        FutureUtils.result(dlm.getWriterMetadataStore().getLog(uri, name, true, true));
+        Utils.ioResult(dlm.getWriterMetadataStore().getLog(uri, name, true, true));
 
         // Log exists but is empty, better not throw.
         AppendOnlyStreamWriter writer = dlm.getAppendOnlyStreamWriter();
         byte[] byteStream = DLMTestUtil.repeatString("a", 1025).getBytes();
-        Await.result(writer.write(byteStream));
+        Utils.ioResult(writer.write(byteStream));
 
         writer.close();
         dlm.close();
@@ -266,7 +266,7 @@
         BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
 
         URI uri = createDLMURI("/" + name);
-        FutureUtils.result(dlm.getWriterMetadataStore().getLog(uri, name, true, true));
+        Utils.ioResult(dlm.getWriterMetadataStore().getLog(uri, name, true, true));
 
         // Log exists but is empty, better not throw.
         AppendOnlyStreamWriter writer = dlm.getAppendOnlyStreamWriter();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java
index 139d935..6efd0c1 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java
@@ -17,14 +17,14 @@
  */
 package org.apache.distributedlog;
 
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.exceptions.LogRecordTooLongException;
 import org.apache.distributedlog.exceptions.WriteCancelledException;
 import org.apache.distributedlog.exceptions.WriteException;
 import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
+import org.apache.distributedlog.util.Utils;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
@@ -83,8 +83,8 @@
         records.add(DLMTestUtil.getLogRecordInstance(goodRecs, MAX_LOGRECORD_SIZE + 1));
         records.addAll(DLMTestUtil.getLargeLogRecordInstanceList(1, goodRecs));
 
-        Future<List<Future<DLSN>>> futureResults = writer.writeBulk(records);
-        List<Future<DLSN>> results = validateFutureSucceededAndGetResult(futureResults);
+        CompletableFuture<List<CompletableFuture<DLSN>>> futureResults = writer.writeBulk(records);
+        List<CompletableFuture<DLSN>> results = validateFutureSucceededAndGetResult(futureResults);
 
         // One future returned for each write.
         assertEquals(2*goodRecs + 1, results.size());
@@ -160,14 +160,14 @@
         // Write one record larger than max seg size. Ledger doesn't roll until next write.
         int txid = 1;
         LogRecord record = DLMTestUtil.getLogRecordInstance(txid++, 2048);
-        Future<DLSN> result = writer.write(record);
+        CompletableFuture<DLSN> result = writer.write(record);
         DLSN dlsn = validateFutureSucceededAndGetResult(result);
         assertEquals(1, dlsn.getLogSegmentSequenceNo());
 
         // Write two more via bulk. Ledger doesn't roll because there's a partial failure.
         List<LogRecord> records = null;
-        Future<List<Future<DLSN>>> futureResults = null;
-        List<Future<DLSN>> results = null;
+        CompletableFuture<List<CompletableFuture<DLSN>>> futureResults = null;
+        List<CompletableFuture<DLSN>> results = null;
         records = new ArrayList<LogRecord>(2);
         records.add(DLMTestUtil.getLogRecordInstance(txid++, 2048));
         records.add(DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1));
@@ -309,15 +309,15 @@
                                    long txIndex) throws Exception {
 
         List<LogRecord> records = DLMTestUtil.getLogRecordInstanceList(txIndex, batchSize, recSize);
-        Future<List<Future<DLSN>>> futureResults = writer.writeBulk(records);
+        CompletableFuture<List<CompletableFuture<DLSN>>> futureResults = writer.writeBulk(records);
         assertNotNull(futureResults);
-        List<Future<DLSN>> results = Await.result(futureResults, Duration.fromSeconds(10));
+        List<CompletableFuture<DLSN>> results = Utils.ioResult(futureResults, 10, TimeUnit.SECONDS);
         assertNotNull(results);
         assertEquals(results.size(), records.size());
         long prevEntryId = 0;
         DLSN lastDlsn = null;
-        for (Future<DLSN> result : results) {
-            DLSN dlsn = Await.result(result, Duration.fromSeconds(10));
+        for (CompletableFuture<DLSN> result : results) {
+            DLSN dlsn = Utils.ioResult(result, 10, TimeUnit.SECONDS);
             lastDlsn = dlsn;
 
             // If we cross a transmission boundary, slot id gets reset.
@@ -338,12 +338,12 @@
                                             long txIndex) throws Exception {
 
         List<LogRecord> records = DLMTestUtil.getLogRecordInstanceList(txIndex, batchSize, recSize);
-        Future<List<Future<DLSN>>> futureResults = writer.writeBulk(records);
+        CompletableFuture<List<CompletableFuture<DLSN>>> futureResults = writer.writeBulk(records);
         assertNotNull(futureResults);
-        List<Future<DLSN>> results = Await.result(futureResults, Duration.fromSeconds(10));
+        List<CompletableFuture<DLSN>> results = Utils.ioResult(futureResults, 10, TimeUnit.SECONDS);
         assertNotNull(results);
         assertEquals(results.size(), records.size());
-        for (Future<DLSN> result : results) {
+        for (CompletableFuture<DLSN> result : results) {
             validateFutureFailed(result, IOException.class);
         }
     }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java
index adceaf9..62ac5ef 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java
@@ -17,36 +17,34 @@
  */
 package org.apache.distributedlog;
 
-import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.exceptions.LockCancelledException;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
 import org.apache.distributedlog.lock.LockClosedException;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.subscription.SubscriptionsStore;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Await;
-import com.twitter.util.ExceptionalFunction;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -76,9 +74,9 @@
         writer.write(DLMTestUtil.getLogRecordInstance(1L));
         writer.closeAndComplete();
 
-        Future<AsyncLogReader> futureReader1 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        BKAsyncLogReader reader1 = (BKAsyncLogReader) Await.result(futureReader1);
-        LogRecordWithDLSN record = Await.result(reader1.readNext());
+        CompletableFuture<AsyncLogReader> futureReader1 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        BKAsyncLogReader reader1 = (BKAsyncLogReader) Utils.ioResult(futureReader1);
+        LogRecordWithDLSN record = Utils.ioResult(reader1.readNext());
         assertEquals(1L, record.getTransactionId());
         assertEquals(0L, record.getSequenceId());
         DLMTestUtil.verifyLogRecord(record);
@@ -89,9 +87,9 @@
         // simulate a old stream created without readlock path
         NamespaceDriver driver = dlm.getNamespaceDriver();
         ((BKNamespaceDriver) driver).getWriterZKC().get().delete(readLockPath, -1);
-        Future<AsyncLogReader> futureReader2 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        AsyncLogReader reader2 = Await.result(futureReader2);
-        record = Await.result(reader2.readNext());
+        CompletableFuture<AsyncLogReader> futureReader2 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        AsyncLogReader reader2 = Utils.ioResult(futureReader2);
+        record = Utils.ioResult(reader2.readNext());
         assertEquals(1L, record.getTransactionId());
         assertEquals(0L, record.getSequenceId());
         DLMTestUtil.verifyLogRecord(record);
@@ -107,19 +105,14 @@
 
         final CountDownLatch latch = new CountDownLatch(1);
 
-        Future<AsyncLogReader> futureReader1 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        futureReader1.flatMap(new ExceptionalFunction<AsyncLogReader, Future<Void>>() {
-            @Override
-            public Future<Void> applyE(AsyncLogReader reader) throws IOException {
-                return reader.asyncClose().map(new AbstractFunction1<Void, Void>() {
-                    @Override
-                    public Void apply(Void result) {
+        CompletableFuture<AsyncLogReader> futureReader1 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        futureReader1
+            .thenCompose(
+                reader -> reader.asyncClose()
+                    .thenApply(result -> {
                         latch.countDown();
                         return null;
-                    }
-                });
-            }
-        });
+                    }));
 
         latch.await();
         dlm.close();
@@ -133,8 +126,8 @@
         writer.write(DLMTestUtil.getLogRecordInstance(1L));
         writer.closeAndComplete();
 
-        Future<AsyncLogReader> futureReader1 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        AsyncLogReader reader1 = Await.result(futureReader1);
+        CompletableFuture<AsyncLogReader> futureReader1 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        AsyncLogReader reader1 = Utils.ioResult(futureReader1);
         reader1.readNext();
 
         final CountDownLatch acquiredLatch = new CountDownLatch(1);
@@ -142,12 +135,12 @@
         Thread acquireThread = new Thread(new Runnable() {
             @Override
             public void run() {
-                Future<AsyncLogReader> futureReader2 = null;
+                CompletableFuture<AsyncLogReader> futureReader2 = null;
                 DistributedLogManager dlm2 = null;
                 try {
                     dlm2 = createNewDLM(conf, name);
                     futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-                    AsyncLogReader reader2 = Await.result(futureReader2);
+                    AsyncLogReader reader2 = Utils.ioResult(futureReader2);
                     acquired.set(true);
                     acquiredLatch.countDown();
                 } catch (Exception ex) {
@@ -172,10 +165,10 @@
         dlm.close();
     }
 
-    int countDefined(ArrayList<Future<AsyncLogReader>> readers) {
+    int countDefined(ArrayList<CompletableFuture<AsyncLogReader>> readers) {
         int done = 0;
-        for (Future<AsyncLogReader> futureReader : readers) {
-            if (futureReader.isDefined()) {
+        for (CompletableFuture<AsyncLogReader> futureReader : readers) {
+            if (futureReader.isDone()) {
                 done++;
             }
         }
@@ -193,7 +186,7 @@
 
         int count = 5;
         final CountDownLatch acquiredLatch = new CountDownLatch(count);
-        final ArrayList<Future<AsyncLogReader>> readers = new ArrayList<Future<AsyncLogReader>>(count);
+        final ArrayList<CompletableFuture<AsyncLogReader>> readers = new ArrayList<CompletableFuture<AsyncLogReader>>(count);
         for (int i = 0; i < count; i++) {
             readers.add(null);
         }
@@ -201,7 +194,7 @@
         for (int i = 0; i < count; i++) {
             dlms[i] = createNewDLM(conf, name);
             readers.set(i, dlms[i].getAsyncLogReaderWithLock(DLSN.InitialDLSN));
-            readers.get(i).addEventListener(new FutureEventListener<AsyncLogReader>() {
+            readers.get(i).whenComplete(new FutureEventListener<AsyncLogReader>() {
                 @Override
                 public void onSuccess(AsyncLogReader reader) {
                     acquiredLatch.countDown();
@@ -232,16 +225,17 @@
         writer.closeAndComplete();
 
         DistributedLogManager dlm1 = createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        AsyncLogReader reader1 = Await.result(futureReader1);
+        CompletableFuture<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        AsyncLogReader reader1 = Utils.ioResult(futureReader1);
 
         BKDistributedLogManager dlm2 = (BKDistributedLogManager) createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        CompletableFuture<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
 
         dlm2.close();
         try {
-            Await.result(futureReader2);
+            Utils.ioResult(futureReader2);
             fail("should have thrown exception!");
+        } catch (CancellationException ce) {
         } catch (LockClosedException ex) {
         } catch (LockCancelledException ex) {
         }
@@ -256,7 +250,7 @@
         String name = runtime.getMethodName();
         URI uri = createDLMURI("/" + name);
         ensureURICreated(uri);
-        DistributedLogNamespace ns0 = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace ns0 = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .build();
@@ -266,13 +260,13 @@
         writer.write(DLMTestUtil.getLogRecordInstance(2L));
         writer.closeAndComplete();
 
-        DistributedLogNamespace ns1 = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace ns1 = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .build();
         DistributedLogManager dlm1 = ns1.openLog(name);
-        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        AsyncLogReader reader1 = Await.result(futureReader1);
+        CompletableFuture<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        AsyncLogReader reader1 = Utils.ioResult(futureReader1);
         ZooKeeperClientUtils.expireSession(((BKNamespaceDriver) ns1.getNamespaceDriver()).getWriterZKC(), zkServers, 1000);
 
         // The result of expireSession is somewhat non-deterministic with this lock.
@@ -280,12 +274,12 @@
         // the moment rather than make it deterministic we accept either result.
         boolean success = false;
         try {
-            Await.result(reader1.readNext());
+            Utils.ioResult(reader1.readNext());
             success = true;
         } catch (LockingException ex) {
         }
         if (success) {
-            Await.result(reader1.readNext());
+            Utils.ioResult(reader1.readNext());
         }
 
         Utils.close(reader1);
@@ -305,15 +299,16 @@
         writer.closeAndComplete();
 
         DistributedLogManager dlm1 = createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        AsyncLogReader reader1 = Await.result(futureReader1);
+        CompletableFuture<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        AsyncLogReader reader1 = Utils.ioResult(futureReader1);
 
         DistributedLogManager dlm2 = createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        CompletableFuture<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
         try {
-            FutureUtils.cancel(futureReader2);
-            Await.result(futureReader2);
+            futureReader2.cancel(true);
+            Utils.ioResult(futureReader2);
             fail("Should fail getting log reader as it is cancelled");
+        } catch (CancellationException ce) {
         } catch (LockClosedException ex) {
         } catch (LockCancelledException ex) {
         } catch (OwnershipAcquireFailedException oafe) {
@@ -322,7 +317,7 @@
         futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
         Utils.close(reader1);
 
-        Await.result(futureReader2);
+        Utils.ioResult(futureReader2);
 
         dlm0.close();
         dlm1.close();
@@ -339,13 +334,13 @@
         writer.closeAndComplete();
 
         DistributedLogManager dlm1 = createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        CompletableFuture<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
 
         // Must not throw or cancel or do anything bad, future already completed.
-        Await.result(futureReader1);
-        FutureUtils.cancel(futureReader1);
-        AsyncLogReader reader1 = Await.result(futureReader1);
-        Await.result(reader1.readNext());
+        Utils.ioResult(futureReader1);
+        futureReader1.cancel(true);
+        AsyncLogReader reader1 = Utils.ioResult(futureReader1);
+        Utils.ioResult(reader1.readNext());
 
         dlm0.close();
         dlm1.close();
@@ -361,13 +356,13 @@
         writer.closeAndComplete();
 
         DistributedLogManager dlm1 = createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        Future<AsyncLogReader> futureReader2 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        CompletableFuture<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        CompletableFuture<AsyncLogReader> futureReader2 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
 
         // Both use the same client id, so there's no lock conflict. Not necessarily ideal, but how the
         // system currently works.
-        Await.result(futureReader1);
-        Await.result(futureReader2);
+        Utils.ioResult(futureReader1);
+        Utils.ioResult(futureReader2);
 
         dlm0.close();
         dlm1.close();
@@ -413,7 +408,7 @@
         private void readEntries(AsyncLogReader reader) {
             try {
                 for (int i = 0; i < 300; i++) {
-                    LogRecordWithDLSN record = Await.result(reader.readNext());
+                    LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
                     currentDLSN.set(record.getDlsn());
                 }
             } catch (Exception ex) {
@@ -446,7 +441,7 @@
         localConf.setNumWorkerThreads(2);
         localConf.setLockTimeout(Long.MAX_VALUE);
 
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(localConf).uri(uri).clientId("main").build();
 
         DistributedLogManager dlm0 = namespace.openLog(name);
@@ -457,27 +452,27 @@
         AtomicReference<DLSN> currentDLSN = new AtomicReference<DLSN>(DLSN.InitialDLSN);
 
         String clientId1 = "reader1";
-        DistributedLogNamespace namespace1 = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace1 = NamespaceBuilder.newBuilder()
                 .conf(localConf).uri(uri).clientId(clientId1).build();
         DistributedLogManager dlm1 = namespace1.openLog(name);
         String clientId2 = "reader2";
-        DistributedLogNamespace namespace2 = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace2 = NamespaceBuilder.newBuilder()
                 .conf(localConf).uri(uri).clientId(clientId2).build();
         DistributedLogManager dlm2 = namespace2.openLog(name);
         String clientId3 = "reader3";
-        DistributedLogNamespace namespace3 = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace3 = NamespaceBuilder.newBuilder()
                 .conf(localConf).uri(uri).clientId(clientId3).build();
         DistributedLogManager dlm3 = namespace3.openLog(name);
 
         LOG.info("{} is opening reader on stream {}", clientId1, name);
-        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        AsyncLogReader reader1 = Await.result(futureReader1);
+        CompletableFuture<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        AsyncLogReader reader1 = Utils.ioResult(futureReader1);
         LOG.info("{} opened reader on stream {}", clientId1, name);
 
         LOG.info("{} is opening reader on stream {}", clientId2, name);
-        Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        CompletableFuture<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
         LOG.info("{} is opening reader on stream {}", clientId3, name);
-        Future<AsyncLogReader> futureReader3 = dlm3.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        CompletableFuture<AsyncLogReader> futureReader3 = dlm3.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
 
         ExecutorService executorService = Executors.newCachedThreadPool();
 
@@ -485,26 +480,26 @@
                 new ReadRecordsListener(currentDLSN, clientId2, executorService);
         ReadRecordsListener listener3 =
                 new ReadRecordsListener(currentDLSN, clientId3, executorService);
-        futureReader2.addEventListener(listener2);
-        futureReader3.addEventListener(listener3);
+        futureReader2.whenComplete(listener2);
+        futureReader3.whenComplete(listener3);
 
         // Get reader1 and start reading.
         for ( ; recordCount < 200; recordCount++) {
-            LogRecordWithDLSN record = Await.result(reader1.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader1.readNext());
             currentDLSN.set(record.getDlsn());
         }
 
         // Take a break, reader2 decides to stop waiting and cancels.
         Thread.sleep(1000);
         assertFalse(listener2.done());
-        FutureUtils.cancel(futureReader2);
+        futureReader2.cancel(true);
         listener2.getLatch().await();
         assertTrue(listener2.done());
         assertTrue(listener2.failed());
 
         // Reader1 starts reading again.
         for (; recordCount < 300; recordCount++) {
-            LogRecordWithDLSN record = Await.result(reader1.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader1.readNext());
             currentDLSN.set(record.getDlsn());
         }
 
@@ -519,12 +514,12 @@
         assertEquals(new DLSN(3, 99, 0), currentDLSN.get());
 
         try {
-            Await.result(futureReader2);
+            Utils.ioResult(futureReader2);
         } catch (Exception ex) {
             // Can't get this one to close it--the dlm will take care of it.
         }
 
-        Utils.close(Await.result(futureReader3));
+        Utils.close(Utils.ioResult(futureReader3));
 
         dlm1.close();
         namespace1.close();
@@ -553,7 +548,7 @@
         for (long i = 0; i < 3; i++) {
             BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
             for (long j = 1; j <= 10; j++) {
-                DLSN dlsn = Await.result(writer.write(DLMTestUtil.getEmptyLogRecordInstance(txid++)));
+                DLSN dlsn = Utils.ioResult(writer.write(DLMTestUtil.getEmptyLogRecordInstance(txid++)));
                 if (i == 1 && j == 1L) {
                     readDLSN = dlsn;
                 }
@@ -561,10 +556,10 @@
             writer.closeAndComplete();
         }
 
-        BKAsyncLogReader reader0 = (BKAsyncLogReader) Await.result(dlm.getAsyncLogReaderWithLock(subscriberId));
+        BKAsyncLogReader reader0 = (BKAsyncLogReader) Utils.ioResult(dlm.getAsyncLogReaderWithLock(subscriberId));
         assertEquals(DLSN.NonInclusiveLowerBound, reader0.getStartDLSN());
         long numTxns = 0;
-        LogRecordWithDLSN record = Await.result(reader0.readNext());
+        LogRecordWithDLSN record = Utils.ioResult(reader0.readNext());
         while (null != record) {
             DLMTestUtil.verifyEmptyLogRecord(record);
             ++numTxns;
@@ -574,18 +569,18 @@
             if (txid - 1 == numTxns) {
                 break;
             }
-            record = Await.result(reader0.readNext());
+            record = Utils.ioResult(reader0.readNext());
         }
         assertEquals(txid - 1, numTxns);
         Utils.close(reader0);
 
         SubscriptionsStore subscriptionsStore = dlm.getSubscriptionsStore();
-        Await.result(subscriptionsStore.advanceCommitPosition(subscriberId, readDLSN));
-        BKAsyncLogReader reader1 = (BKAsyncLogReader) Await.result(dlm.getAsyncLogReaderWithLock(subscriberId));
+        Utils.ioResult(subscriptionsStore.advanceCommitPosition(subscriberId, readDLSN));
+        BKAsyncLogReader reader1 = (BKAsyncLogReader) Utils.ioResult(dlm.getAsyncLogReaderWithLock(subscriberId));
         assertEquals(readDLSN, reader1.getStartDLSN());
         numTxns = 0;
         long startTxID =  10L;
-        record = Await.result(reader1.readNext());
+        record = Utils.ioResult(reader1.readNext());
         while (null != record) {
             DLMTestUtil.verifyEmptyLogRecord(record);
             ++numTxns;
@@ -596,7 +591,7 @@
             if (startTxID == txid - 1) {
                 break;
             }
-            record = Await.result(reader1.readNext());
+            record = Utils.ioResult(reader1.readNext());
         }
         assertEquals(txid - 1, startTxID);
         assertEquals(20, numTxns);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
index 2f5766d..5123178 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
@@ -22,6 +22,7 @@
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -30,16 +31,22 @@
 
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
-import org.apache.distributedlog.annotations.DistributedLogAnnotations;
-import org.apache.distributedlog.config.ConcurrentBaseConfiguration;
-import org.apache.distributedlog.config.ConcurrentConstConfiguration;
+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.api.LogWriter;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
+import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
 import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeperAccessor;
 import org.apache.bookkeeper.client.LedgerHandle;
@@ -63,15 +70,10 @@
 import org.apache.distributedlog.exceptions.ReadCancelledException;
 import org.apache.distributedlog.exceptions.WriteException;
 import org.apache.distributedlog.lock.DistributedLock;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.SimplePermitLimiter;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
 
 import junit.framework.Assert;
 import static com.google.common.base.Charsets.UTF_8;
@@ -111,13 +113,13 @@
         for (long i = 0; i < 3; i++) {
             final long currentLogSegmentSeqNo = i + 1;
             BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-            DLSN dlsn = Await.result(writer.writeControlRecord(new LogRecord(txid++, "control".getBytes(UTF_8))));
+            DLSN dlsn = Utils.ioResult(writer.writeControlRecord(new LogRecord(txid++, "control".getBytes(UTF_8))));
             assertEquals(currentLogSegmentSeqNo, dlsn.getLogSegmentSequenceNo());
             assertEquals(0, dlsn.getEntryId());
             assertEquals(0, dlsn.getSlotId());
             for (long j = 1; j < 10; j++) {
                 final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
-                Await.result(writer.write(record));
+                Utils.ioResult(writer.write(record));
             }
             writer.closeAndComplete();
         }
@@ -161,8 +163,8 @@
         // Write one record larger than max seg size. Ledger doesn't roll until next write.
         int txid = 1;
         LogRecord record = DLMTestUtil.getLogRecordInstance(txid++, 2048);
-        Future<DLSN> result = writer.write(record);
-        DLSN dlsn = Await.result(result, Duration.fromSeconds(10));
+        CompletableFuture<DLSN> result = writer.write(record);
+        DLSN dlsn = Utils.ioResult(result, 10, TimeUnit.SECONDS);
         assertEquals(1, dlsn.getLogSegmentSequenceNo());
 
         record = DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1);
@@ -207,8 +209,8 @@
             for (long j = 0; j < numRecordsPerLogSegment; j++) {
                 final long currentEntryId = j;
                 final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
-                Future<DLSN> dlsnFuture = writer.write(record);
-                dlsnFuture.addEventListener(new FutureEventListener<DLSN>() {
+                CompletableFuture<DLSN> dlsnFuture = writer.write(record);
+                dlsnFuture.whenComplete(new FutureEventListener<DLSN>() {
                     @Override
                     public void onSuccess(DLSN value) {
                         if(value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) {
@@ -245,7 +247,7 @@
         assertEquals("Last DLSN" + last.getDlsn() + " isn't the maximum DLSN " + maxDLSN.get(),
                 last.getDlsn(), maxDLSN.get());
         assertEquals(last.getDlsn(), dlm.getLastDLSN());
-        assertEquals(last.getDlsn(), Await.result(dlm.getLastDLSNAsync()));
+        assertEquals(last.getDlsn(), Utils.ioResult(dlm.getLastDLSNAsync()));
         DLMTestUtil.verifyLargeLogRecord(last);
 
         dlm.close();
@@ -330,8 +332,8 @@
                                  final CountDownLatch syncLatch,
                                  final CountDownLatch completionLatch,
                                  final AtomicBoolean errorsFound) {
-        Future<LogRecordWithDLSN> record = reader.readNext();
-        record.addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+        CompletableFuture<LogRecordWithDLSN> record = reader.readNext();
+        record.whenComplete(new FutureEventListener<LogRecordWithDLSN>() {
             @Override
             public void onSuccess(LogRecordWithDLSN value) {
                 try {
@@ -455,7 +457,7 @@
             if (expectedTxID == numLogSegments * numRecordsPerLogSegment) {
                 break;
             }
-            List<LogRecordWithDLSN> records = Await.result(reader.readBulk(20));
+            List<LogRecordWithDLSN> records = Utils.ioResult(reader.readBulk(20));
             LOG.info("Bulk read {} entries.", records.size());
 
             assertTrue(records.size() >= 1);
@@ -495,7 +497,7 @@
         for (long i = 0; i < 3; i++) {
             // since we batched 20 entries into single bookkeeper entry
             // we should be able to read 20 entries as a batch.
-            List<LogRecordWithDLSN> records = Await.result(reader.readBulk(20));
+            List<LogRecordWithDLSN> records = Utils.ioResult(reader.readBulk(20));
             assertEquals(20, records.size());
             for (LogRecordWithDLSN record : records) {
                 assertEquals(expectedTxID, record.getTransactionId());
@@ -531,7 +533,7 @@
                 name, asyncReader.getStreamName());
         long numTrans = 0;
         DLSN lastDLSN = DLSN.InvalidDLSN;
-        LogRecordWithDLSN record = Await.result(asyncReader.readNext());
+        LogRecordWithDLSN record = Utils.ioResult(asyncReader.readNext());
         while (null != record) {
             DLMTestUtil.verifyEmptyLogRecord(record);
             assertEquals(0, record.getDlsn().getSlotId());
@@ -541,7 +543,7 @@
             if (numTrans >= (txid - 1)) {
                 break;
             }
-            record = Await.result(asyncReader.readNext());
+            record = Utils.ioResult(asyncReader.readNext());
         }
         assertEquals((txid - 1), numTrans);
         Utils.close(asyncReader);
@@ -715,8 +717,8 @@
             for (long j = 0; j < 10; j++) {
                 final long currentEntryId = j;
                 final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
-                Future<DLSN> dlsnFuture = writer.write(record);
-                dlsnFuture.addEventListener(new WriteFutureEventListener(
+                CompletableFuture<DLSN> dlsnFuture = writer.write(record);
+                dlsnFuture.whenComplete(new WriteFutureEventListener(
                         record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
                 if (i == 0 && j == 0) {
                     boolean monotonic = LogSegmentMetadata.supportsSequenceId(logSegmentVersion);
@@ -793,8 +795,8 @@
             for (long j = 0; j < 10; j++) {
                 final long currentEntryId = j;
                 final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
-                Future<DLSN> dlsnFuture = writer.write(record);
-                dlsnFuture.addEventListener(new WriteFutureEventListener(
+                CompletableFuture<DLSN> dlsnFuture = writer.write(record);
+                dlsnFuture.whenComplete(new WriteFutureEventListener(
                         record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
             }
             writer.closeAndComplete();
@@ -835,7 +837,7 @@
 
         URI uri = createDLMURI("/" + name);
         ensureURICreated(uri);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(confLocal).uri(uri).build();
         final DistributedLogManager[] dlms = new DistributedLogManager[count];
         final TestReader[] readers = new TestReader[count];
@@ -868,8 +870,8 @@
                 final long currentEntryId = j;
                 final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
                 for (int s = 0; s < count; s++) {
-                    Future<DLSN> dlsnFuture = writers[s].write(record);
-                    dlsnFuture.addEventListener(new WriteFutureEventListener(
+                    CompletableFuture<DLSN> dlsnFuture = writers[s].write(record);
+                    dlsnFuture.whenComplete(new WriteFutureEventListener(
                             record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
                 }
             }
@@ -937,8 +939,8 @@
             for (long j = 0; j < numRecordsPerLogSegment; j++) {
                 final long currentEntryId = j;
                 final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
-                Future<DLSN> dlsnFuture = writer.write(record);
-                dlsnFuture.addEventListener(new WriteFutureEventListener(
+                CompletableFuture<DLSN> dlsnFuture = writer.write(record);
+                dlsnFuture.whenComplete(new WriteFutureEventListener(
                         record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
             }
             writer.closeAndComplete();
@@ -988,8 +990,8 @@
             for (long j = 0; j < numRecordsPerLogSegment; j++) {
                 Thread.sleep(50);
                 final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
-                Future<DLSN> dlsnFuture = writer.write(record);
-                dlsnFuture.addEventListener(new WriteFutureEventListener(
+                CompletableFuture<DLSN> dlsnFuture = writer.write(record);
+                dlsnFuture.whenComplete(new WriteFutureEventListener(
                         record, currentLogSegmentSeqNo, j, writeLatch, writeErrors, false));
                 if (i == 0 && j == 0) {
                     boolean monotonic = LogSegmentMetadata.supportsSequenceId(confLocal.getDLLedgerMetadataLayoutVersion());
@@ -1027,7 +1029,7 @@
         writer.closeAndComplete();
 
         final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        LogRecordWithDLSN record = Await.result(reader.readNext());
+        LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
         assertEquals(1L, record.getTransactionId());
         DLMTestUtil.verifyLogRecord(record);
 
@@ -1037,7 +1039,7 @@
             @Override
             public void run() {
                 try {
-                    Await.result(reader.readNext());
+                    Utils.ioResult(reader.readNext());
                 } catch (ReadCancelledException rce) {
                     receiveExpectedException.set(true);
                 } catch (Throwable t) {
@@ -1060,7 +1062,7 @@
 
         // closed reader should reject any readNext
         try {
-            Await.result(reader.readNext());
+            Utils.ioResult(reader.readNext());
             fail("Reader should reject readNext if it is closed.");
         } catch (ReadCancelledException rce) {
             // expected
@@ -1087,8 +1089,8 @@
         for (long i = 0; i < COUNT; i++) {
             Thread.sleep(1);
             final LogRecord record = DLMTestUtil.getLogRecordInstance(txid++);
-            Future<DLSN> dlsnFuture = writer.write(record);
-            dlsnFuture.addEventListener(new FutureEventListener<DLSN>() {
+            CompletableFuture<DLSN> dlsnFuture = writer.write(record);
+            dlsnFuture.whenComplete(new FutureEventListener<DLSN>() {
                 @Override
                 public void onSuccess(DLSN value) {
                     syncLatch.countDown();
@@ -1142,10 +1144,10 @@
         URI uri = createDLMURI("/" + name);
         ensureURICreated(uri);
 
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(confLocal).uri(uri).clientId("gabbagoo").build();
         DistributedLogManager dlm = namespace.openLog(name);
-        DistributedLogNamespace namespace1 = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace1 = NamespaceBuilder.newBuilder()
                 .conf(confLocal).uri(uri).clientId("tortellini").build();
         DistributedLogManager dlm1 = namespace1.openLog(name);
 
@@ -1153,12 +1155,12 @@
         BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
 
         // First write succeeds since lock isnt checked until transmit, which is scheduled
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
         writer.flushAndCommit();
 
         BKLogSegmentWriter perStreamWriter = writer.getCachedLogWriter();
         DistributedLock lock = perStreamWriter.getLock();
-        FutureUtils.result(lock.asyncClose());
+        Utils.ioResult(lock.asyncClose());
 
         // Get second writer, steal lock
         BKAsyncLogWriter writer2 = (BKAsyncLogWriter)(dlm1.startAsyncLogSegmentNonPartitioned());
@@ -1169,7 +1171,7 @@
 
             // Succeeds, kicks off scheduled flush
             Thread.sleep(100);
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
             fail("should have thrown");
         } catch (LockingException ex) {
             LOG.debug("caught exception ", ex);
@@ -1194,13 +1196,13 @@
             dlm = createNewDLM(confLocal, runtime.getMethodName());
         }
         BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(1000);
+        ArrayList<CompletableFuture<DLSN>> results = new ArrayList<CompletableFuture<DLSN>>(1000);
         for (int i = 0; i < 1000; i++) {
             results.add(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
         }
-        for (Future<DLSN> result : results) {
+        for (CompletableFuture<DLSN> result : results) {
             try {
-                Await.result(result);
+                Utils.ioResult(result);
                 if (shouldFail) {
                     fail("should fail due to no outstanding writes permitted");
                 }
@@ -1242,12 +1244,12 @@
         confLocal.setOutstandingWriteLimitDarkmode(true);
         DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
         BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(1000);
+        ArrayList<CompletableFuture<DLSN>> results = new ArrayList<CompletableFuture<DLSN>>(1000);
         for (int i = 0; i < 1000; i++) {
             results.add(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
         }
-        for (Future<DLSN> result : results) {
-            Await.result(result);
+        for (CompletableFuture<DLSN> result : results) {
+            Utils.ioResult(result);
         }
         writer.closeAndComplete();
         dlm.close();
@@ -1266,7 +1268,7 @@
 
         long txId = 1L;
         for (int i = 0; i < 5; i++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
         }
 
         BKLogSegmentWriter logWriter = writer.getCachedLogWriter();
@@ -1277,7 +1279,7 @@
                 BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
 
         try {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
             fail("Should fail write to a fenced ledger with BKTransmitException");
         } catch (BKTransmitException bkte) {
             // expected
@@ -1310,7 +1312,7 @@
 
         long txId = 1L;
         for (int i = 0; i < 5; i++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
         }
 
         BKLogSegmentWriter logWriter = writer.getCachedLogWriter();
@@ -1408,8 +1410,8 @@
         int recordCount = 0;
         try {
             while (true) {
-                Future<LogRecordWithDLSN> record = reader.readNext();
-                Await.result(record);
+                CompletableFuture<LogRecordWithDLSN> record = reader.readNext();
+                Utils.ioResult(record);
                 recordCount++;
 
                 if (recordCount >= segmentSize * numSegments) {
@@ -1465,7 +1467,7 @@
         BKAsyncLogWriter writer =
                 (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
 
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
         writer.abort();
 
         for (int i = 0; i < 2; i++) {
@@ -1548,8 +1550,8 @@
         int recordCount = 0;
         try {
             while (true) {
-                Future<LogRecordWithDLSN> record = reader.readNext();
-                Await.result(record);
+                CompletableFuture<LogRecordWithDLSN> record = reader.readNext();
+                Utils.ioResult(record);
                 if (recordCount == 0) {
                     readLatch.countDown();
                 }
@@ -1582,7 +1584,7 @@
 
         int numRecords = 10;
         for (int i = 0; i < numRecords; i++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(i)));
             assertEquals("last tx id should become " + i,
                     i, writer.getLastTxId());
         }
@@ -1612,16 +1614,16 @@
 
         int numRecords = 40;
         for (int i = 1; i <= numRecords; i++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(i)));
             assertEquals("last tx id should become " + i,
                     i, writer.getLastTxId());
         }
         LogRecord record = DLMTestUtil.getLogRecordInstance(numRecords);
         record.setControl();
-        Await.result(writer.write(record));
+        Utils.ioResult(writer.write(record));
 
         BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        record = Await.result(reader.readNext());
+        record = Utils.ioResult(reader.readNext());
         LOG.info("Read record {}", record);
         assertEquals(1L, record.getTransactionId());
 
@@ -1629,7 +1631,7 @@
         assertTrue(reader.getReadAheadReader().getNumCachedEntries() <= maxAllowedCachedRecords);
 
         for (int i = 2; i <= numRecords; i++) {
-            record = Await.result(reader.readNext());
+            record = Utils.ioResult(reader.readNext());
             LOG.info("Read record {}", record);
             assertEquals((long) i, record.getTransactionId());
             TimeUnit.MILLISECONDS.sleep(20);
@@ -1656,18 +1658,18 @@
         final int NUM_RECORDS = 10;
         int i = 1;
         for (; i <= NUM_RECORDS; i++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(i)));
             assertEquals("last tx id should become " + i,
                     i, writer.getLastTxId());
         }
 
-        Await.result(writer.markEndOfStream());
+        Utils.ioResult(writer.markEndOfStream());
 
         // Multiple end of streams are ok.
-        Await.result(writer.markEndOfStream());
+        Utils.ioResult(writer.markEndOfStream());
 
         try {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(i)));
             fail("Should have thrown");
         } catch (EndOfStreamException ex) {
         }
@@ -1675,12 +1677,12 @@
         BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
         LogRecord record = null;
         for (int j = 0; j < NUM_RECORDS; j++) {
-            record = Await.result(reader.readNext());
+            record = Utils.ioResult(reader.readNext());
             assertEquals(j+1, record.getTransactionId());
         }
 
         try {
-            record = Await.result(reader.readNext());
+            record = Utils.ioResult(reader.readNext());
             fail("Should have thrown");
         } catch (EndOfStreamException ex) {
         }
@@ -1698,9 +1700,9 @@
 
         DistributedLogManager dlm = createNewDLM(confLocal, name);
         BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        Await.result(writer.markEndOfStream());
+        Utils.ioResult(writer.markEndOfStream());
         try {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(1)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(1)));
             fail("Should have thrown");
         } catch (EndOfStreamException ex) {
         }
@@ -1708,7 +1710,7 @@
 
         BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
         try {
-            LogRecord record = Await.result(reader.readNext());
+            LogRecord record = Utils.ioResult(reader.readNext());
             fail("Should have thrown");
         } catch (EndOfStreamException ex) {
         }
@@ -1726,32 +1728,32 @@
 
         DistributedLogManager dlm = createNewDLM(confLocal, name);
         BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
         LogRecord controlRecord = DLMTestUtil.getLogRecordInstance(1L);
         controlRecord.setControl();
-        FutureUtils.result(writer.write(controlRecord));
+        Utils.ioResult(writer.write(controlRecord));
 
         BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        Future<List<LogRecordWithDLSN>> bulkReadFuture = reader.readBulk(2, Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        Future<LogRecordWithDLSN> readFuture = reader.readNext();
+        CompletableFuture<List<LogRecordWithDLSN>> bulkReadFuture = reader.readBulk(2, Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        CompletableFuture<LogRecordWithDLSN> readFuture = reader.readNext();
 
         // write another records
         for (int i = 0; i < 5; i++) {
             long txid = 2L + i;
-            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid)));
             controlRecord = DLMTestUtil.getLogRecordInstance(txid);
             controlRecord.setControl();
-            FutureUtils.result(writer.write(controlRecord));
+            Utils.ioResult(writer.write(controlRecord));
         }
 
-        List<LogRecordWithDLSN> bulkReadRecords = FutureUtils.result(bulkReadFuture);
+        List<LogRecordWithDLSN> bulkReadRecords = Utils.ioResult(bulkReadFuture);
         assertEquals(2, bulkReadRecords.size());
         assertEquals(1L, bulkReadRecords.get(0).getTransactionId());
         assertEquals(2L, bulkReadRecords.get(1).getTransactionId());
         for (LogRecordWithDLSN record : bulkReadRecords) {
             DLMTestUtil.verifyLogRecord(record);
         }
-        LogRecordWithDLSN record = FutureUtils.result(readFuture);
+        LogRecordWithDLSN record = Utils.ioResult(readFuture);
         assertEquals(3L, record.getTransactionId());
         DLMTestUtil.verifyLogRecord(record);
 
@@ -1771,16 +1773,16 @@
 
         DistributedLogManager dlm = createNewDLM(confLocal, name);
         BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
         LogRecord controlRecord = DLMTestUtil.getLogRecordInstance(1L);
         controlRecord.setControl();
-        FutureUtils.result(writer.write(controlRecord));
+        Utils.ioResult(writer.write(controlRecord));
 
         BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        Future<List<LogRecordWithDLSN>> bulkReadFuture = reader.readBulk(2, 0, TimeUnit.MILLISECONDS);
-        Future<LogRecordWithDLSN> readFuture = reader.readNext();
+        CompletableFuture<List<LogRecordWithDLSN>> bulkReadFuture = reader.readBulk(2, 0, TimeUnit.MILLISECONDS);
+        CompletableFuture<LogRecordWithDLSN> readFuture = reader.readNext();
 
-        List<LogRecordWithDLSN> bulkReadRecords = FutureUtils.result(bulkReadFuture);
+        List<LogRecordWithDLSN> bulkReadRecords = Utils.ioResult(bulkReadFuture);
         assertEquals(1, bulkReadRecords.size());
         assertEquals(1L, bulkReadRecords.get(0).getTransactionId());
         for (LogRecordWithDLSN record : bulkReadRecords) {
@@ -1790,13 +1792,13 @@
         // write another records
         for (int i = 0; i < 5; i++) {
             long txid = 2L + i;
-            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid)));
             controlRecord = DLMTestUtil.getLogRecordInstance(txid);
             controlRecord.setControl();
-            FutureUtils.result(writer.write(controlRecord));
+            Utils.ioResult(writer.write(controlRecord));
         }
 
-        LogRecordWithDLSN record = FutureUtils.result(readFuture);
+        LogRecordWithDLSN record = Utils.ioResult(readFuture);
         assertEquals(2L, record.getTransactionId());
         DLMTestUtil.verifyLogRecord(record);
 
@@ -1832,7 +1834,7 @@
         // 3 segments, 10 records each, immediate flush, batch size 1, so just the first
         // record in each ledger is discarded, for 30 - 3 = 27 records.
         for (int i = 0; i < 27; i++) {
-            LogRecordWithDLSN record = Await.result(reader.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
             assertFalse(record.getDlsn().getEntryId() % 10 == 0);
         }
 
@@ -1868,7 +1870,7 @@
             // 3 segments, 10 records each, immediate flush, batch size 1, so just the first
             // record in each ledger is discarded, for 30 - 3 = 27 records.
             for (int i = 0; i < 30; i++) {
-                LogRecordWithDLSN record = Await.result(reader.readNext());
+                LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
                 assertFalse(record.getDlsn().getEntryId() % 10 == 0);
             }
             fail("should have thrown");
@@ -1909,7 +1911,7 @@
         // 3. ranges 6-10, 7-11, 8-12, 9-13 will be bad
         // And so on, so 5 records in each 10 will be discarded, for 50 good records.
         for (int i = 0; i < 50; i++) {
-            LogRecordWithDLSN record = Await.result(reader.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
             assertFalse(record.getDlsn().getEntryId() % 10 == 0);
         }
 
@@ -1946,7 +1948,7 @@
         // 2. range 1-8 will be good, but only contain 4 records
         // And so on for the next segment, so 4 records in each segment, for 12 good records
         for (int i = 0; i < 12; i++) {
-            LogRecordWithDLSN record = Await.result(reader.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
             assertFalse(record.getDlsn().getEntryId() % 10 == 0);
         }
 
@@ -1970,13 +1972,13 @@
 
         URI uri = createDLMURI("/" + name);
         ensureURICreated(uri);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(confLocal).uri(uri).build();
 
         // use the pool
         DistributedLogManager dlm = namespace.openLog(name + "-pool");
         AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
         List<LogSegmentMetadata> segments = dlm.getLogSegments();
         assertEquals(1, segments.size());
         long ledgerId = segments.get(0).getLogSegmentId();
@@ -1995,7 +1997,7 @@
                 Optional.of(dynConf),
                 Optional.<StatsLogger>absent());
         writer = dlm.startAsyncLogSegmentNonPartitioned();
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
         segments = dlm.getLogSegments();
         assertEquals(1, segments.size());
         ledgerId = segments.get(0).getLogSegmentId();
@@ -2023,17 +2025,17 @@
 
         DistributedLogManager dlm = createNewDLM(confLocal, name);
         BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        List<Future<DLSN>> writeFutures = Lists.newArrayList();
+        List<CompletableFuture<DLSN>> writeFutures = Lists.newArrayList();
         for (int i = 0; i < 5; i++) {
             LogRecord record = DLMTestUtil.getLogRecordInstance(1L + i);
             writeFutures.add(writer.write(record));
         }
-        List<Future<DLSN>> recordSetFutures = Lists.newArrayList();
+        List<CompletableFuture<DLSN>> recordSetFutures = Lists.newArrayList();
         // write another 5 records
         final LogRecordSet.Writer recordSetWriter = LogRecordSet.newWriter(4096, CompressionCodec.Type.LZ4);
         for (int i = 0; i < 5; i++) {
             LogRecord record = DLMTestUtil.getLogRecordInstance(6L + i);
-            Promise<DLSN> writePromise = new Promise<DLSN>();
+            CompletableFuture<DLSN> writePromise = new CompletableFuture<DLSN>();
             recordSetWriter.writeRecord(ByteBuffer.wrap(record.getPayload()), writePromise);
             recordSetFutures.add(writePromise);
         }
@@ -2042,8 +2044,8 @@
         recordSetBuffer.get(data);
         LogRecord setRecord = new LogRecord(6L, data);
         setRecord.setRecordSet();
-        Future<DLSN> writeRecordSetFuture = writer.write(setRecord);
-        writeRecordSetFuture.addEventListener(new FutureEventListener<DLSN>() {
+        CompletableFuture<DLSN> writeRecordSetFuture = writer.write(setRecord);
+        writeRecordSetFuture.whenComplete(new FutureEventListener<DLSN>() {
             @Override
             public void onSuccess(DLSN dlsn) {
                 recordSetWriter.completeTransmit(
@@ -2058,20 +2060,20 @@
             }
         });
         writeFutures.add(writeRecordSetFuture);
-        FutureUtils.result(writeRecordSetFuture);
+        Utils.ioResult(writeRecordSetFuture);
         // write last 5 records
         for (int i = 0; i < 5; i++) {
             LogRecord record = DLMTestUtil.getLogRecordInstance(11L + i);
-            Future<DLSN> writeFuture = writer.write(record);
+            CompletableFuture<DLSN> writeFuture = writer.write(record);
             writeFutures.add(writeFuture);
             // make sure get log record count returns the right count
             if (i == 0) {
-                FutureUtils.result(writeFuture);
+                Utils.ioResult(writeFuture);
                 assertEquals(10, dlm.getLogRecordCount());
             }
         }
 
-        List<DLSN> writeResults = FutureUtils.result(Future.collect(writeFutures));
+        List<DLSN> writeResults = Utils.ioResult(FutureUtils.collect(writeFutures));
 
         for (int i = 0; i < 5; i++) {
             Assert.assertEquals(new DLSN(1L, i, 0L), writeResults.get(i));
@@ -2080,12 +2082,12 @@
         for (int i = 0; i < 5; i++) {
             Assert.assertEquals(new DLSN(1L, 6L + i, 0L), writeResults.get(6 + i));
         }
-        List<DLSN> recordSetWriteResults = Await.result(Future.collect(recordSetFutures));
+        List<DLSN> recordSetWriteResults = Utils.ioResult(FutureUtils.collect(recordSetFutures));
         for (int i = 0; i < 5; i++) {
             Assert.assertEquals(new DLSN(1L, 5L, i), recordSetWriteResults.get(i));
         }
 
-        FutureUtils.result(writer.flushAndCommit());
+        Utils.ioResult(writer.flushAndCommit());
 
         DistributedLogConfiguration readConf1 = new DistributedLogConfiguration();
         readConf1.addConfiguration(confLocal);
@@ -2094,7 +2096,7 @@
         DistributedLogManager readDLM1 = createNewDLM(readConf1, name);
         AsyncLogReader reader1 = readDLM1.getAsyncLogReader(DLSN.InitialDLSN);
         for (int i = 0; i < 15; i++) {
-            LogRecordWithDLSN record = FutureUtils.result(reader1.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader1.readNext());
             if (i < 5) {
                 assertEquals(new DLSN(1L, i, 0L), record.getDlsn());
                 assertEquals(1L + i, record.getTransactionId());
@@ -2118,7 +2120,7 @@
         DistributedLogManager readDLM2 = createNewDLM(readConf2, name);
         AsyncLogReader reader2 = readDLM2.getAsyncLogReader(DLSN.InitialDLSN);
         for (int i = 0; i < 11; i++) {
-            LogRecordWithDLSN record = FutureUtils.result(reader2.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader2.readNext());
             LOG.info("Read record {}", record);
             if (i < 5) {
                 assertEquals(new DLSN(1L, i, 0L), record.getDlsn());
@@ -2159,12 +2161,12 @@
         ensureURICreated(uri);
 
         DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) FutureUtils.result(dlm.openAsyncLogWriter());
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) Utils.ioResult(dlm.openAsyncLogWriter());
         writer.write(DLMTestUtil.getLogRecordInstance(1L));
 
-        AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(DLSN.InitialDLSN));
+        AsyncLogReader reader = Utils.ioResult(dlm.openAsyncLogReader(DLSN.InitialDLSN));
         try {
-            FutureUtils.result(reader.readNext());
+            Utils.ioResult(reader.readNext());
             fail("Should fail when stream is idle");
         } catch (IdleReaderException ire) {
             // expected
@@ -2191,11 +2193,11 @@
         ensureURICreated(uri);
 
         DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) FutureUtils.result(dlm.openAsyncLogWriter());
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) Utils.ioResult(dlm.openAsyncLogWriter());
         writer.write(DLMTestUtil.getLogRecordInstance(1L));
 
-        AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(DLSN.InitialDLSN));
-        LogRecordWithDLSN record = FutureUtils.result(reader.readNext());
+        AsyncLogReader reader = Utils.ioResult(dlm.openAsyncLogReader(DLSN.InitialDLSN));
+        LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
         assertEquals(1L, record.getTransactionId());
         DLMTestUtil.verifyLogRecord(record);
 
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
index dff0133..18e097f 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
@@ -23,10 +23,19 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
+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.api.LogWriter;
+import org.apache.distributedlog.api.MetadataAccessor;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.exceptions.LogEmptyException;
@@ -35,7 +44,6 @@
 import org.apache.distributedlog.impl.ZKLogSegmentMetadataStore;
 import org.apache.distributedlog.io.Abortables;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
 import org.apache.bookkeeper.client.BKException;
@@ -54,12 +62,8 @@
 import org.apache.distributedlog.metadata.LogMetadata;
 import org.apache.distributedlog.metadata.MetadataUpdater;
 import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import org.apache.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
 
 import static org.junit.Assert.*;
 import static org.junit.Assert.assertEquals;
@@ -89,7 +93,7 @@
             BKLogWriteHandler blplm = dlm.createWriteHandler(true);
             assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
                                                                 perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
+            Utils.ioResult(blplm.asyncClose());
         }
 
         LogWriter writer = dlm.startLogSegmentNonPartitioned();
@@ -129,7 +133,7 @@
         BKLogWriteHandler blplm = dlm.createWriteHandler(true);
         assertNotNull(zkc.exists(blplm.completedLedgerZNode(1, 100,
                 perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-        FutureUtils.result(blplm.asyncClose());
+        Utils.ioResult(blplm.asyncClose());
     }
 
     @Test(timeout = 60000)
@@ -167,7 +171,7 @@
             assertNotNull(
                 zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
                                                       perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
+            Utils.ioResult(blplm.asyncClose());
         }
 
         BKSyncLogWriter out = dlm.startLogSegmentNonPartitioned();
@@ -263,14 +267,14 @@
         confLocal.setWriteLockEnabled(false);
         String name = "distrlog-two-writers-lock-disabled";
         DistributedLogManager manager = createNewDLM(confLocal, name);
-        AsyncLogWriter writer1 = FutureUtils.result(manager.openAsyncLogWriter());
-        FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(1L)));
-        AsyncLogWriter writer2 = FutureUtils.result(manager.openAsyncLogWriter());
-        FutureUtils.result(writer2.write(DLMTestUtil.getLogRecordInstance(2L)));
+        AsyncLogWriter writer1 = Utils.ioResult(manager.openAsyncLogWriter());
+        Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(1L)));
+        AsyncLogWriter writer2 = Utils.ioResult(manager.openAsyncLogWriter());
+        Utils.ioResult(writer2.write(DLMTestUtil.getLogRecordInstance(2L)));
 
         // write a record to writer 1 again
         try {
-            FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(3L)));
+            Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(3L)));
             fail("Should fail writing record to writer 1 again as writer 2 took over the ownership");
         } catch (BKTransmitException bkte) {
             assertEquals(BKException.Code.LedgerFencedException, bkte.getBKResultCode());
@@ -311,7 +315,7 @@
             assertNotNull(
                 zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
                                                       perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
+            Utils.ioResult(blplm.asyncClose());
         }
         BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
         for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
@@ -394,7 +398,7 @@
             assertNotNull(
                 zkc.exists(blplm.completedLedgerZNode(txid - 1, txid - 1,
                                                       perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
+            Utils.ioResult(blplm.asyncClose());
         }
 
         BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
@@ -411,14 +415,14 @@
 
         AsyncLogReader asyncreader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
         long numTrans = 0;
-        LogRecordWithDLSN record = Await.result(asyncreader.readNext());
+        LogRecordWithDLSN record = Utils.ioResult(asyncreader.readNext());
         while (null != record) {
             DLMTestUtil.verifyLogRecord(record);
             numTrans++;
             if (numTrans >= (txid - 1)) {
                 break;
             }
-            record = Await.result(asyncreader.readNext());
+            record = Utils.ioResult(asyncreader.readNext());
         }
         assertEquals((txid - 1), numTrans);
         Utils.close(asyncreader);
@@ -459,12 +463,12 @@
         dlm.close();
 
         URI uri = createDLMURI("/" + name);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf).uri(uri).build();
         assertTrue(namespace.logExists(name));
         assertFalse(namespace.logExists("non-existent-log"));
         URI nonExistentUri = createDLMURI("/" + "non-existent-ns");
-        DistributedLogNamespace nonExistentNS = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace nonExistentNS = NamespaceBuilder.newBuilder()
                 .conf(conf).uri(nonExistentUri).build();
         assertFalse(nonExistentNS.logExists(name));
 
@@ -508,31 +512,31 @@
         SubscriptionsStore store = dlm.getSubscriptionsStore();
 
         // no data
-        assertEquals(Await.result(store.getLastCommitPosition(subscriber0)), DLSN.NonInclusiveLowerBound);
-        assertEquals(Await.result(store.getLastCommitPosition(subscriber1)), DLSN.NonInclusiveLowerBound);
-        assertEquals(Await.result(store.getLastCommitPosition(subscriber2)), DLSN.NonInclusiveLowerBound);
+        assertEquals(Utils.ioResult(store.getLastCommitPosition(subscriber0)), DLSN.NonInclusiveLowerBound);
+        assertEquals(Utils.ioResult(store.getLastCommitPosition(subscriber1)), DLSN.NonInclusiveLowerBound);
+        assertEquals(Utils.ioResult(store.getLastCommitPosition(subscriber2)), DLSN.NonInclusiveLowerBound);
         // empty
-        assertTrue(Await.result(store.getLastCommitPositions()).isEmpty());
+        assertTrue(Utils.ioResult(store.getLastCommitPositions()).isEmpty());
 
         // subscriber 0 advance
-        Await.result(store.advanceCommitPosition(subscriber0, commitPosition0));
-        assertEquals(commitPosition0, Await.result(store.getLastCommitPosition(subscriber0)));
-        Map<String, DLSN> committedPositions = Await.result(store.getLastCommitPositions());
+        Utils.ioResult(store.advanceCommitPosition(subscriber0, commitPosition0));
+        assertEquals(commitPosition0, Utils.ioResult(store.getLastCommitPosition(subscriber0)));
+        Map<String, DLSN> committedPositions = Utils.ioResult(store.getLastCommitPositions());
         assertEquals(1, committedPositions.size());
         assertEquals(commitPosition0, committedPositions.get(subscriber0));
 
         // subscriber 1 advance
-        Await.result(store.advanceCommitPosition(subscriber1, commitPosition1));
-        assertEquals(commitPosition1, Await.result(store.getLastCommitPosition(subscriber1)));
-        committedPositions = Await.result(store.getLastCommitPositions());
+        Utils.ioResult(store.advanceCommitPosition(subscriber1, commitPosition1));
+        assertEquals(commitPosition1, Utils.ioResult(store.getLastCommitPosition(subscriber1)));
+        committedPositions = Utils.ioResult(store.getLastCommitPositions());
         assertEquals(2, committedPositions.size());
         assertEquals(commitPosition0, committedPositions.get(subscriber0));
         assertEquals(commitPosition1, committedPositions.get(subscriber1));
 
         // subscriber 2 advance
-        Await.result(store.advanceCommitPosition(subscriber2, commitPosition2));
-        assertEquals(commitPosition2, Await.result(store.getLastCommitPosition(subscriber2)));
-        committedPositions = Await.result(store.getLastCommitPositions());
+        Utils.ioResult(store.advanceCommitPosition(subscriber2, commitPosition2));
+        assertEquals(commitPosition2, Utils.ioResult(store.getLastCommitPosition(subscriber2)));
+        committedPositions = Utils.ioResult(store.getLastCommitPositions());
         assertEquals(3, committedPositions.size());
         assertEquals(commitPosition0, committedPositions.get(subscriber0));
         assertEquals(commitPosition1, committedPositions.get(subscriber1));
@@ -541,11 +545,11 @@
         // subscriber 2 advance again
         DistributedLogManager newDLM = createNewDLM(conf, name);
         SubscriptionsStore newStore = newDLM.getSubscriptionsStore();
-        Await.result(newStore.advanceCommitPosition(subscriber2, commitPosition3));
+        Utils.ioResult(newStore.advanceCommitPosition(subscriber2, commitPosition3));
         newStore.close();
         newDLM.close();
 
-        committedPositions = Await.result(store.getLastCommitPositions());
+        committedPositions = Utils.ioResult(store.getLastCommitPositions());
         assertEquals(3, committedPositions.size());
         assertEquals(commitPosition0, committedPositions.get(subscriber0));
         assertEquals(commitPosition1, committedPositions.get(subscriber1));
@@ -570,13 +574,13 @@
                 BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
                 assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
                                                                     perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-                FutureUtils.result(blplm.asyncClose());
+                Utils.ioResult(blplm.asyncClose());
             } else {
                 writer.markEndOfStream();
                 BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
                 assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, DistributedLogConstants.MAX_TXID,
                                                                     perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-                FutureUtils.result(blplm.asyncClose());
+                Utils.ioResult(blplm.asyncClose());
             }
         }
         return txid;
@@ -698,8 +702,8 @@
     @Test(timeout = 60000, expected = LogRecordTooLongException.class)
     public void testMaxLogRecSize() throws Exception {
         DistributedLogManager dlm = createNewDLM(conf, "distrlog-maxlogRecSize");
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
-        FutureUtils.result(writer.write(new LogRecord(1L, DLMTestUtil.repeatString(
+        AsyncLogWriter writer = Utils.ioResult(dlm.openAsyncLogWriter());
+        Utils.ioResult(writer.write(new LogRecord(1L, DLMTestUtil.repeatString(
                                 DLMTestUtil.repeatString("abcdefgh", 256), 512).getBytes())));
     }
 
@@ -710,21 +714,21 @@
         confLocal.setOutputBufferSize(1024 * 1024);
         BKDistributedLogManager dlm =
                 createNewDLM(confLocal, "distrlog-transmissionSize");
-        AsyncLogWriter out = FutureUtils.result(dlm.openAsyncLogWriter());
+        AsyncLogWriter out = Utils.ioResult(dlm.openAsyncLogWriter());
         boolean exceptionEncountered = false;
         byte[] largePayload = new byte[(LogRecord.MAX_LOGRECORDSET_SIZE / 2) + 2];
         RAND.nextBytes(largePayload);
         try {
             LogRecord op = new LogRecord(1L, largePayload);
-            Future<DLSN> firstWriteFuture = out.write(op);
+            CompletableFuture<DLSN> firstWriteFuture = out.write(op);
             op = new LogRecord(2L, largePayload);
             // the second write will flush the first one, since we reached the maximum transmission size.
             out.write(op);
-            FutureUtils.result(firstWriteFuture);
+            Utils.ioResult(firstWriteFuture);
         } catch (LogRecordTooLongException exc) {
             exceptionEncountered = true;
         } finally {
-            FutureUtils.result(out.asyncClose());
+            Utils.ioResult(out.asyncClose());
         }
         assertFalse(exceptionEncountered);
         Abortables.abortQuietly(out);
@@ -750,7 +754,7 @@
             BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
             assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
                                                                 perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
+            Utils.ioResult(blplm.asyncClose());
         }
 
         LogReader reader = dlm.getInputStream(1);
@@ -819,7 +823,7 @@
             assertNotNull(
                 zkc.exists(blplm.completedLedgerZNode(txid - 1, txid - 1,
                                                       writer.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
+            Utils.ioResult(blplm.asyncClose());
         }
 
         BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
@@ -857,7 +861,7 @@
 
         BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
 
-        FutureUtils.result(dlm.getWriterMetadataStore().getLog(dlm.getUri(), name, true, true));
+        Utils.ioResult(dlm.getWriterMetadataStore().getLog(dlm.getUri(), name, true, true));
         dlm.registerListener(new LogSegmentListener() {
             @Override
             public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
@@ -931,7 +935,7 @@
         for (int i = 0; i < 10; i++) {
             LogRecord record = DLMTestUtil.getLogRecordInstance(txid++);
             record.setControl();
-            Await.result(writer.writeControlRecord(record));
+            Utils.ioResult(writer.writeControlRecord(record));
         }
         LOG.info("10 control records are written");
 
@@ -946,14 +950,14 @@
         LOG.info("Completed first log segment");
 
         writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
         LOG.info("Completed second log segment");
 
         LOG.info("Writing another 10 control records");
         for (int i = 1; i < 10; i++) {
             LogRecord record = DLMTestUtil.getLogRecordInstance(txid++);
             record.setControl();
-            Await.result(writer.write(record));
+            Utils.ioResult(writer.write(record));
         }
 
         assertEquals(new DLSN(2, 0, 0), dlm.getLastDLSN());
@@ -973,8 +977,8 @@
         BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
         DLMTestUtil.generateCompletedLogSegments(dlm, conf, 2, 10);
 
-        Future<Long> futureCount = dlm.getLogRecordCountAsync(DLSN.InitialDLSN);
-        Long count = Await.result(futureCount, Duration.fromSeconds(2));
+        CompletableFuture<Long> futureCount = dlm.getLogRecordCountAsync(DLSN.InitialDLSN);
+        Long count = Utils.ioResult(futureCount, 2, TimeUnit.SECONDS);
         assertEquals(20, count.longValue());
 
         writer.close();
@@ -986,7 +990,7 @@
         String baseName = testNames.getMethodName();
         String streamName = "\0blah";
         URI uri = createDLMURI("/" + baseName);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf).uri(uri).build();
 
         DistributedLogManager dlm = null;
@@ -1036,15 +1040,15 @@
             BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
             for (long j = 1; j <= 10; j++) {
                 LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
-                Future<DLSN> dlsn = writer.write(record);
+                CompletableFuture<DLSN> dlsn = writer.write(record);
 
                 if (i == 1 && j == 2) {
-                    truncDLSN = Await.result(dlsn);
+                    truncDLSN = Utils.ioResult(dlsn);
                 } else if (i == 2 && j == 3) {
-                    beyondTruncDLSN = Await.result(dlsn);
+                    beyondTruncDLSN = Utils.ioResult(dlsn);
                     beyondTruncTxId = record.getTransactionId();
                 } else if (j == 10) {
-                    Await.ready(dlsn);
+                    Utils.ioResult(dlsn);
                 }
             }
 
@@ -1065,7 +1069,7 @@
 
         MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(
                 confLocal, metadataStore);
-        FutureUtils.result(updater.setLogSegmentTruncated(segmentList.get(1L)));
+        Utils.ioResult(updater.setLogSegmentTruncated(segmentList.get(1L)));
 
         segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
                 LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
@@ -1088,7 +1092,7 @@
         }
 
         updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, metadataStore);
-        FutureUtils.result(updater.setLogSegmentActive(segmentList.get(1L)));
+        Utils.ioResult(updater.setLogSegmentActive(segmentList.get(1L)));
 
         segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
                 LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
@@ -1096,7 +1100,7 @@
         LOG.info("Read segments after marked first segment as active : {}", segmentList);
 
         updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, metadataStore);
-        FutureUtils.result(updater.setLogSegmentTruncated(segmentList.get(2L)));
+        Utils.ioResult(updater.setLogSegmentTruncated(segmentList.get(2L)));
 
         segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
                 LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
@@ -1109,7 +1113,7 @@
             boolean exceptionEncountered = false;
             try {
                 for (int i = 0; i < 3 * 10; i++) {
-                    LogRecordWithDLSN record = Await.result(reader.readNext());
+                    LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
                     DLMTestUtil.verifyLargeLogRecord(record);
                     assertEquals(expectedTxId, record.getTransactionId());
                     expectedTxId++;
@@ -1122,10 +1126,10 @@
         }
 
         updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-        FutureUtils.result(updater.setLogSegmentActive(segmentList.get(2L)));
+        Utils.ioResult(updater.setLogSegmentActive(segmentList.get(2L)));
 
         BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-        Assert.assertTrue(Await.result(writer.truncate(truncDLSN)));
+        Assert.assertTrue(Utils.ioResult(writer.truncate(truncDLSN)));
         BKLogWriteHandler handler = writer.getCachedWriteHandler();
         List<LogSegmentMetadata> cachedSegments = handler.getCachedLogSegments(LogSegmentMetadata.COMPARATOR);
         for (LogSegmentMetadata segment: cachedSegments) {
@@ -1164,7 +1168,7 @@
 
         {
             AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
-            LogRecordWithDLSN record = Await.result(reader.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
             assertTrue(record != null);
             assertEquals(truncDLSN, record.getDlsn());
             Utils.close(reader);
@@ -1190,7 +1194,7 @@
 
         {
             AsyncLogReader reader = dlm.getAsyncLogReader(beyondTruncDLSN);
-            LogRecordWithDLSN record = Await.result(reader.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
             assertTrue(record != null);
             assertEquals(beyondTruncDLSN, record.getDlsn());
             Utils.close(reader);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
index e0f2bab..2078a88 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
@@ -28,14 +28,17 @@
 import java.util.concurrent.atomic.AtomicReference;
 
 import com.google.common.collect.Sets;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.api.LogWriter;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.callback.NamespaceListener;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.InvalidStreamNameException;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.util.DLUtils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -95,8 +98,8 @@
         DistributedLogConfiguration newConf = new DistributedLogConfiguration();
         newConf.addConfiguration(conf);
         newConf.setCreateStreamIfNotExists(false);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(newConf).uri(uri).build();
+        Namespace namespace = NamespaceBuilder.newBuilder()
+            .conf(newConf).uri(uri).build();
         DistributedLogManager dlm = namespace.openLog(logName);
         LogWriter writer;
         try {
@@ -118,7 +121,7 @@
         newConf.addConfiguration(conf);
         newConf.setCreateStreamIfNotExists(false);
         String streamName = "test-stream";
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(newConf).uri(uri).build();
         DistributedLogManager dlm = namespace.openLog(streamName);
         LogWriter writer;
@@ -148,7 +151,7 @@
 
         URI uri = createDLMURI("/" + runtime.getMethodName());
 
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf).uri(uri).build();
 
         try {
@@ -225,7 +228,7 @@
     public void testNamespaceListener() throws Exception {
         URI uri = createDLMURI("/" + runtime.getMethodName());
         zooKeeperClient.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf).uri(uri).build();
         final CountDownLatch[] latches = new CountDownLatch[3];
         for (int i = 0; i < 3; i++) {
@@ -268,7 +271,7 @@
         newConf.addConfiguration(conf);
         newConf.setCreateStreamIfNotExists(true);
         newConf.setZkAclId(un);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(newConf).uri(uri).build();
         DistributedLogManager dlm = namespace.openLog(streamName);
         LogWriter writer = dlm.startLogSegmentNonPartitioned();
@@ -400,7 +403,7 @@
     @Test(timeout = 60000)
     public void testUseNamespaceAfterCloseShouldFailFast() throws Exception {
         URI uri = createDLMURI("/" + runtime.getMethodName());
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
             .conf(conf)
             .uri(uri)
             .build();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
index 854fd61..4915137 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
@@ -18,20 +18,21 @@
 package org.apache.distributedlog;
 
 import com.google.common.base.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogWriter;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
 import org.apache.distributedlog.logsegment.LogSegmentFilter;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import com.twitter.util.Await;
 
 import java.util.List;
 import java.util.ArrayList;
 import java.util.concurrent.TimeUnit;
 
-import com.twitter.util.TimeoutException;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
@@ -78,21 +79,21 @@
         DistributedLogManager dlm1 = createNewDLM(confLocal, dlName);
         long txid = 1;
 
-        ArrayList<Future<DLSN>> futures = new ArrayList<Future<DLSN>>(numEntriesPerSegment);
+        ArrayList<CompletableFuture<DLSN>> futures = new ArrayList<CompletableFuture<DLSN>>(numEntriesPerSegment);
         AsyncLogWriter out = dlm1.startAsyncLogSegmentNonPartitioned();
         for (int eid = 0; eid < numEntriesPerSegment; ++eid) {
             futures.add(out.write(DLMTestUtil.getLogRecordInstance(txid)));
             ++txid;
         }
-        FutureUtils.result(Future.collect(futures));
+        Utils.ioResult(FutureUtils.collect(futures));
         // commit
         LogRecord controlRecord = new LogRecord(txid, DistributedLogConstants.CONTROL_RECORD_CONTENT);
         controlRecord.setControl();
-        FutureUtils.result(out.write(controlRecord));
+        Utils.ioResult(out.write(controlRecord));
 
         DLSN last = dlm1.getLastDLSN();
         assertEquals(new DLSN(1,9,0), last);
-        DLSN first = Await.result(dlm1.getFirstDLSNAsync());
+        DLSN first = Utils.ioResult(dlm1.getFirstDLSNAsync());
         assertEquals(new DLSN(1,0,0), first);
         Utils.close(out);
     }
@@ -102,9 +103,9 @@
         String dlName = runtime.getMethodName();
         BKDistributedLogManager dlm = createNewDLM(conf, dlName);
         BKLogReadHandler readHandler = dlm.createReadHandler();
-        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
+        CompletableFuture<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
         try {
-            Await.result(futureRecord);
+            Utils.ioResult(futureRecord);
             fail("should have thrown exception");
         } catch (LogNotFoundException ex) {
         }
@@ -116,9 +117,9 @@
         BKDistributedLogManager dlm = createNewDLM(conf, dlName);
         DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 3);
         BKLogReadHandler readHandler = dlm.createReadHandler();
-        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
+        CompletableFuture<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
         try {
-            LogRecordWithDLSN record = Await.result(futureRecord);
+            LogRecordWithDLSN record = Utils.ioResult(futureRecord);
             assertEquals(new DLSN(1, 0, 0), record.getDlsn());
         } catch (Exception ex) {
             fail("should not have thrown exception: " + ex);
@@ -133,11 +134,11 @@
         BKLogReadHandler readHandler =
             ((BKDistributedLogManager) dlm).createReadHandler();
         AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-        Future<Boolean> futureSuccess = writer.truncate(new DLSN(2, 0, 0));
-        Boolean success = Await.result(futureSuccess);
+        CompletableFuture<Boolean> futureSuccess = writer.truncate(new DLSN(2, 0, 0));
+        Boolean success = Utils.ioResult(futureSuccess);
         assertTrue(success);
-        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
-        LogRecordWithDLSN record = Await.result(futureRecord);
+        CompletableFuture<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
+        LogRecordWithDLSN record = Utils.ioResult(futureRecord);
         assertEquals(new DLSN(2, 0, 0), record.getDlsn());
     }
 
@@ -151,11 +152,11 @@
         AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
 
         // Only truncates at ledger boundary.
-        Future<Boolean> futureSuccess = writer.truncate(new DLSN(2, 5, 0));
-        Boolean success = Await.result(futureSuccess);
+        CompletableFuture<Boolean> futureSuccess = writer.truncate(new DLSN(2, 5, 0));
+        Boolean success = Utils.ioResult(futureSuccess);
         assertTrue(success);
-        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
-        LogRecordWithDLSN record = Await.result(futureRecord);
+        CompletableFuture<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
+        LogRecordWithDLSN record = Utils.ioResult(futureRecord);
         assertEquals(new DLSN(2, 0, 0), record.getDlsn());
     }
 
@@ -164,10 +165,10 @@
         String dlName = runtime.getMethodName();
         DistributedLogManager dlm = createNewDLM(conf, dlName);
         BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
+        CompletableFuture<Long> count = null;
         count = readHandler.asyncGetLogRecordCount(DLSN.InitialDLSN);
         try {
-            Await.result(count);
+            Utils.ioResult(count);
             fail("log is empty, should have returned log empty ex");
         } catch (LogNotFoundException ex) {
         }
@@ -179,9 +180,9 @@
         prepareLogSegmentsNonPartitioned(dlName, 11, 3);
         DistributedLogManager dlm = createNewDLM(conf, dlName);
         BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
+        CompletableFuture<Long> count = null;
         count = readHandler.asyncGetLogRecordCount(DLSN.InitialDLSN);
-        assertEquals(33, Await.result(count).longValue());
+        assertEquals(33, Utils.ioResult(count).longValue());
     }
 
     @Test(timeout = 60000)
@@ -190,11 +191,11 @@
         prepareLogSegmentsNonPartitioned(dlName, 11, 3);
         DistributedLogManager dlm = createNewDLM(conf, dlName);
         BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
+        CompletableFuture<Long> count = null;
         count = readHandler.asyncGetLogRecordCount(new DLSN(2, 0, 0));
-        assertEquals(30, Await.result(count).longValue());
+        assertEquals(30, Utils.ioResult(count).longValue());
         count = readHandler.asyncGetLogRecordCount(new DLSN(3, 0, 0));
-        assertEquals(27, Await.result(count).longValue());
+        assertEquals(27, Utils.ioResult(count).longValue());
     }
 
     @Test(timeout = 60000)
@@ -203,9 +204,9 @@
         prepareLogSegmentsNonPartitioned(dlName, 11, 3);
         DistributedLogManager dlm = createNewDLM(conf, dlName);
         BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
+        CompletableFuture<Long> count = null;
         count = readHandler.asyncGetLogRecordCount(new DLSN(12, 0, 0));
-        assertEquals(0, Await.result(count).longValue());
+        assertEquals(0, Utils.ioResult(count).longValue());
     }
 
     @Test(timeout = 60000)
@@ -214,9 +215,9 @@
         prepareLogSegmentsNonPartitioned(dlName, 11, 3);
         DistributedLogManager dlm = createNewDLM(conf, dlName);
         BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
+        CompletableFuture<Long> count = null;
         count = readHandler.asyncGetLogRecordCount(new DLSN(11, 2, 0));
-        assertEquals(1, Await.result(count).longValue());
+        assertEquals(1, Utils.ioResult(count).longValue());
     }
 
     @Test(timeout = 60000)
@@ -225,11 +226,11 @@
         prepareLogSegmentsNonPartitioned(dlName, 5, 10);
         DistributedLogManager dlm = createNewDLM(conf, dlName);
         BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
+        CompletableFuture<Long> count = null;
         count = readHandler.asyncGetLogRecordCount(new DLSN(3, 5, 0));
-        assertEquals(25, Await.result(count).longValue());
+        assertEquals(25, Utils.ioResult(count).longValue());
         count = readHandler.asyncGetLogRecordCount(new DLSN(2, 5, 0));
-        assertEquals(35, Await.result(count).longValue());
+        assertEquals(35, Utils.ioResult(count).longValue());
     }
 
     @Test(timeout = 60000)
@@ -239,9 +240,9 @@
         txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 5, 5, txid);
         txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 0, 10, txid);
         BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
+        CompletableFuture<Long> count = null;
         count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
-        assertEquals(15, Await.result(count).longValue());
+        assertEquals(15, Utils.ioResult(count).longValue());
     }
 
     @Test(timeout = 60000)
@@ -251,9 +252,9 @@
         txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 5, 0, txid);
         txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 10, 0, txid);
         BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
+        CompletableFuture<Long> count = null;
         count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
-        assertEquals(0, Await.result(count).longValue());
+        assertEquals(0, Utils.ioResult(count).longValue());
     }
 
     @Test(timeout = 60000)
@@ -264,12 +265,12 @@
         AsyncLogWriter out = bkdlm.startAsyncLogSegmentNonPartitioned();
         int txid = 1;
 
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
 
         BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        List<LogSegmentMetadata> ledgerList = FutureUtils.result(
+        List<LogSegmentMetadata> ledgerList = Utils.ioResult(
                 readHandler.readLogSegmentsFromStore(
                         LogSegmentMetadata.COMPARATOR,
                         LogSegmentFilter.DEFAULT_FILTER,
@@ -279,9 +280,9 @@
         assertEquals(1, ledgerList.size());
         assertTrue(ledgerList.get(0).isInProgress());
 
-        Future<Long> count = null;
+        CompletableFuture<Long> count = null;
         count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
-        assertEquals(2, Await.result(count).longValue());
+        assertEquals(2, Utils.ioResult(count).longValue());
 
         Utils.close(out);
     }
@@ -294,12 +295,12 @@
         long txid = 1;
         txid += DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, txid);
         AsyncLogWriter out = bkdlm.startAsyncLogSegmentNonPartitioned();
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
 
         BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        List<LogSegmentMetadata> ledgerList = FutureUtils.result(
+        List<LogSegmentMetadata> ledgerList = Utils.ioResult(
                 readHandler.readLogSegmentsFromStore(
                         LogSegmentMetadata.COMPARATOR,
                         LogSegmentFilter.DEFAULT_FILTER,
@@ -309,9 +310,9 @@
         assertFalse(ledgerList.get(0).isInProgress());
         assertTrue(ledgerList.get(1).isInProgress());
 
-        Future<Long> count = null;
+        CompletableFuture<Long> count = null;
         count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
-        assertEquals(7, Await.result(count).longValue());
+        assertEquals(7, Utils.ioResult(count).longValue());
 
         Utils.close(out);
     }
@@ -322,14 +323,14 @@
         BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
         BKLogReadHandler readHandler = bkdlm.createReadHandler();
         try {
-            Await.result(readHandler.lockStream());
+            Utils.ioResult(readHandler.lockStream());
             fail("Should fail lock stream if log not found");
         } catch (LogNotFoundException ex) {
         }
 
         BKLogReadHandler subscriberReadHandler = bkdlm.createReadHandler(Optional.of("test-subscriber"));
         try {
-            Await.result(subscriberReadHandler.lockStream());
+            Utils.ioResult(subscriberReadHandler.lockStream());
             fail("Subscriber should fail lock stream if log not found");
         } catch (LogNotFoundException ex) {
             // expected
@@ -342,17 +343,17 @@
         BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1);
         BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        Await.result(readHandler.lockStream());
+        Utils.ioResult(readHandler.lockStream());
 
         // two subscribers could lock stream in parallel
         BKDistributedLogManager bkdlm10 = createNewDLM(conf, streamName);
         BKLogReadHandler s10Handler =
                 bkdlm10.createReadHandler(Optional.of("s1"));
-        Await.result(s10Handler.lockStream());
+        Utils.ioResult(s10Handler.lockStream());
         BKDistributedLogManager bkdlm20 = createNewDLM(conf, streamName);
         BKLogReadHandler s20Handler =
                 bkdlm20.createReadHandler(Optional.of("s2"));
-        Await.result(s20Handler.lockStream());
+        Utils.ioResult(s20Handler.lockStream());
 
         readHandler.asyncClose();
         bkdlm.close();
@@ -368,19 +369,19 @@
         BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1);
         BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        Await.result(readHandler.lockStream());
+        Utils.ioResult(readHandler.lockStream());
 
         // same subscrbiers couldn't lock stream in parallel
         BKDistributedLogManager bkdlm10 = createNewDLM(conf, streamName);
         BKLogReadHandler s10Handler =
                 bkdlm10.createReadHandler(Optional.of("s1"));
-        Await.result(s10Handler.lockStream());
+        Utils.ioResult(s10Handler.lockStream());
 
         BKDistributedLogManager bkdlm11 = createNewDLM(conf, streamName);
         BKLogReadHandler s11Handler =
                 bkdlm11.createReadHandler(Optional.of("s1"));
         try {
-            Await.result(s11Handler.lockStream(), Duration.apply(10000, TimeUnit.MILLISECONDS));
+            Utils.ioResult(s11Handler.lockStream(), 10000, TimeUnit.MILLISECONDS);
             fail("Should fail lock stream using same subscriber id");
         } catch (OwnershipAcquireFailedException oafe) {
             // expected
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
index ccbfc44..4ad0bc0 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
@@ -17,24 +17,23 @@
  */
 package org.apache.distributedlog;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.exceptions.BKTransmitException;
 import org.apache.distributedlog.exceptions.EndOfStreamException;
 import org.apache.distributedlog.exceptions.WriteCancelledException;
 import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
 import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
-import org.apache.distributedlog.io.Abortables;
 import org.apache.distributedlog.lock.SessionLockFactory;
 import org.apache.distributedlog.lock.ZKDistributedLock;
 import org.apache.distributedlog.lock.ZKSessionLockFactory;
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
 import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.PermitLimiter;
+import org.apache.distributedlog.common.util.PermitLimiter;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
@@ -42,14 +41,12 @@
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
-import scala.runtime.AbstractFunction0;
 
 import java.io.IOException;
 import java.net.URI;
@@ -129,9 +126,9 @@
                                          boolean acquireLock)
             throws Exception {
         try {
-            Await.result(Utils.zkAsyncCreateFullPathOptimistic(zkClient, path, new byte[0],
+            Utils.ioResult(Utils.zkAsyncCreateFullPathOptimistic(zkClient, path, new byte[0],
                     ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT));
-        } catch (KeeperException.NodeExistsException nee) {
+        } catch (ZKException zke) {
             // node already exists
         }
         SessionLockFactory lockFactory = new ZKSessionLockFactory(
@@ -150,7 +147,7 @@
                 Long.MAX_VALUE,
                 NullStatsLogger.INSTANCE);
         if (acquireLock) {
-            return FutureUtils.result(lock.asyncAcquire());
+            return Utils.ioResult(lock.asyncAcquire());
         } else {
             return lock;
         }
@@ -158,9 +155,9 @@
 
     private void closeWriterAndLock(BKLogSegmentWriter writer,
                                     ZKDistributedLock lock)
-            throws IOException {
+            throws Exception {
         try {
-            FutureUtils.result(writer.asyncClose());
+            Utils.ioResult(writer.asyncClose());
         } finally {
             Utils.closeQuietly(lock);
         }
@@ -170,7 +167,7 @@
                                     ZKDistributedLock lock)
             throws IOException {
         try {
-            Abortables.abort(writer, false);
+            Utils.abort(writer, false);
         } finally {
             Utils.closeQuietly(lock);
         }
@@ -231,10 +228,10 @@
                 createLogSegmentWriter(confLocal, 0L, -1L, lock);
         // Use another lock to wait for writer releasing lock
         ZKDistributedLock lock0 = createLock("/test/lock-" + runtime.getMethodName(), zkc0, false);
-        Future<ZKDistributedLock> lockFuture0 = lock0.asyncAcquire();
+        CompletableFuture<ZKDistributedLock> lockFuture0 = lock0.asyncAcquire();
         // add 10 records
         int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        List<CompletableFuture<DLSN>> futureList = new ArrayList<CompletableFuture<DLSN>>(numRecords);
         for (int i = 0; i < numRecords; i++) {
             futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
         }
@@ -248,7 +245,7 @@
                 10, writer.getPositionWithinLogSegment());
         // close the writer should flush buffered data and release lock
         closeWriterAndLock(writer, lock);
-        Await.result(lockFuture0);
+        Utils.ioResult(lockFuture0);
         lock0.checkOwnership();
         assertEquals("Last tx id should still be " + (numRecords - 1),
                 numRecords - 1, writer.getLastTxId());
@@ -256,7 +253,7 @@
                 numRecords - 1, writer.getLastTxIdAcknowledged());
         assertEquals("Position should still be " + numRecords,
                 10, writer.getPositionWithinLogSegment());
-        List<DLSN> dlsns = Await.result(Future.collect(futureList));
+        List<DLSN> dlsns = Utils.ioResult(FutureUtils.collect(futureList));
         assertEquals("All records should be written",
                 numRecords, dlsns.size());
         for (int i = 0; i < numRecords; i++) {
@@ -293,10 +290,10 @@
                 createLogSegmentWriter(confLocal, 0L, -1L, lock);
         // Use another lock to wait for writer releasing lock
         ZKDistributedLock lock0 = createLock("/test/lock-" + runtime.getMethodName(), zkc0, false);
-        Future<ZKDistributedLock> lockFuture0 = lock0.asyncAcquire();
+        CompletableFuture<ZKDistributedLock> lockFuture0 = lock0.asyncAcquire();
         // add 10 records
         int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        List<CompletableFuture<DLSN>> futureList = new ArrayList<CompletableFuture<DLSN>>(numRecords);
         for (int i = 0; i < numRecords; i++) {
             futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
         }
@@ -310,7 +307,7 @@
                 10, writer.getPositionWithinLogSegment());
         // close the writer should flush buffered data and release lock
         abortWriterAndLock(writer, lock);
-        Await.result(lockFuture0);
+        Utils.ioResult(lockFuture0);
         lock0.checkOwnership();
         assertEquals("Last tx id should still be " + (numRecords - 1),
                 numRecords - 1, writer.getLastTxId());
@@ -323,7 +320,7 @@
 
         for (int i = 0; i < numRecords; i++) {
             try {
-                Await.result(futureList.get(i));
+                Utils.ioResult(futureList.get(i));
                 fail("Should be aborted record " + i + " with transmit exception");
             } catch (WriteCancelledException wce) {
                 assertTrue("Record " + i + " should be aborted because of ledger fenced",
@@ -369,10 +366,10 @@
                 createLogSegmentWriter(confLocal, 0L, -1L, lock);
         // Use another lock to wait for writer releasing lock
         ZKDistributedLock lock0 = createLock("/test/lock-" + runtime.getMethodName(), zkc0, false);
-        Future<ZKDistributedLock> lockFuture0 = lock0.asyncAcquire();
+        CompletableFuture<ZKDistributedLock> lockFuture0 = lock0.asyncAcquire();
         // add 10 records
         int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        List<CompletableFuture<DLSN>> futureList = new ArrayList<CompletableFuture<DLSN>>(numRecords);
         for (int i = 0; i < numRecords; i++) {
             futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
         }
@@ -393,7 +390,7 @@
             assertEquals("Inconsistent rc is thrown",
                     rcToFailComplete, bkte.getBKResultCode());
         }
-        Await.result(lockFuture0);
+        Utils.ioResult(lockFuture0);
         lock0.checkOwnership();
         assertEquals("Last tx id should still be " + (numRecords - 1),
                 numRecords - 1, writer.getLastTxId());
@@ -406,7 +403,7 @@
 
         for (int i = 0; i < numRecords; i++) {
             try {
-                Await.result(futureList.get(i));
+                Utils.ioResult(futureList.get(i));
                 fail("Should be aborted record " + i + " with transmit exception");
             } catch (WriteCancelledException wce) {
                 assertTrue("Record " + i + " should be aborted because of ledger fenced",
@@ -441,10 +438,10 @@
                 createLogSegmentWriter(confLocal, 0L, -1L, lock);
         // Use another lock to wait for writer releasing lock
         ZKDistributedLock lock0 = createLock("/test/lock-" + runtime.getMethodName(), zkc0, false);
-        Future<ZKDistributedLock> lockFuture0 = lock0.asyncAcquire();
+        CompletableFuture<ZKDistributedLock> lockFuture0 = lock0.asyncAcquire();
         // add 10 records
         int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        List<CompletableFuture<DLSN>> futureList = new ArrayList<CompletableFuture<DLSN>>(numRecords);
         for (int i = 0; i < numRecords; i++) {
             futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
         }
@@ -467,7 +464,7 @@
                     BKException.Code.LedgerFencedException, bkte.getBKResultCode());
         }
 
-        Await.result(lockFuture0);
+        Utils.ioResult(lockFuture0);
         lock0.checkOwnership();
 
         assertEquals("Last tx id should still be " + (numRecords - 1),
@@ -481,7 +478,7 @@
 
         for (int i = 0; i < numRecords; i++) {
             try {
-                Await.result(futureList.get(i));
+                Utils.ioResult(futureList.get(i));
                 fail("Should be aborted record " + i + " with transmit exception");
             } catch (BKTransmitException bkte) {
                 assertEquals("Record " + i + " should be aborted",
@@ -513,10 +510,10 @@
                 createLogSegmentWriter(confLocal, 0L, -1L, lock);
         // Use another lock to wait for writer releasing lock
         ZKDistributedLock lock0 = createLock("/test/lock-" + runtime.getMethodName(), zkc0, false);
-        Future<ZKDistributedLock> lockFuture0 = lock0.asyncAcquire();
+        CompletableFuture<ZKDistributedLock> lockFuture0 = lock0.asyncAcquire();
         // add 10 records
         int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        List<CompletableFuture<DLSN>> futureList = new ArrayList<CompletableFuture<DLSN>>(numRecords);
         for (int i = 0; i < numRecords; i++) {
             futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
         }
@@ -530,23 +527,19 @@
                 numRecords, writer.getPositionWithinLogSegment());
 
         final CountDownLatch deferLatch = new CountDownLatch(1);
-        writer.getFuturePool().apply(new AbstractFunction0<Object>() {
-            @Override
-            public Object apply() {
-                try {
-                    deferLatch.await();
-                } catch (InterruptedException e) {
-                    LOG.warn("Interrupted on deferring completion : ", e);
-                }
-                return null;
+        writer.getFuturePool().submit(() -> {
+            try {
+                deferLatch.await();
+            } catch (InterruptedException e) {
+                LOG.warn("Interrupted on deferring completion : ", e);
             }
         });
 
         // transmit the buffered data
-        FutureUtils.result(writer.flush());
+        Utils.ioResult(writer.flush());
 
         // add another 10 records
-        List<Future<DLSN>> anotherFutureList = new ArrayList<Future<DLSN>>(numRecords);
+        List<CompletableFuture<DLSN>> anotherFutureList = new ArrayList<CompletableFuture<DLSN>>(numRecords);
         for (int i = numRecords; i < 2 * numRecords; i++) {
             anotherFutureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
         }
@@ -562,13 +555,13 @@
         // abort the writer: it waits for outstanding transmits and abort buffered data
         abortWriterAndLock(writer, lock);
 
-        Await.result(lockFuture0);
+        Utils.ioResult(lockFuture0);
         lock0.checkOwnership();
 
         // release defer latch so completion would go through
         deferLatch.countDown();
 
-        List<DLSN> dlsns = Await.result(Future.collect(futureList));
+        List<DLSN> dlsns = Utils.ioResult(FutureUtils.collect(futureList));
         assertEquals("All first 10 records should be written",
                 numRecords, dlsns.size());
         for (int i = 0; i < numRecords; i++) {
@@ -582,7 +575,7 @@
         }
         for (int i = 0; i < numRecords; i++) {
             try {
-                Await.result(anotherFutureList.get(i));
+                Utils.ioResult(anotherFutureList.get(i));
                 fail("Should be aborted record " + (numRecords + i) + " with transmit exception");
             } catch (WriteCancelledException wce) {
                 // writes should be cancelled.
@@ -622,7 +615,7 @@
                 createLogSegmentWriter(confLocal, 0L, -1L, lock);
         // add 10 records
         int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        List<CompletableFuture<DLSN>> futureList = new ArrayList<CompletableFuture<DLSN>>(numRecords);
         for (int i = 0; i < numRecords; i++) {
             futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
         }
@@ -639,7 +632,7 @@
         // close the writer to flush the output buffer
         closeWriterAndLock(writer, lock);
 
-        List<DLSN> dlsns = Await.result(Future.collect(futureList));
+        List<DLSN> dlsns = Utils.ioResult(FutureUtils.collect(futureList));
         assertEquals("All 11 records should be written",
                 numRecords + 1, dlsns.size());
         for (int i = 0; i < numRecords; i++) {
@@ -687,10 +680,10 @@
 
         // close the writer
         closeWriterAndLock(writer, lock);
-        FutureUtils.result(writer.asyncClose());
+        Utils.ioResult(writer.asyncClose());
 
         try {
-            Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(1)));
+            Utils.ioResult(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(1)));
             fail("Should fail the write if the writer is closed");
         } catch (WriteException we) {
             // expected
@@ -713,10 +706,10 @@
         BKLogSegmentWriter writer =
                 createLogSegmentWriter(confLocal, 0L, -1L, lock);
 
-        FutureUtils.result(writer.markEndOfStream());
+        Utils.ioResult(writer.markEndOfStream());
 
         try {
-            Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(1)));
+            Utils.ioResult(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(1)));
             fail("Should fail the write if the writer is marked as end of stream");
         } catch (EndOfStreamException we) {
             // expected
@@ -747,7 +740,7 @@
         LogRecord record = DLMTestUtil.getLogRecordInstance(1);
         record.setControl();
         try {
-            Await.result(writer.asyncWrite(record));
+            Utils.ioResult(writer.asyncWrite(record));
             fail("Should fail the writer if the log segment is already fenced");
         } catch (BKTransmitException bkte) {
             // expected
@@ -755,7 +748,7 @@
         }
 
         try {
-            Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(2)));
+            Utils.ioResult(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(2)));
             fail("Should fail the writer if the log segment is already fenced");
         } catch (WriteException we) {
             // expected
@@ -781,7 +774,7 @@
                 createLogSegmentWriter(confLocal, 0L, -1L, lock);
 
         assertEquals(DLSN.InvalidDLSN,
-                Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(2))));
+                Utils.ioResult(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(2))));
         assertEquals(-1L, ((BKLogSegmentEntryWriter) writer.getEntryWriter())
                 .getLedgerHandle().getLastAddPushed());
 
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
index 2566d34..c0f208f 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
@@ -17,12 +17,13 @@
  */
 package org.apache.distributedlog;
 
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.bk.LedgerAllocator;
 import org.apache.distributedlog.bk.LedgerAllocatorPool;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.Utils;
 import org.junit.Rule;
 import org.junit.Test;
@@ -58,7 +59,7 @@
         confLocal.setLedgerAllocatorPoolName("test-allocator-pool");
 
         BKDistributedLogNamespace namespace = (BKDistributedLogNamespace)
-                DistributedLogNamespaceBuilder.newBuilder()
+                NamespaceBuilder.newBuilder()
                         .conf(confLocal)
                         .uri(uri)
                         .build();
@@ -66,8 +67,8 @@
         FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber,
                 FailpointUtils.FailPointActions.FailPointAction_Throw);
         try {
-            AsyncLogWriter writer =  FutureUtils.result(dlm.openAsyncLogWriter());
-            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+            AsyncLogWriter writer =  Utils.ioResult(dlm.openAsyncLogWriter());
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
             fail("Should fail opening the writer");
         } catch (IOException ioe) {
             // expected
@@ -82,7 +83,7 @@
         LedgerAllocatorPool allocatorPool = (LedgerAllocatorPool) allocator;
         assertEquals(0, allocatorPool.obtainMapSize());
 
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+        AsyncLogWriter writer = Utils.ioResult(dlm.openAsyncLogWriter());
         writer.write(DLMTestUtil.getLogRecordInstance(1L));
         Utils.close(writer);
     }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java
index bb8503f..07f0db5 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java
@@ -17,6 +17,8 @@
  */
 package org.apache.distributedlog;
 
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.junit.Rule;
 import org.junit.Test;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
index a766d3e..5e4ba07 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
@@ -21,6 +21,9 @@
 
 import com.google.common.base.Optional;
 import com.google.common.base.Ticker;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
 import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
 import org.apache.distributedlog.injector.AsyncFailureInjector;
@@ -29,14 +32,13 @@
 import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.namespace.NamespaceDriver;
 import org.apache.distributedlog.util.ConfUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.PermitLimiter;
-import org.apache.distributedlog.util.SchedulerUtils;
-import com.twitter.util.Future;
+import org.apache.distributedlog.common.util.PermitLimiter;
+import org.apache.distributedlog.common.util.SchedulerUtils;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.feature.SettableFeatureProvider;
 import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
@@ -171,7 +173,7 @@
             throws Exception {
         URI uri = createDLMURI("/" + name);
         ensureURICreated(uri);
-        final DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        final Namespace namespace = NamespaceBuilder.newBuilder()
                 .uri(uri)
                 .conf(conf)
                 .build();
@@ -181,14 +183,14 @@
                 .build();
         AsyncCloseable resourcesCloseable = new AsyncCloseable() {
             @Override
-            public Future<Void> asyncClose() {
+            public CompletableFuture<Void> asyncClose() {
                 LOG.info("Shutting down the scheduler");
                 SchedulerUtils.shutdownScheduler(scheduler, 1, TimeUnit.SECONDS);
                 LOG.info("Shut down the scheduler");
                 LOG.info("Closing the namespace");
                 namespace.close();
                 LOG.info("Closed the namespace");
-                return Future.Void();
+                return FutureUtils.Void();
             }
         };
         AsyncFailureInjector failureInjector = AsyncRandomFailureInjector.newBuilder()
@@ -217,20 +219,20 @@
                 Optional.of(resourcesCloseable));
     }
 
-    protected LogSegmentMetadataStore getLogSegmentMetadataStore(DistributedLogNamespace namespace)
+    protected LogSegmentMetadataStore getLogSegmentMetadataStore(Namespace namespace)
             throws IOException {
         return namespace.getNamespaceDriver().getLogStreamMetadataStore(NamespaceDriver.Role.READER)
                 .getLogSegmentMetadataStore();
     }
 
-    protected ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) throws Exception {
+    protected ZooKeeperClient getZooKeeperClient(Namespace namespace) throws Exception {
         NamespaceDriver driver = namespace.getNamespaceDriver();
         assertTrue(driver instanceof BKNamespaceDriver);
         return ((BKNamespaceDriver) driver).getWriterZKC();
     }
 
     @SuppressWarnings("deprecation")
-    protected BookKeeperClient getBookKeeperClient(DistributedLogNamespace namespace) throws Exception {
+    protected BookKeeperClient getBookKeeperClient(Namespace namespace) throws Exception {
         NamespaceDriver driver = namespace.getNamespaceDriver();
         assertTrue(driver instanceof BKNamespaceDriver);
         return ((BKNamespaceDriver) driver).getReaderBKC();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java
index 6d8bd0c..30ef481 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java
@@ -17,19 +17,17 @@
  */
 package org.apache.distributedlog;
 
-import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.Entry.Reader;
 import org.apache.distributedlog.Entry.Writer;
 import org.apache.distributedlog.exceptions.LogRecordTooLongException;
 import org.apache.distributedlog.io.Buffer;
 import org.apache.distributedlog.io.CompressionCodec;
-import com.twitter.io.Buf;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.util.Utils;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -80,7 +78,7 @@
 
         LogRecord largeRecord = new LogRecord(1L, new byte[MAX_LOGRECORD_SIZE + 1]);
         try {
-            writer.writeRecord(largeRecord, new Promise<DLSN>());
+            writer.writeRecord(largeRecord, new CompletableFuture<DLSN>());
             Assert.fail("Should fail on writing large record");
         } catch (LogRecordTooLongException lrtle) {
             // expected
@@ -103,12 +101,12 @@
         assertEquals("zero bytes", 0, writer.getNumBytes());
         assertEquals("zero records", 0, writer.getNumRecords());
 
-        List<Future<DLSN>> writePromiseList = Lists.newArrayList();
+        List<CompletableFuture<DLSN>> writePromiseList = Lists.newArrayList();
         // write first 5 records
         for (int i = 0; i < 5; i++) {
             LogRecord record = new LogRecord(i, ("record-" + i).getBytes(UTF_8));
             record.setPositionWithinLogSegment(i);
-            Promise<DLSN> writePromise = new Promise<DLSN>();
+            CompletableFuture<DLSN> writePromise = new CompletableFuture<DLSN>();
             writer.writeRecord(record, writePromise);
             writePromiseList.add(writePromise);
             assertEquals((i + 1) + " records", (i + 1), writer.getNumRecords());
@@ -117,7 +115,7 @@
         // write large record
         LogRecord largeRecord = new LogRecord(1L, new byte[MAX_LOGRECORD_SIZE + 1]);
         try {
-            writer.writeRecord(largeRecord, new Promise<DLSN>());
+            writer.writeRecord(largeRecord, new CompletableFuture<DLSN>());
             Assert.fail("Should fail on writing large record");
         } catch (LogRecordTooLongException lrtle) {
             // expected
@@ -128,7 +126,7 @@
         for (int i = 0; i < 5; i++) {
             LogRecord record = new LogRecord(i + 5, ("record-" + (i + 5)).getBytes(UTF_8));
             record.setPositionWithinLogSegment(i + 5);
-            Promise<DLSN> writePromise = new Promise<DLSN>();
+            CompletableFuture<DLSN> writePromise = new CompletableFuture<DLSN>();
             writer.writeRecord(record, writePromise);
             writePromiseList.add(writePromise);
             assertEquals((i + 6) + " records", (i + 6), writer.getNumRecords());
@@ -138,7 +136,7 @@
 
         // Test transmit complete
         writer.completeTransmit(1L, 1L);
-        List<DLSN> writeResults = Await.result(Future.collect(writePromiseList));
+        List<DLSN> writeResults = Utils.ioResult(FutureUtils.collect(writePromiseList));
         for (int i = 0; i < 10; i++) {
             Assert.assertEquals(new DLSN(1L, 1L, i), writeResults.get(i));
         }
@@ -175,23 +173,23 @@
         assertEquals("zero bytes", 0, writer.getNumBytes());
         assertEquals("zero records", 0, writer.getNumRecords());
 
-        List<Future<DLSN>> writePromiseList = Lists.newArrayList();
+        List<CompletableFuture<DLSN>> writePromiseList = Lists.newArrayList();
         // write first 5 records
         for (int i = 0; i < 5; i++) {
             LogRecord record = new LogRecord(i, ("record-" + i).getBytes(UTF_8));
             record.setPositionWithinLogSegment(i);
-            Promise<DLSN> writePromise = new Promise<DLSN>();
+            CompletableFuture<DLSN> writePromise = new CompletableFuture<DLSN>();
             writer.writeRecord(record, writePromise);
             writePromiseList.add(writePromise);
             assertEquals((i + 1) + " records", (i + 1), writer.getNumRecords());
         }
 
         final LogRecordSet.Writer recordSetWriter = LogRecordSet.newWriter(1024, CompressionCodec.Type.NONE);
-        List<Future<DLSN>> recordSetPromiseList = Lists.newArrayList();
+        List<CompletableFuture<DLSN>> recordSetPromiseList = Lists.newArrayList();
         // write another 5 records as a batch
         for (int i = 0; i < 5; i++) {
             ByteBuffer record = ByteBuffer.wrap(("record-" + (i + 5)).getBytes(UTF_8));
-            Promise<DLSN> writePromise = new Promise<DLSN>();
+            CompletableFuture<DLSN> writePromise = new CompletableFuture<DLSN>();
             recordSetWriter.writeRecord(record, writePromise);
             recordSetPromiseList.add(writePromise);
             assertEquals((i + 1) + " records", (i + 1), recordSetWriter.getNumRecords());
@@ -202,8 +200,8 @@
         LogRecord setRecord = new LogRecord(5L, data);
         setRecord.setPositionWithinLogSegment(5);
         setRecord.setRecordSet();
-        Promise<DLSN> writePromise = new Promise<DLSN>();
-        writePromise.addEventListener(new FutureEventListener<DLSN>() {
+        CompletableFuture<DLSN> writePromise = new CompletableFuture<DLSN>();
+        writePromise.whenComplete(new FutureEventListener<DLSN>() {
             @Override
             public void onSuccess(DLSN dlsn) {
                 recordSetWriter.completeTransmit(
@@ -224,7 +222,7 @@
         for (int i = 0; i < 5; i++) {
             LogRecord record = new LogRecord(i + 10, ("record-" + (i + 10)).getBytes(UTF_8));
             record.setPositionWithinLogSegment(i + 10);
-            writePromise = new Promise<DLSN>();
+            writePromise = new CompletableFuture<DLSN>();
             writer.writeRecord(record, writePromise);
             writePromiseList.add(writePromise);
             assertEquals((i + 11) + " records", (i + 11), writer.getNumRecords());
@@ -234,7 +232,7 @@
 
         // Test transmit complete
         writer.completeTransmit(1L, 1L);
-        List<DLSN> writeResults = Await.result(Future.collect(writePromiseList));
+        List<DLSN> writeResults = Utils.ioResult(FutureUtils.collect(writePromiseList));
         for (int i = 0; i < 5; i++) {
             Assert.assertEquals(new DLSN(1L, 1L, i), writeResults.get(i));
         }
@@ -242,7 +240,7 @@
         for (int i = 0; i < 5; i++) {
             Assert.assertEquals(new DLSN(1L, 1L, (10 + i)), writeResults.get(6 + i));
         }
-        List<DLSN> recordSetWriteResults = Await.result(Future.collect(recordSetPromiseList));
+        List<DLSN> recordSetWriteResults = Utils.ioResult(FutureUtils.collect(recordSetPromiseList));
         for (int i = 0; i < 5; i++) {
             Assert.assertEquals(new DLSN(1L, 1L, (5 + i)), recordSetWriteResults.get(i));
         }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
index fd3c4ee..c111baf 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog;
 
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.util.Utils;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -81,11 +83,11 @@
         BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
         for (long j = 1; j <= 4; j++) {
             for (int k = 1; k <= 10; k++) {
-                FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                FutureUtils.result(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                Utils.ioResult(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
             }
-            FutureUtils.result(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
-            FutureUtils.result(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
             if (null == reader0) {
                 reader0 = dlmreader0.getInputStream(1);
             }
@@ -124,13 +126,13 @@
                 writer1.setForceRolling(true);
             }
             for (int k = 1; k <= 2; k++) {
-                FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                FutureUtils.result(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                Utils.ioResult(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
                 writer0.setForceRolling(false);
                 writer1.setForceRolling(false);
             }
-            FutureUtils.result(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
-            FutureUtils.result(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
             LOG.info("Completed {} write", j);
             if (null == reader0) {
                 reader0 = dlmreader0.getInputStream(1);
@@ -170,13 +172,13 @@
                     writer0.setForceRolling(true);
                     writer1.setForceRolling(true);
                 }
-                FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                FutureUtils.result(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                Utils.ioResult(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
                 writer0.setForceRolling(false);
                 writer1.setForceRolling(false);
             }
-            FutureUtils.result(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
-            FutureUtils.result(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
             if (null == reader0) {
                 reader0 = dlmreader0.getInputStream(1);
             }
@@ -212,9 +214,9 @@
                     writer1.setForceRolling(true);
                     writer1.overRideMinTimeStampToKeep(retentionPeriodOverride);
                 }
-                DLSN dlsn1 = FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                DLSN dlsn1 = Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
                 LOG.info("writer1 write record {}", dlsn1);
-                DLSN dlsn0 = FutureUtils.result(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                DLSN dlsn0 = Utils.ioResult(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
                 LOG.info("writer0 write record {}", dlsn0);
                 if (k == 5) {
                     writer0.setForceRolling(false);
@@ -223,8 +225,8 @@
                 }
                 Thread.sleep(5);
             }
-            FutureUtils.result(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
-            FutureUtils.result(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
         }
         writer0.close();
         writer1.close();
@@ -264,15 +266,15 @@
                     writer0.setForceRecovery(true);
                     writer1.setForceRecovery(true);
                 }
-                DLSN dlsn1 = FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                DLSN dlsn1 = Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
                 LOG.info("writer1 write record {} - txid = {}", dlsn1, txid-1);
-                DLSN dlsn0 = FutureUtils.result(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                DLSN dlsn0 = Utils.ioResult(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
                 LOG.info("writer0 write record {} - txid = {}", dlsn0, txid-1);
                 writer0.setForceRecovery(false);
                 writer1.setForceRecovery(false);
             }
-            FutureUtils.result(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
-            FutureUtils.result(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
             if (null == reader0) {
                 reader0 = dlmreader0.getInputStream(1);
             }
@@ -313,13 +315,13 @@
                 writer1.setForceRolling(true);
             }
             for (int k = 1; k <= 2; k++) {
-                FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                FutureUtils.result(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                Utils.ioResult(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
                 writer0.setForceRolling(false);
                 writer1.setForceRolling(false);
             }
-            FutureUtils.result(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
-            FutureUtils.result(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
             if (null == reader0) {
                 reader0 = dlmreader0.getInputStream(1);
             }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
index 152e4d8..8bdf86d 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
@@ -20,9 +20,10 @@
 import java.net.URI;
 import java.util.List;
 
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.util.Await;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.util.Utils;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,7 +44,7 @@
                 .setImmediateFlushEnabled(true)
                 .setEnableLedgerAllocatorPool(true)
                 .setLedgerAllocatorPoolName("test");
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf).uri(uri).build();
         DistributedLogManager dlm = namespace.openLog(name);
         final int numSegments = 3;
@@ -68,7 +69,7 @@
         writer2.closeAndComplete();
 
         try {
-            Await.result(writer1.write(DLMTestUtil.getLogRecordInstance(numSegments + 1)));
+            Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(numSegments + 1)));
             fail("Should fail on writing new log records.");
         } catch (Throwable t) {
             LOG.error("Failed to write entry : ", t);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
index 31df059..39ffe85 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
@@ -21,8 +21,7 @@
 import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
 import org.apache.distributedlog.LogSegmentMetadata.TruncationStatus;
 import org.apache.distributedlog.exceptions.UnsupportedMetadataVersionException;
-
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.Utils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -64,7 +63,7 @@
         LogSegmentMetadata metadata1 = new LogSegmentMetadataBuilder("/metadata1",
             LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION, 1000, 1).setRegionId(TEST_REGION_ID).build();
         metadata1.write(zkc);
-        LogSegmentMetadata read1 = FutureUtils.result(LogSegmentMetadata.read(zkc, "/metadata1"));
+        LogSegmentMetadata read1 = Utils.ioResult(LogSegmentMetadata.read(zkc, "/metadata1"));
         assertEquals(metadata1, read1);
         assertEquals(TEST_REGION_ID, read1.getRegionId());
     }
@@ -75,7 +74,7 @@
             1, 1000, 1).setRegionId(TEST_REGION_ID).build();
         metadata1.write(zkc);
         // synchronous read
-        LogSegmentMetadata read1 = FutureUtils.result(LogSegmentMetadata.read(zkc, "/metadata2", true));
+        LogSegmentMetadata read1 = Utils.ioResult(LogSegmentMetadata.read(zkc, "/metadata2", true));
         assertEquals(read1.getLogSegmentId(), metadata1.getLogSegmentId());
         assertEquals(read1.getFirstTxId(), metadata1.getFirstTxId());
         assertEquals(read1.getLastTxId(), metadata1.getLastTxId());
@@ -90,7 +89,7 @@
         metadata1.write(zkc);
         // synchronous read
         try {
-            LogSegmentMetadata read1 = FutureUtils.result(LogSegmentMetadata.read(zkc, "/metadata-failure"));
+            LogSegmentMetadata read1 = Utils.ioResult(LogSegmentMetadata.read(zkc, "/metadata-failure"));
             fail("The previous statement should throw an exception");
         } catch (UnsupportedMetadataVersionException e) {
             // Expected
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java
index 8c01a5c..fcc3395 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java
@@ -17,11 +17,12 @@
  */
 package org.apache.distributedlog;
 
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
 import org.apache.distributedlog.metadata.LogMetadata;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.util.DLUtils;
 import org.apache.bookkeeper.meta.ZkVersion;
 import org.apache.bookkeeper.versioning.Versioned;
@@ -80,7 +81,7 @@
                 .setImmediateFlushEnabled(true)
                 .setEnableLedgerAllocatorPool(true)
                 .setLedgerAllocatorPoolName("test");
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
+        Namespace namespace = NamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
 
         namespace.createLog(streamName);
         MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
@@ -111,7 +112,7 @@
                 .setImmediateFlushEnabled(true)
                 .setEnableLedgerAllocatorPool(true)
                 .setLedgerAllocatorPoolName("test");
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
+        Namespace namespace = NamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
 
         namespace.createLog(streamName);
         MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
@@ -167,7 +168,7 @@
                 .setImmediateFlushEnabled(true)
                 .setEnableLedgerAllocatorPool(true)
                 .setLedgerAllocatorPoolName("test");
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
+        Namespace namespace = NamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
 
         namespace.createLog(streamName);
         MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
@@ -223,7 +224,7 @@
                 .setImmediateFlushEnabled(true)
                 .setEnableLedgerAllocatorPool(true)
                 .setLedgerAllocatorPoolName("test");
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
+        Namespace namespace = NamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
 
         namespace.createLog(streamName);
         DistributedLogManager dlm1 = namespace.openLog(streamName);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
index 5bfbf45..2b02704 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
@@ -21,9 +21,11 @@
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.distributedlog.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
 import org.apache.distributedlog.exceptions.IdleReaderException;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.Utils;
 import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -229,7 +231,7 @@
             BKAsyncLogWriter out = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
             for (long j = 1; j <= segmentSize; j++) {
                 LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                FutureUtils.result(out.write(op));
+                Utils.ioResult(out.write(op));
                 numRecordsWritten++;
             }
             out.closeAndComplete();
@@ -237,7 +239,7 @@
 
         BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
         String completedZNode = blplm.completedLedgerZNode(txid - segmentSize, txid - 1, 3);
-        LogSegmentMetadata metadata = FutureUtils.result(LogSegmentMetadata.read(zkClient, completedZNode));
+        LogSegmentMetadata metadata = Utils.ioResult(LogSegmentMetadata.read(zkClient, completedZNode));
         zkClient.get().delete(completedZNode, -1);
         LogSegmentMetadata metadataToChange =
                 metadata.mutator()
@@ -253,7 +255,7 @@
             BKAsyncLogWriter out = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
             for (long j = 1; j <= segmentSize; j++) {
                 LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                FutureUtils.result(out.write(op));
+                Utils.ioResult(out.write(op));
                 numRecordsWritten++;
             }
             out.closeAndComplete();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
index 8f445c4..6c9e354 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
@@ -18,8 +18,10 @@
 package org.apache.distributedlog;
 
 import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.Utils;
 import org.junit.Test;
 
@@ -93,8 +95,8 @@
         DistributedLogManager dlmwrite = createNewDLM(confLocal, name);
 
         final AsyncLogWriter writer = dlmwrite.startAsyncLogSegmentNonPartitioned();
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(0)));
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(0)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(1)));
         final AtomicInteger writeCount = new AtomicInteger(2);
 
         DistributedLogManager dlmread = createNewDLM(conf, name);
@@ -116,7 +118,7 @@
                         while (running.get()) {
                             limiter.acquire();
                             long curTxId = txid++;
-                            dlsn = FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
+                            dlsn = Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
                             writeCount.incrementAndGet();
                             if (curTxId % 1000 == 0) {
                                 LOG.info("writer write {}", curTxId);
@@ -126,7 +128,7 @@
                         Utils.close(writer);
                     } catch (DLInterruptedException die) {
                         Thread.currentThread().interrupt();
-                    } catch (IOException e) {
+                    } catch (Exception e) {
 
                     }
                 }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
index ac9984b..eda8eb2 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
@@ -20,16 +20,18 @@
 import com.google.common.base.Optional;
 import com.google.common.base.Ticker;
 import com.google.common.collect.Lists;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
 import org.apache.distributedlog.exceptions.DLIllegalStateException;
 import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
 import org.apache.distributedlog.injector.AsyncFailureInjector;
 import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
 import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.stats.AlertStatsLogger;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.junit.After;
@@ -130,14 +132,14 @@
     }
 
     private void ensureOrderSchedulerEmpty(String streamName) throws Exception {
-        final Promise<Void> promise = new Promise<Void>();
+        final CompletableFuture<Void> promise = new CompletableFuture<Void>();
         scheduler.submit(streamName, new Runnable() {
             @Override
             public void run() {
-                FutureUtils.setValue(promise, null);
+                FutureUtils.complete(promise, null);
             }
         });
-        FutureUtils.result(promise);
+        Utils.ioResult(promise);
     }
 
     void generateCompletedLogSegments(DistributedLogManager dlm,
@@ -153,12 +155,12 @@
 
         long txid = startTxId;
         for (long i = 0; i < numCompletedSegments; i++) {
-            AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+            AsyncLogWriter writer = Utils.ioResult(dlm.openAsyncLogWriter());
             for (long j = 1; j <= segmentSize; j++) {
-                FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
                 LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(txid);
                 ctrlRecord.setControl();
-                FutureUtils.result(writer.write(ctrlRecord));
+                Utils.ioResult(writer.write(ctrlRecord));
             }
             Utils.close(writer);
         }
@@ -167,12 +169,12 @@
     AsyncLogWriter createInprogressLogSegment(DistributedLogManager dlm,
                                               DistributedLogConfiguration conf,
                                               long segmentSize) throws Exception {
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+        AsyncLogWriter writer = Utils.ioResult(dlm.openAsyncLogWriter());
         for (long i = 1L; i <= segmentSize; i++) {
-            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(i)));
             LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(i);
             ctrlRecord.setControl();
-            FutureUtils.result(writer.write(ctrlRecord));
+            Utils.ioResult(writer.write(ctrlRecord));
         }
         return writer;
     }
@@ -325,8 +327,8 @@
 
         // generate list of log segments
         generateCompletedLogSegments(dlm, 3, 3);
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
-        FutureUtils.result(writer.truncate(new DLSN(2L, 1L, 0L)));
+        AsyncLogWriter writer = Utils.ioResult(dlm.openAsyncLogWriter());
+        Utils.ioResult(writer.truncate(new DLSN(2L, 1L, 0L)));
 
         List<LogSegmentMetadata> segments = dlm.getLogSegments();
 
@@ -382,8 +384,8 @@
 
         // generate list of log segments
         generateCompletedLogSegments(dlm, 3, 2);
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
-        FutureUtils.result(writer.truncate(new DLSN(2L, 1L, 0L)));
+        AsyncLogWriter writer = Utils.ioResult(dlm.openAsyncLogWriter());
+        Utils.ioResult(writer.truncate(new DLSN(2L, 1L, 0L)));
 
         List<LogSegmentMetadata> segments = dlm.getLogSegments();
 
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java
index 029e872..efc9ac6 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java
@@ -18,16 +18,15 @@
 package org.apache.distributedlog;
 
 import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.logsegment.LogSegmentFilter;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -47,7 +46,7 @@
     @Rule
     public TestName runtime = new TestName();
 
-    private Future<Optional<LogRecordWithDLSN>> getLogRecordNotLessThanTxId(
+    private CompletableFuture<Optional<LogRecordWithDLSN>> getLogRecordNotLessThanTxId(
             BKDistributedLogManager bkdlm, int logsegmentIdx, long transactionId) throws Exception {
         List<LogSegmentMetadata> logSegments = bkdlm.getLogSegments();
         return ReadUtils.getLogRecordNotLessThanTxId(
@@ -60,7 +59,7 @@
         );
     }
 
-    private Future<LogRecordWithDLSN> getFirstGreaterThanRecord(BKDistributedLogManager bkdlm, int ledgerNo, DLSN dlsn) throws Exception {
+    private CompletableFuture<LogRecordWithDLSN> getFirstGreaterThanRecord(BKDistributedLogManager bkdlm, int ledgerNo, DLSN dlsn) throws Exception {
         List<LogSegmentMetadata> ledgerList = bkdlm.getLogSegments();
         return ReadUtils.asyncReadFirstUserRecord(
                 bkdlm.getStreamName(), ledgerList.get(ledgerNo), 2, 16, new AtomicInteger(0), Executors.newFixedThreadPool(1),
@@ -68,9 +67,9 @@
         );
     }
 
-    private Future<LogRecordWithDLSN> getLastUserRecord(BKDistributedLogManager bkdlm, int ledgerNo) throws Exception {
+    private CompletableFuture<LogRecordWithDLSN> getLastUserRecord(BKDistributedLogManager bkdlm, int ledgerNo) throws Exception {
         BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        List<LogSegmentMetadata> ledgerList = FutureUtils.result(
+        List<LogSegmentMetadata> ledgerList = Utils.ioResult(
                 readHandler.readLogSegmentsFromStore(
                         LogSegmentMetadata.COMPARATOR,
                         LogSegmentFilter.DEFAULT_FILTER,
@@ -89,8 +88,8 @@
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1 /* txid */);
 
         DLSN dlsn = new DLSN(1,0,0);
-        Future<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
-        LogRecordWithDLSN logrec = Await.result(futureLogrec);
+        CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
+        LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
         assertEquals("should be an exact match", dlsn, logrec.getDlsn());
         bkdlm.close();
     }
@@ -102,8 +101,8 @@
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1 /* txid */);
 
         DLSN dlsn = new DLSN(1,1,0);
-        Future<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
-        LogRecordWithDLSN logrec = Await.result(futureLogrec);
+        CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
+        LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
         assertEquals("should be an exact match", dlsn, logrec.getDlsn());
         bkdlm.close();
     }
@@ -115,8 +114,8 @@
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1 /* txid */);
 
         DLSN dlsn = new DLSN(1,0,1);
-        Future<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
-        LogRecordWithDLSN logrec = Await.result(futureLogrec);
+        CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
+        LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
         assertEquals(new DLSN(1,1,0), logrec.getDlsn());
         bkdlm.close();
     }
@@ -128,8 +127,8 @@
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5 /* user recs */ , 1 /* txid */);
 
         DLSN dlsn = new DLSN(2,0,0);
-        Future<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
-        LogRecordWithDLSN logrec = Await.result(futureLogrec);
+        CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
+        LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
         assertEquals(null, logrec);
         bkdlm.close();
     }
@@ -144,8 +143,8 @@
         txid += DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5 /* user recs */ , txid);
 
         DLSN dlsn = new DLSN(1,3,0);
-        Future<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 1, dlsn);
-        LogRecordWithDLSN logrec = Await.result(futureLogrec);
+        CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 1, dlsn);
+        LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
         assertEquals(new DLSN(2,0,0), logrec.getDlsn());
         bkdlm.close();
     }
@@ -157,8 +156,8 @@
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 5 /* control recs */, 5, 1 /* txid */);
 
         DLSN dlsn = new DLSN(1,3,0);
-        Future<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
-        LogRecordWithDLSN logrec = Await.result(futureLogrec);
+        CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
+        LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
         assertEquals(new DLSN(1,5,0), logrec.getDlsn());
         bkdlm.close();
     }
@@ -169,8 +168,8 @@
         BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 5 /* control recs */, 5, 1 /* txid */);
 
-        Future<LogRecordWithDLSN> futureLogrec = getLastUserRecord(bkdlm, 0);
-        LogRecordWithDLSN logrec = Await.result(futureLogrec);
+        CompletableFuture<LogRecordWithDLSN> futureLogrec = getLastUserRecord(bkdlm, 0);
+        LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
         assertEquals(new DLSN(1,9,0), logrec.getDlsn());
         bkdlm.close();
     }
@@ -182,15 +181,15 @@
 
         AsyncLogWriter out = bkdlm.startAsyncLogSegmentNonPartitioned();
         int txid = 1;
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, true)));
-        Await.result(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, true)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, false)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, true)));
+        Utils.ioResult(out.write(DLMTestUtil.getLargeLogRecordInstance(txid++, true)));
         Utils.close(out);
 
-        Future<LogRecordWithDLSN> futureLogrec = getLastUserRecord(bkdlm, 0);
-        LogRecordWithDLSN logrec = Await.result(futureLogrec);
+        CompletableFuture<LogRecordWithDLSN> futureLogrec = getLastUserRecord(bkdlm, 0);
+        LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
         assertEquals(new DLSN(1,2,0), logrec.getDlsn());
         bkdlm.close();
     }
@@ -201,8 +200,8 @@
         BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 5 /* control recs */, 0, 1 /* txid */);
 
-        Future<LogRecordWithDLSN> futureLogrec = getLastUserRecord(bkdlm, 0);
-        LogRecordWithDLSN logrec = Await.result(futureLogrec);
+        CompletableFuture<LogRecordWithDLSN> futureLogrec = getLastUserRecord(bkdlm, 0);
+        LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
         assertEquals(null, logrec);
         bkdlm.close();
     }
@@ -259,7 +258,7 @@
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 1, 1 /* txid */);
 
         Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 999L));
+                Utils.ioResult(getLogRecordNotLessThanTxId(bkdlm, 0, 999L));
         assertFalse(result.isPresent());
     }
 
@@ -270,7 +269,7 @@
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 1, 999L /* txid */);
 
         Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 99L));
+                Utils.ioResult(getLogRecordNotLessThanTxId(bkdlm, 0, 99L));
         assertTrue(result.isPresent());
         assertEquals(999L, result.get().getTransactionId());
         assertEquals(0L, result.get().getDlsn().getEntryId());
@@ -284,7 +283,7 @@
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 5, 1L /* txid */);
 
         Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 3L));
+                Utils.ioResult(getLogRecordNotLessThanTxId(bkdlm, 0, 3L));
         assertTrue(result.isPresent());
         assertEquals(3L, result.get().getTransactionId());
     }
@@ -296,7 +295,7 @@
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 100, 1L /* txid */);
 
         Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 9L));
+                Utils.ioResult(getLogRecordNotLessThanTxId(bkdlm, 0, 9L));
         assertTrue(result.isPresent());
         assertEquals(9L, result.get().getTransactionId());
     }
@@ -308,7 +307,7 @@
         DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 100, 1L /* txid */, 3L);
 
         Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 23L));
+                Utils.ioResult(getLogRecordNotLessThanTxId(bkdlm, 0, 23L));
         assertTrue(result.isPresent());
         assertEquals(25L, result.get().getTransactionId());
     }
@@ -321,22 +320,22 @@
         long txid = 1L;
         for (int i = 0; i < 10; ++i) {
             LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
-            Await.result(out.write(record));
+            Utils.ioResult(out.write(record));
             txid += 1;
         }
         long txidToSearch = txid;
         for (int i = 0; i < 10; ++i) {
             LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txidToSearch);
-            Await.result(out.write(record));
+            Utils.ioResult(out.write(record));
         }
         for (int i = 0; i < 10; ++i) {
             LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
-            Await.result(out.write(record));
+            Utils.ioResult(out.write(record));
             txid += 1;
         }
         Utils.close(out);
         Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, txidToSearch));
+                Utils.ioResult(getLogRecordNotLessThanTxId(bkdlm, 0, txidToSearch));
         assertTrue(result.isPresent());
         assertEquals(10L, result.get().getDlsn().getEntryId());
     }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReader.java
index ad5bf8e..8d9f846 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReader.java
@@ -17,13 +17,13 @@
  */
 package org.apache.distributedlog;
 
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
 
 import java.io.IOException;
 import java.util.concurrent.CountDownLatch;
@@ -140,8 +140,8 @@
     }
 
     private void readNext() {
-        Future<LogRecordWithDLSN> record = reader.readNext();
-        record.addEventListener(this);
+        CompletableFuture<LogRecordWithDLSN> record = reader.readNext();
+        record.whenComplete(this);
     }
 
     @Override
@@ -184,12 +184,8 @@
 
     private void closeReader() {
         if (null != reader) {
-            reader.asyncClose().onFailure(new AbstractFunction1<Throwable, BoxedUnit>() {
-                @Override
-                public BoxedUnit apply(Throwable cause) {
-                    LOG.warn("Exception on closing reader {} : ", readerName, cause);
-                    return BoxedUnit.UNIT;
-                }
+            reader.asyncClose().whenComplete((value, cause) -> {
+                LOG.warn("Exception on closing reader {} : ", readerName, cause);
             });
         }
     }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
index 9032866..0111e4d 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
@@ -20,14 +20,16 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.feature.CoreFeatureKeys;
 import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryReader;
 import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Future;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.feature.SettableFeature;
@@ -35,9 +37,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.distributedlog.annotations.DistributedLogAnnotations.FlakyTest;
-import com.twitter.util.Await;
-import com.twitter.util.FutureEventListener;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations.FlakyTest;
 
 import static com.google.common.base.Charsets.UTF_8;
 import static org.junit.Assert.*;
@@ -79,7 +79,7 @@
         // send requests in parallel
         for (int i = 1; i <= numEntries; i++) {
             final int entryId = i;
-            writer.write(DLMTestUtil.getLogRecordInstance(entryId)).addEventListener(new FutureEventListener<DLSN>() {
+            writer.write(DLMTestUtil.getLogRecordInstance(entryId)).whenComplete(new FutureEventListener<DLSN>() {
 
                 @Override
                 public void onSuccess(DLSN value) {
@@ -125,7 +125,9 @@
         // send requests in parallel to have outstanding requests
         for (int i = 1; i <= numEntries; i++) {
             final int entryId = i;
-            Future<DLSN> writeFuture = writer.write(DLMTestUtil.getLogRecordInstance(entryId)).addEventListener(new FutureEventListener<DLSN>() {
+            CompletableFuture<DLSN> writeFuture =
+                writer.write(DLMTestUtil.getLogRecordInstance(entryId))
+                    .whenComplete(new FutureEventListener<DLSN>() {
 
                 @Override
                 public void onSuccess(DLSN value) {
@@ -146,7 +148,7 @@
             });
             if (i == 1) {
                 // wait for first log segment created
-                FutureUtils.result(writeFuture);
+                Utils.ioResult(writeFuture);
             }
         }
         latch.await();
@@ -191,7 +193,7 @@
         long txId = 1L;
 
         // Create Log Segments
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
 
         FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_StartLogSegmentBeforeLedgerCreate,
                 FailpointUtils.FailPointActions.FailPointAction_Throw);
@@ -201,7 +203,7 @@
             final int numRecords = 10;
             final CountDownLatch latch = new CountDownLatch(numRecords);
             for (int i = 0; i < numRecords; i++) {
-                writer.write(DLMTestUtil.getLogRecordInstance(++txId)).addEventListener(new FutureEventListener<DLSN>() {
+                writer.write(DLMTestUtil.getLogRecordInstance(++txId)).whenComplete(new FutureEventListener<DLSN>() {
                     @Override
                     public void onSuccess(DLSN value) {
                         logger.info("Completed entry : {}.", value);
@@ -266,7 +268,7 @@
         // send requests in parallel to have outstanding requests
         for (int i = 1; i <= numLogSegments; i++) {
             final int entryId = i;
-            Future<DLSN> writeFuture = writer.write(DLMTestUtil.getLogRecordInstance(entryId)).addEventListener(new FutureEventListener<DLSN>() {
+            CompletableFuture<DLSN> writeFuture = writer.write(DLMTestUtil.getLogRecordInstance(entryId)).whenComplete(new FutureEventListener<DLSN>() {
                 @Override
                 public void onSuccess(DLSN value) {
                     logger.info("Completed entry {} : {}.", entryId, value);
@@ -279,7 +281,7 @@
             });
             if (i == 1) {
                 // wait for first log segment created
-                FutureUtils.result(writeFuture);
+                Utils.ioResult(writeFuture);
             }
         }
         latch.await();
@@ -297,7 +299,7 @@
         // writer should work after rolling log segments
         // there would be (numLogSegments/2) segments based on current rolling policy
         for (int i = 1; i <= numLogSegments; i++) {
-            DLSN newDLSN = Await.result(writer.write(DLMTestUtil.getLogRecordInstance(numLogSegments + i)));
+            DLSN newDLSN = Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(numLogSegments + i)));
             logger.info("Completed entry {} : {}", numLogSegments + i, newDLSN);
         }
 
@@ -364,7 +366,7 @@
 
         // 2) reader should be able to read 5 entries.
         for (long i = 1; i <= numEntries; i++) {
-            LogRecordWithDLSN record = Await.result(reader.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
             DLMTestUtil.verifyLogRecord(record);
             assertEquals(i, record.getTransactionId());
             assertEquals(record.getTransactionId() - 1, record.getSequenceId());
@@ -418,7 +420,7 @@
         anotherWriter.closeAndComplete();
 
         for (long i = numEntries + 1; i <= numEntries + 3; i++) {
-            LogRecordWithDLSN record = Await.result(reader.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
             DLMTestUtil.verifyLogRecord(record);
             assertEquals(i, record.getTransactionId());
         }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java
index 9258922..da4ef81 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java
@@ -18,8 +18,9 @@
 package org.apache.distributedlog;
 
 import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
-import com.twitter.util.Await;
-import com.twitter.util.FutureEventListener;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.util.Utils;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -72,7 +73,7 @@
 
         BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
         BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(0L)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(0L)));
 
         dlm.close();
 
@@ -126,16 +127,16 @@
         for (int i = 0; i < 3; i++) {
             BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
             for (int j = 0; j < 2; j++) {
-                Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+                Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
 
                 if (null == reader) {
                     reader = readDLM.getAsyncLogReader(DLSN.InitialDLSN);
                     final AsyncLogReader r = reader;
-                    reader.readNext().addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+                    reader.readNext().whenComplete(new FutureEventListener<LogRecordWithDLSN>() {
                         @Override
                         public void onSuccess(LogRecordWithDLSN record) {
                             readRecords.add(record);
-                            r.readNext().addEventListener(this);
+                            r.readNext().whenComplete(this);
                         }
 
                         @Override
@@ -149,7 +150,7 @@
         }
 
         BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
 
         List<LogSegmentMetadata> segments = dlm.getLogSegments();
         assertEquals(4, segments.size());
@@ -174,12 +175,12 @@
         for (int i = 0; i < 3; i++) {
             BKAsyncLogWriter writerv5 = dlmv5.startAsyncLogSegmentNonPartitioned();
             for (int j = 0; j < 2; j++) {
-                Await.result(writerv5.write(DLMTestUtil.getLogRecordInstance(txId++)));
+                Utils.ioResult(writerv5.write(DLMTestUtil.getLogRecordInstance(txId++)));
             }
             writerv5.closeAndComplete();
         }
         BKAsyncLogWriter writerv5 = dlmv5.startAsyncLogSegmentNonPartitioned();
-        Await.result(writerv5.write(DLMTestUtil.getLogRecordInstance(txId++)));
+        Utils.ioResult(writerv5.write(DLMTestUtil.getLogRecordInstance(txId++)));
 
         List<LogSegmentMetadata> segmentsv5 = dlmv5.getLogSegments();
         assertEquals(8, segmentsv5.size());
@@ -205,7 +206,7 @@
         for (int i = 0; i < 3; i++) {
             BKAsyncLogWriter writerv4 = dlmv4.startAsyncLogSegmentNonPartitioned();
             for (int j = 0; j < 2; j++) {
-                Await.result(writerv4.write(DLMTestUtil.getLogRecordInstance(txId++)));
+                Utils.ioResult(writerv4.write(DLMTestUtil.getLogRecordInstance(txId++)));
             }
             writerv4.closeAndComplete();
         }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java
index 5b26a70..06708c8 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java
@@ -22,7 +22,9 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.util.Utils;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
@@ -31,7 +33,6 @@
 import org.slf4j.LoggerFactory;
 
 import org.apache.distributedlog.LogSegmentMetadata.TruncationStatus;
-import com.twitter.util.Await;
 
 import static org.junit.Assert.*;
 
@@ -96,11 +97,11 @@
         AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
         long txid = 1 + 10 * 10;
         for (int j = 1; j <= 10; j++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
         }
 
         // to make sure the truncation task is executed
-        DLSN lastDLSN = Await.result(dlm.getLastDLSNAsync());
+        DLSN lastDLSN = Utils.ioResult(dlm.getLastDLSNAsync());
         LOG.info("Get last dlsn of stream {} : {}", name, lastDLSN);
 
         assertEquals(6, distributedLogManager.getLogSegments().size());
@@ -123,20 +124,20 @@
         Thread.sleep(1000);
 
         // delete invalid dlsn
-        assertFalse(Await.result(pair.getRight().truncate(DLSN.InvalidDLSN)));
+        assertFalse(Utils.ioResult(pair.getRight().truncate(DLSN.InvalidDLSN)));
         verifyEntries(name, 1, 1, 5 * 10);
 
         for (int i = 1; i <= 4; i++) {
             int txn = (i-1) * 10 + i;
             DLSN dlsn = txid2DLSN.get((long)txn);
-            assertTrue(Await.result(pair.getRight().truncate(dlsn)));
+            assertTrue(Utils.ioResult(pair.getRight().truncate(dlsn)));
             verifyEntries(name, 1, (i - 1) * 10 + 1, (5 - i + 1) * 10);
         }
 
         // Delete higher dlsn
         int txn = 43;
         DLSN dlsn = txid2DLSN.get((long) txn);
-        assertTrue(Await.result(pair.getRight().truncate(dlsn)));
+        assertTrue(Utils.ioResult(pair.getRight().truncate(dlsn)));
         verifyEntries(name, 1, 41, 10);
 
         Utils.close(pair.getRight());
@@ -160,14 +161,14 @@
         for (int i = 1; i <= 4; i++) {
             int txn = (i-1) * 10 + i;
             DLSN dlsn = txid2DLSN.get((long)txn);
-            assertTrue(Await.result(pair.getRight().truncate(dlsn)));
+            assertTrue(Utils.ioResult(pair.getRight().truncate(dlsn)));
             verifyEntries(name, 1, (i - 1) * 10 + 1, (5 - i + 1) * 10);
         }
 
         // Delete higher dlsn
         int txn = 43;
         DLSN dlsn = txid2DLSN.get((long) txn);
-        assertTrue(Await.result(pair.getRight().truncate(dlsn)));
+        assertTrue(Utils.ioResult(pair.getRight().truncate(dlsn)));
         verifyEntries(name, 1, 41, 10);
 
         Utils.close(pair.getRight());
@@ -176,7 +177,7 @@
         // Try force truncation
         BKDistributedLogManager dlm = (BKDistributedLogManager)createNewDLM(confLocal, name);
         BKLogWriteHandler handler = dlm.createWriteHandler(true);
-        FutureUtils.result(handler.purgeLogSegmentsOlderThanTxnId(Integer.MAX_VALUE));
+        Utils.ioResult(handler.purgeLogSegmentsOlderThanTxnId(Integer.MAX_VALUE));
 
         verifyEntries(name, 1, 41, 10);
     }
@@ -230,11 +231,11 @@
         AsyncLogWriter newWriter = newDLM.startAsyncLogSegmentNonPartitioned();
         long txid = 1 + 4 * 10;
         for (int j = 1; j <= 10; j++) {
-            Await.result(newWriter.write(DLMTestUtil.getLogRecordInstance(txid++)));
+            Utils.ioResult(newWriter.write(DLMTestUtil.getLogRecordInstance(txid++)));
         }
 
         // to make sure the truncation task is executed
-        DLSN lastDLSN = Await.result(newDLM.getLastDLSNAsync());
+        DLSN lastDLSN = Utils.ioResult(newDLM.getLastDLSNAsync());
         LOG.info("Get last dlsn of stream {} : {}", name, lastDLSN);
 
         assertEquals(5, newDLM.getLogSegments().size());
@@ -277,7 +278,7 @@
 
         DistributedLogManager newDLM = createNewDLM(confLocal, name);
         AsyncLogWriter newWriter = newDLM.startAsyncLogSegmentNonPartitioned();
-        Await.result(newWriter.truncate(dlsnMap.get(15L)));
+        Utils.ioResult(newWriter.truncate(dlsnMap.get(15L)));
 
         List<LogSegmentMetadata> newSegments2 = newDLM.getLogSegments();
         assertArrayEquals(newSegments.toArray(new LogSegmentMetadata[4]),
@@ -299,7 +300,7 @@
             AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
             for (int j = 1; j <= numEntriesPerLogSegment; j++) {
                 long curTxId = txid++;
-                DLSN dlsn = Await.result(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
+                DLSN dlsn = Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
                 txid2DLSN.put(curTxId, dlsn);
             }
             Utils.close(writer);
@@ -311,7 +312,7 @@
             AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
             for (int j = 1; j <= 10; j++) {
                 long curTxId = txid++;
-                DLSN dlsn = Await.result(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
+                DLSN dlsn = Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
                 txid2DLSN.put(curTxId, dlsn);
             }
             return new ImmutablePair<DistributedLogManager, AsyncLogWriter>(dlm, writer);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
index c28437f..0d0ca99 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
@@ -18,7 +18,6 @@
 package org.apache.distributedlog;
 
 import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.util.PermitLimiter;
 import org.apache.distributedlog.util.SimplePermitLimiter;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.feature.SettableFeature;
@@ -30,7 +29,6 @@
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import scala.runtime.BoxedUnit;
 
 public class TestWriteLimiter {
     static final Logger LOG = LoggerFactory.getLogger(TestWriteLimiter.class);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
index a1c075f..75bcda2 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
@@ -19,7 +19,7 @@
 
 import org.apache.distributedlog.ZooKeeperClient.Credentials;
 import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
-import org.apache.distributedlog.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
 import org.apache.zookeeper.CreateMode;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java b/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java
index 8d88a37..45fc1f3 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java
@@ -17,20 +17,20 @@
  */
 package org.apache.distributedlog.acl;
 
+import java.net.URI;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClusterTestCase;
 import org.apache.distributedlog.impl.acl.ZKAccessControl;
 import org.apache.distributedlog.thrift.AccessControlEntry;
-import com.twitter.util.Await;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.net.URI;
-
 import static com.google.common.base.Charsets.UTF_8;
 import static org.junit.Assert.*;
 
@@ -60,14 +60,14 @@
         ace.setDenyWrite(true);
         String zkPath = "/create-zk-access-control";
         ZKAccessControl zkac = new ZKAccessControl(ace, zkPath);
-        Await.result(zkac.create(zkc));
+        Utils.ioResult(zkac.create(zkc));
 
-        ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+        ZKAccessControl readZKAC = Utils.ioResult(ZKAccessControl.read(zkc, zkPath, null));
         assertEquals(zkac, readZKAC);
 
         ZKAccessControl another = new ZKAccessControl(ace, zkPath);
         try {
-            Await.result(another.create(zkc));
+            FutureUtils.result(another.create(zkc));
         } catch (KeeperException.NodeExistsException ke) {
             // expected
         }
@@ -81,19 +81,19 @@
         ace.setDenyDelete(true);
 
         ZKAccessControl zkac = new ZKAccessControl(ace, zkPath);
-        Await.result(zkac.create(zkc));
+        Utils.ioResult(zkac.create(zkc));
 
-        ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+        ZKAccessControl readZKAC = Utils.ioResult(ZKAccessControl.read(zkc, zkPath, null));
         assertEquals(zkac, readZKAC);
 
-        Await.result(ZKAccessControl.delete(zkc, zkPath));
+        Utils.ioResult(ZKAccessControl.delete(zkc, zkPath));
 
         try {
-            Await.result(ZKAccessControl.read(zkc, zkPath, null));
+            FutureUtils.result(ZKAccessControl.read(zkc, zkPath, null));
         } catch (KeeperException.NoNodeException nne) {
             // expected.
         }
-        Await.result(ZKAccessControl.delete(zkc, zkPath));
+        Utils.ioResult(ZKAccessControl.delete(zkc, zkPath));
     }
 
     @Test(timeout = 60000)
@@ -102,7 +102,7 @@
 
         zkc.get().create(zkPath, new byte[0], zkc.getDefaultACL(), CreateMode.PERSISTENT);
 
-        ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+        ZKAccessControl readZKAC = Utils.ioResult(ZKAccessControl.read(zkc, zkPath, null));
 
         assertEquals(zkPath, readZKAC.getZKPath());
         assertEquals(ZKAccessControl.DEFAULT_ACCESS_CONTROL_ENTRY, readZKAC.getAccessControlEntry());
@@ -116,7 +116,7 @@
         zkc.get().create(zkPath, "corrupted-data".getBytes(UTF_8), zkc.getDefaultACL(), CreateMode.PERSISTENT);
 
         try {
-            Await.result(ZKAccessControl.read(zkc, zkPath, null));
+            Utils.ioResult(ZKAccessControl.read(zkc, zkPath, null));
         } catch (ZKAccessControl.CorruptedAccessControlException cace) {
             // expected
         }
@@ -130,25 +130,25 @@
         ace.setDenyDelete(true);
 
         ZKAccessControl zkac = new ZKAccessControl(ace, zkPath);
-        Await.result(zkac.create(zkc));
+        Utils.ioResult(zkac.create(zkc));
 
-        ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+        ZKAccessControl readZKAC = Utils.ioResult(ZKAccessControl.read(zkc, zkPath, null));
         assertEquals(zkac, readZKAC);
 
         ace.setDenyRelease(true);
         ZKAccessControl newZKAC = new ZKAccessControl(ace, zkPath);
-        Await.result(newZKAC.update(zkc));
-        ZKAccessControl readZKAC2 = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+        Utils.ioResult(newZKAC.update(zkc));
+        ZKAccessControl readZKAC2 = Utils.ioResult(ZKAccessControl.read(zkc, zkPath, null));
         assertEquals(newZKAC, readZKAC2);
 
         try {
-            Await.result(readZKAC.update(zkc));
+            FutureUtils.result(readZKAC.update(zkc));
         } catch (KeeperException.BadVersionException bve) {
             // expected
         }
         readZKAC2.getAccessControlEntry().setDenyTruncate(true);
-        Await.result(readZKAC2.update(zkc));
-        ZKAccessControl readZKAC3 = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+        Utils.ioResult(readZKAC2.update(zkc));
+        ZKAccessControl readZKAC3 = Utils.ioResult(ZKAccessControl.read(zkc, zkPath, null));
         assertEquals(readZKAC2, readZKAC3);
     }
 }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java b/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java
index 19c301b..868549e 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java
@@ -25,7 +25,7 @@
 import org.apache.distributedlog.impl.acl.ZKAccessControl;
 import org.apache.distributedlog.impl.acl.ZKAccessControlManager;
 import org.apache.distributedlog.thrift.AccessControlEntry;
-import com.twitter.util.Await;
+import org.apache.distributedlog.util.Utils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -136,7 +136,7 @@
             verifyStreamPermissions(zkcm, stream2, true, false, true, true, true);
 
             // delete stream2
-            Await.result(ZKAccessControl.delete(zkc, zkPath2));
+            Utils.ioResult(ZKAccessControl.delete(zkc, zkPath2));
             logger.info("Delete ACL for stream {}", stream2);
             while (!zkcm.allowTruncate(stream2)) {
                 Thread.sleep(100);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java b/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java
index 4f968b6..8a2c476 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java
@@ -17,21 +17,20 @@
  */
 package org.apache.distributedlog.admin;
 
-import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.DLMTestUtil;
 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.LogSegmentMetadata;
 import org.apache.distributedlog.TestDistributedLogBase;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
 import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.SchedulerUtils;
+import org.apache.distributedlog.common.util.SchedulerUtils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs;
 import org.junit.After;
@@ -105,7 +104,7 @@
         confLocal.setLogSegmentCacheEnabled(false);
         URI uri = createDLMURI("/check-and-repair-dl-namespace");
         zkc.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(confLocal)
                 .uri(uri)
                 .build();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java b/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java
index f911f15..f7f859c 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java
@@ -18,14 +18,14 @@
 package org.apache.distributedlog.admin;
 
 import java.net.URI;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.CompletableFuture;
 
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
-import org.apache.distributedlog.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
 import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs;
@@ -36,19 +36,16 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.distributedlog.AsyncLogReader;
+import org.apache.distributedlog.api.AsyncLogReader;
 import org.apache.distributedlog.DLMTestUtil;
 import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.LogRecordWithDLSN;
 import org.apache.distributedlog.TestDistributedLogBase;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
 import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
 
 import static org.junit.Assert.*;
 
@@ -92,11 +89,11 @@
 
         URI uri = createDLMURI("/change-sequence-number");
         zooKeeperClient.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(confLocal)
                 .uri(uri)
                 .build();
-        DistributedLogNamespace readNamespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace readNamespace = NamespaceBuilder.newBuilder()
                 .conf(readConf)
                 .uri(uri)
                 .build();
@@ -117,7 +114,7 @@
         long expectedTxId = 1L;
         DLSN lastDLSN = DLSN.InitialDLSN;
         for (int i = 0; i < 4 * 10; i++) {
-            LogRecordWithDLSN record = Await.result(reader.readNext());
+            LogRecordWithDLSN record = Utils.ioResult(reader.readNext());
             assertNotNull(record);
             DLMTestUtil.verifyLogRecord(record);
             assertEquals(expectedTxId, record.getTransactionId());
@@ -133,9 +130,9 @@
         LOG.info("Injected bad log segment '3'");
 
         // there isn't records should be read
-        Future<LogRecordWithDLSN> readFuture = reader.readNext();
+        CompletableFuture<LogRecordWithDLSN> readFuture = reader.readNext();
         try {
-            LogRecordWithDLSN record = Await.result(readFuture);
+            LogRecordWithDLSN record = Utils.ioResult(readFuture);
             fail("Should fail reading next record "
                     + record
                     + " when there is a corrupted log segment");
@@ -151,7 +148,7 @@
 
         try {
             reader = readDLM.getAsyncLogReader(lastDLSN);
-            Await.result(reader.readNext());
+            Utils.ioResult(reader.readNext());
             fail("Should fail reading next when there is a corrupted log segment");
         } catch (UnexpectedException ue) {
             // expected
@@ -166,18 +163,18 @@
         // be able to read more after fix
         reader = readDLM.getAsyncLogReader(lastDLSN);
         // skip the first record
-        Await.result(reader.readNext());
+        Utils.ioResult(reader.readNext());
         readFuture = reader.readNext();
 
         expectedTxId = 51L;
-        LogRecord record = Await.result(readFuture);
+        LogRecord record = Utils.ioResult(readFuture);
         assertNotNull(record);
         DLMTestUtil.verifyLogRecord(record);
         assertEquals(expectedTxId, record.getTransactionId());
         expectedTxId++;
 
         for (int i = 1; i < 10; i++) {
-            record = Await.result(reader.readNext());
+            record = Utils.ioResult(reader.readNext());
             assertNotNull(record);
             DLMTestUtil.verifyLogRecord(record);
             assertEquals(expectedTxId, record.getTransactionId());
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java b/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java
index 2492c06..925cad5 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java
@@ -17,22 +17,21 @@
  */
 package org.apache.distributedlog.bk;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.BookKeeperClientBuilder;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
-import org.apache.distributedlog.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
 import org.apache.distributedlog.bk.SimpleLedgerAllocator.AllocationException;
 import org.apache.distributedlog.bk.SimpleLedgerAllocator.Phase;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.TestDistributedLogBase;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.Transaction.OpListener;
 import org.apache.distributedlog.util.Utils;
 import org.apache.distributedlog.zk.DefaultZKOp;
 import org.apache.distributedlog.zk.ZKTransaction;
-import com.twitter.util.Future;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerEntry;
@@ -53,7 +52,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.net.URI;
 import java.util.Enumeration;
 import java.util.HashSet;
@@ -116,13 +114,13 @@
         return new ZKTransaction(zkc);
     }
 
-    private SimpleLedgerAllocator createAllocator(String allocationPath) throws IOException {
+    private SimpleLedgerAllocator createAllocator(String allocationPath) throws Exception {
         return createAllocator(allocationPath, dlConf);
     }
 
     private SimpleLedgerAllocator createAllocator(String allocationPath,
-                                                  DistributedLogConfiguration conf) throws IOException {
-        return FutureUtils.result(SimpleLedgerAllocator.of(allocationPath, null, newQuorumConfigProvider(conf), zkc, bkc));
+                                                  DistributedLogConfiguration conf) throws Exception {
+        return Utils.ioResult(SimpleLedgerAllocator.of(allocationPath, null, newQuorumConfigProvider(conf), zkc, bkc));
     }
 
     /**
@@ -136,13 +134,13 @@
         SimpleLedgerAllocator allocator = createAllocator(allocationPath);
         allocator.allocate();
         ZKTransaction txn = newTxn();
-        LedgerHandle lh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER));
+        LedgerHandle lh = Utils.ioResult(allocator.tryObtain(txn, NULL_LISTENER));
         logger.info("Try obtaining ledger handle {}", lh.getId());
         byte[] data = zkc.get().getData(allocationPath, false, null);
         assertEquals((Long) lh.getId(), Long.valueOf(new String(data, UTF_8)));
         txn.addOp(DefaultZKOp.of(Op.setData("/unexistedpath", "data".getBytes(UTF_8), -1), null));
         try {
-            FutureUtils.result(txn.execute());
+            Utils.ioResult(txn.execute());
             fail("Should fail the transaction when setting unexisted path");
         } catch (ZKException ke) {
             // expected
@@ -154,9 +152,9 @@
         // Create new transaction to obtain the ledger again.
         txn = newTxn();
         // we could obtain the ledger if it was obtained
-        LedgerHandle newLh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER));
+        LedgerHandle newLh = Utils.ioResult(allocator.tryObtain(txn, NULL_LISTENER));
         assertEquals(lh.getId(), newLh.getId());
-        FutureUtils.result(txn.execute());
+        Utils.ioResult(txn.execute());
         data = zkc.get().getData(allocationPath, false, null);
         assertEquals(0, data.length);
         Utils.close(allocator);
@@ -177,16 +175,16 @@
         allocator1.allocate();
         // wait until allocated
         ZKTransaction txn1 = newTxn();
-        LedgerHandle lh = FutureUtils.result(allocator1.tryObtain(txn1, NULL_LISTENER));
+        LedgerHandle lh = Utils.ioResult(allocator1.tryObtain(txn1, NULL_LISTENER));
         allocator2.allocate();
         ZKTransaction txn2 = newTxn();
         try {
-            FutureUtils.result(allocator2.tryObtain(txn2, NULL_LISTENER));
+            Utils.ioResult(allocator2.tryObtain(txn2, NULL_LISTENER));
             fail("Should fail allocating on second allocator as allocator1 is starting allocating something.");
-        } catch (ZKException zke) {
-            assertEquals(KeeperException.Code.BADVERSION, zke.getKeeperExceptionCode());
+        } catch (ZKException ke) {
+            assertEquals(KeeperException.Code.BADVERSION, ke.getKeeperExceptionCode());
         }
-        FutureUtils.result(txn1.execute());
+        Utils.ioResult(txn1.execute());
         Utils.close(allocator1);
         Utils.close(allocator2);
 
@@ -217,7 +215,7 @@
         ZKTransaction txn1 = newTxn();
 
         try {
-            FutureUtils.result(allocator1.tryObtain(txn1, NULL_LISTENER));
+            Utils.ioResult(allocator1.tryObtain(txn1, NULL_LISTENER));
             fail("Should fail allocating ledger if there aren't enough bookies");
         } catch (AllocationException ioe) {
             // expected
@@ -241,7 +239,7 @@
         allocator1.allocate();
         // wait until allocated
         ZKTransaction txn1 = newTxn();
-        LedgerHandle lh1 = FutureUtils.result(allocator1.tryObtain(txn1, NULL_LISTENER));
+        LedgerHandle lh1 = Utils.ioResult(allocator1.tryObtain(txn1, NULL_LISTENER));
 
         // Second allocator kicks in
         stat = new Stat();
@@ -252,16 +250,16 @@
         allocator2.allocate();
         // wait until allocated
         ZKTransaction txn2 = newTxn();
-        LedgerHandle lh2 = FutureUtils.result(allocator2.tryObtain(txn2, NULL_LISTENER));
+        LedgerHandle lh2 = Utils.ioResult(allocator2.tryObtain(txn2, NULL_LISTENER));
 
         // should fail to commit txn1 as version is changed by second allocator
         try {
-            FutureUtils.result(txn1.execute());
+            Utils.ioResult(txn1.execute());
             fail("Should fail commit obtaining ledger handle from first allocator as allocator is modified by second allocator.");
         } catch (ZKException ke) {
             // as expected
         }
-        FutureUtils.result(txn2.execute());
+        Utils.ioResult(txn2.execute());
         Utils.close(allocator1);
         Utils.close(allocator2);
 
@@ -298,7 +296,7 @@
         allocator.allocate();
         ZKTransaction txn = newTxn();
         // close during obtaining ledger.
-        LedgerHandle lh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER));
+        LedgerHandle lh = Utils.ioResult(allocator.tryObtain(txn, NULL_LISTENER));
         Utils.close(allocator);
         byte[] data = zkc.get().getData(allocationPath, false, null);
         assertEquals((Long) lh.getId(), Long.valueOf(new String(data, UTF_8)));
@@ -319,8 +317,8 @@
         allocator.allocate();
         ZKTransaction txn = newTxn();
         // close during obtaining ledger.
-        LedgerHandle lh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER));
-        FutureUtils.result(txn.execute());
+        LedgerHandle lh = Utils.ioResult(allocator.tryObtain(txn, NULL_LISTENER));
+        Utils.ioResult(txn.execute());
         Utils.close(allocator);
         byte[] data = zkc.get().getData(allocationPath, false, null);
         assertEquals(0, data.length);
@@ -336,10 +334,10 @@
         allocator.allocate();
         ZKTransaction txn = newTxn();
         // close during obtaining ledger.
-        LedgerHandle lh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER));
+        LedgerHandle lh = Utils.ioResult(allocator.tryObtain(txn, NULL_LISTENER));
         txn.addOp(DefaultZKOp.of(Op.setData("/unexistedpath", "data".getBytes(UTF_8), -1), null));
         try {
-            FutureUtils.result(txn.execute());
+            Utils.ioResult(txn.execute());
             fail("Should fail the transaction when setting unexisted path");
         } catch (ZKException ke) {
             // expected
@@ -358,13 +356,13 @@
         SimpleLedgerAllocator allocator = createAllocator(allcationPath);
         allocator.allocate();
         ZKTransaction txn1 = newTxn();
-        Future<LedgerHandle> obtainFuture1 = allocator.tryObtain(txn1, NULL_LISTENER);
+        CompletableFuture<LedgerHandle> obtainFuture1 = allocator.tryObtain(txn1, NULL_LISTENER);
         ZKTransaction txn2 = newTxn();
-        Future<LedgerHandle> obtainFuture2 = allocator.tryObtain(txn2, NULL_LISTENER);
-        assertTrue(obtainFuture2.isDefined());
-        assertTrue(obtainFuture2.isThrow());
+        CompletableFuture<LedgerHandle> obtainFuture2 = allocator.tryObtain(txn2, NULL_LISTENER);
+        assertTrue(obtainFuture2.isDone());
+        assertTrue(obtainFuture2.isCompletedExceptionally());
         try {
-            FutureUtils.result(obtainFuture2);
+            Utils.ioResult(obtainFuture2);
             fail("Should fail the concurrent obtain since there is already a transaction obtaining the ledger handle");
         } catch (SimpleLedgerAllocator.ConcurrentObtainException cbe) {
             // expected
@@ -380,8 +378,8 @@
         for (int i = 0; i < numLedgers; i++) {
             allocator.allocate();
             ZKTransaction txn = newTxn();
-            LedgerHandle lh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER));
-            FutureUtils.result(txn.execute());
+            LedgerHandle lh = Utils.ioResult(allocator.tryObtain(txn, NULL_LISTENER));
+            Utils.ioResult(txn.execute());
             allocatedLedgers.add(lh);
         }
         assertEquals(numLedgers, allocatedLedgers.size());
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java b/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java
index e1aaa0b..a42d688 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java
@@ -24,7 +24,6 @@
 import org.apache.distributedlog.TestDistributedLogBase;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.Transaction.OpListener;
 import org.apache.distributedlog.util.Utils;
 import org.apache.distributedlog.zk.ZKTransaction;
@@ -127,7 +126,7 @@
         for (int i = 0; i < numAllocators; i++) {
             try {
                 pool.allocate();
-                FutureUtils.result(pool.tryObtain(newTxn(), NULL_LISTENER));
+                Utils.ioResult(pool.tryObtain(newTxn(), NULL_LISTENER));
                 fail("Should fail to allocate ledger if there are enought bookies");
             } catch (SimpleLedgerAllocator.AllocationException ae) {
                 assertEquals(SimpleLedgerAllocator.Phase.ERROR, ae.getPhase());
@@ -136,7 +135,7 @@
         for (int i = 0; i < numAllocators; i++) {
             try {
                 pool.allocate();
-                FutureUtils.result(pool.tryObtain(newTxn(), NULL_LISTENER));
+                Utils.ioResult(pool.tryObtain(newTxn(), NULL_LISTENER));
                 fail("Should fail to allocate ledger if there aren't available allocators");
             } catch (SimpleLedgerAllocator.AllocationException ae) {
                 assertEquals(SimpleLedgerAllocator.Phase.ERROR, ae.getPhase());
@@ -159,7 +158,7 @@
         for (int i = 0; i < numAllocators; i++) {
             ZKTransaction txn = newTxn();
             pool.allocate();
-            LedgerHandle lh = FutureUtils.result(pool.tryObtain(txn, NULL_LISTENER));
+            LedgerHandle lh = Utils.ioResult(pool.tryObtain(txn, NULL_LISTENER));
 
             // get the corresponding ledger allocator
             SimpleLedgerAllocator sla = pool.getLedgerAllocator(lh);
@@ -176,7 +175,7 @@
             String slaPath = allocatePaths.get(i);
 
             // execute the transaction to confirm/abort obtain
-            FutureUtils.result(txn.execute());
+            Utils.ioResult(txn.execute());
 
             // introduce error to individual ledger allocator
             byte[] data = zkc.get().getData(slaPath, false, new Stat());
@@ -188,7 +187,7 @@
             try {
                 pool.allocate();
                 ZKTransaction txn = newTxn();
-                LedgerHandle lh = FutureUtils.result(pool.tryObtain(txn, NULL_LISTENER));
+                LedgerHandle lh = Utils.ioResult(pool.tryObtain(txn, NULL_LISTENER));
 
                 // get the corresponding ledger allocator
                 SimpleLedgerAllocator sla = pool.getLedgerAllocator(lh);
@@ -197,7 +196,7 @@
                 logger.info("Allocated ledger {} from path {}", lh.getId(), slaPath);
                 allocatedPathSet.add(slaPath);
 
-                FutureUtils.result(txn.execute());
+                Utils.ioResult(txn.execute());
                 ++numSuccess;
             } catch (IOException ioe) {
                 // continue
@@ -229,7 +228,7 @@
         LedgerAllocatorPool pool = new LedgerAllocatorPool(allocationPath, 0, dlConf, zkc, bkc, allocationExecutor);
         ZKTransaction txn = newTxn();
         try {
-            FutureUtils.result(pool.tryObtain(txn, NULL_LISTENER));
+            Utils.ioResult(pool.tryObtain(txn, NULL_LISTENER));
             fail("Should fail obtain ledger handle if there is no allocator.");
         } catch (SimpleLedgerAllocator.AllocationException ae) {
             fail("Should fail obtain ledger handle if there is no allocator.");
@@ -251,8 +250,8 @@
         for (int i = 0; i < numLedgers; i++) {
             pool.allocate();
             ZKTransaction txn = newTxn();
-            LedgerHandle lh = FutureUtils.result(pool.tryObtain(txn, NULL_LISTENER));
-            FutureUtils.result(txn.execute());
+            LedgerHandle lh = Utils.ioResult(pool.tryObtain(txn, NULL_LISTENER));
+            Utils.ioResult(txn.execute());
             allocatedLedgers.add(lh);
         }
         assertEquals(numLedgers, allocatedLedgers.size());
@@ -280,8 +279,8 @@
                         for (int i = 0; i < numLedgers; i++) {
                             pool.allocate();
                             ZKTransaction txn = newTxn();
-                            LedgerHandle lh = FutureUtils.result(pool.tryObtain(txn, NULL_LISTENER));
-                            FutureUtils.result(txn.execute());
+                            LedgerHandle lh = Utils.ioResult(pool.tryObtain(txn, NULL_LISTENER));
+                            Utils.ioResult(txn.execute());
                             lh.close();
                             allocatedLedgers.putIfAbsent(lh.getId(), lh);
                             logger.info("[thread {}] allocate {}th ledger {}",
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/config/PropertiesWriter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/config/PropertiesWriter.java
index f371007..5efa7e4 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/config/PropertiesWriter.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/config/PropertiesWriter.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
@@ -20,10 +20,12 @@
 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);
 
@@ -57,7 +59,7 @@
     public void save() throws Exception {
         FileOutputStream outputStream = new FileOutputStream(configFile);
         properties.store(outputStream, null);
-        configFile.setLastModified(configFile.lastModified()+1000);
+        configFile.setLastModified(configFile.lastModified() + 1000);
         LOG.debug("save modified={}", configFile.lastModified());
     }
 
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java
index b5d6300..21aa1c9 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java
@@ -23,12 +23,13 @@
 import org.apache.distributedlog.DistributedLogConfiguration;
 
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.configuration.ConfigurationException;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
+import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
+import org.apache.distributedlog.common.config.PropertiesWriter;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java
index c1ac98a..2731af3 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java
@@ -20,6 +20,8 @@
 import org.apache.distributedlog.DistributedLogConfiguration;
 
 import org.apache.distributedlog.bk.QuorumConfig;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
+import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
 import org.junit.Test;
 
 import static org.apache.distributedlog.DistributedLogConfiguration.*;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java b/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java
index 3ce4952..1064a6f 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.feature;
 
-import org.apache.distributedlog.config.ConcurrentBaseConfiguration;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.feature.SettableFeature;
 import org.junit.Test;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java b/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
index 5d4472d..f8dd245 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
@@ -18,8 +18,8 @@
 package org.apache.distributedlog.feature;
 
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.annotations.DistributedLogAnnotations;
-import org.apache.distributedlog.config.PropertiesWriter;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.common.config.PropertiesWriter;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.junit.Ignore;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java
index b2fcbf6..db9fb31 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java
@@ -23,7 +23,6 @@
 import org.apache.distributedlog.TestDistributedLogBase;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.CreateMode;
@@ -89,12 +88,12 @@
 
     @Test(timeout = 60000)
     public void testCreateLog() throws Exception {
-        assertEquals(uri, FutureUtils.result(metadataStore.createLog("test")));
+        assertEquals(uri, Utils.ioResult(metadataStore.createLog("test")));
     }
 
     @Test(timeout = 60000)
     public void testGetLogLocation() throws Exception {
-        Optional<URI> uriOptional = FutureUtils.result(metadataStore.getLogLocation("test"));
+        Optional<URI> uriOptional = Utils.ioResult(metadataStore.getLogLocation("test"));
         assertTrue(uriOptional.isPresent());
         assertEquals(uri, uriOptional.get());
     }
@@ -107,7 +106,7 @@
             logs.add(logName);
             createLogInNamespace(uri, logName);
         }
-        Set<String> result = Sets.newHashSet(FutureUtils.result(metadataStore.getLogs()));
+        Set<String> result = Sets.newHashSet(Utils.ioResult(metadataStore.getLogs()));
         assertEquals(10, result.size());
         assertTrue(Sets.difference(logs, result).isEmpty());
     }
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java
index eb81900..721bfbf 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java
@@ -18,6 +18,7 @@
 package org.apache.distributedlog.impl;
 
 import com.google.common.collect.Lists;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DLMTestUtil;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.LogSegmentMetadata;
@@ -30,15 +31,13 @@
 import org.apache.distributedlog.metadata.LogMetadata;
 import org.apache.distributedlog.metadata.LogMetadataForWriter;
 import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Transaction;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.meta.ZkVersion;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
@@ -137,7 +136,7 @@
         LogSegmentMetadata segment = createLogSegment(1L);
         Transaction<Object> createTxn = lsmStore.transaction();
         lsmStore.createLogSegment(createTxn, segment, null);
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         // the log segment should be created
         assertNotNull("LogSegment " + segment + " should be created",
                 zkc.get().exists(segment.getZkPath(), false));
@@ -145,7 +144,7 @@
         Transaction<Object> createTxn2 = lsmStore.transaction();
         lsmStore.createLogSegment(createTxn2, segment2, null);
         try {
-            FutureUtils.result(createTxn2.execute());
+            Utils.ioResult(createTxn2.execute());
             fail("Should fail if log segment exists");
         } catch (Throwable t) {
             // expected
@@ -162,13 +161,13 @@
         LogSegmentMetadata segment = createLogSegment(1L);
         Transaction<Object> createTxn = lsmStore.transaction();
         lsmStore.createLogSegment(createTxn, segment, null);
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         // the log segment should be created
         assertNotNull("LogSegment " + segment + " should be created",
                 zkc.get().exists(segment.getZkPath(), false));
         Transaction<Object> deleteTxn = lsmStore.transaction();
         lsmStore.deleteLogSegment(deleteTxn, segment, null);
-        FutureUtils.result(deleteTxn.execute());
+        Utils.ioResult(deleteTxn.execute());
         assertNull("LogSegment " + segment + " should be deleted",
                 zkc.get().exists(segment.getZkPath(), false));
     }
@@ -179,7 +178,7 @@
         Transaction<Object> deleteTxn = lsmStore.transaction();
         lsmStore.deleteLogSegment(deleteTxn, segment, null);
         try {
-            FutureUtils.result(deleteTxn.execute());
+            Utils.ioResult(deleteTxn.execute());
             fail("Should fail deletion if log segment doesn't exist");
         } catch (Throwable t) {
             assertTrue("Should throw NoNodeException if log segment doesn't exist",
@@ -196,7 +195,7 @@
         Transaction<Object> updateTxn = lsmStore.transaction();
         lsmStore.updateLogSegment(updateTxn, segment);
         try {
-            FutureUtils.result(updateTxn.execute());
+            Utils.ioResult(updateTxn.execute());
             fail("Should fail update if log segment doesn't exist");
         } catch (Throwable t) {
             assertTrue("Should throw NoNodeException if log segment doesn't exist",
@@ -212,17 +211,17 @@
         LogSegmentMetadata segment = createLogSegment(1L, 99L);
         Transaction<Object> createTxn = lsmStore.transaction();
         lsmStore.createLogSegment(createTxn, segment, null);
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         // the log segment should be created
         assertNotNull("LogSegment " + segment + " should be created",
                 zkc.get().exists(segment.getZkPath(), false));
         LogSegmentMetadata modifiedSegment = createLogSegment(1L, 999L);
         Transaction<Object> updateTxn = lsmStore.transaction();
         lsmStore.updateLogSegment(updateTxn, modifiedSegment);
-        FutureUtils.result(updateTxn.execute());
+        Utils.ioResult(updateTxn.execute());
         // the log segment should be updated
         LogSegmentMetadata readSegment =
-                FutureUtils.result(LogSegmentMetadata.read(zkc, segment.getZkPath(), true));
+                Utils.ioResult(LogSegmentMetadata.read(zkc, segment.getZkPath(), true));
         assertEquals("Last entry id should be changed from 99L to 999L",
                 999L, readSegment.getLastEntryId());
     }
@@ -234,7 +233,7 @@
         // create log segment 1
         Transaction<Object> createTxn = lsmStore.transaction();
         lsmStore.createLogSegment(createTxn, segment1, null);
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         // the log segment should be created
         assertNotNull("LogSegment " + segment1 + " should be created",
                 zkc.get().exists(segment1.getZkPath(), false));
@@ -242,7 +241,7 @@
         Transaction<Object> createDeleteTxn = lsmStore.transaction();
         lsmStore.createLogSegment(createDeleteTxn, segment2, null);
         lsmStore.deleteLogSegment(createDeleteTxn, segment1, null);
-        FutureUtils.result(createDeleteTxn.execute());
+        Utils.ioResult(createDeleteTxn.execute());
         // segment 1 should be deleted, segment 2 should be created
         assertNull("LogSegment " + segment1 + " should be deleted",
                 zkc.get().exists(segment1.getZkPath(), false));
@@ -258,7 +257,7 @@
         // create log segment 1
         Transaction<Object> createTxn = lsmStore.transaction();
         lsmStore.createLogSegment(createTxn, segment1, null);
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         // the log segment should be created
         assertNotNull("LogSegment " + segment1 + " should be created",
                 zkc.get().exists(segment1.getZkPath(), false));
@@ -268,7 +267,7 @@
         lsmStore.deleteLogSegment(createDeleteTxn, segment2, null);
         lsmStore.createLogSegment(createDeleteTxn, segment3, null);
         try {
-            FutureUtils.result(createDeleteTxn.execute());
+            Utils.ioResult(createDeleteTxn.execute());
             fail("Should fail transaction if one operation failed");
         } catch (Throwable t) {
             assertTrue("Transaction is aborted",
@@ -290,12 +289,12 @@
         LogSegmentMetadata segment = createLogSegment(1L, 99L);
         Transaction<Object> createTxn = lsmStore.transaction();
         lsmStore.createLogSegment(createTxn, segment, null);
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         // the log segment should be created
         assertNotNull("LogSegment " + segment + " should be created",
                 zkc.get().exists(segment.getZkPath(), false));
         LogSegmentMetadata readSegment =
-                FutureUtils.result(lsmStore.getLogSegment(segment.getZkPath()));
+                Utils.ioResult(lsmStore.getLogSegment(segment.getZkPath()));
         assertEquals("Log segment should match",
                 segment, readSegment);
     }
@@ -309,24 +308,24 @@
             createdSegments.add(segment);
             lsmStore.createLogSegment(createTxn, segment, null);
         }
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         String rootPath = "/" + runtime.getMethodName();
         List<String> children = zkc.get().getChildren(rootPath, false);
         Collections.sort(children);
         assertEquals("Should find 10 log segments",
                 10, children.size());
         List<String> logSegmentNames =
-                FutureUtils.result(lsmStore.getLogSegmentNames(rootPath, null)).getValue();
+                Utils.ioResult(lsmStore.getLogSegmentNames(rootPath, null)).getValue();
         Collections.sort(logSegmentNames);
         assertEquals("Should find 10 log segments",
                 10, logSegmentNames.size());
         assertEquals(children, logSegmentNames);
-        List<Future<LogSegmentMetadata>> getFutures = Lists.newArrayListWithExpectedSize(10);
+        List<CompletableFuture<LogSegmentMetadata>> getFutures = Lists.newArrayListWithExpectedSize(10);
         for (int i = 0; i < 10; i++) {
             getFutures.add(lsmStore.getLogSegment(rootPath + "/" + logSegmentNames.get(i)));
         }
         List<LogSegmentMetadata> segments =
-                FutureUtils.result(Future.collect(getFutures));
+                Utils.ioResult(FutureUtils.collect(getFutures));
         for (int i = 0; i < 10; i++) {
             assertEquals(createdSegments.get(i), segments.get(i));
         }
@@ -358,7 +357,7 @@
             LogSegmentMetadata segment = createLogSegment(i);
             lsmStore.createLogSegment(createTxn, segment, null);
         }
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         String rootPath = "/" + runtime.getMethodName();
         List<String> children = zkc.get().getChildren(rootPath, false);
         Collections.sort(children);
@@ -399,7 +398,7 @@
             LogSegmentMetadata segment = createLogSegment(i);
             lsmStore.createLogSegment(anotherCreateTxn, segment, null);
         }
-        FutureUtils.result(anotherCreateTxn.execute());
+        Utils.ioResult(anotherCreateTxn.execute());
         List<String> newChildren = zkc.get().getChildren(rootPath, false);
         Collections.sort(newChildren);
         logger.info("All log segments become {}", newChildren);
@@ -424,7 +423,7 @@
             LogSegmentMetadata segment = createLogSegment(i);
             lsmStore.createLogSegment(createTxn, segment, null);
         }
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         String rootPath = "/" + runtime.getMethodName();
         List<String> children = zkc.get().getChildren(rootPath, false);
         Collections.sort(children);
@@ -464,7 +463,7 @@
             LogSegmentMetadata segment = createLogSegment(i);
             lsmStore.deleteLogSegment(deleteTxn, segment, null);
         }
-        FutureUtils.result(deleteTxn.execute());
+        Utils.ioResult(deleteTxn.execute());
         List<String> newChildren = zkc.get().getChildren(rootPath, false);
         Collections.sort(newChildren);
         while (numNotifications.get() < 2) {
@@ -496,7 +495,7 @@
             LogSegmentMetadata segment = createLogSegment(i);
             lsmStore.createLogSegment(createTxn, segment, null);
         }
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         String rootPath = "/" + runtime.getMethodName();
         List<String> children = zkc.get().getChildren(rootPath, false);
         Collections.sort(children);
@@ -541,7 +540,7 @@
             LogSegmentMetadata segment = createLogSegment(i);
             lsmStore.createLogSegment(anotherCreateTxn, segment, null);
         }
-        FutureUtils.result(anotherCreateTxn.execute());
+        Utils.ioResult(anotherCreateTxn.execute());
         List<String> newChildren = zkc.get().getChildren(rootPath, false);
         Collections.sort(newChildren);
         logger.info("All log segments become {}", newChildren);
@@ -566,7 +565,7 @@
             LogSegmentMetadata segment = createLogSegment(i);
             lsmStore.createLogSegment(createTxn, segment, null);
         }
-        FutureUtils.result(createTxn.execute());
+        Utils.ioResult(createTxn.execute());
         String rootPath = "/" + runtime.getMethodName();
         List<String> children = zkc.get().getChildren(rootPath, false);
         Collections.sort(children);
@@ -607,7 +606,7 @@
             LogSegmentMetadata segment = createLogSegment(i);
             lsmStore.deleteLogSegment(deleteTxn, segment, null);
         }
-        FutureUtils.result(deleteTxn.execute());
+        Utils.ioResult(deleteTxn.execute());
         List<String> newChildren = zkc.get().getChildren(rootPath, false);
         Collections.sort(newChildren);
         while (numNotifications.get() < 2) {
@@ -636,23 +635,23 @@
     public void testStoreMaxLogSegmentSequenceNumber() throws Exception {
         Transaction<Object> updateTxn = lsmStore.transaction();
         Versioned<Long> value = new Versioned<Long>(999L, new ZkVersion(0));
-        final Promise<Version> result = new Promise<Version>();
+        final CompletableFuture<Version> result = new CompletableFuture<Version>();
         LogMetadata metadata = mock(LogMetadata.class);
         when(metadata.getLogSegmentsPath()).thenReturn(rootZkPath);
         lsmStore.storeMaxLogSegmentSequenceNumber(updateTxn, metadata, value,
                 new Transaction.OpListener<Version>() {
             @Override
             public void onCommit(Version r) {
-                result.setValue(r);
+                result.complete(r);
             }
 
             @Override
             public void onAbort(Throwable t) {
-                result.setException(t);
+                result.completeExceptionally(t);
             }
         });
-        FutureUtils.result(updateTxn.execute());
-        assertEquals(1, ((ZkVersion) FutureUtils.result(result)).getZnodeVersion());
+        Utils.ioResult(updateTxn.execute());
+        assertEquals(1, ((ZkVersion) Utils.ioResult(result)).getZnodeVersion());
         Stat stat = new Stat();
         byte[] data = zkc.get().getData(rootZkPath, false, stat);
         assertEquals(999L, DLUtils.deserializeLogSegmentSequenceNumber(data));
@@ -663,32 +662,32 @@
     public void testStoreMaxLogSegmentSequenceNumberBadVersion() throws Exception {
         Transaction<Object> updateTxn = lsmStore.transaction();
         Versioned<Long> value = new Versioned<Long>(999L, new ZkVersion(10));
-        final Promise<Version> result = new Promise<Version>();
+        final CompletableFuture<Version> result = new CompletableFuture<Version>();
         LogMetadata metadata = mock(LogMetadata.class);
         when(metadata.getLogSegmentsPath()).thenReturn(rootZkPath);
         lsmStore.storeMaxLogSegmentSequenceNumber(updateTxn, metadata, value,
                 new Transaction.OpListener<Version>() {
                     @Override
                     public void onCommit(Version r) {
-                        result.setValue(r);
+                        result.complete(r);
                     }
 
                     @Override
                     public void onAbort(Throwable t) {
-                        result.setException(t);
+                        result.completeExceptionally(t);
                     }
                 });
         try {
-            FutureUtils.result(updateTxn.execute());
+            Utils.ioResult(updateTxn.execute());
             fail("Should fail on storing log segment sequence number if providing bad version");
         } catch (ZKException zke) {
             assertEquals(KeeperException.Code.BADVERSION, zke.getKeeperExceptionCode());
         }
         try {
-            Await.result(result);
+            Utils.ioResult(result);
             fail("Should fail on storing log segment sequence number if providing bad version");
-        } catch (KeeperException ke) {
-            assertEquals(KeeperException.Code.BADVERSION, ke.code());
+        } catch (ZKException ze) {
+            assertEquals(KeeperException.Code.BADVERSION, ze.getKeeperExceptionCode());
         }
         Stat stat = new Stat();
         byte[] data = zkc.get().getData(rootZkPath, false, stat);
@@ -700,7 +699,7 @@
     public void testStoreMaxLogSegmentSequenceNumberOnNonExistentPath() throws Exception {
         Transaction<Object> updateTxn = lsmStore.transaction();
         Versioned<Long> value = new Versioned<Long>(999L, new ZkVersion(10));
-        final Promise<Version> result = new Promise<Version>();
+        final CompletableFuture<Version> result = new CompletableFuture<Version>();
         String nonExistentPath = rootZkPath + "/non-existent";
         LogMetadata metadata = mock(LogMetadata.class);
         when(metadata.getLogSegmentsPath()).thenReturn(nonExistentPath);
@@ -708,25 +707,25 @@
                 new Transaction.OpListener<Version>() {
                     @Override
                     public void onCommit(Version r) {
-                        result.setValue(r);
+                        result.complete(r);
                     }
 
                     @Override
                     public void onAbort(Throwable t) {
-                        result.setException(t);
+                        result.completeExceptionally(t);
                     }
                 });
         try {
-            FutureUtils.result(updateTxn.execute());
+            Utils.ioResult(updateTxn.execute());
             fail("Should fail on storing log segment sequence number if path doesn't exist");
         } catch (ZKException zke) {
             assertEquals(KeeperException.Code.NONODE, zke.getKeeperExceptionCode());
         }
         try {
-            Await.result(result);
+            Utils.ioResult(result);
             fail("Should fail on storing log segment sequence number if path doesn't exist");
-        } catch (KeeperException ke) {
-            assertEquals(KeeperException.Code.NONODE, ke.code());
+        } catch (ZKException ke) {
+            assertEquals(KeeperException.Code.NONODE, ke.getKeeperExceptionCode());
         }
     }
 
@@ -734,23 +733,23 @@
     public void testStoreMaxTxnId() throws Exception {
         Transaction<Object> updateTxn = lsmStore.transaction();
         Versioned<Long> value = new Versioned<Long>(999L, new ZkVersion(0));
-        final Promise<Version> result = new Promise<Version>();
+        final CompletableFuture<Version> result = new CompletableFuture<Version>();
         LogMetadataForWriter metadata = mock(LogMetadataForWriter.class);
         when(metadata.getMaxTxIdPath()).thenReturn(rootZkPath);
         lsmStore.storeMaxTxnId(updateTxn, metadata, value,
                 new Transaction.OpListener<Version>() {
             @Override
             public void onCommit(Version r) {
-                result.setValue(r);
+                result.complete(r);
             }
 
             @Override
             public void onAbort(Throwable t) {
-                result.setException(t);
+                result.completeExceptionally(t);
             }
         });
-        FutureUtils.result(updateTxn.execute());
-        assertEquals(1, ((ZkVersion) FutureUtils.result(result)).getZnodeVersion());
+        Utils.ioResult(updateTxn.execute());
+        assertEquals(1, ((ZkVersion) Utils.ioResult(result)).getZnodeVersion());
         Stat stat = new Stat();
         byte[] data = zkc.get().getData(rootZkPath, false, stat);
         assertEquals(999L, DLUtils.deserializeTransactionId(data));
@@ -761,32 +760,32 @@
     public void testStoreMaxTxnIdBadVersion() throws Exception {
         Transaction<Object> updateTxn = lsmStore.transaction();
         Versioned<Long> value = new Versioned<Long>(999L, new ZkVersion(10));
-        final Promise<Version> result = new Promise<Version>();
+        final CompletableFuture<Version> result = new CompletableFuture<Version>();
         LogMetadataForWriter metadata = mock(LogMetadataForWriter.class);
         when(metadata.getMaxTxIdPath()).thenReturn(rootZkPath);
         lsmStore.storeMaxTxnId(updateTxn, metadata, value,
                 new Transaction.OpListener<Version>() {
                     @Override
                     public void onCommit(Version r) {
-                        result.setValue(r);
+                        result.complete(r);
                     }
 
                     @Override
                     public void onAbort(Throwable t) {
-                        result.setException(t);
+                        result.completeExceptionally(t);
                     }
                 });
         try {
-            FutureUtils.result(updateTxn.execute());
+            Utils.ioResult(updateTxn.execute());
             fail("Should fail on storing log record transaction id if providing bad version");
         } catch (ZKException zke) {
             assertEquals(KeeperException.Code.BADVERSION, zke.getKeeperExceptionCode());
         }
         try {
-            Await.result(result);
+            Utils.ioResult(result);
             fail("Should fail on storing log record transaction id if providing bad version");
-        } catch (KeeperException ke) {
-            assertEquals(KeeperException.Code.BADVERSION, ke.code());
+        } catch (ZKException ze) {
+            assertEquals(KeeperException.Code.BADVERSION, ze.getKeeperExceptionCode());
         }
         Stat stat = new Stat();
         byte[] data = zkc.get().getData(rootZkPath, false, stat);
@@ -798,7 +797,7 @@
     public void testStoreMaxTxnIdOnNonExistentPath() throws Exception {
         Transaction<Object> updateTxn = lsmStore.transaction();
         Versioned<Long> value = new Versioned<Long>(999L, new ZkVersion(10));
-        final Promise<Version> result = new Promise<Version>();
+        final CompletableFuture<Version> result = new CompletableFuture<Version>();
         String nonExistentPath = rootZkPath + "/non-existent";
         LogMetadataForWriter metadata = mock(LogMetadataForWriter.class);
         when(metadata.getMaxTxIdPath()).thenReturn(nonExistentPath);
@@ -806,25 +805,25 @@
                 new Transaction.OpListener<Version>() {
                     @Override
                     public void onCommit(Version r) {
-                        result.setValue(r);
+                        result.complete(r);
                     }
 
                     @Override
                     public void onAbort(Throwable t) {
-                        result.setException(t);
+                        result.completeExceptionally(t);
                     }
                 });
         try {
-            FutureUtils.result(updateTxn.execute());
+            Utils.ioResult(updateTxn.execute());
             fail("Should fail on storing log record transaction id if path doesn't exist");
         } catch (ZKException zke) {
             assertEquals(KeeperException.Code.NONODE, zke.getKeeperExceptionCode());
         }
         try {
-            Await.result(result);
+            Utils.ioResult(result);
             fail("Should fail on storing log record transaction id if path doesn't exist");
-        } catch (KeeperException ke) {
-            assertEquals(KeeperException.Code.NONODE, ke.code());
+        } catch (ZKException ze) {
+            assertEquals(KeeperException.Code.NONODE, ze.getKeeperExceptionCode());
         }
     }
 
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java
index 3c6e77c..4bd513b 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java
@@ -24,7 +24,6 @@
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientUtils;
 import org.apache.distributedlog.callback.NamespaceListener;
-import org.apache.distributedlog.util.DLUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.CreateMode;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
index 9c46d96..c81eb1d 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
@@ -25,16 +25,14 @@
 import org.apache.distributedlog.TestDistributedLogBase;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.ZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClientUtils;
 import org.apache.distributedlog.callback.NamespaceListener;
 import org.apache.distributedlog.exceptions.LogExistsException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
 import org.apache.distributedlog.metadata.LogMetadataStore;
-import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.Utils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -153,7 +151,7 @@
     }
 
     private void deleteLog(String logName) throws Exception {
-        Optional<URI> logUriOptional = FutureUtils.result(metadataStore.getLogLocation(logName));
+        Optional<URI> logUriOptional = Utils.ioResult(metadataStore.getLogLocation(logName));
         assertTrue(logUriOptional.isPresent());
         URI logUri = logUriOptional.get();
         zkc.get().delete(logUri.getPath() + "/" + logName, -1);
@@ -164,12 +162,12 @@
         TestNamespaceListener listener = new TestNamespaceListener();
         metadataStore.registerNamespaceListener(listener);
         String logName = "test-log-1";
-        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
+        URI logUri = Utils.ioResult(metadataStore.createLog(logName));
         assertEquals(uri, logUri);
-        Optional<URI> logLocation = FutureUtils.result(metadataStore.getLogLocation(logName));
+        Optional<URI> logLocation = Utils.ioResult(metadataStore.getLogLocation(logName));
         assertTrue(logLocation.isPresent());
         assertEquals(uri, logLocation.get());
-        Optional<URI> notExistLogLocation = FutureUtils.result(metadataStore.getLogLocation("non-existent-log"));
+        Optional<URI> notExistLogLocation = Utils.ioResult(metadataStore.getLogLocation("non-existent-log"));
         assertFalse(notExistLogLocation.isPresent());
         // listener should receive notification
         listener.waitForDone();
@@ -178,7 +176,7 @@
         assertEquals(logName, logsIter.next());
         assertFalse(logsIter.hasNext());
         // get logs should return the log
-        Iterator<String> newLogsIter = FutureUtils.result(metadataStore.getLogs());
+        Iterator<String> newLogsIter = Utils.ioResult(metadataStore.getLogs());
         assertTrue(newLogsIter.hasNext());
         assertEquals(logName, newLogsIter.next());
         assertFalse(newLogsIter.hasNext());
@@ -191,7 +189,7 @@
         metadataStore.registerNamespaceListener(listener1);
         metadataStore.registerNamespaceListener(listener2);
         String logName = "test-multiple-listeners";
-        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
+        URI logUri = Utils.ioResult(metadataStore.createLog(logName));
         assertEquals(uri, logUri);
         listener1.waitForDone();
         listener2.waitForDone();
@@ -220,8 +218,8 @@
                 checkStore = metadataStore;
             }
             String logName = "test-create-log-" + i;
-            URI logUri = FutureUtils.result(createStore.createLog(logName));
-            Optional<URI> logLocation = FutureUtils.result(checkStore.getLogLocation(logName));
+            URI logUri = Utils.ioResult(createStore.createLog(logName));
+            Optional<URI> logLocation = Utils.ioResult(checkStore.getLogLocation(logName));
             assertTrue("Log " + logName + " doesn't exist", logLocation.isPresent());
             assertEquals("Different log location " + logLocation.get() + " is found",
                     logUri, logLocation.get());
@@ -236,10 +234,10 @@
         conf.addConfiguration(baseConf);
 
         String logName = "test-log";
-        FutureUtils.result(metadataStore.createLog(logName));
+        Utils.ioResult(metadataStore.createLog(logName));
 
-        URI subNs1 = FutureUtils.result(metadataStore.createSubNamespace());
-        URI subNs2 = FutureUtils.result(metadataStore.createSubNamespace());
+        URI subNs1 = Utils.ioResult(metadataStore.createSubNamespace());
+        URI subNs2 = Utils.ioResult(metadataStore.createSubNamespace());
 
         String duplicatedLogName = "test-duplicated-logs";
         // Create same log in different sub namespaces
@@ -247,35 +245,35 @@
         metadataStore.createLogInNamespaceSync(subNs2, duplicatedLogName);
 
         try {
-            FutureUtils.result(metadataStore.createLog("non-existent-log"));
+            Utils.ioResult(metadataStore.createLog("non-existent-log"));
             fail("should throw exception when duplicated log found");
         } catch (UnexpectedException ue) {
             // should throw unexpected exception
             assertTrue(metadataStore.duplicatedLogFound.get());
         }
         try {
-            FutureUtils.result(metadataStore.getLogLocation(logName));
+            Utils.ioResult(metadataStore.getLogLocation(logName));
             fail("should throw exception when duplicated log found");
         } catch (UnexpectedException ue) {
             // should throw unexpected exception
             assertTrue(metadataStore.duplicatedLogFound.get());
         }
         try {
-            FutureUtils.result(metadataStore.getLogLocation("non-existent-log"));
+            Utils.ioResult(metadataStore.getLogLocation("non-existent-log"));
             fail("should throw exception when duplicated log found");
         } catch (UnexpectedException ue) {
             // should throw unexpected exception
             assertTrue(metadataStore.duplicatedLogFound.get());
         }
         try {
-            FutureUtils.result(metadataStore.getLogLocation(duplicatedLogName));
+            Utils.ioResult(metadataStore.getLogLocation(duplicatedLogName));
             fail("should throw exception when duplicated log found");
         } catch (UnexpectedException ue) {
             // should throw unexpected exception
             assertTrue(metadataStore.duplicatedLogFound.get());
         }
         try {
-            FutureUtils.result(metadataStore.getLogs());
+            Utils.ioResult(metadataStore.getLogs());
             fail("should throw exception when duplicated log found");
         } catch (UnexpectedException ue) {
             // should throw unexpected exception
@@ -286,10 +284,10 @@
     @Test(timeout = 60000)
     public void testGetLogLocationWhenCacheMissed() throws Exception {
         String logName = "test-get-location-when-cache-missed";
-        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
+        URI logUri = Utils.ioResult(metadataStore.createLog(logName));
         assertEquals(uri, logUri);
         metadataStore.removeLogFromCache(logName);
-        Optional<URI> logLocation = FutureUtils.result(metadataStore.getLogLocation(logName));
+        Optional<URI> logLocation = Utils.ioResult(metadataStore.getLogLocation(logName));
         assertTrue(logLocation.isPresent());
         assertEquals(logUri, logLocation.get());
     }
@@ -297,25 +295,25 @@
     @Test(timeout = 60000, expected = LogExistsException.class)
     public void testCreateLogWhenCacheMissed() throws Exception {
         String logName = "test-create-log-when-cache-missed";
-        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
+        URI logUri = Utils.ioResult(metadataStore.createLog(logName));
         assertEquals(uri, logUri);
         metadataStore.removeLogFromCache(logName);
-        FutureUtils.result(metadataStore.createLog(logName));
+        Utils.ioResult(metadataStore.createLog(logName));
     }
 
     @Test(timeout = 60000, expected = LogExistsException.class)
     public void testCreateLogWhenLogExists() throws Exception {
         String logName = "test-create-log-when-log-exists";
-        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
+        URI logUri = Utils.ioResult(metadataStore.createLog(logName));
         assertEquals(uri, logUri);
-        FutureUtils.result(metadataStore.createLog(logName));
+        Utils.ioResult(metadataStore.createLog(logName));
     }
 
     private Set<String> createLogs(int numLogs, String prefix) throws Exception {
         Set<String> expectedLogs = Sets.newTreeSet();
         for (int i = 0; i < numLogs; i++) {
             String logName = prefix + i;
-            FutureUtils.result(metadataStore.createLog(logName));
+            Utils.ioResult(metadataStore.createLog(logName));
             expectedLogs.add(logName);
         }
         return expectedLogs;
@@ -339,7 +337,7 @@
         do {
             TimeUnit.MILLISECONDS.sleep(20);
             receivedLogs = new TreeSet<String>();
-            Iterator<String> logs = FutureUtils.result(metadataStore.getLogs());
+            Iterator<String> logs = Utils.ioResult(metadataStore.getLogs());
             receivedLogs.addAll(Lists.newArrayList(logs));
         } while (receivedLogs.size() < numLogs);
         assertEquals(numLogs, receivedLogs.size());
@@ -372,8 +370,8 @@
 
     @Test(timeout = 60000)
     public void testCreateLogPickingFirstAvailableSubNamespace() throws Exception {
-        URI subNs1 = FutureUtils.result(metadataStore.createSubNamespace());
-        URI subNs2 = FutureUtils.result(metadataStore.createSubNamespace());
+        URI subNs1 = Utils.ioResult(metadataStore.createSubNamespace());
+        URI subNs2 = Utils.ioResult(metadataStore.createSubNamespace());
 
         Set<String> logs0 = createLogs(uri, maxLogsPerSubnamespace - 1, "test-ns0-");
         Set<String> logs1 = createLogs(subNs1, maxLogsPerSubnamespace, "test-ns1-");
@@ -388,7 +386,7 @@
         do {
             TimeUnit.MILLISECONDS.sleep(20);
             receivedLogs = new TreeSet<String>();
-            Iterator<String> logs = FutureUtils.result(metadataStore.getLogs());
+            Iterator<String> logs = Utils.ioResult(metadataStore.getLogs());
             receivedLogs.addAll(Lists.newArrayList(logs));
         } while (receivedLogs.size() < 3 * maxLogsPerSubnamespace - 1);
 
@@ -396,19 +394,19 @@
                 new TestNamespaceListenerWithExpectedSize(3 * maxLogsPerSubnamespace + 1);
         metadataStore.registerNamespaceListener(listener);
 
-        Set<URI> uris = FutureUtils.result(metadataStore.fetchSubNamespaces(null));
+        Set<URI> uris = Utils.ioResult(metadataStore.fetchSubNamespaces(null));
         assertEquals(3, uris.size());
         String testLogName = "test-pick-first-available-ns";
-        URI createdURI = FutureUtils.result(metadataStore.createLog(testLogName));
+        URI createdURI = Utils.ioResult(metadataStore.createLog(testLogName));
         allLogs.add(testLogName);
         assertEquals(uri, createdURI);
-        uris = FutureUtils.result(metadataStore.fetchSubNamespaces(null));
+        uris = Utils.ioResult(metadataStore.fetchSubNamespaces(null));
         assertEquals(3, uris.size());
         testLogName = "test-create-new-ns";
-        URI newURI = FutureUtils.result(metadataStore.createLog(testLogName));
+        URI newURI = Utils.ioResult(metadataStore.createLog(testLogName));
         allLogs.add(testLogName);
         assertFalse(uris.contains(newURI));
-        uris = FutureUtils.result(metadataStore.fetchSubNamespaces(null));
+        uris = Utils.ioResult(metadataStore.fetchSubNamespaces(null));
         assertEquals(4, uris.size());
 
         listener.waitForDone();
@@ -435,7 +433,7 @@
                 .build();
         FederatedZKLogMetadataStore anotherMetadataStore =
                 new FederatedZKLogMetadataStore(anotherConf, uri, anotherZkc, scheduler);
-        FutureUtils.result(anotherMetadataStore.createLog(testLogName));
+        Utils.ioResult(anotherMetadataStore.createLog(testLogName));
 
         listener.waitForDone();
         Set<String> receivedLogs = listener.getResult();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
index 5505259..a70edf5 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
@@ -18,13 +18,14 @@
 package org.apache.distributedlog.impl.logsegment;
 
 import com.google.common.collect.Lists;
-import org.apache.distributedlog.AsyncLogWriter;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.BookKeeperClient;
 import org.apache.distributedlog.BookKeeperClientBuilder;
 import org.apache.distributedlog.DLMTestUtil;
 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.Entry;
 import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.LogRecordWithDLSN;
@@ -37,10 +38,8 @@
 import org.apache.distributedlog.injector.AsyncFailureInjector;
 import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
 import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
-import com.twitter.util.Future;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.junit.After;
 import org.junit.Before;
@@ -112,7 +111,7 @@
                 null,
                 NullStatsLogger.INSTANCE,
                 AsyncFailureInjector.NULL);
-        return (BKLogSegmentEntryReader) FutureUtils.result(store.openReader(segment, startEntryId));
+        return (BKLogSegmentEntryReader) Utils.ioResult(store.openReader(segment, startEntryId));
     }
 
     void generateCompletedLogSegments(DistributedLogManager dlm,
@@ -121,12 +120,12 @@
                                       long segmentSize) throws Exception {
         long txid = 1L;
         for (long i = 0; i < numCompletedSegments; i++) {
-            AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+            AsyncLogWriter writer = Utils.ioResult(dlm.openAsyncLogWriter());
             for (long j = 1; j <= segmentSize; j++) {
-                FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
                 LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(txid);
                 ctrlRecord.setControl();
-                FutureUtils.result(writer.write(ctrlRecord));
+                Utils.ioResult(writer.write(ctrlRecord));
             }
             Utils.close(writer);
         }
@@ -135,12 +134,12 @@
     AsyncLogWriter createInprogressLogSegment(DistributedLogManager dlm,
                                               DistributedLogConfiguration conf,
                                               long segmentSize) throws Exception {
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+        AsyncLogWriter writer = Utils.ioResult(dlm.openAsyncLogWriter());
         for (long i = 1L; i <= segmentSize; i++) {
-            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(i)));
             LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(i);
             ctrlRecord.setControl();
-            FutureUtils.result(writer.write(ctrlRecord));
+            Utils.ioResult(writer.write(ctrlRecord));
         }
         return writer;
     }
@@ -168,7 +167,7 @@
         while (!done) {
             Entry.Reader entryReader;
             try {
-                entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+                entryReader = Utils.ioResult(reader.readNext(1)).get(0);
             } catch (EndOfLogSegmentException eol) {
                 done = true;
                 continue;
@@ -205,15 +204,15 @@
                 1, segments.size());
 
         BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
-        List<Future<List<Entry.Reader>>> futures = Lists.newArrayList();
+        List<CompletableFuture<List<Entry.Reader>>> futures = Lists.newArrayList();
         for (int i = 0; i < 5; i++) {
             futures.add(reader.readNext(1));
         }
         assertFalse("Reader should not be closed yet", reader.isClosed());
         Utils.close(reader);
-        for (Future<List<Entry.Reader>> future : futures) {
+        for (CompletableFuture<List<Entry.Reader>> future : futures) {
             try {
-                FutureUtils.result(future);
+                Utils.ioResult(future);
                 fail("The read request should be cancelled");
             } catch (ReadCancelledException rce) {
                 // expected
@@ -253,7 +252,7 @@
         assertEquals(10, reader.getNextEntryId());
         assertFalse(reader.hasCaughtUpOnInprogress());
         // read first entry
-        Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+        Entry.Reader entryReader = Utils.ioResult(reader.readNext(1)).get(0);
         LogRecordWithDLSN record = entryReader.nextRecord();
         while (null != record) {
             if (!record.isControl()) {
@@ -309,7 +308,7 @@
         assertEquals(5, reader.readAheadEntries.size());
         assertEquals(5, reader.getNextEntryId());
         // read first entry
-        Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+        Entry.Reader entryReader = Utils.ioResult(reader.readNext(1)).get(0);
         LogRecordWithDLSN record = entryReader.nextRecord();
         while (null != record) {
             if (!record.isControl()) {
@@ -365,7 +364,7 @@
         assertEquals((reader.getLastAddConfirmed() + 1), reader.readAheadEntries.size());
         assertEquals((reader.getLastAddConfirmed() + 1), reader.getNextEntryId());
         // read first entry
-        Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+        Entry.Reader entryReader = Utils.ioResult(reader.readNext(1)).get(0);
         LogRecordWithDLSN record = entryReader.nextRecord();
         while (null != record) {
             if (!record.isControl()) {
@@ -415,7 +414,7 @@
         long txId = 1L;
         long entryId = 0L;
         while (true) {
-            Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+            Entry.Reader entryReader = Utils.ioResult(reader.readNext(1)).get(0);
             LogRecordWithDLSN record = entryReader.nextRecord();
             while (null != record) {
                 if (!record.isControl()) {
@@ -435,11 +434,11 @@
         }
         assertEquals(6L, txId);
 
-        Future<List<Entry.Reader>> nextReadFuture = reader.readNext(1);
+        CompletableFuture<List<Entry.Reader>> nextReadFuture = reader.readNext(1);
         // write another record to commit previous writes
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
         // the long poll will be satisfied
-        List<Entry.Reader> nextReadEntries = FutureUtils.result(nextReadFuture);
+        List<Entry.Reader> nextReadEntries = Utils.ioResult(nextReadFuture);
         assertEquals(1, nextReadEntries.size());
         assertTrue(reader.hasCaughtUpOnInprogress());
         Entry.Reader entryReader = nextReadEntries.get(0);
@@ -486,7 +485,7 @@
         long txId = 1L;
         long entryId = 0L;
         while (true) {
-            Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+            Entry.Reader entryReader = Utils.ioResult(reader.readNext(1)).get(0);
             LogRecordWithDLSN record = entryReader.nextRecord();
             while (null != record) {
                 if (!record.isControl()) {
@@ -506,11 +505,11 @@
         }
         assertEquals(6L, txId);
 
-        Future<List<Entry.Reader>> nextReadFuture = reader.readNext(1);
+        CompletableFuture<List<Entry.Reader>> nextReadFuture = reader.readNext(1);
         // write another record to commit previous writes
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
+        Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
         // the long poll will be satisfied
-        List<Entry.Reader> nextReadEntries = FutureUtils.result(nextReadFuture);
+        List<Entry.Reader> nextReadEntries = Utils.ioResult(nextReadFuture);
         assertEquals(1, nextReadEntries.size());
         Entry.Reader entryReader = nextReadEntries.get(0);
         LogRecordWithDLSN record = entryReader.nextRecord();
@@ -528,7 +527,7 @@
         ++entryId;
         // close the writer, the write will be committed
         Utils.close(writer);
-        entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+        entryReader = Utils.ioResult(reader.readNext(1)).get(0);
         record = entryReader.nextRecord();
         assertNotNull(record);
         assertFalse(record.isControl());
@@ -549,8 +548,8 @@
         try {
             // when we closed the log segment, another control record will be
             // written, so we loop over the reader until we reach end of log segment.
-            FutureUtils.result(reader.readNext(1));
-            FutureUtils.result(reader.readNext(1));
+            Utils.ioResult(reader.readNext(1));
+            Utils.ioResult(reader.readNext(1));
             fail("Should reach end of log segment");
         } catch (EndOfLogSegmentException eol) {
             // expected
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
index f67de35..813501b 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
@@ -20,19 +20,17 @@
 import com.google.common.collect.Lists;
 import org.apache.distributedlog.DLMTestUtil;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.MetadataAccessor;
+import org.apache.distributedlog.api.MetadataAccessor;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
-import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.metadata.DLMetadata;
 import org.apache.distributedlog.metadata.LogMetadataForWriter;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.distributedlog.DistributedLogConstants;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClusterTestCase;
 import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.Utils;
 import org.apache.bookkeeper.meta.ZkVersion;
 import org.apache.bookkeeper.util.ZkUtils;
@@ -129,7 +127,7 @@
     public void testCheckLogMetadataPathsWithAllocator() throws Exception {
         String logRootPath = "/" + testName.getMethodName();
         List<Versioned<byte[]>> metadatas =
-                FutureUtils.result(checkLogMetadataPaths(
+                Utils.ioResult(checkLogMetadataPaths(
                         zkc.get(), logRootPath, true));
         assertEquals("Should have 8 paths",
                 8, metadatas.size());
@@ -143,7 +141,7 @@
     public void testCheckLogMetadataPathsWithoutAllocator() throws Exception {
         String logRootPath = "/" + testName.getMethodName();
         List<Versioned<byte[]>> metadatas =
-                FutureUtils.result(checkLogMetadataPaths(
+                Utils.ioResult(checkLogMetadataPaths(
                         zkc.get(), logRootPath, false));
         assertEquals("Should have 7 paths",
                 7, metadatas.size());
@@ -169,12 +167,12 @@
         }
 
         LogMetadataForWriter logMetadata =
-                FutureUtils.result(getLog(uri, logName, logIdentifier, zkc, ownAllocator, true));
+                Utils.ioResult(getLog(uri, logName, logIdentifier, zkc, ownAllocator, true));
 
         final String logRootPath = getLogRootPath(uri, logName, logIdentifier);
 
         List<Versioned<byte[]>> metadatas =
-                FutureUtils.result(checkLogMetadataPaths(zkc.get(), logRootPath, ownAllocator));
+                Utils.ioResult(checkLogMetadataPaths(zkc.get(), logRootPath, ownAllocator));
 
         if (ownAllocator) {
             assertEquals("Should have 8 paths : ownAllocator = " + ownAllocator,
@@ -301,7 +299,7 @@
     public void testCreateLogMetadataWithCreateIfNotExistsSetToFalse() throws Exception {
         String logName = testName.getMethodName();
         String logIdentifier = "<default>";
-        FutureUtils.result(getLog(uri, logName, logIdentifier, zkc, true, false));
+        Utils.ioResult(getLog(uri, logName, logIdentifier, zkc, true, false));
     }
 
     @Test(timeout = 60000)
@@ -312,7 +310,7 @@
 
         DLMetadata.create(new BKDLConfig(zkServers, "/ledgers")).update(uri);
 
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
             .conf(new DistributedLogConfiguration())
             .uri(uri)
             .build();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java b/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java
index b2eee34..26cf979 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java
@@ -17,20 +17,18 @@
  */
 package org.apache.distributedlog.lock;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.exceptions.UnexpectedException;
 import org.apache.distributedlog.util.FailpointUtils;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.TestDistributedLogBase;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.Utils;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.ZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClientUtils;
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.zookeeper.CreateMode;
@@ -192,9 +190,9 @@
 
     private static void checkLockAndReacquire(ZKDistributedLock lock, boolean sync) throws Exception {
         lock.checkOwnershipAndReacquire();
-        Future<ZKDistributedLock> reacquireFuture = lock.getLockReacquireFuture();
+        CompletableFuture<ZKDistributedLock> reacquireFuture = lock.getLockReacquireFuture();
         if (null != reacquireFuture && sync) {
-            FutureUtils.result(reacquireFuture);
+            Utils.ioResult(reacquireFuture);
         }
     }
 
@@ -212,7 +210,7 @@
             try {
                 ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
                         Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-                FutureUtils.result(lock.asyncAcquire());
+                Utils.ioResult(lock.asyncAcquire());
                 fail("Should fail on creating lock if couldn't establishing connections to zookeeper");
             } catch (IOException ioe) {
                 // expected.
@@ -228,7 +226,7 @@
             try {
                 ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
                         Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-                FutureUtils.result(lock.asyncAcquire());
+                Utils.ioResult(lock.asyncAcquire());
                 fail("Should fail on creating lock if couldn't establishing connections to zookeeper after 3 retries");
             } catch (IOException ioe) {
                 // expected.
@@ -243,14 +241,14 @@
         try {
             ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
                 Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-            FutureUtils.result(lock.asyncAcquire());
+            Utils.ioResult(lock.asyncAcquire());
 
             Pair<String, Long> lockId1 = ((ZKSessionLock) lock.getInternalLock()).getLockId();
 
             List<String> children = getLockWaiters(zkc, lockPath);
             assertEquals(1, children.size());
             assertTrue(lock.haveLock());
-            assertEquals(lockId1, Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+            assertEquals(lockId1, Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
             lock.asyncClose();
         } finally {
@@ -268,16 +266,16 @@
         SessionLockFactory lockFactory = createLockFactory(clientId, zkc);
         ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
                 Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock.asyncAcquire());
+        Utils.ioResult(lock.asyncAcquire());
 
         Pair<String, Long> lockId1 = ((ZKSessionLock) lock.getInternalLock()).getLockId();
 
         List<String> children = getLockWaiters(zkc, lockPath);
         assertEquals(1, children.size());
         assertTrue(lock.haveLock());
-        assertEquals(lockId1, Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lockId1, Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
-        FutureUtils.result(lock.asyncClose());
+        Utils.ioResult(lock.asyncClose());
 
         children = getLockWaiters(zkc, lockPath);
         assertEquals(0, children.size());
@@ -285,25 +283,25 @@
 
         lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
                 Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock.asyncAcquire());
+        Utils.ioResult(lock.asyncAcquire());
 
         Pair<String, Long> lockId2 = ((ZKSessionLock) lock.getInternalLock()).getLockId();
 
         children = getLockWaiters(zkc, lockPath);
         assertEquals(1, children.size());
         assertTrue(lock.haveLock());
-        assertEquals(lockId2, Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lockId2, Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
         assertEquals(lockId1, lockId2);
 
-        FutureUtils.result(lock.asyncClose());
+        Utils.ioResult(lock.asyncClose());
 
         children = getLockWaiters(zkc, lockPath);
         assertEquals(0, children.size());
         assertFalse(lock.haveLock());
 
         try {
-            FutureUtils.result(lock.asyncAcquire());
+            Utils.ioResult(lock.asyncAcquire());
             fail("Should fail on acquiring a closed lock");
         } catch (UnexpectedException le) {
             // expected.
@@ -324,7 +322,7 @@
         ZKDistributedLock lock0 =
                 new ZKDistributedLock(lockStateExecutor, lockFactory0, lockPath,
                         Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock0.asyncAcquire());
+        Utils.ioResult(lock0.asyncAcquire());
 
         Pair<String, Long> lockId0_1 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
 
@@ -332,7 +330,7 @@
         assertEquals(1, children.size());
         assertTrue(lock0.haveLock());
         assertEquals(lockId0_1,
-                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+                Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
         // expire the session
         ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
@@ -347,7 +345,7 @@
         assertEquals(1, children.size());
         assertTrue(lock0.haveLock());
         assertEquals(lockId0_2,
-                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+                Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
 
         SessionLockFactory lockFactory = createLockFactory(clientId, zkc);
@@ -359,9 +357,9 @@
             @Override
             public void run() {
                 try {
-                    FutureUtils.result(lock1.asyncAcquire());
+                    Utils.ioResult(lock1.asyncAcquire());
                     lockLatch.countDown();
-                } catch (IOException e) {
+                } catch (Exception e) {
                     logger.error("Failed on locking lock1 : ", e);
                 }
             }
@@ -424,7 +422,7 @@
         ZKDistributedLock lock0 =
                 new ZKDistributedLock(lockStateExecutor, lockFactory0, lockPath,
                         Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock0.asyncAcquire());
+        Utils.ioResult(lock0.asyncAcquire());
 
         Pair<String, Long> lockId0_1 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
 
@@ -432,7 +430,7 @@
         assertEquals(1, children.size());
         assertTrue(lock0.haveLock());
         assertEquals(lockId0_1,
-                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+                Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
         ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
 
@@ -441,13 +439,13 @@
             checkLockAndReacquire(lock0, false);
         } else {
             // session expire will trigger lock re-acquisition
-            Future<ZKDistributedLock> asyncLockAcquireFuture;
+            CompletableFuture<ZKDistributedLock> asyncLockAcquireFuture;
             do {
                 Thread.sleep(1);
                 asyncLockAcquireFuture = lock0.getLockReacquireFuture();
             } while (null == asyncLockAcquireFuture && lock0.getReacquireCount() < 1);
             if (null != asyncLockAcquireFuture) {
-                Await.result(asyncLockAcquireFuture);
+                Utils.ioResult(asyncLockAcquireFuture);
             }
             checkLockAndReacquire(lock0, false);
         }
@@ -456,11 +454,11 @@
         assertTrue(lock0.haveLock());
         Pair<String, Long> lock0_2 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
         assertEquals(lock0_2,
-                Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+                Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
         assertEquals(clientId, lock0_2.getLeft());
         assertFalse(lockId0_1.equals(lock0_2));
 
-        FutureUtils.result(lock0.asyncClose());
+        Utils.ioResult(lock0.asyncClose());
 
         children = getLockWaiters(zkc, lockPath);
         assertEquals(0, children.size());
@@ -495,7 +493,7 @@
         ZKDistributedLock lock0 =
                 new ZKDistributedLock(lockStateExecutor, lockFactory0, lockPath,
                         Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock0.asyncAcquire());
+        Utils.ioResult(lock0.asyncAcquire());
 
         final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
         SessionLockFactory lockFactory1 = createLockFactory(clientId, zkc);
@@ -506,9 +504,9 @@
             @Override
             public void run() {
                 try {
-                    FutureUtils.result(lock1.asyncAcquire());
+                    Utils.ioResult(lock1.asyncAcquire());
                     lock1DoneLatch.countDown();
-                } catch (IOException e) {
+                } catch (Exception e) {
                     logger.error("Error on acquiring lock1 : ", e);
                 }
             }
@@ -524,9 +522,9 @@
         assertTrue(lock0.haveLock());
         assertFalse(lock1.haveLock());
         assertEquals(((ZKSessionLock) lock0.getInternalLock()).getLockId(),
-                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+                Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
         assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId(),
-                Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
+                Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
 
         logger.info("Expiring session on lock0");
         ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
@@ -553,14 +551,14 @@
         } else {
             logger.info("Waiting lock0 to attempt acquisition after session expired");
             // session expire will trigger lock re-acquisition
-            Future<ZKDistributedLock> asyncLockAcquireFuture;
+            CompletableFuture<ZKDistributedLock> asyncLockAcquireFuture;
             do {
                 Thread.sleep(1);
                 asyncLockAcquireFuture = lock0.getLockReacquireFuture();
             } while (null == asyncLockAcquireFuture);
 
             try {
-                Await.result(asyncLockAcquireFuture);
+                Utils.ioResult(asyncLockAcquireFuture);
                 fail("Should fail check write lock since lock is already held by other people");
             } catch (OwnershipAcquireFailedException oafe) {
                 assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId().getLeft(),
@@ -579,10 +577,10 @@
         assertFalse(lock0.haveLock());
         assertTrue(lock1.haveLock());
         assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId(),
-                Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+                Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
 
-        FutureUtils.result(lock0.asyncClose());
-        FutureUtils.result(lock1.asyncClose());
+        Utils.ioResult(lock0.asyncClose());
+        Utils.ioResult(lock1.asyncClose());
 
         children = getLockWaiters(zkc, lockPath);
         assertEquals(0, children.size());
@@ -597,7 +595,7 @@
         SessionLockFactory lockFactory = createLockFactory(clientId, zkc, conf.getLockTimeoutMilliSeconds(), 0);
         ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
             conf.getLockTimeoutMilliSeconds(), NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock.asyncAcquire());
+        Utils.ioResult(lock.asyncAcquire());
 
         // try and cleanup the underlying lock
         lock.getInternalLock().unlock();
@@ -614,14 +612,14 @@
 
         boolean exceptionEncountered = false;
         try {
-            FutureUtils.result(lock2.asyncAcquire());
+            Utils.ioResult(lock2.asyncAcquire());
         } catch (OwnershipAcquireFailedException exc) {
             assertEquals(clientId, exc.getCurrentOwner());
             exceptionEncountered = true;
         }
         assertTrue(exceptionEncountered);
-        FutureUtils.result(lock.asyncClose());
-        FutureUtils.result(lock2.asyncClose());
+        Utils.ioResult(lock.asyncClose());
+        Utils.ioResult(lock2.asyncClose());
     }
 
     @Test(timeout = 60000)
@@ -633,7 +631,7 @@
         SessionLockFactory factory = createLockFactory(clientId, zkc, conf.getLockTimeoutMilliSeconds(), 0);
         ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, factory, lockPath,
             conf.getLockTimeoutMilliSeconds(), NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock.asyncAcquire());
+        Utils.ioResult(lock.asyncAcquire());
 
         // try and cleanup the underlying lock
         lock.getInternalLock().unlock();
@@ -650,15 +648,15 @@
 
         boolean exceptionEncountered = false;
         try {
-            FutureUtils.result(lock2.asyncAcquire());
+            Utils.ioResult(lock2.asyncAcquire());
         } catch (OwnershipAcquireFailedException exc) {
             assertEquals(clientId, exc.getCurrentOwner());
             exceptionEncountered = true;
         }
         assertTrue(exceptionEncountered);
-        FutureUtils.result(lock2.asyncClose());
+        Utils.ioResult(lock2.asyncClose());
 
-        FutureUtils.result(lock.asyncClose());
+        Utils.ioResult(lock.asyncClose());
         assertEquals(false, lock.haveLock());
         assertEquals(false, lock.getInternalLock().isLockHeld());
 
@@ -666,10 +664,10 @@
         ZKDistributedLock lock3 = new ZKDistributedLock(lockStateExecutor, factory, lockPath,
             0, NullStatsLogger.INSTANCE);
 
-        FutureUtils.result(lock3.asyncAcquire());
+        Utils.ioResult(lock3.asyncAcquire());
         assertEquals(true, lock3.haveLock());
         assertEquals(true, lock3.getInternalLock().isLockHeld());
-        FutureUtils.result(lock3.asyncClose());
+        Utils.ioResult(lock3.asyncClose());
     }
 
     void assertLatchesSet(CountDownLatch[] latches, int endIndex) {
@@ -697,8 +695,8 @@
         TestLockFactory locks = new TestLockFactory(runtime.getMethodName(), zkc, lockStateExecutor);
 
         int count = 3;
-        ArrayList<Future<ZKDistributedLock>> results =
-                new ArrayList<Future<ZKDistributedLock>>(count);
+        ArrayList<CompletableFuture<ZKDistributedLock>> results =
+                new ArrayList<CompletableFuture<ZKDistributedLock>>(count);
         ZKDistributedLock[] lockArray = new ZKDistributedLock[count];
         final CountDownLatch[] latches = new CountDownLatch[count];
 
@@ -708,7 +706,7 @@
             latches[i] = new CountDownLatch(1);
             lockArray[i] = locks.createLock(i, zkc);
             final int index = i;
-            results.add(lockArray[i].asyncAcquire().addEventListener(
+            results.add(lockArray[i].asyncAcquire().whenComplete(
                 new FutureEventListener<ZKDistributedLock>() {
                     @Override
                     public void onSuccess(ZKDistributedLock lock) {
@@ -727,8 +725,8 @@
         for (int i = 0; i < count; i++) {
             latches[i].await();
             assertLatchesSet(latches, i+1);
-            Await.result(results.get(i));
-            FutureUtils.result(lockArray[i].asyncClose());
+            Utils.ioResult(results.get(i));
+            Utils.ioResult(lockArray[i].asyncClose());
         }
     }
 
@@ -738,7 +736,7 @@
         final ZKDistributedLock lock0 = locks.createLock(0, zkc);
         final ZKDistributedLock lock1 = locks.createLock(1, zkc0);
 
-        FutureUtils.result(lock0.asyncAcquire());
+        Utils.ioResult(lock0.asyncAcquire());
 
         // Initial state.
         assertLockState(lock0, true, true, lock1, false, false, 1, locks.getLockPath());
@@ -747,8 +745,8 @@
             @Override
             public void run() {
                 try {
-                    FutureUtils.result(lock1.asyncAcquire());
-                } catch (IOException e) {
+                    Utils.ioResult(lock1.asyncAcquire());
+                } catch (Exception e) {
                     fail("shouldn't fail to acquire");
                 }
             }
@@ -761,13 +759,13 @@
         }
         assertLockState(lock0, true, true, lock1, false, false, 2, locks.getLockPath());
 
-        FutureUtils.result(lock0.asyncClose());
-        Await.result(lock1.getLockAcquireFuture());
+        Utils.ioResult(lock0.asyncClose());
+        Utils.ioResult(lock1.getLockAcquireFuture());
 
         assertLockState(lock0, false, false, lock1, true, true, 1, locks.getLockPath());
 
         // Release lock1
-        FutureUtils.result(lock1.asyncClose());
+        Utils.ioResult(lock1.asyncClose());
         assertLockState(lock0, false, false, lock1, false, false, 0, locks.getLockPath());
     }
 
@@ -777,8 +775,8 @@
         final ZKDistributedLock lock0 = locks.createLock(0, zkc);
         final ZKDistributedLock lock1 = locks.createLock(1, zkc0);
 
-        FutureUtils.result(lock0.asyncAcquire());
-        Future<ZKDistributedLock> result = lock1.asyncAcquire();
+        Utils.ioResult(lock0.asyncAcquire());
+        CompletableFuture<ZKDistributedLock> result = lock1.asyncAcquire();
         // make sure we place a waiter for lock1
         while (null == lock1.getLockWaiter()) {
             TimeUnit.MILLISECONDS.sleep(20);
@@ -787,7 +785,7 @@
         // Expire causes acquire future to be failed and unset.
         ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
         try {
-            Await.result(result);
+            Utils.ioResult(result);
             fail("future should have been failed");
         } catch (OwnershipAcquireFailedException ex) {
         }
@@ -803,11 +801,11 @@
         final ZKDistributedLock lock0 = locks.createLock(0, zkc);
         final ZKDistributedLock lock1 = locks.createLock(1, zkc0);
 
-        FutureUtils.result(lock0.asyncAcquire());
-        Future<ZKDistributedLock> result = lock1.asyncAcquire();
-        FutureUtils.result(lock1.asyncClose());
+        Utils.ioResult(lock0.asyncAcquire());
+        CompletableFuture<ZKDistributedLock> result = lock1.asyncAcquire();
+        Utils.ioResult(lock1.asyncClose());
         try {
-            Await.result(result);
+            Utils.ioResult(result);
             fail("future should have been failed");
         } catch (LockClosedException ex) {
         }
@@ -821,12 +819,12 @@
         TestLockFactory locks = new TestLockFactory(runtime.getMethodName(), zkc, lockStateExecutor);
         final ZKDistributedLock lock0 = locks.createLock(0, zkc);
 
-        Future<ZKDistributedLock> result = lock0.asyncAcquire();
-        Await.result(result);
-        FutureUtils.result(lock0.asyncClose());
+        CompletableFuture<ZKDistributedLock> result = lock0.asyncAcquire();
+        Utils.ioResult(result);
+        Utils.ioResult(lock0.asyncClose());
 
         // Already have this, stays satisfied.
-        Await.result(result);
+        Utils.ioResult(result);
 
         // But we no longer have the lock.
         assertEquals(false, lock0.haveLock());
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java b/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java
index 4d4a008..5189104 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java
@@ -17,6 +17,7 @@
  */
 package org.apache.distributedlog.lock;
 
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DLMTestUtil;
 import org.apache.distributedlog.exceptions.LockingException;
 import org.apache.distributedlog.ZooKeeperClient;
@@ -27,11 +28,10 @@
 import org.apache.distributedlog.lock.ZKSessionLock.State;
 import org.apache.distributedlog.util.FailpointUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Await;
-import com.twitter.util.Promise;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.util.SafeRunnable;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
@@ -43,7 +43,6 @@
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -149,9 +148,9 @@
         String node2 = getLockIdFromPath(createLockNodeV2(zk, lockPath, clientId));
         String node3 = getLockIdFromPath(createLockNodeV3(zk, lockPath, clientId));
 
-        assertEquals(lockId, Await.result(asyncParseClientID(zk, lockPath, node1)));
-        assertEquals(lockId, Await.result(asyncParseClientID(zk, lockPath, node2)));
-        assertEquals(lockId, Await.result(asyncParseClientID(zk, lockPath, node3)));
+        assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node1)));
+        assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node2)));
+        assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node3)));
 
         // Bad Lock Node Name
         String node4 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member"));
@@ -160,15 +159,15 @@
         String node7 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode_badnode_badnode"));
         String node8 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode_badnode_badnode_badnode"));
 
-        assertEquals(lockId, Await.result(asyncParseClientID(zk, lockPath, node4)));
-        assertEquals(lockId, Await.result(asyncParseClientID(zk, lockPath, node5)));
-        assertEquals(lockId, Await.result(asyncParseClientID(zk, lockPath, node6)));
-        assertEquals(lockId, Await.result(asyncParseClientID(zk, lockPath, node7)));
-        assertEquals(lockId, Await.result(asyncParseClientID(zk, lockPath, node8)));
+        assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node4)));
+        assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node5)));
+        assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node6)));
+        assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node7)));
+        assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node8)));
 
         // Malformed Node Name
         String node9 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_malformed_s12345678_999999"));
-        assertEquals(Pair.of("malformed", 12345678L), Await.result(asyncParseClientID(zk, lockPath, node9)));
+        assertEquals(Pair.of("malformed", 12345678L), Utils.ioResult(asyncParseClientID(zk, lockPath, node9)));
     }
 
     @Test(timeout = 60000)
@@ -256,7 +255,7 @@
         assertEquals("counter should not be increased in different epochs", 1, counter.get());
 
         // lock action would not be executed in same epoch and promise would be satisfied with exception
-        Promise<BoxedUnit> promise = new Promise<BoxedUnit>();
+        CompletableFuture<Void> promise = new CompletableFuture<Void>();
         lock.executeLockAction(lock.getEpoch().get() + 1, new LockAction() {
             @Override
             public void execute() {
@@ -269,7 +268,7 @@
             }
         }, promise);
         try {
-            Await.result(promise);
+            Utils.ioResult(promise);
             fail("Should satisfy promise with epoch changed exception.");
         } catch (EpochChangedException ece) {
             // expected
@@ -457,7 +456,7 @@
         assertEquals(State.CLAIMED, lock.getLockState());
         List<String> children = getLockWaiters(zkc, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+        assertEquals(lock.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
 
         // lock should fail on a success lock
         try {
@@ -469,7 +468,7 @@
         assertEquals(State.CLAIMED, lock.getLockState());
         children = getLockWaiters(zkc, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+        assertEquals(lock.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
 
         // unlock
         lock.unlock();
@@ -546,7 +545,7 @@
         assertEquals(State.CLAIMED, lock0.getLockState());
         List<String> children = getLockWaiters(zkc0, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
         try {
             lock1.tryLock(timeout, TimeUnit.MILLISECONDS);
@@ -559,7 +558,7 @@
         assertEquals(State.CLOSED, lock1.getLockState());
         children = getLockWaiters(zkc0, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
         lock0.unlock();
         // verification after unlock lock0
@@ -574,7 +573,7 @@
         assertEquals(State.CLAIMED, lock2.getLockState());
         children = getLockWaiters(zkc, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock2.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+        assertEquals(lock2.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
 
         lock2.unlock();
     }
@@ -649,7 +648,7 @@
         assertEquals(State.CLAIMED, lock0.getLockState());
         List<String> children = getLockWaiters(zkc0, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
         final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
 
@@ -687,7 +686,7 @@
         assertEquals(State.CLAIMED, lock1.getLockState());
         children = getLockWaiters(zkc, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+        assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
 
         lock1.unlock();
     }
@@ -719,7 +718,7 @@
         assertEquals(State.CLAIMED, lock.getLockState());
         List<String> children = getLockWaiters(zkc, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+        assertEquals(lock.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
 
         ZooKeeperClientUtils.expireSession(zkc, zkServers, sessionTimeoutMs);
         expiredLatch.await();
@@ -806,7 +805,7 @@
         assertEquals(State.CLAIMED, lock0.getLockState());
         List<String> children = getLockWaiters(zkc0, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
         final ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId1, lockStateExecutor);
         final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
@@ -830,9 +829,9 @@
 
         assertEquals(2, children.size());
         assertEquals(State.CLAIMED, lock0.getLockState());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
         awaitState(State.WAITING, lock1);
-        assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
+        assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
 
         // expire lock1
         ZooKeeperClientUtils.expireSession(zkc, zkServers, sessionTimeoutMs);
@@ -843,7 +842,7 @@
         assertEquals(State.CLOSED, lock1.getLockState());
         children = getLockWaiters(zkc0, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
     }
 
     public void awaitState(State state, ZKSessionLock lock) throws InterruptedException {
@@ -891,7 +890,7 @@
         assertEquals(State.CLOSED, lock1_0.getLockState());
         List<String> children = getLockWaiters(zkc0, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
         // lock1_1 would wait the ownership
         final ZKSessionLock lock1_1 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
@@ -917,9 +916,9 @@
 
         assertEquals(2, children.size());
         assertEquals(State.CLAIMED, lock0.getLockState());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
         awaitState(State.WAITING, lock1_1);
-        assertEquals(lock1_1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
+        assertEquals(lock1_1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
 
         if (isUnlock) {
             lock0.unlock();
@@ -938,7 +937,7 @@
         assertEquals(State.CLAIMED, lock1_1.getLockState());
         children = getLockWaiters(zkc, lockPath);
         assertEquals(1, children.size());
-        assertEquals(lock1_1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+        assertEquals(lock1_1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
 
         lock1_1.unlock();
     }
@@ -1040,9 +1039,9 @@
 
         assertEquals(2, children.size());
         assertEquals(State.CLAIMED, lock0_0.getLockState());
-        assertEquals(lock0_0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0_0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
         awaitState(State.WAITING, lock1);
-        assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
+        assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
 
         final CountDownLatch lock0DoneLatch = new CountDownLatch(1);
         final AtomicReference<String> ownerFromLock0 = new AtomicReference<String>(null);
@@ -1058,9 +1057,9 @@
             children = getLockWaiters(zkc, lockPath);
             assertEquals(2, children.size());
             assertEquals(State.CLAIMED, lock0_0.getLockState());
-            assertEquals(lock0_0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+            assertEquals(lock0_0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
             assertEquals(State.WAITING, lock1.getLockState());
-            assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
+            assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
         } else {
             lock0Thread = new Thread(new Runnable() {
                 @Override
@@ -1087,11 +1086,11 @@
 
             assertEquals(3, children.size());
             assertEquals(State.CLAIMED, lock0_0.getLockState());
-            assertEquals(lock0_0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+            assertEquals(lock0_0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
             awaitState(State.WAITING, lock1);
-            assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
+            assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
             awaitState(State.WAITING, lock0_1);
-            assertEquals(lock0_1.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(2))));
+            assertEquals(lock0_1.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(2))));
         }
 
         if (isUnlock) {
@@ -1114,7 +1113,7 @@
             children = getLockWaiters(zkc, lockPath);
             assertEquals(1, children.size());
             assertEquals(State.CLAIMED, lock1.getLockState());
-            assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+            assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
         } else {
             assertNotNull(lock0Thread);
             if (!isUnlock) {
@@ -1128,14 +1127,14 @@
                 children = getLockWaiters(zkc, lockPath);
                 assertEquals(1, children.size());
                 assertEquals(State.CLAIMED, lock1.getLockState());
-                assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+                assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
             } else {
                 children = getLockWaiters(zkc, lockPath);
                 assertEquals(2, children.size());
                 assertEquals(State.CLAIMED, lock1.getLockState());
-                assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+                assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
                 assertEquals(State.WAITING, lock0_1.getLockState());
-                assertEquals(lock0_1.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(1))));
+                assertEquals(lock0_1.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(1))));
             }
         }
 
@@ -1148,7 +1147,7 @@
             children = getLockWaiters(zkc, lockPath);
             assertEquals(1, children.size());
             assertEquals(State.CLAIMED, lock0_1.getLockState());
-            assertEquals(lock0_1.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+            assertEquals(lock0_1.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
         }
     }
 
@@ -1186,15 +1185,15 @@
         List<String> children = getLockWaiters(zkc0, lockPath);
         assertEquals(1, children.size());
         assertEquals(State.CLAIMED, lock0.getLockState());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
 
         lock1.tryLock(timeout, TimeUnit.MILLISECONDS);
         children = getLockWaiters(zkc0, lockPath);
         assertEquals(2, children.size());
         assertEquals(State.CLAIMED, lock0.getLockState());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(lock0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
         assertEquals(State.CLAIMED, lock1.getLockState());
-        assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(1))));
+        assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(1))));
 
         if (isUnlock) {
             lock0.unlock();
@@ -1202,7 +1201,7 @@
             children = getLockWaiters(zkc0, lockPath);
             assertEquals(1, children.size());
             assertEquals(State.CLAIMED, lock1.getLockState());
-            assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+            assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
             lock1.unlock();
         } else {
             ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java b/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java
index 5943b64..6687b7b 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java
@@ -17,7 +17,7 @@
  */
 package org.apache.distributedlog.logsegment;
 
-import org.apache.distributedlog.util.Sizable;
+import org.apache.distributedlog.common.util.Sizable;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
index e18fb3f..2090828 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
@@ -24,12 +24,11 @@
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.ZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClusterTestCase;
 import org.apache.distributedlog.impl.ZKLogSegmentMetadataStore;
 import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
-import org.apache.distributedlog.util.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs;
 import org.junit.After;
@@ -105,7 +104,7 @@
 
         // Dryrun
         MetadataUpdater dryrunUpdater = new DryrunLogSegmentMetadataStoreUpdater(conf, metadataStore);
-        FutureUtils.result(dryrunUpdater.changeSequenceNumber(segment, 6L));
+        Utils.ioResult(dryrunUpdater.changeSequenceNumber(segment, 6L));
 
         segmentList = readLogSegments(ledgerPath);
         assertEquals(5, segmentList.size());
@@ -113,7 +112,7 @@
         // Fix the inprogress log segments
 
         MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-        FutureUtils.result(updater.changeSequenceNumber(segment, 6L));
+        Utils.ioResult(updater.changeSequenceNumber(segment, 6L));
 
         segmentList = readLogSegments(ledgerPath);
         assertEquals(6, segmentList.size());
@@ -156,19 +155,19 @@
         // Dryrun
         MetadataUpdater dryrunUpdater = new DryrunLogSegmentMetadataStoreUpdater(conf, metadataStore);
         try {
-            FutureUtils.result(dryrunUpdater.updateLastRecord(completedLogSegment, badRecord));
+            Utils.ioResult(dryrunUpdater.updateLastRecord(completedLogSegment, badRecord));
             fail("Should fail on updating dlsn that in different log segment");
         } catch (IllegalArgumentException iae) {
             // expected
         }
         try {
-            FutureUtils.result(dryrunUpdater.updateLastRecord(inprogressLogSegment, goodRecord2));
+            Utils.ioResult(dryrunUpdater.updateLastRecord(inprogressLogSegment, goodRecord2));
             fail("Should fail on updating dlsn for an inprogress log segment");
         } catch (IllegalStateException ise) {
             // expected
         }
         LogSegmentMetadata updatedCompletedLogSegment =
-                FutureUtils.result(dryrunUpdater.updateLastRecord(completedLogSegment, goodRecord1));
+                Utils.ioResult(dryrunUpdater.updateLastRecord(completedLogSegment, goodRecord1));
         assertEquals(goodLastDLSN1, updatedCompletedLogSegment.getLastDLSN());
         assertEquals(goodRecord1.getTransactionId(), updatedCompletedLogSegment.getLastTxId());
         assertTrue(updatedCompletedLogSegment.isRecordLastPositioninThisSegment(goodRecord1));
@@ -187,18 +186,18 @@
         // Fix the last dlsn
         MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
         try {
-            FutureUtils.result(updater.updateLastRecord(completedLogSegment, badRecord));
+            Utils.ioResult(updater.updateLastRecord(completedLogSegment, badRecord));
             fail("Should fail on updating dlsn that in different log segment");
         } catch (IllegalArgumentException iae) {
             // expected
         }
         try {
-            FutureUtils.result(updater.updateLastRecord(inprogressLogSegment, goodRecord2));
+            Utils.ioResult(updater.updateLastRecord(inprogressLogSegment, goodRecord2));
             fail("Should fail on updating dlsn for an inprogress log segment");
         } catch (IllegalStateException ise) {
             // expected
         }
-        updatedCompletedLogSegment = FutureUtils.result(updater.updateLastRecord(completedLogSegment, goodRecord1));
+        updatedCompletedLogSegment = Utils.ioResult(updater.updateLastRecord(completedLogSegment, goodRecord1));
         assertEquals(goodLastDLSN1, updatedCompletedLogSegment.getLastDLSN());
         assertEquals(goodRecord1.getTransactionId(), updatedCompletedLogSegment.getLastTxId());
         assertTrue(updatedCompletedLogSegment.isRecordLastPositioninThisSegment(goodRecord1));
@@ -245,28 +244,28 @@
 
         // Dryrun
         MetadataUpdater dryrunUpdater = new DryrunLogSegmentMetadataStoreUpdater(conf, metadataStore);
-        FutureUtils.result(dryrunUpdater.setLogSegmentTruncated(segmentList.get(segmentToModify)));
+        Utils.ioResult(dryrunUpdater.setLogSegmentTruncated(segmentList.get(segmentToModify)));
 
         segmentList = readLogSegments(ledgerPath);
         assertEquals(false, segmentList.get(segmentToModify).isTruncated());
 
         // change truncation for the 1st log segment
         MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-        FutureUtils.result(updater.setLogSegmentTruncated(segmentList.get(segmentToModify)));
+        Utils.ioResult(updater.setLogSegmentTruncated(segmentList.get(segmentToModify)));
 
         segmentList = readLogSegments(ledgerPath);
         assertEquals(true, segmentList.get(segmentToModify).isTruncated());
         assertEquals(false, segmentList.get(segmentToModify).isPartiallyTruncated());
 
         updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-        FutureUtils.result(updater.setLogSegmentActive(segmentList.get(segmentToModify)));
+        Utils.ioResult(updater.setLogSegmentActive(segmentList.get(segmentToModify)));
 
         segmentList = readLogSegments(ledgerPath);
         assertEquals(false, segmentList.get(segmentToModify).isTruncated());
         assertEquals(false, segmentList.get(segmentToModify).isPartiallyTruncated());
 
         updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-        FutureUtils.result(updater.setLogSegmentPartiallyTruncated(segmentList.get(segmentToModify),
+        Utils.ioResult(updater.setLogSegmentPartiallyTruncated(segmentList.get(segmentToModify),
                 segmentList.get(segmentToModify).getFirstDLSN()));
 
         segmentList = readLogSegments(ledgerPath);
@@ -274,7 +273,7 @@
         assertEquals(true, segmentList.get(segmentToModify).isPartiallyTruncated());
 
         updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-        FutureUtils.result(updater.setLogSegmentActive(segmentList.get(segmentToModify)));
+        Utils.ioResult(updater.setLogSegmentActive(segmentList.get(segmentToModify)));
 
         segmentList = readLogSegments(ledgerPath);
         assertEquals(false, segmentList.get(segmentToModify).isTruncated());
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java b/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java
similarity index 86%
rename from distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java
rename to distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java
index 46a3a6f..89b4852 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java
@@ -20,6 +20,8 @@
 import org.apache.distributedlog.BKDistributedLogNamespace;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.TestDistributedLogBase;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.junit.Test;
 
 import java.net.URI;
@@ -30,23 +32,23 @@
 /**
  * Test Namespace Builder
  */
-public class TestDistributedLogNamespaceBuilder extends TestDistributedLogBase {
+public class TestNamespaceBuilder extends TestDistributedLogBase {
 
     @Test(timeout = 60000, expected = NullPointerException.class)
     public void testEmptyBuilder() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder().build();
+        NamespaceBuilder.newBuilder().build();
     }
 
     @Test(timeout = 60000, expected = NullPointerException.class)
     public void testMissingUri() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
+        NamespaceBuilder.newBuilder()
                 .conf(new DistributedLogConfiguration())
                 .build();
     }
 
     @Test(timeout = 60000, expected = NullPointerException.class)
     public void testMissingSchemeInUri() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
+        NamespaceBuilder.newBuilder()
                 .conf(new DistributedLogConfiguration())
                 .uri(new URI("/test"))
                 .build();
@@ -54,7 +56,7 @@
 
     @Test(timeout = 60000, expected = IllegalArgumentException.class)
     public void testInvalidSchemeInUri() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
+        NamespaceBuilder.newBuilder()
                 .conf(new DistributedLogConfiguration())
                 .uri(new URI("dist://invalid/scheme/in/uri"))
                 .build();
@@ -62,7 +64,7 @@
 
     @Test(timeout = 60000, expected = IllegalArgumentException.class)
     public void testInvalidSchemeCorrectBackendInUri() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
+        NamespaceBuilder.newBuilder()
                 .conf(new DistributedLogConfiguration())
                 .uri(new URI("dist-bk://invalid/scheme/in/uri"))
                 .build();
@@ -70,7 +72,7 @@
 
     @Test(timeout = 60000, expected = IllegalArgumentException.class)
     public void testUnknownBackendInUri() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
+        NamespaceBuilder.newBuilder()
                 .conf(new DistributedLogConfiguration())
                 .uri(new URI("distributedlog-unknown://invalid/scheme/in/uri"))
                 .build();
@@ -78,7 +80,7 @@
 
     @Test(timeout = 60000, expected = NullPointerException.class)
     public void testNullStatsLogger() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
+        NamespaceBuilder.newBuilder()
                 .conf(new DistributedLogConfiguration())
                 .uri(new URI("distributedlog-bk://localhost/distributedlog"))
                 .statsLogger(null)
@@ -87,7 +89,7 @@
 
     @Test(timeout = 60000, expected = NullPointerException.class)
     public void testNullClientId() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
+        NamespaceBuilder.newBuilder()
                 .conf(new DistributedLogConfiguration())
                 .uri(new URI("distributedlog-bk://localhost/distributedlog"))
                 .clientId(null)
@@ -96,7 +98,7 @@
 
     @Test(timeout = 60000)
     public void testBuildBKDistributedLogNamespace() throws Exception {
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(new DistributedLogConfiguration())
                 .uri(new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace"))
                 .build();
@@ -110,7 +112,7 @@
 
     @Test(timeout = 60000)
     public void testBuildWhenMissingBackendInUri() throws Exception {
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(new DistributedLogConfiguration())
                 .uri(new URI("distributedlog://" + zkServers + DLOG_NAMESPACE + "/defaultnamespace"))
                 .build();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/rate/TestMovingAverageRate.java b/distributedlog-core/src/test/java/org/apache/distributedlog/rate/TestMovingAverageRate.java
deleted file mode 100644
index 8949bec..0000000
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/rate/TestMovingAverageRate.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.rate;
-
-import com.twitter.util.Duration;
-import com.twitter.util.Function;
-import com.twitter.util.MockTimer;
-import com.twitter.util.Time$;
-import com.twitter.util.TimeControl;
-
-import org.junit.Test;
-import scala.runtime.BoxedUnit;
-
-import static org.junit.Assert.*;
-
-public class TestMovingAverageRate {
-    interface TcCallback {
-        void apply(TimeControl tc);
-    }
-
-    void withCurrentTimeFrozen(final TcCallback cb) {
-        Time$.MODULE$.withCurrentTimeFrozen(new Function<TimeControl, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(TimeControl time) {
-                cb.apply(time);
-                return BoxedUnit.UNIT;
-            }
-        });
-    }
-
-    private void advance(TimeControl time, MockTimer timer, int timeMs) {
-        Duration duration = Duration.fromMilliseconds(timeMs);
-        time.advance(duration);
-        timer.tick();
-    }
-
-    @Test(timeout = 60000)
-    public void testNoChangeInUnderMinInterval() {
-        withCurrentTimeFrozen(new TcCallback() {
-            @Override
-            public void apply(TimeControl time) {
-                MockTimer timer = new MockTimer();
-                MovingAverageRateFactory factory = new MovingAverageRateFactory(timer);
-                MovingAverageRate avg60 = factory.create(60);
-                avg60.add(1000);
-                assertEquals(0, avg60.get(), 0);
-                advance(time, timer, 1);
-                assertEquals(0, avg60.get(), 0);
-                advance(time, timer, 1);
-                assertEquals(0, avg60.get(), 0);
-            }
-        });
-    }
-
-    @Test(timeout = 60000)
-    public void testFactoryWithMultipleTimers() {
-        withCurrentTimeFrozen(new TcCallback() {
-            @Override
-            public void apply(TimeControl time) {
-                MockTimer timer = new MockTimer();
-                MovingAverageRateFactory factory = new MovingAverageRateFactory(timer);
-                MovingAverageRate avg60 = factory.create(60);
-                MovingAverageRate avg30 = factory.create(30);
-
-                // Can't test this precisely because the Rate class uses its own
-                // ticker. So we can control when it gets sampled but not the time
-                // value it uses. So, just do basic validation.
-                for (int i = 0; i < 30; i++) {
-                    avg60.add(100);
-                    avg30.add(100);
-                    advance(time, timer, 1000);
-                }
-                double s1 = avg60.get();
-                assertTrue(avg30.get() > 0);
-                for (int i = 0; i < 30; i++) {
-                    advance(time, timer, 1000);
-                }
-                assertTrue(avg60.get() > 0);
-                assertTrue(avg60.get() < s1);
-                assertEquals(0.0, avg30.get(), 0);
-            }
-        });
-    }
-}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java b/distributedlog-core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java
index 47e2fae..71bf68d 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java
@@ -22,11 +22,11 @@
 import org.apache.distributedlog.DLMTestUtil;
 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.TestDistributedLogBase;
 import org.apache.distributedlog.LocalDLMEmulator;
 import org.apache.distributedlog.LogRecordWithDLSN;
-import org.apache.distributedlog.LogReader;
+import org.apache.distributedlog.api.LogReader;
 import org.apache.distributedlog.exceptions.ZKException;
 import org.apache.distributedlog.tools.DistributedLogTool.*;
 import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestFutureUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestFutureUtils.java
deleted file mode 100644
index f9e4eb8..0000000
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestFutureUtils.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.util;
-
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.fail;
-
-/**
- * Test Case for {@link FutureUtils}
- */
-public class TestFutureUtils {
-
-    static class TestException extends IOException {
-    }
-
-    @Test(timeout = 60000)
-    public void testWithin() throws Exception {
-        OrderedScheduler scheduler = OrderedScheduler.newBuilder()
-                .corePoolSize(1)
-                .name("test-within")
-                .build();
-        final Promise<Void> promiseToTimeout = new Promise<Void>();
-        final Promise<Void> finalPromise = new Promise<Void>();
-        FutureUtils.within(
-                promiseToTimeout,
-                10,
-                TimeUnit.MILLISECONDS,
-                new TestException(),
-                scheduler,
-                "test-within"
-        ).addEventListener(new FutureEventListener<Void>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                FutureUtils.setException(finalPromise, cause);
-            }
-
-            @Override
-            public void onSuccess(Void value) {
-                FutureUtils.setValue(finalPromise, value);
-            }
-        });
-        try {
-            FutureUtils.result(finalPromise);
-            fail("Should fail with TestException");
-        } catch (TestException te) {
-            // expected
-        }
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java
index 802649d..807ce02 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java
@@ -17,6 +17,7 @@
  */
 package org.apache.distributedlog.util;
 
+import org.apache.distributedlog.common.util.PermitManager;
 import org.apache.distributedlog.zk.LimitedPermitManager;
 import org.junit.Test;
 
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestSafeQueueingFuturePool.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestSafeQueueingFuturePool.java
deleted file mode 100644
index 7bfe5ed..0000000
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestSafeQueueingFuturePool.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.distributedlog.util;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Function0;
-import com.twitter.util.FuturePool;
-import com.twitter.util.ExecutorServiceFuturePool;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Future;
-
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-import scala.runtime.BoxedUnit;
-
-public class TestSafeQueueingFuturePool {
-    static final Logger LOG = LoggerFactory.getLogger(TestSafeQueueingFuturePool.class);
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    class TestFuturePool<T> {
-        final ScheduledExecutorService executor;
-        final FuturePool pool;
-        final SafeQueueingFuturePool<T> wrapper;
-        TestFuturePool() {
-            executor = Executors.newScheduledThreadPool(1);
-            pool = new ExecutorServiceFuturePool(executor);
-            wrapper = new SafeQueueingFuturePool<T>(pool);
-        }
-        public void shutdown() {
-            executor.shutdown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testSimpleSuccess() throws Exception {
-        TestFuturePool<Void> pool = new TestFuturePool<Void>();
-        final AtomicBoolean result = new AtomicBoolean(false);
-        Future<Void> future = pool.wrapper.apply(new Function0<Void>() {
-            public Void apply() {
-                result.set(true);
-                return null;
-            }
-        });
-        Await.result(future);
-        assertTrue(result.get());
-        pool.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testSimpleFailure() throws Exception {
-        TestFuturePool<Void> pool = new TestFuturePool<Void>();
-        Future<Void> future = pool.wrapper.apply(new Function0<Void>() {
-            public Void apply() {
-                throw new RuntimeException("failed");
-            }
-        });
-        try {
-            Await.result(future);
-            fail("should have thrown");
-        } catch (Exception ex) {
-        }
-        pool.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testFailedDueToClosed() throws Exception {
-        TestFuturePool<Void> pool = new TestFuturePool<Void>();
-        pool.wrapper.close();
-        Future<Void> future = pool.wrapper.apply(new Function0<Void>() {
-            public Void apply() {
-                throw new RuntimeException("failed");
-            }
-        });
-        try {
-            Await.result(future);
-            fail("should have thrown");
-        } catch (RejectedExecutionException ex) {
-        }
-        pool.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testRejectedFailure() throws Exception {
-        TestFuturePool<Void> pool = new TestFuturePool<Void>();
-        final AtomicBoolean result = new AtomicBoolean(false);
-        pool.executor.shutdown();
-        final CountDownLatch latch = new CountDownLatch(1);
-        Future<Void> future = pool.wrapper.apply(new Function0<Void>() {
-            public Void apply() {
-                result.set(true);
-                latch.countDown();
-                return null;
-            }
-        });
-        try {
-            Await.result(future);
-            fail("should have thrown");
-        } catch (RejectedExecutionException ex) {
-        }
-        assertFalse(result.get());
-        pool.wrapper.close();
-        latch.await();
-        assertTrue(result.get());
-        pool.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testRejectedBackupFailure() throws Exception {
-        TestFuturePool<Void> pool = new TestFuturePool<Void>();
-        final AtomicBoolean result = new AtomicBoolean(false);
-        pool.executor.shutdownNow();
-        final CountDownLatch latch1 = new CountDownLatch(1);
-        final CountDownLatch latch2 = new CountDownLatch(1);
-        Future<Void> future1 = pool.wrapper.apply(new Function0<Void>() {
-            public Void apply() {
-                try {
-                    latch1.await();
-                } catch (Exception ex) {
-                }
-                return null;
-            }
-        });
-
-        // Enqueue a set of futures behind.
-        final int blockedCount = 100;
-        final ArrayList<Future<Void>> blockedFutures = new ArrayList<Future<Void>>(blockedCount);
-        final int[] doneArray = new int[blockedCount];
-        final AtomicInteger doneCount = new AtomicInteger(0);
-        for (int i = 0; i < blockedCount; i++) {
-            final int index = i;
-            blockedFutures.add(pool.wrapper.apply(new Function0<Void>() {
-                public Void apply() {
-                    doneArray[index] = doneCount.getAndIncrement();
-                    return null;
-                }
-            }));
-        }
-
-        // All the futures fail when the executor is force closed.
-        latch1.countDown();
-        pool.executor.shutdownNow();
-        for (int i = 0; i < blockedCount; i++) {
-            try {
-                Await.result(blockedFutures.get(i));
-                fail("should have thrown");
-            } catch (RejectedExecutionException ex) {
-            }
-        }
-
-        // None of them have completed.
-        for (int i = 0; i < blockedCount; i++) {
-            assertEquals(0, doneArray[i]);
-        }
-
-        // Close cleans up all pending ops in order.
-        pool.wrapper.close();
-        for (int i = 0; i < blockedCount; i++) {
-            assertEquals(i, doneArray[i]);
-        }
-
-        pool.shutdown();
-    }
-}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestUtils.java
index a9db6e0..acd441c 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestUtils.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestUtils.java
@@ -108,15 +108,15 @@
     @Test(timeout = 60000)
     public void testZkGetData() throws Exception {
         String path1 = "/zk-get-data/non-existent-path";
-        Versioned<byte[]> data = FutureUtils.result(Utils.zkGetData(zkc.get(), path1, false));
+        Versioned<byte[]> data = Utils.ioResult(Utils.zkGetData(zkc.get(), path1, false));
         assertNull("No data should return from non-existent-path", data.getValue());
         assertNull("No version should return from non-existent-path", data.getVersion());
 
         String path2 = "/zk-get-data/path2";
         byte[] rawData = "test-data".getBytes(UTF_8);
-        FutureUtils.result(Utils.zkAsyncCreateFullPathOptimistic(zkc, path2, rawData,
+        Utils.ioResult(Utils.zkAsyncCreateFullPathOptimistic(zkc, path2, rawData,
                 zkc.getDefaultACL(), CreateMode.PERSISTENT));
-        data = FutureUtils.result(Utils.zkGetData(zkc.get(), path2, false));
+        data = Utils.ioResult(Utils.zkGetData(zkc.get(), path2, false));
         assertArrayEquals("Data should return as written",
                 rawData, data.getValue());
         assertEquals("Version should be zero",
diff --git a/distributedlog-protocol/pom.xml b/distributedlog-protocol/pom.xml
index a483444..7197feb 100644
--- a/distributedlog-protocol/pom.xml
+++ b/distributedlog-protocol/pom.xml
@@ -26,35 +26,15 @@
   <name>Apache DistributedLog :: Protocol</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>
+      <groupId>org.apache.distributedlog</groupId>
+      <artifactId>distributedlog-common</artifactId>
+      <version>${project.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>com.twitter</groupId>
-      <artifactId>finagle-core_2.11</artifactId>
-      <version>${finagle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-      <version>${slf4j.version}</version>
+      <groupId>org.projectlombok</groupId>
+      <artifactId>lombok</artifactId>
+      <version>${lombok.version}</version>
+      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>net.jpountz.lz4</groupId>
@@ -67,6 +47,18 @@
       <version>${junit.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.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <version>${slf4j.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   <build>
     <plugins>
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java
index 9d2d7a7..2a60ff3 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java
@@ -26,12 +26,6 @@
 import static org.apache.distributedlog.LogRecordSet.NULL_OP_STATS_LOGGER;
 import static org.apache.distributedlog.LogRecordSet.VERSION;
 
-import org.apache.distributedlog.exceptions.LogRecordTooLongException;
-import org.apache.distributedlog.exceptions.WriteException;
-import org.apache.distributedlog.io.Buffer;
-import org.apache.distributedlog.io.CompressionCodec;
-import org.apache.distributedlog.io.CompressionUtils;
-import com.twitter.util.Promise;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -39,6 +33,12 @@
 import java.nio.channels.WritableByteChannel;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.io.Buffer;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.io.CompressionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,7 +52,7 @@
     private final Buffer buffer;
     private final DataOutputStream writer;
     private final WritableByteChannel writeChannel;
-    private final List<Promise<DLSN>> promiseList;
+    private final List<CompletableFuture<DLSN>> promiseList;
     private final CompressionCodec.Type codec;
     private final int codecCode;
     private int count = 0;
@@ -61,7 +61,7 @@
     EnvelopedRecordSetWriter(int initialBufferSize,
                              CompressionCodec.Type codec) {
         this.buffer = new Buffer(Math.max(initialBufferSize, HEADER_LEN));
-        this.promiseList = new LinkedList<Promise<DLSN>>();
+        this.promiseList = new LinkedList<CompletableFuture<DLSN>>();
         this.codec = codec;
         switch (codec) {
             case LZ4:
@@ -84,13 +84,13 @@
         this.writeChannel = Channels.newChannel(writer);
     }
 
-    synchronized List<Promise<DLSN>> getPromiseList() {
+    synchronized List<CompletableFuture<DLSN>> getPromiseList() {
         return promiseList;
     }
 
     @Override
     public synchronized void writeRecord(ByteBuffer record,
-                                         Promise<DLSN> transmitPromise)
+                                         CompletableFuture<DLSN> transmitPromise)
             throws LogRecordTooLongException, WriteException {
         int logRecordSize = record.remaining();
         if (logRecordSize > MAX_LOGRECORD_SIZE) {
@@ -111,16 +111,16 @@
 
     private synchronized void satisfyPromises(long lssn, long entryId, long startSlotId) {
         long nextSlotId = startSlotId;
-        for (Promise<DLSN> promise : promiseList) {
-            promise.setValue(new DLSN(lssn, entryId, nextSlotId));
+        for (CompletableFuture<DLSN> promise : promiseList) {
+            promise.complete(new DLSN(lssn, entryId, nextSlotId));
             nextSlotId++;
         }
         promiseList.clear();
     }
 
     private synchronized void cancelPromises(Throwable reason) {
-        for (Promise<DLSN> promise : promiseList) {
-            promise.setException(reason);
+        for (CompletableFuture<DLSN> promise : promiseList) {
+            promise.completeExceptionally(reason);
         }
         promiseList.clear();
     }
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordSet.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordSet.java
index 375ed3f..55b20ff 100644
--- a/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordSet.java
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordSet.java
@@ -19,15 +19,15 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 
-import org.apache.distributedlog.exceptions.LogRecordTooLongException;
-import org.apache.distributedlog.exceptions.WriteException;
-import org.apache.distributedlog.io.CompressionCodec;
-import com.twitter.util.Promise;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
 import org.apache.bookkeeper.stats.NullStatsLogger;
 import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.io.CompressionCodec;
 
 /**
  * A set of {@link LogRecord}s.
@@ -134,7 +134,7 @@
          * @throws LogRecordTooLongException if the record is too long
          * @throws WriteException when encountered exception writing the record
          */
-        void writeRecord(ByteBuffer record, Promise<DLSN> transmitPromise)
+        void writeRecord(ByteBuffer record, CompletableFuture<DLSN> transmitPromise)
                 throws LogRecordTooLongException, WriteException;
     }
 
diff --git a/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java b/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java
index 95e03ab..1c5db24 100644
--- a/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java
+++ b/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java
@@ -25,15 +25,14 @@
 import static org.junit.Assert.fail;
 
 import com.google.common.collect.Lists;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.LogRecordSet.Reader;
 import org.apache.distributedlog.LogRecordSet.Writer;
 import org.apache.distributedlog.exceptions.LogRecordTooLongException;
 import org.apache.distributedlog.io.CompressionCodec.Type;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import java.nio.ByteBuffer;
-import java.util.List;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.junit.Test;
 
 /**
@@ -72,7 +71,7 @@
 
         ByteBuffer dataBuf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
         try {
-            writer.writeRecord(dataBuf, new Promise<DLSN>());
+            writer.writeRecord(dataBuf, new CompletableFuture<DLSN>());
             fail("Should fail on writing large record");
         } catch (LogRecordTooLongException lrtle) {
             // expected
@@ -111,18 +110,18 @@
         assertEquals("zero user bytes", HEADER_LEN, writer.getNumBytes());
         assertEquals("zero records", 0, writer.getNumRecords());
 
-        List<Future<DLSN>> writePromiseList = Lists.newArrayList();
+        List<CompletableFuture<DLSN>> writePromiseList = Lists.newArrayList();
         /// write first 5 records
         for (int i = 0; i < 5; i++) {
             ByteBuffer record = ByteBuffer.wrap(("record-" + i).getBytes(UTF_8));
-            Promise<DLSN> writePromise = new Promise<DLSN>();
+            CompletableFuture<DLSN> writePromise = new CompletableFuture<>();
             writer.writeRecord(record, writePromise);
             writePromiseList.add(writePromise);
             assertEquals((i + 1) + " records", (i + 1), writer.getNumRecords());
         }
         ByteBuffer dataBuf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
         try {
-            writer.writeRecord(dataBuf, new Promise<DLSN>());
+            writer.writeRecord(dataBuf, new CompletableFuture<>());
             fail("Should fail on writing large record");
         } catch (LogRecordTooLongException lrtle) {
             // expected
@@ -132,7 +131,7 @@
         /// write another 5 records
         for (int i = 0; i < 5; i++) {
             ByteBuffer record = ByteBuffer.wrap(("record-" + (i + 5)).getBytes(UTF_8));
-            Promise<DLSN> writePromise = new Promise<DLSN>();
+            CompletableFuture<DLSN> writePromise = new CompletableFuture<>();
             writer.writeRecord(record, writePromise);
             writePromiseList.add(writePromise);
             assertEquals((i + 6) + " records", (i + 6), writer.getNumRecords());
@@ -143,7 +142,7 @@
 
         // Test transmit complete
         writer.completeTransmit(1L, 1L, 10L);
-        List<DLSN> writeResults = Await.result(Future.collect(writePromiseList));
+        List<DLSN> writeResults = FutureUtils.result(FutureUtils.collect(writePromiseList));
         for (int i = 0; i < 10; i++) {
             assertEquals(new DLSN(1L, 1L, 10L + i), writeResults.get(i));
         }
diff --git a/distributedlog-protocol/src/test/resources/log4j.properties b/distributedlog-protocol/src/test/resources/log4j.properties
new file mode 100644
index 0000000..3e51059
--- /dev/null
+++ b/distributedlog-protocol/src/test/resources/log4j.properties
@@ -0,0 +1,51 @@
+#/**
+# * 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.
+# */
+
+#
+# DisributedLog Logging Configuration
+#
+
+# Example with rolling log file
+log4j.rootLogger=INFO, CONSOLE
+
+#disable zookeeper logging
+log4j.logger.org.apache.zookeeper=OFF
+#Set the bookkeeper level to warning
+log4j.logger.org.apache.bookkeeper=INFO
+
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.Threshold=INFO
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+# Add ROLLINGFILE to rootLogger to get log file output
+#    Log DEBUG level and above messages to a log file
+#log4j.appender.ROLLINGFILE=org.apache.log4j.DailyRollingFileAppender
+#log4j.appender.ROLLINGFILE.Threshold=INFO
+#log4j.appender.ROLLINGFILE.File=distributedlog.log
+#log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout
+#log4j.appender.ROLLINGFILE.DatePattern='.'yyyy-MM-dd-HH-mm
+#log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
+
+log4j.appender.R=org.apache.log4j.RollingFileAppender
+log4j.appender.R.Threshold=TRACE
+log4j.appender.R.File=target/error.log
+log4j.appender.R.MaxFileSize=200MB
+log4j.appender.R.MaxBackupIndex=7
+log4j.appender.R.layout=org.apache.log4j.PatternLayout
+log4j.appender.R.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n
diff --git a/distributedlog-proxy-client/pom.xml b/distributedlog-proxy-client/pom.xml
index 7392d90..25ad732 100644
--- a/distributedlog-proxy-client/pom.xml
+++ b/distributedlog-proxy-client/pom.xml
@@ -86,7 +86,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>
@@ -129,7 +129,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-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
index b3f3368..781005c 100644
--- a/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
+++ b/distributedlog-proxy-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
@@ -21,6 +21,7 @@
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
 import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+import static org.apache.distributedlog.protocol.util.TwitterFutureUtils.newJFuture;
 
 import com.google.common.base.Stopwatch;
 import com.google.common.base.Ticker;
@@ -440,7 +441,7 @@
         }
         Promise<DLSN> writePromise = new Promise<DLSN>();
         try {
-            recordSetWriter.writeRecord(buffer, writePromise);
+            recordSetWriter.writeRecord(buffer, newJFuture(writePromise));
         } catch (LogRecordTooLongException e) {
             return Future.exception(e);
         } catch (WriteException e) {
diff --git a/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/TwitterFutureUtils.java b/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/TwitterFutureUtils.java
new file mode 100644
index 0000000..6ce1fa4
--- /dev/null
+++ b/distributedlog-proxy-protocol/src/main/java/org/apache/distributedlog/protocol/util/TwitterFutureUtils.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
+ * <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.protocol.util;
+
+import com.google.common.collect.Lists;
+import com.twitter.util.Future;
+import com.twitter.util.Promise;
+import com.twitter.util.Return;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+
+/**
+ * Utils for Twitter's {@link com.twitter.util.Future}.
+ */
+public final class TwitterFutureUtils {
+
+    private TwitterFutureUtils() {}
+
+    public static <T> CompletableFuture<T> newJFuture(Promise<T> promise) {
+        CompletableFuture<T> jFuture = 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;
+    }
+
+    public static <T> void setValue(Promise<T> promise, T value) {
+        promise.updateIfEmpty(new Return<T>(value));
+    }
+
+}
diff --git a/distributedlog-proxy-server/pom.xml b/distributedlog-proxy-server/pom.xml
index d7cbd56..83b2bef 100644
--- a/distributedlog-proxy-server/pom.xml
+++ b/distributedlog-proxy-server/pom.xml
@@ -135,7 +135,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>
@@ -189,7 +189,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-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
index 81e476b..c904499 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
@@ -37,7 +37,7 @@
 import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
 import org.apache.distributedlog.thrift.service.DistributedLogService;
 import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.util.SchedulerUtils;
+import org.apache.distributedlog.common.util.SchedulerUtils;
 import com.twitter.finagle.Stack;
 import com.twitter.finagle.ThriftMuxServer$;
 import com.twitter.finagle.builder.Server;
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
index c37cd53..72f2758 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
@@ -25,6 +25,7 @@
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
 import org.apache.distributedlog.client.resolver.RegionResolver;
 import org.apache.distributedlog.client.routing.RoutingService;
@@ -35,10 +36,9 @@
 import org.apache.distributedlog.exceptions.StreamUnavailableException;
 import org.apache.distributedlog.exceptions.TooManyStreamsException;
 import org.apache.distributedlog.feature.AbstractFeatureProvider;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import org.apache.distributedlog.rate.MovingAverageRate;
-import org.apache.distributedlog.rate.MovingAverageRateFactory;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.rate.MovingAverageRate;
+import org.apache.distributedlog.common.rate.MovingAverageRateFactory;
 import org.apache.distributedlog.service.config.ServerConfiguration;
 import org.apache.distributedlog.service.config.StreamConfigProvider;
 import org.apache.distributedlog.service.placement.LeastLoadPlacementPolicy;
@@ -76,7 +76,7 @@
 import org.apache.distributedlog.thrift.service.WriteResponse;
 import org.apache.distributedlog.util.ConfUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.SchedulerUtils;
+import org.apache.distributedlog.common.util.SchedulerUtils;
 import com.twitter.util.Await;
 import com.twitter.util.Duration;
 import com.twitter.util.Function;
@@ -116,7 +116,7 @@
 
     private final ServerConfiguration serverConfig;
     private final DistributedLogConfiguration dlConfig;
-    private final DistributedLogNamespace dlNamespace;
+    private final Namespace dlNamespace;
     private final int serverRegionId;
     private final PlacementPolicy placementPolicy;
     private ServerStatus serverStatus = ServerStatus.WRITE_AND_ACCEPT;
@@ -199,7 +199,7 @@
         }
 
         // Build the namespace
-        this.dlNamespace = DistributedLogNamespaceBuilder.newBuilder()
+        this.dlNamespace = NamespaceBuilder.newBuilder()
                 .conf(dlConf)
                 .uri(uri)
                 .statsLogger(statsLogger)
@@ -218,8 +218,6 @@
         this.scheduler = OrderedScheduler.newBuilder()
                 .corePoolSize(numThreads)
                 .name("DistributedLogService-Executor")
-                .traceTaskExecution(true)
-                .statsLogger(statsLogger.scope("scheduler"))
                 .build();
 
         // Timer, kept separate to ensure reliability of timeouts.
@@ -261,7 +259,7 @@
 
         // Resource limiting
         this.timer = new ScheduledThreadPoolTimer(1, "timer", true);
-        this.movingAvgFactory = new MovingAverageRateFactory(timer);
+        this.movingAvgFactory = new MovingAverageRateFactory(scheduler);
         this.windowedRps = movingAvgFactory.create(MOVING_AVERAGE_WINDOW_SECS);
         this.windowedBps = movingAvgFactory.create(MOVING_AVERAGE_WINDOW_SECS);
         this.limiter = new ServiceRequestLimiter(
@@ -783,7 +781,7 @@
     }
 
     @VisibleForTesting
-    public DistributedLogNamespace getDistributedLogNamespace() {
+    public Namespace getDistributedLogNamespace() {
         return dlNamespace;
     }
 
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorService.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorService.java
index b1e2879..969c598 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorService.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/MonitorService.java
@@ -27,14 +27,14 @@
 import com.twitter.common.zookeeper.ServerSet;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.LogSegmentMetadata;
 import org.apache.distributedlog.callback.LogSegmentListener;
 import org.apache.distributedlog.callback.NamespaceListener;
 import org.apache.distributedlog.client.monitor.MonitorServiceClient;
 import org.apache.distributedlog.client.serverset.DLZkServerSet;
-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 com.twitter.finagle.builder.ClientBuilder;
 import com.twitter.finagle.stats.Stat;
 import com.twitter.finagle.stats.StatsReceiver;
@@ -71,7 +71,7 @@
 
     private static final Logger logger = LoggerFactory.getLogger(MonitorService.class);
 
-    private DistributedLogNamespace dlNamespace = null;
+    private Namespace dlNamespace = null;
     private MonitorServiceClient dlClient = null;
     private DLZkServerSet[] zkServerSets = null;
     private final ScheduledExecutorService executorService =
@@ -411,7 +411,7 @@
         // stats
         statsProvider.getStatsLogger("monitor").registerGauge("num_streams", numOfStreamsGauge);
         logger.info("Construct dl namespace @ {}", dlUri);
-        dlNamespace = DistributedLogNamespaceBuilder.newBuilder()
+        dlNamespace = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(dlUri)
                 .build();
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ResponseUtils.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
index 08f4b41..b327867 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
@@ -17,6 +17,7 @@
  */
 package org.apache.distributedlog.service;
 
+import java.util.concurrent.CompletionException;
 import org.apache.distributedlog.exceptions.DLException;
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
 import org.apache.distributedlog.thrift.service.BulkWriteResponse;
@@ -53,6 +54,8 @@
             }
             response.setCode(StatusCode.findByValue(dle.getCode()));
             response.setErrMsg(dle.getMessage());
+        } else if (t instanceof CompletionException) {
+            return exceptionToHeader(t.getCause());
         } else {
             response.setCode(StatusCode.INTERNAL_SERVER_ERROR);
             response.setErrMsg("Internal server error : " + t.getMessage());
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java
index 7d72093..53e16b4 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java
@@ -20,11 +20,11 @@
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.config.ConcurrentConstConfiguration;
-import org.apache.distributedlog.config.ConfigurationSubscription;
+import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
+import org.apache.distributedlog.common.config.ConfigurationSubscription;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.config.FileConfigurationBuilder;
-import org.apache.distributedlog.config.PropertiesConfigurationBuilder;
+import org.apache.distributedlog.common.config.FileConfigurationBuilder;
+import org.apache.distributedlog.common.config.PropertiesConfigurationBuilder;
 import java.io.File;
 import java.net.MalformedURLException;
 import java.util.List;
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
index 257b4be..1e62302 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
@@ -22,6 +22,7 @@
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.DistributedLogConfiguration;
 import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
 import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
 import org.apache.bookkeeper.util.ReflectionUtils;
@@ -160,7 +161,7 @@
     }
 
     /**
-     * Set the region id used to instantiate DistributedLogNamespace.
+     * Set the region id used to instantiate Namespace.
      *
      * @param regionId
      *          region id
@@ -172,9 +173,9 @@
     }
 
     /**
-     * Get the region id used to instantiate {@link org.apache.distributedlog.namespace.DistributedLogNamespace}.
+     * Get the region id used to instantiate {@link Namespace}.
      *
-     * @return region id used to instantiate DistributedLogNamespace
+     * @return region id used to instantiate Namespace
      */
     public int getRegionId() {
         return getInt(SERVER_REGION_ID, SERVER_REGION_ID_DEFAULT);
@@ -216,7 +217,7 @@
     /**
      * Get the shard id of this server.
      *
-     * <p>It would be used to instantiate the client id used for DistributedLogNamespace.
+     * <p>It would be used to instantiate the client id used for Namespace.
      *
      * @return shard id of this server.
      */
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
index 2e9dd6b..1336ddd 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
@@ -18,7 +18,7 @@
 package org.apache.distributedlog.service.placement;
 
 import org.apache.distributedlog.client.routing.RoutingService;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.api.namespace.Namespace;
 import com.twitter.util.Duration;
 import com.twitter.util.Function;
 import com.twitter.util.Future;
@@ -52,7 +52,7 @@
     private Map<String, String> streamToServer = new HashMap<String, String>();
 
     public LeastLoadPlacementPolicy(LoadAppraiser loadAppraiser, RoutingService routingService,
-                                    DistributedLogNamespace namespace, PlacementStateManager placementStateManager,
+                                    Namespace namespace, PlacementStateManager placementStateManager,
                                     Duration refreshInterval, StatsLogger statsLogger) {
         super(loadAppraiser, routingService, namespace, placementStateManager, refreshInterval, statsLogger);
         statsLogger.registerGauge("placement/load.diff", new Gauge<Number>() {
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
index ac952aa..17edc22 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
@@ -17,8 +17,8 @@
  */
 package org.apache.distributedlog.service.placement;
 
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.client.routing.RoutingService;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
 import org.apache.distributedlog.service.DLSocketAddress;
 import com.twitter.util.Duration;
 import com.twitter.util.Function0;
@@ -53,14 +53,14 @@
 
     protected final LoadAppraiser loadAppraiser;
     protected final RoutingService routingService;
-    protected final DistributedLogNamespace namespace;
+    protected final Namespace namespace;
     protected final PlacementStateManager placementStateManager;
     private final Duration refreshInterval;
     protected final OpStatsLogger placementCalcStats;
     private Timer placementRefreshTimer;
 
     public PlacementPolicy(LoadAppraiser loadAppraiser, RoutingService routingService,
-                           DistributedLogNamespace namespace, PlacementStateManager placementStateManager,
+                           Namespace namespace, PlacementStateManager placementStateManager,
                            Duration refreshInterval, StatsLogger statsLogger) {
         this.loadAppraiser = loadAppraiser;
         this.routingService = routingService;
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
index 862f05a..5dcea73 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
@@ -63,8 +63,7 @@
         serverLoadPath = uri.getPath() + SERVER_LOAD_DIR;
     }
 
-    private void createServerLoadPathIfNoExists(byte[] data)
-        throws ZooKeeperClient.ZooKeeperConnectionException, KeeperException, InterruptedException {
+    private void createServerLoadPathIfNoExists(byte[] data) throws KeeperException, IOException {
         try {
             Utils.zkCreateFullPathOptimistic(
                 zkClient, serverLoadPath, data, zkClient.getDefaultACL(), CreateMode.PERSISTENT);
@@ -152,7 +151,7 @@
                 watching = false;
                 watch(callback);
             }
-        } catch (ZooKeeperClient.ZooKeeperConnectionException | InterruptedException | KeeperException e) {
+        } catch (InterruptedException | IOException | KeeperException e) {
             logger.error("Watch of Ownership failed", e);
             watching = false;
             watch(callback);
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java
index 83ac668..7700184 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java
@@ -18,13 +18,13 @@
 package org.apache.distributedlog.service.stream;
 
 import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.exceptions.ChecksumFailedException;
 import org.apache.distributedlog.exceptions.DLException;
 import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
 import org.apache.distributedlog.service.ResponseUtils;
 import org.apache.distributedlog.thrift.service.ResponseHeader;
-import org.apache.distributedlog.util.Sequencer;
+import org.apache.distributedlog.common.util.Sequencer;
 import com.twitter.util.Future;
 import com.twitter.util.FutureEventListener;
 import com.twitter.util.Promise;
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
index 6c98468..372703a 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog.service.stream;
 
-import org.apache.distributedlog.AsyncLogWriter;
+import static org.apache.distributedlog.protocol.util.TwitterFutureUtils.newTFutureList;
+
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.acl.AccessControlManager;
@@ -33,7 +35,7 @@
 import org.apache.distributedlog.thrift.service.ResponseHeader;
 import org.apache.distributedlog.thrift.service.StatusCode;
 import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
+import org.apache.distributedlog.common.util.Sequencer;
 import com.twitter.util.ConstFuture;
 import com.twitter.util.Future;
 import com.twitter.util.Future$;
@@ -157,7 +159,7 @@
         Future<List<Future<DLSN>>> futureList;
         synchronized (txnLock) {
             records = asRecordList(buffers, sequencer);
-            futureList = writer.writeBulk(records);
+            futureList = newTFutureList(writer.writeBulk(records));
         }
 
         // Collect into a list of tries to make it easier to extract exception or DLSN.
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java
index 3ecb46f..24ce0be 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java
@@ -17,13 +17,13 @@
  */
 package org.apache.distributedlog.service.stream;
 
-import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.acl.AccessControlManager;
 import org.apache.distributedlog.exceptions.DLException;
 import org.apache.distributedlog.exceptions.RequestDeniedException;
 import org.apache.distributedlog.service.ResponseUtils;
 import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
+import org.apache.distributedlog.common.util.Sequencer;
 import com.twitter.util.Future;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.stats.Counter;
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java
index 0ffa619..c9dec80 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java
@@ -18,8 +18,9 @@
 package org.apache.distributedlog.service.stream;
 
 import static com.google.common.base.Charsets.UTF_8;
+import static org.apache.distributedlog.protocol.util.TwitterFutureUtils.newTFuture;
 
-import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.acl.AccessControlManager;
@@ -27,7 +28,7 @@
 import org.apache.distributedlog.exceptions.RequestDeniedException;
 import org.apache.distributedlog.service.ResponseUtils;
 import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
+import org.apache.distributedlog.common.util.Sequencer;
 import com.twitter.util.Future;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.stats.Counter;
@@ -78,7 +79,7 @@
                 txnId = sequencer.nextId();
                 LogRecord hbRecord = new LogRecord(txnId, HEARTBEAT_DATA);
                 hbRecord.setControl();
-                writeResult = writer.write(hbRecord);
+                writeResult = newTFuture(writer.write(hbRecord));
             }
             return writeResult.map(new AbstractFunction1<DLSN, WriteResponse>() {
                 @Override
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java
index 6ec8642..d657660 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java
@@ -17,13 +17,13 @@
  */
 package org.apache.distributedlog.service.stream;
 
-import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.acl.AccessControlManager;
 import org.apache.distributedlog.exceptions.DLException;
 import org.apache.distributedlog.exceptions.RequestDeniedException;
 import org.apache.distributedlog.service.ResponseUtils;
 import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
+import org.apache.distributedlog.common.util.Sequencer;
 import com.twitter.util.Future;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.stats.Counter;
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java
index 2b90d55..98362b5 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java
@@ -18,8 +18,8 @@
 package org.apache.distributedlog.service.stream;
 
 import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
 import org.apache.distributedlog.service.FatalErrorHandler;
 import org.apache.distributedlog.service.config.ServerConfiguration;
 import org.apache.distributedlog.service.config.StreamConfigProvider;
@@ -40,7 +40,7 @@
     private final FeatureProvider featureProvider;
     private final StreamConfigProvider streamConfigProvider;
     private final StreamPartitionConverter streamPartitionConverter;
-    private final DistributedLogNamespace dlNamespace;
+    private final Namespace dlNamespace;
     private final OrderedScheduler scheduler;
     private final FatalErrorHandler fatalErrorHandler;
     private final HashedWheelTimer requestTimer;
@@ -53,7 +53,7 @@
         FeatureProvider featureProvider,
         StreamConfigProvider streamConfigProvider,
         StreamPartitionConverter streamPartitionConverter,
-        DistributedLogNamespace dlNamespace,
+        Namespace dlNamespace,
         OrderedScheduler scheduler,
         FatalErrorHandler fatalErrorHandler,
         HashedWheelTimer requestTimer) {
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
index c0c0972..df3d64f 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
@@ -20,9 +20,10 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.AsyncLogWriter;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.AlreadyClosedException;
 import org.apache.distributedlog.exceptions.DLException;
@@ -33,15 +34,16 @@
 import org.apache.distributedlog.exceptions.StreamUnavailableException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
 import org.apache.distributedlog.io.Abortables;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.protocol.util.TwitterFutureUtils;
 import org.apache.distributedlog.service.FatalErrorHandler;
 import org.apache.distributedlog.service.ServerFeatureKeys;
 import org.apache.distributedlog.service.config.ServerConfiguration;
 import org.apache.distributedlog.service.config.StreamConfigProvider;
 import org.apache.distributedlog.service.stream.limiter.StreamRequestLimiter;
 import org.apache.distributedlog.service.streamset.Partition;
-import org.apache.distributedlog.stats.BroadCastStatsLogger;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.stats.BroadCastStatsLogger;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import org.apache.distributedlog.util.OrderedScheduler;
 import org.apache.distributedlog.util.TimeSequencer;
 import org.apache.distributedlog.util.Utils;
@@ -50,7 +52,6 @@
 import com.twitter.util.Future;
 import com.twitter.util.FutureEventListener;
 import com.twitter.util.Promise;
-import com.twitter.util.TimeoutException;
 import com.twitter.util.Timer;
 import java.io.IOException;
 import java.util.ArrayDeque;
@@ -126,7 +127,7 @@
     private final StreamRequestLimiter limiter;
     private final DynamicDistributedLogConfiguration dynConf;
     private final DistributedLogConfiguration dlConfig;
-    private final DistributedLogNamespace dlNamespace;
+    private final Namespace dlNamespace;
     private final String clientId;
     private final OrderedScheduler scheduler;
     private final ReentrantReadWriteLock closeLock = new ReentrantReadWriteLock();
@@ -169,7 +170,7 @@
                DynamicDistributedLogConfiguration streamConf,
                FeatureProvider featureProvider,
                StreamConfigProvider streamConfigProvider,
-               DistributedLogNamespace dlNamespace,
+               Namespace dlNamespace,
                OrderedScheduler scheduler,
                FatalErrorHandler fatalErrorHandler,
                HashedWheelTimer requestTimer,
@@ -555,8 +556,8 @@
     Future<Boolean> acquireStream() {
         final Stopwatch stopwatch = Stopwatch.createStarted();
         final Promise<Boolean> acquirePromise = new Promise<Boolean>();
-        manager.openAsyncLogWriter().addEventListener(
-            FutureUtils.OrderedFutureEventListener.of(new FutureEventListener<AsyncLogWriter>() {
+        manager.openAsyncLogWriter().whenCompleteAsync(
+            new org.apache.distributedlog.common.concurrent.FutureEventListener<AsyncLogWriter>() {
 
             @Override
             public void onSuccess(AsyncLogWriter w) {
@@ -568,7 +569,7 @@
                 onAcquireStreamFailure(cause, stopwatch, acquirePromise);
             }
 
-        }, scheduler, getStreamName()));
+        }, scheduler.chooseExecutor(getStreamName()));
         return acquirePromise;
     }
 
@@ -662,7 +663,7 @@
             pendingOpsCounter.dec();
         }
         Abortables.asyncAbort(oldWriter, true);
-        FutureUtils.setValue(acquirePromise, success);
+        TwitterFutureUtils.setValue(acquirePromise, success);
     }
 
     //
@@ -802,7 +803,7 @@
                 logger.info("Removed cached stream {}.", getStreamName());
             }
         }
-        FutureUtils.setValue(closePromise, null);
+        TwitterFutureUtils.setValue(closePromise, null);
     }
 
     /**
@@ -825,7 +826,7 @@
         }
         logger.info("Closing stream {} ...", name);
         // Close the writers to release the locks before failing the requests
-        Future<Void> closeWriterFuture;
+        CompletableFuture<Void> closeWriterFuture;
         if (abort) {
             closeWriterFuture = Abortables.asyncAbort(writer, true);
         } else {
@@ -839,25 +840,38 @@
             closeWaitDuration = Duration.fromMilliseconds(writerCloseTimeoutMs);
         }
 
-        FutureUtils.stats(
+        CompletableFuture<Void> maskedFuture = FutureUtils.createFuture();
+        FutureUtils.proxyTo(
+            FutureUtils.stats(
                 closeWriterFuture,
                 writerCloseStatLogger,
                 Stopwatch.createStarted()
-        ).masked().within(futureTimer, closeWaitDuration)
-                .addEventListener(FutureUtils.OrderedFutureEventListener.of(
-                new FutureEventListener<Void>() {
-                    @Override
-                    public void onSuccess(Void value) {
-                        postClose(uncache);
+            ),
+            maskedFuture);
+
+        FutureUtils.within(
+            maskedFuture,
+            closeWaitDuration.inMillis(),
+            TimeUnit.MILLISECONDS,
+            new java.util.concurrent.TimeoutException("Timeout on closing"),
+            scheduler,
+            name
+        ).whenCompleteAsync(
+            new org.apache.distributedlog.common.concurrent.FutureEventListener<Void>() {
+                @Override
+                public void onSuccess(Void value) {
+                    postClose(uncache);
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+                    if (cause instanceof java.util.concurrent.TimeoutException) {
+                        writerCloseTimeoutCounter.inc();
                     }
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        if (cause instanceof TimeoutException) {
-                            writerCloseTimeoutCounter.inc();
-                        }
-                        postClose(uncache);
-                    }
-                }, scheduler, name));
+                }
+            },
+            scheduler.chooseExecutor(name)
+        );
         return closePromise;
     }
 
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
index 5d54738..fd57c17 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
@@ -21,11 +21,11 @@
 import com.google.common.base.Optional;
 import com.google.common.util.concurrent.RateLimiter;
 import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.ServiceUnavailableException;
 import org.apache.distributedlog.exceptions.StreamUnavailableException;
 import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
 import org.apache.distributedlog.service.config.StreamConfigProvider;
 import org.apache.distributedlog.service.streamset.Partition;
 import org.apache.distributedlog.service.streamset.PartitionMap;
@@ -86,7 +86,7 @@
     private final String clientId;
     private boolean closed = false;
     private final StreamFactory streamFactory;
-    private final DistributedLogNamespace dlNamespace;
+    private final Namespace dlNamespace;
 
     public StreamManagerImpl(String clientId,
                              DistributedLogConfiguration dlConfig,
@@ -94,7 +94,7 @@
                              StreamFactory streamFactory,
                              StreamPartitionConverter partitionConverter,
                              StreamConfigProvider streamConfigProvider,
-                             DistributedLogNamespace dlNamespace) {
+                             Namespace dlNamespace) {
         this.clientId = clientId;
         this.executorService = executorService;
         this.streamFactory = streamFactory;
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java
index d0b8de4..b608e11 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java
@@ -18,10 +18,10 @@
 package org.apache.distributedlog.service.stream;
 
 import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.exceptions.DLException;
 import org.apache.distributedlog.thrift.service.ResponseHeader;
-import org.apache.distributedlog.util.Sequencer;
+import org.apache.distributedlog.common.util.Sequencer;
 import com.twitter.util.Future;
 
 /**
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java
index f3fc610..feb2c6a 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java
@@ -18,7 +18,7 @@
 package org.apache.distributedlog.service.stream;
 
 import org.apache.distributedlog.service.streamset.Partition;
-import org.apache.distributedlog.stats.BroadCastStatsLogger;
+import org.apache.distributedlog.common.stats.BroadCastStatsLogger;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.OpStatsLogger;
 import org.apache.bookkeeper.stats.StatsLogger;
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java
index 0036a5c..5d6dd1c 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog.service.stream;
 
-import org.apache.distributedlog.AsyncLogWriter;
+import static org.apache.distributedlog.protocol.util.TwitterFutureUtils.newTFuture;
+
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.acl.AccessControlManager;
 import org.apache.distributedlog.exceptions.DLException;
@@ -25,14 +27,13 @@
 import org.apache.distributedlog.protocol.util.ProtocolUtils;
 import org.apache.distributedlog.service.ResponseUtils;
 import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
+import org.apache.distributedlog.common.util.Sequencer;
 import com.twitter.util.Future;
 import org.apache.bookkeeper.feature.Feature;
 import org.apache.bookkeeper.stats.Counter;
 import org.apache.bookkeeper.stats.StatsLogger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
 
 /**
  * Operation to truncate a log stream.
@@ -72,12 +73,7 @@
             logger.error("Truncate: Stream Name Mismatch in the Stream Map {}, {}", stream, writer.getStreamName());
             return Future.exception(new IllegalStateException("The stream mapping is incorrect, fail the request"));
         }
-        return writer.truncate(dlsn).map(new AbstractFunction1<Boolean, WriteResponse>() {
-            @Override
-            public WriteResponse apply(Boolean v1) {
-                return ResponseUtils.writeSuccess();
-            }
-        });
+        return newTFuture(writer.truncate(dlsn).thenApply((value) -> ResponseUtils.writeSuccess()));
     }
 
     @Override
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
index 2e7ffb8..0a8a2da 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
@@ -17,7 +17,9 @@
  */
 package org.apache.distributedlog.service.stream;
 
-import org.apache.distributedlog.AsyncLogWriter;
+import static org.apache.distributedlog.protocol.util.TwitterFutureUtils.newTFuture;
+
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.acl.AccessControlManager;
@@ -31,7 +33,7 @@
 import org.apache.distributedlog.thrift.service.ResponseHeader;
 import org.apache.distributedlog.thrift.service.StatusCode;
 import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
+import org.apache.distributedlog.common.util.Sequencer;
 import com.twitter.util.Future;
 import com.twitter.util.FutureEventListener;
 import java.nio.ByteBuffer;
@@ -150,7 +152,7 @@
             if (isRecordSet) {
                 record.setRecordSet();
             }
-            writeResult = writer.write(record);
+            writeResult = newTFuture(writer.write(record));
         }
         return writeResult.map(new AbstractFunction1<DLSN, WriteResponse>() {
             @Override
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
index de805aa..549262d 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
@@ -22,7 +22,7 @@
 import org.apache.distributedlog.limiter.ChainedRequestLimiter;
 import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
 import org.apache.distributedlog.limiter.RequestLimiter;
-import org.apache.distributedlog.rate.MovingAverageRate;
+import org.apache.distributedlog.common.rate.MovingAverageRate;
 import org.apache.distributedlog.service.stream.StreamManager;
 import org.apache.distributedlog.service.stream.StreamOp;
 import org.apache.bookkeeper.feature.Feature;
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
index 7675d6f..2551a5e 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
@@ -20,7 +20,7 @@
 import org.apache.distributedlog.exceptions.OverCapacityException;
 import org.apache.distributedlog.exceptions.TooManyStreamsException;
 import org.apache.distributedlog.limiter.RequestLimiter;
-import org.apache.distributedlog.rate.MovingAverageRate;
+import org.apache.distributedlog.common.rate.MovingAverageRate;
 import org.apache.distributedlog.service.stream.StreamManager;
 import org.apache.distributedlog.service.stream.StreamOp;
 import org.apache.bookkeeper.stats.Counter;
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
index 4a5dd01..16e36c9 100644
--- a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
@@ -26,23 +26,23 @@
 import static org.junit.Assert.fail;
 
 import com.google.common.base.Optional;
-import org.apache.distributedlog.AsyncLogReader;
+import org.apache.distributedlog.api.AsyncLogReader;
 import org.apache.distributedlog.DLMTestUtil;
 import org.apache.distributedlog.DLSN;
-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.LogRecordWithDLSN;
 import org.apache.distributedlog.TestZooKeeperClientBuilder;
 import org.apache.distributedlog.ZooKeeperClient;
 import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
 import org.apache.distributedlog.client.routing.LocalRoutingService;
 import org.apache.distributedlog.exceptions.DLException;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
 import org.apache.distributedlog.impl.acl.ZKAccessControl;
 import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.service.stream.StreamManagerImpl;
 import org.apache.distributedlog.thrift.AccessControlEntry;
 import org.apache.distributedlog.thrift.service.BulkWriteResponse;
@@ -50,7 +50,7 @@
 import org.apache.distributedlog.thrift.service.StatusCode;
 import org.apache.distributedlog.thrift.service.WriteContext;
 import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import com.twitter.finagle.builder.ClientBuilder;
 import com.twitter.finagle.thrift.ClientId$;
 import com.twitter.util.Await;
@@ -105,7 +105,7 @@
         HeartbeatOptions hbOptions = new HeartbeatOptions();
         hbOptions.setSendHeartBeatToReader(true);
         // make sure the first log segment of each stream created
-        FutureUtils.result(dlClient.dlClient.heartbeat(name));
+        Await.result(dlClient.dlClient.heartbeat(name));
 
         DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
         LogReader reader = dlm.getInputStream(1);
@@ -305,7 +305,7 @@
         }
 
         validateFailedAsLogRecordTooLong(futures.get(writeCount));
-        FutureUtils.result(Futures.collect(futures.subList(writeCount + 1, 2 * writeCount + 1)));
+        Await.result(Futures.collect(futures.subList(writeCount + 1, 2 * writeCount + 1)));
         assertEquals(writeCount, succeeded);
     }
 
@@ -325,7 +325,7 @@
 
         List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
         validateFailedAsLogRecordTooLong(futures.get(0));
-        FutureUtils.result(Futures.collect(futures.subList(1, writeCount + 1)));
+        Await.result(Futures.collect(futures.subList(1, writeCount + 1)));
     }
 
     @Test(timeout = 60000)
@@ -601,7 +601,7 @@
                 .connectionTimeoutMs(60000)
                 .sessionTimeoutMs(60000)
                 .build();
-        DistributedLogNamespace dlNamespace = dlServer.dlServer.getLeft().getDistributedLogNamespace();
+        Namespace dlNamespace = dlServer.dlServer.getLeft().getDistributedLogNamespace();
         BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, getUri());
         String zkPath = getUri().getPath() + "/" + bkdlConfig.getACLRootPath() + "/" + name;
         ZKAccessControl accessControl = new ZKAccessControl(ace, zkPath);
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
index 4a2d65f..60f814e 100644
--- a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
@@ -50,7 +50,6 @@
 import org.apache.distributedlog.thrift.service.WriteContext;
 import org.apache.distributedlog.thrift.service.WriteResponse;
 import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.util.FutureUtils;
 import com.twitter.util.Await;
 import com.twitter.util.Future;
 import java.net.URI;
@@ -446,7 +445,8 @@
             assertTrue("Write should not fail before closing",
                     futureList.get(i).isDefined());
             WriteResponse response = Await.result(futureList.get(i));
-            assertTrue("Op should fail with " + StatusCode.WRITE_CANCELLED_EXCEPTION,
+            assertTrue("Op should fail with " + StatusCode.WRITE_CANCELLED_EXCEPTION
+                    + " but " + response.getHeader().getCode() + " is received.",
                     StatusCode.BK_TRANSMIT_ERROR == response.getHeader().getCode()
                         || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
                         || StatusCode.WRITE_CANCELLED_EXCEPTION == response.getHeader().getCode());
@@ -500,7 +500,7 @@
         }
         assertTrue("Stream " + streamName + " should be cached",
                 streamManager.getCachedStreams().containsKey(streamName));
-        List<WriteResponse> resultList = FutureUtils.result(Future.collect(futureList));
+        List<WriteResponse> resultList = Await.result(Future.collect(futureList));
         for (WriteResponse wr : resultList) {
             assertEquals(DLSN.InvalidDLSN, DLSN.deserialize(wr.getDlsn()));
         }
@@ -689,7 +689,7 @@
             HeartbeatOptions hbOptions = new HeartbeatOptions();
             hbOptions.setSendHeartBeatToReader(true);
             // make sure the first log segment of each stream created
-            FutureUtils.result(localService.heartbeatWithOptions(streamName, new WriteContext(), hbOptions));
+            Await.result(localService.heartbeatWithOptions(streamName, new WriteContext(), hbOptions));
             for (int j = 0; j < numWrites; j++) {
                 futureList.add(localService.write(streamName, createRecord(i * numWrites + j)));
             }
@@ -741,7 +741,7 @@
             HeartbeatOptions hbOptions = new HeartbeatOptions();
             hbOptions.setSendHeartBeatToReader(true);
             // make sure the first log segment of each stream created
-            FutureUtils.result(localService.heartbeatWithOptions(streamName, new WriteContext(), hbOptions));
+            Await.result(localService.heartbeatWithOptions(streamName, new WriteContext(), hbOptions));
             for (int j = 0; j < numWrites; j++) {
                 futureList.add(localService.write(streamName, createRecord(i * numWrites + j)));
             }
@@ -803,7 +803,7 @@
 
         service.startPlacementPolicy();
 
-        WriteResponse response = FutureUtils.result(service.getOwner("stream-1", new WriteContext()));
+        WriteResponse response = Await.result(service.getOwner("stream-1", new WriteContext()));
         assertEquals(StatusCode.FOUND, response.getHeader().getCode());
         assertEquals(service.getServiceAddress().toString(),
                 response.getHeader().getLocation());
@@ -824,7 +824,7 @@
         assertNull(stream.getLastException());
 
         // the stream is acquired
-        response = FutureUtils.result(service.getOwner("stream-2", new WriteContext()));
+        response = Await.result(service.getOwner("stream-2", new WriteContext()));
         assertEquals(StatusCode.FOUND, response.getHeader().getCode());
         assertEquals(service.getServiceAddress().toString(),
                 response.getHeader().getLocation());
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
index 5f5ecd4..5b8e3a6 100644
--- a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
@@ -26,8 +26,8 @@
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.client.routing.RoutingService;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
 import com.twitter.util.Await;
 import com.twitter.util.Duration;
 import com.twitter.util.Future;
@@ -55,7 +55,7 @@
         int numSevers = new Random().nextInt(20) + 1;
         int numStreams = new Random().nextInt(200) + 1;
         RoutingService mockRoutingService = mock(RoutingService.class);
-        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
+        Namespace mockNamespace = mock(Namespace.class);
         LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
             new EqualLoadAppraiser(),
             mockRoutingService,
@@ -81,7 +81,7 @@
         int numStreams = new Random().nextInt(200) + 1;
         RoutingService mockRoutingService = mock(RoutingService.class);
         when(mockRoutingService.getHosts()).thenReturn(generateSocketAddresses(numSevers));
-        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
+        Namespace mockNamespace = mock(Namespace.class);
         try {
             when(mockNamespace.getLogs()).thenReturn(generateStreams(numStreams).iterator());
         } catch (IOException e) {
@@ -112,7 +112,7 @@
     /* use AtomicInteger to have a final object in answer method */
         final AtomicInteger maxLoad = new AtomicInteger(Integer.MIN_VALUE);
         RoutingService mockRoutingService = mock(RoutingService.class);
-        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
+        Namespace mockNamespace = mock(Namespace.class);
         LoadAppraiser mockLoadAppraiser = mock(LoadAppraiser.class);
         when(mockLoadAppraiser.getStreamLoad(anyString())).then(new Answer<Future<StreamLoad>>() {
             @Override
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
index 56e9483..2e87b71 100644
--- a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
@@ -26,8 +26,8 @@
 import static org.mockito.Mockito.when;
 
 import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
 import org.apache.distributedlog.service.config.StreamConfigProvider;
 import org.apache.distributedlog.service.streamset.Partition;
 import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
@@ -67,7 +67,7 @@
                 mockStreamFactory,
                 mockPartitionConverter,
                 mockStreamConfigProvider,
-                mock(DistributedLogNamespace.class));
+                mock(Namespace.class));
 
         assertFalse(streamManager.isAcquired("stream1"));
         assertEquals(0, streamManager.numAcquired());
@@ -117,7 +117,7 @@
             (DynamicDistributedLogConfiguration) any(),
             (StreamManager) any())
         ).thenReturn(mockStream);
-        DistributedLogNamespace dlNamespace = mock(DistributedLogNamespace.class);
+        Namespace dlNamespace = mock(Namespace.class);
         ScheduledExecutorService executorService = new ScheduledThreadPoolExecutor(1);
 
         StreamManager streamManager = new StreamManagerImpl(
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java
index a18fda1..dc861a4 100644
--- a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java
@@ -22,7 +22,7 @@
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.api.AsyncLogWriter;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.LogRecord;
 import org.apache.distributedlog.acl.DefaultAccessControlManager;
@@ -32,9 +32,9 @@
 import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
 import org.apache.distributedlog.thrift.service.StatusCode;
 import org.apache.distributedlog.thrift.service.WriteResponse;
-import org.apache.distributedlog.util.Sequencer;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.util.Sequencer;
 import com.twitter.util.Await;
-import com.twitter.util.Future;
 import java.nio.ByteBuffer;
 import org.apache.bookkeeper.feature.SettableFeature;
 import org.apache.bookkeeper.stats.NullStatsLogger;
@@ -79,7 +79,7 @@
     @Test(timeout = 60000)
     public void testResponseSucceededThenFailed() throws Exception {
         AsyncLogWriter writer = mock(AsyncLogWriter.class);
-        when(writer.write((LogRecord) any())).thenReturn(Future.value(new DLSN(1, 2, 3)));
+        when(writer.write((LogRecord) any())).thenReturn(FutureUtils.value(new DLSN(1, 2, 3)));
         when(writer.getStreamName()).thenReturn("test");
         WriteOp writeOp = getWriteOp();
         writeOp.execute(writer, new Sequencer() {
diff --git a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
index 431bfa4..ccf3188 100644
--- a/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
+++ b/distributedlog-proxy-server/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
@@ -21,7 +21,7 @@
 import static org.junit.Assert.fail;
 
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.config.ConcurrentConstConfiguration;
+import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
 import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
 import org.apache.distributedlog.exceptions.OverCapacityException;
 import org.apache.distributedlog.limiter.ChainedRequestLimiter;
diff --git a/distributedlog-tutorials/distributedlog-basic/conf/log4j.properties b/distributedlog-tutorials/distributedlog-basic/conf/log4j.properties
index 56a6417..7aa93f6 100644
--- a/distributedlog-tutorials/distributedlog-basic/conf/log4j.properties
+++ b/distributedlog-tutorials/distributedlog-basic/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-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/AtomicWriter.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/AtomicWriter.java
index 8ef5c46..006f832 100644
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/AtomicWriter.java
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/AtomicWriter.java
@@ -17,22 +17,21 @@
  */
 package org.apache.distributedlog.basic;
 
+import static com.google.common.base.Charsets.UTF_8;
+
 import com.google.common.collect.Lists;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Await;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.LogRecordSet;
 import org.apache.distributedlog.io.CompressionCodec.Type;
 import org.apache.distributedlog.service.DistributedLogClient;
 import org.apache.distributedlog.service.DistributedLogClientBuilder;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import static com.google.common.base.Charsets.UTF_8;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 
 /**
  * Write multiple record atomically
@@ -60,12 +59,12 @@
                 .build();
 
         final LogRecordSet.Writer recordSetWriter = LogRecordSet.newWriter(16 * 1024, Type.NONE);
-        List<Future<DLSN>> writeFutures = Lists.newArrayListWithExpectedSize(messages.length);
+        List<CompletableFuture<DLSN>> writeFutures = Lists.newArrayListWithExpectedSize(messages.length);
         for (String msg : messages) {
             final String message = msg;
             ByteBuffer msgBuf = ByteBuffer.wrap(msg.getBytes(UTF_8));
-            Promise<DLSN> writeFuture = new Promise<DLSN>();
-            writeFuture.addEventListener(new FutureEventListener<DLSN>() {
+            CompletableFuture<DLSN> writeFuture = FutureUtils.createFuture();
+            writeFuture.whenComplete(new FutureEventListener<DLSN>() {
                 @Override
                 public void onFailure(Throwable cause) {
                     System.out.println("Encountered error on writing data");
@@ -81,9 +80,9 @@
             recordSetWriter.writeRecord(msgBuf, writeFuture);
             writeFutures.add(writeFuture);
         }
-        FutureUtils.result(
+        Await.result(
             client.writeRecordSet(streamName, recordSetWriter)
-                .addEventListener(new FutureEventListener<DLSN>() {
+                .addEventListener(new com.twitter.util.FutureEventListener<DLSN>() {
                     @Override
                     public void onFailure(Throwable cause) {
                         recordSetWriter.abortTransmit(cause);
@@ -101,7 +100,7 @@
                     }
                 })
         );
-        FutureUtils.result(Future.collect(writeFutures));
+        FutureUtils.result(FutureUtils.collect(writeFutures));
         client.close();
     }
 }
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleWriter.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleWriter.java
index 4322224..833c0ce 100644
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleWriter.java
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleWriter.java
@@ -17,18 +17,21 @@
  */
 package org.apache.distributedlog.basic;
 
-import org.apache.distributedlog.*;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-import jline.ConsoleReader;
+import static com.google.common.base.Charsets.UTF_8;
 
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 import java.net.URI;
 import java.util.concurrent.TimeUnit;
-
-import static com.google.common.base.Charsets.UTF_8;
+import jline.ConsoleReader;
 
 /**
  * Writer write records from console
@@ -53,7 +56,7 @@
         conf.setOutputBufferSize(0);
         conf.setPeriodicFlushFrequencyMilliSeconds(0);
         conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .regionId(DistributedLogConstants.LOCAL_REGION_ID)
@@ -73,7 +76,7 @@
                 String line;
                 while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
                     writer.write(new LogRecord(System.currentTimeMillis(), line.getBytes(UTF_8)))
-                            .addEventListener(new FutureEventListener<DLSN>() {
+                            .whenComplete(new FutureEventListener<DLSN>() {
                                 @Override
                                 public void onFailure(Throwable cause) {
                                     System.out.println("Encountered error on writing data");
@@ -89,7 +92,7 @@
                 }
             } finally {
                 if (null != writer) {
-                    FutureUtils.result(writer.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
+                    FutureUtils.result(writer.asyncClose(), 5, TimeUnit.SECONDS);
                 }
             }
         } finally {
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/MultiReader.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/MultiReader.java
index 9fe2013..29370de 100644
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/MultiReader.java
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/MultiReader.java
@@ -18,15 +18,17 @@
 package org.apache.distributedlog.basic;
 
 import org.apache.distributedlog.*;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.exceptions.LogEmptyException;
 import org.apache.distributedlog.exceptions.LogNotFoundException;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.util.FutureEventListener;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.commons.lang.StringUtils;
 
 import java.net.URI;
 import java.util.concurrent.CountDownLatch;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
 
 import static com.google.common.base.Charsets.UTF_8;
 
@@ -48,7 +50,7 @@
 
         URI uri = URI.create(dlUriStr);
         DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .build();
@@ -67,7 +69,7 @@
 
         for (DistributedLogManager dlm : managers) {
             final DistributedLogManager manager = dlm;
-            dlm.getLastLogRecordAsync().addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+            dlm.getLastLogRecordAsync().whenComplete(new FutureEventListener<LogRecordWithDLSN>() {
                 @Override
                 public void onFailure(Throwable cause) {
                     if (cause instanceof LogNotFoundException) {
@@ -99,7 +101,7 @@
                                  final DLSN dlsn,
                                  final CountDownLatch keepAliveLatch) {
         System.out.println("Wait for records from " + dlm.getStreamName() + " starting from " + dlsn);
-        dlm.openAsyncLogReader(dlsn).addEventListener(new FutureEventListener<AsyncLogReader>() {
+        dlm.openAsyncLogReader(dlsn).whenComplete(new FutureEventListener<AsyncLogReader>() {
             @Override
             public void onFailure(Throwable cause) {
                 System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
@@ -131,10 +133,10 @@
                 System.out.println("\"\"\"");
                 System.out.println(new String(record.getPayload(), UTF_8));
                 System.out.println("\"\"\"");
-                reader.readNext().addEventListener(this);
+                reader.readNext().whenComplete(this);
             }
         };
-        reader.readNext().addEventListener(readListener);
+        reader.readNext().whenComplete(readListener);
     }
 
 }
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/StreamRewinder.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/StreamRewinder.java
index 50a456d..b43b90c 100644
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/StreamRewinder.java
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/StreamRewinder.java
@@ -17,19 +17,20 @@
  */
 package org.apache.distributedlog.basic;
 
-import org.apache.distributedlog.*;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.CountDownLatch;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
+import static com.google.common.base.Charsets.UTF_8;
 
 import java.net.URI;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-
-import static com.google.common.base.Charsets.UTF_8;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 
 /**
  * Rewind a stream to read data back in a while
@@ -50,7 +51,7 @@
 
         URI uri = URI.create(dlUriStr);
         DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .build();
@@ -101,13 +102,13 @@
                     caughtup.set(true);
                 }
 
-                reader.readNext().addEventListener(this);
+                reader.readNext().whenComplete(this);
             }
         };
-        reader.readNext().addEventListener(readListener);
+        reader.readNext().whenComplete(readListener);
 
         keepAliveLatch.await();
-        FutureUtils.result(reader.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
+        FutureUtils.result(reader.asyncClose(), 5, TimeUnit.SECONDS);
     }
 
 }
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/TailReader.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/TailReader.java
index 8b43b45..6a3acf6 100644
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/TailReader.java
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/TailReader.java
@@ -17,20 +17,22 @@
  */
 package org.apache.distributedlog.basic;
 
-import org.apache.distributedlog.*;
-import org.apache.distributedlog.exceptions.LogEmptyException;
-import org.apache.distributedlog.exceptions.LogNotFoundException;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
+import static com.google.common.base.Charsets.UTF_8;
 
 import java.net.URI;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
-
-import static com.google.common.base.Charsets.UTF_8;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.exceptions.LogEmptyException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
 
 /**
  * A reader is tailing a log
@@ -50,7 +52,7 @@
 
         URI uri = URI.create(dlUriStr);
         DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .build();
@@ -100,13 +102,13 @@
                 System.out.println("\"\"\"");
                 System.out.println(new String(record.getPayload(), UTF_8));
                 System.out.println("\"\"\"");
-                reader.readNext().addEventListener(this);
+                reader.readNext().whenComplete(this);
             }
         };
-        reader.readNext().addEventListener(readListener);
+        reader.readNext().whenComplete(readListener);
 
         keepAliveLatch.await();
-        FutureUtils.result(reader.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
+        FutureUtils.result(reader.asyncClose(), 5, TimeUnit.SECONDS);
     }
 
 }
diff --git a/distributedlog-tutorials/distributedlog-kafka/conf/log4j.properties b/distributedlog-tutorials/distributedlog-kafka/conf/log4j.properties
index 56a6417..7aa93f6 100644
--- a/distributedlog-tutorials/distributedlog-kafka/conf/log4j.properties
+++ b/distributedlog-tutorials/distributedlog-kafka/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-tutorials/distributedlog-kafka/src/main/java/org/apache/distributedlog/kafka/DLFutureRecordMetadata.java b/distributedlog-tutorials/distributedlog-kafka/src/main/java/org/apache/distributedlog/kafka/DLFutureRecordMetadata.java
index f1490d4..9cf1cf9 100644
--- a/distributedlog-tutorials/distributedlog-kafka/src/main/java/org/apache/distributedlog/kafka/DLFutureRecordMetadata.java
+++ b/distributedlog-tutorials/distributedlog-kafka/src/main/java/org/apache/distributedlog/kafka/DLFutureRecordMetadata.java
@@ -17,20 +17,18 @@
  */
 package org.apache.distributedlog.kafka;
 
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.util.FutureUtils;
+import com.twitter.util.Await;
 import com.twitter.util.Duration;
 import com.twitter.util.FutureEventListener;
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.TopicPartition;
-
 import java.io.IOException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import org.apache.distributedlog.DLSN;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.TopicPartition;
 
 class DLFutureRecordMetadata implements Future<RecordMetadata> {
 
@@ -79,25 +77,26 @@
     @Override
     public RecordMetadata get() throws InterruptedException, ExecutionException {
         try {
-            FutureUtils.result(dlsnFuture);
+            Await.result(dlsnFuture);
             // TODO: align the DLSN concepts with kafka concepts
             return new RecordMetadata(new TopicPartition(topic, 0), -1L, -1L);
-        } catch (DLInterruptedException e) {
+        } catch (InterruptedException e) {
             throw new InterruptedException("Interrupted on waiting for response");
-        } catch (IOException e) {
+        } catch (Exception e) {
             throw new ExecutionException("Error on waiting for response", e);
         }
     }
 
     @Override
-    public RecordMetadata get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
+    public RecordMetadata get(long timeout, TimeUnit unit)
+            throws InterruptedException, ExecutionException, TimeoutException {
         try {
-            FutureUtils.result(dlsnFuture, Duration.apply(timeout, unit));
+            Await.result(dlsnFuture, Duration.apply(timeout, unit));
             // TODO: align the DLSN concepts with kafka concepts
             return new RecordMetadata(new TopicPartition(topic, 0), -1L, -1L);
-        } catch (DLInterruptedException e) {
+        } catch (InterruptedException e) {
             throw new InterruptedException("Interrupted on waiting for response");
-        } catch (IOException e) {
+        } catch (Exception e) {
             throw new ExecutionException("Error on waiting for response", e);
         }
     }
diff --git a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/DistributedLogInputFormat.java b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/DistributedLogInputFormat.java
index 94a53d4..6fd017c 100644
--- a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/DistributedLogInputFormat.java
+++ b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/DistributedLogInputFormat.java
@@ -19,13 +19,13 @@
 
 import com.google.common.collect.Lists;
 import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.api.DistributedLogManager;
 import org.apache.distributedlog.DLSN;
 import org.apache.distributedlog.LogRecordWithDLSN;
 import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.api.namespace.Namespace;
 import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeperAccessor;
@@ -59,7 +59,7 @@
     protected Configuration conf;
     protected DistributedLogConfiguration dlConf;
     protected URI dlUri;
-    protected DistributedLogNamespace namespace;
+    protected Namespace namespace;
     protected String streamName;
     protected DistributedLogManager dlm;
 
@@ -71,7 +71,7 @@
         dlUri = URI.create(configuration.get(DL_URI, ""));
         streamName = configuration.get(DL_STREAM, "");
         try {
-            namespace = DistributedLogNamespaceBuilder.newBuilder()
+            namespace = NamespaceBuilder.newBuilder()
                     .conf(dlConf)
                     .uri(dlUri)
                     .build();
diff --git a/distributedlog-tutorials/distributedlog-messaging/conf/log4j.properties b/distributedlog-tutorials/distributedlog-messaging/conf/log4j.properties
index 56a6417..7aa93f6 100644
--- a/distributedlog-tutorials/distributedlog-messaging/conf/log4j.properties
+++ b/distributedlog-tutorials/distributedlog-messaging/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-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ReaderWithOffsets.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ReaderWithOffsets.java
index ecf18fc..c59cc72 100644
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ReaderWithOffsets.java
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ReaderWithOffsets.java
@@ -17,22 +17,27 @@
  */
 package org.apache.distributedlog.messaging;
 
-import org.apache.distributedlog.*;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-import org.iq80.leveldb.DB;
-import org.iq80.leveldb.Options;
+import static com.google.common.base.Charsets.UTF_8;
+import static org.iq80.leveldb.impl.Iq80DBFactory.*;
 
 import java.io.File;
 import java.net.URI;
-import java.util.concurrent.*;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.iq80.leveldb.impl.Iq80DBFactory.*;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.iq80.leveldb.DB;
+import org.iq80.leveldb.Options;
 
 /**
  * Reader with offsets
@@ -54,7 +59,7 @@
 
         URI uri = URI.create(dlUriStr);
         DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .build();
@@ -120,13 +125,13 @@
                 System.out.println(new String(record.getPayload(), UTF_8));
                 System.out.println("\"\"\"");
                 lastDLSN.set(record.getDlsn());
-                reader.readNext().addEventListener(this);
+                reader.readNext().whenComplete(this);
             }
         };
-        reader.readNext().addEventListener(readListener);
+        reader.readNext().whenComplete(readListener);
 
         keepAliveLatch.await();
-        FutureUtils.result(reader.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
+        FutureUtils.result(reader.asyncClose(), 5, TimeUnit.SECONDS);
     }
 
 }
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/StreamTransformer.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/StreamTransformer.java
index 2cf202f..3f874c0 100644
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/StreamTransformer.java
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/StreamTransformer.java
@@ -17,19 +17,7 @@
  */
 package org.apache.distributedlog.messaging;
 
-import org.apache.distributedlog.*;
-import org.apache.distributedlog.exceptions.LogEmptyException;
-import org.apache.distributedlog.exceptions.LogNotFoundException;
-import org.apache.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import org.apache.distributedlog.thrift.messaging.TransformedRecord;
-import org.apache.distributedlog.util.FutureUtils;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.transport.TIOStreamTransport;
+import static com.google.common.base.Charsets.UTF_8;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -37,8 +25,24 @@
 import java.nio.ByteBuffer;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
-
-import static com.google.common.base.Charsets.UTF_8;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.exceptions.LogEmptyException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.thrift.messaging.TransformedRecord;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.transport.TIOStreamTransport;
 
 /**
  * Transform one stream to another stream. And apply transformation
@@ -63,7 +67,7 @@
         DistributedLogConfiguration conf = new DistributedLogConfiguration();
         conf.setOutputBufferSize(16*1024); // 16KB
         conf.setPeriodicFlushFrequencyMilliSeconds(5); // 5ms
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+        Namespace namespace = NamespaceBuilder.newBuilder()
                 .conf(conf)
                 .uri(uri)
                 .build();
@@ -102,7 +106,7 @@
         try {
             readLoop(srcDlm, srcDlsn, targetWriter, replicationTransformer);
         } finally {
-            FutureUtils.result(targetWriter.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
+            FutureUtils.result(targetWriter.asyncClose(), 5, TimeUnit.SECONDS);
             targetDlm.close();
             srcDlm.close();
             namespace.close();
@@ -131,7 +135,7 @@
             @Override
             public void onSuccess(LogRecordWithDLSN record) {
                 if (record.getDlsn().compareTo(fromDLSN) <= 0) {
-                    reader.readNext().addEventListener(this);
+                    reader.readNext().whenComplete(this);
                     return;
                 }
                 System.out.println("Received record " + record.getDlsn());
@@ -146,13 +150,13 @@
                     e.printStackTrace(System.err);
                     keepAliveLatch.countDown();
                 }
-                reader.readNext().addEventListener(this);
+                reader.readNext().whenComplete(this);
             }
         };
-        reader.readNext().addEventListener(readListener);
+        reader.readNext().whenComplete(readListener);
 
         keepAliveLatch.await();
-        FutureUtils.result(reader.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
+        FutureUtils.result(reader.asyncClose(), 5, TimeUnit.SECONDS);
     }
 
     private static void transform(final AsyncLogWriter writer,
@@ -170,7 +174,7 @@
         transformedRecord.write(protocolFactory.getProtocol(new TIOStreamTransport(baos)));
         byte[] data = baos.toByteArray();
         writer.write(new LogRecord(record.getSequenceId(), data))
-                .addEventListener(new FutureEventListener<DLSN>() {
+                .whenComplete(new FutureEventListener<DLSN>() {
             @Override
             public void onFailure(Throwable cause) {
                 System.err.println("Encountered error on writing records to stream " + writer.getStreamName());
diff --git a/pom.xml b/pom.xml
index a24577c..d8c74ac 100644
--- a/pom.xml
+++ b/pom.xml
@@ -83,6 +83,7 @@
   </developers>
   <modules>
     <module>distributedlog-build-tools</module>
+    <module>distributedlog-common</module>
     <module>distributedlog-protocol</module>
     <module>distributedlog-core</module>
     <module>distributedlog-proxy-protocol</module>
@@ -90,6 +91,7 @@
     <module>distributedlog-proxy-server</module>
     <module>distributedlog-benchmark</module>
     <module>distributedlog-tutorials</module>
+    <module>distributedlog-core-twitter</module>
   </modules>
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
@@ -103,11 +105,13 @@
     <commons-lang3.version>3.3.2</commons-lang3.version>
     <curator.version>3.2.1</curator.version>
     <finagle.version>6.34.0</finagle.version>
+    <freebuilder.version>1.12.3</freebuilder.version>
     <guava.version>20.0</guava.version>
     <jetty.version>8.1.19.v20160209</jetty.version>
     <jmock.version>2.8.2</jmock.version>
     <junit.version>4.8.1</junit.version>
     <libthrift.version>0.5.0-1</libthrift.version>
+    <lombok.version>1.16.16</lombok.version>
     <lz4.version>1.2.0</lz4.version>
     <mockito.version>1.9.5</mockito.version>
     <scrooge.version>4.6.0</scrooge.version>
@@ -142,7 +146,7 @@
           <groups>
             <group>
               <title>Core Library</title>
-              <packages>org.apache.distributedlog:org.apache.distributedlog.annotations:org.apache.distributedlog.callback:org.apache.distributedlog.exceptions:org.apache.distributedlog.feature:org.apache.distributedlog.io:org.apache.distributedlog.lock:org.apache.distributedlog.logsegment:org.apache.distributedlog.metadata:org.apache.distributedlog.namespace:org.apache.distributedlog.net:org.apache.distributedlog.stats:org.apache.distributedlog.subscription</packages>
+              <packages>org.apache.distributedlog:org.apache.distributedlog.annotations:org.apache.distributedlog.callback:org.apache.distributedlog.exceptions:org.apache.distributedlog.feature:org.apache.distributedlog.io:org.apache.distributedlog.lock:org.apache.distributedlog.logsegment:org.apache.distributedlog.metadata:org.apache.distributedlog.namespace:org.apache.distributedlog.net:org.apache.distributedlog.stats:org.apache.distributedlog.api.subscription</packages>
             </group>
             <group>
               <title>Proxy Client</title>
@@ -150,7 +154,7 @@
             </group>
           </groups>
           <excludePackageNames>
-            org.apache.distributedlog.acl:org.apache.distributedlog.admin:org.apache.distributedlog.auditor:org.apache.distributedlog.basic:org.apache.distributedlog.benchmark*:org.apache.distributedlog.bk:org.apache.distributedlog.ownership:org.apache.distributedlog.proxy:org.apache.distributedlog.resolver:org.apache.distributedlog.service.*:org.apache.distributedlog.config:org.apache.distributedlog.function:org.apache.distributedlog.impl*:org.apache.distributedlog.injector:org.apache.distributedlog.kafka:org.apache.distributedlog.limiter:org.apache.distributedlog.mapreduce:org.apache.distributedlog.messaging:org.apache.distributedlog.rate:org.apache.distributedlog.readahead:org.apache.distributedlog.selector:org.apache.distributedlog.stats:org.apache.distributedlog.thrift*:org.apache.distributedlog.tools:org.apache.distributedlog.util:org.apache.distributedlog.zk:org.apache.bookkeeper.client:org.apache.bookkeeper.stats 
+            org.apache.distributedlog.acl:org.apache.distributedlog.admin:org.apache.distributedlog.auditor:org.apache.distributedlog.basic:org.apache.distributedlog.benchmark*:org.apache.distributedlog.bk:org.apache.distributedlog.ownership:org.apache.distributedlog.proxy:org.apache.distributedlog.resolver:org.apache.distributedlog.service.*:org.apache.distributedlog.config:org.apache.distributedlog.function:org.apache.distributedlog.impl*:org.apache.distributedlog.injector:org.apache.distributedlog.kafka:org.apache.distributedlog.limiter:org.apache.distributedlog.mapreduce:org.apache.distributedlog.messaging:org.apache.distributedlog.common.rate:org.apache.distributedlog.readahead:org.apache.distributedlog.selector:org.apache.distributedlog.stats:org.apache.distributedlog.thrift*:org.apache.distributedlog.tools:org.apache.distributedlog.util:org.apache.distributedlog.zk:org.apache.bookkeeper.client:org.apache.bookkeeper.stats
           </excludePackageNames>
         </configuration>
         <executions>
@@ -182,8 +186,8 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>${maven-compiler-plugin.version}</version>
         <configuration>
-          <source>1.7</source>
-          <target>1.7</target>
+          <source>1.8</source>
+          <target>1.8</target>
         </configuration>
       </plugin>
       <plugin>